[DS-1424] Implementation of mobile clients first seen (#1934)
* Add initial boilerplate for clients_first_seen * Remove submission_timestamp as a field * [wip] Join data against legacy fennec id if applicable * Remove user facing view * Revert "Remove user facing view" This reverts commit a728a7882170eadad5413c7a7046c0f38297bb87. * Add flag for fennec_id * Update logic to limit rows in partitions to submission_date * Add all sql in glean_usage to format ignores * Separate init and query * Add default encoders for testing sql * Add test for initialization of baseline clients first seen in fenix * Update query to update over previous history * Add test for aggregation * Add generated sql and tests for simple baseline clients first seen * Add dry-run exceptions for clients first seen tables * Add clients first seen to generated sql * Update bigquery_etl/glean_usage/templates/baseline_clients_first_seen.metadata.yaml Co-authored-by: Jeff Klukas <jklukas@mozilla.com> * Update bigquery_etl/glean_usage/templates/baseline_clients_first_seen.metadata.yaml Co-authored-by: Jeff Klukas <jklukas@mozilla.com> * Group by sample id instead of min * Add submission_date as baseline first seen date Co-authored-by: Jeff Klukas <jklukas@mozilla.com>
This commit is contained in:
Родитель
420a1bfdba
Коммит
1503a7fa89
|
@ -178,6 +178,12 @@ SKIP = {
|
|||
# Query templates
|
||||
"sql/moz-fx-data-shared-prod/search_derived/mobile_search_clients_daily_v1/fenix_metrics.template.sql", # noqa E501
|
||||
"sql/moz-fx-data-shared-prod/search_derived/mobile_search_clients_daily_v1/mobile_search_clients_daily.template.sql", # noqa E501
|
||||
# Temporary table does not exist
|
||||
# TODO: remove this in a follow up PR
|
||||
*glob.glob(
|
||||
"sql/moz-fx-data-shared-prod/*/baseline_clients_first_seen_v1/*.sql",
|
||||
recursive=True,
|
||||
),
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -12,11 +12,7 @@ SKIP = {
|
|||
# files that existed before we started to enforce this script
|
||||
*glob.glob("bigquery_etl/glam/templates/*.sql"),
|
||||
*glob.glob("bigquery_etl/events_daily/query_templates/**/*.sql"),
|
||||
"bigquery_etl/glean_usage/templates/baseline_clients_daily.view.sql",
|
||||
"bigquery_etl/glean_usage/templates/baseline_clients_daily_v1.sql",
|
||||
"bigquery_etl/glean_usage/templates/baseline_clients_last_seen.view.sql",
|
||||
"bigquery_etl/glean_usage/templates/baseline_clients_last_seen_v1.sql",
|
||||
"sql/moz-fx-data-shared-prod/telemetry/experiments_aggregates_v1/view.sql",
|
||||
*glob.glob("bigquery_etl/glean_usage/templates/*.sql"),
|
||||
"sql/moz-fx-data-shared-prod/telemetry/fenix_events_v1/view.sql",
|
||||
"sql/moz-fx-data-shared-prod/telemetry/fennec_ios_events_v1/view.sql",
|
||||
"sql/moz-fx-data-shared-prod/telemetry/fire_tv_events_v1/view.sql",
|
||||
|
|
|
@ -0,0 +1,183 @@
|
|||
"""Generate and run baseline_clients_first_seen queries for Glean apps."""
|
||||
|
||||
import logging
|
||||
from argparse import ArgumentParser
|
||||
from datetime import datetime
|
||||
from functools import partial
|
||||
from multiprocessing.pool import ThreadPool
|
||||
|
||||
from google.cloud import bigquery
|
||||
from google.cloud.bigquery import ScalarQueryParameter, WriteDisposition
|
||||
|
||||
from bigquery_etl.glean_usage.common import (
|
||||
list_baseline_tables,
|
||||
referenced_table_exists,
|
||||
render,
|
||||
table_names_from_baseline,
|
||||
write_sql,
|
||||
)
|
||||
from bigquery_etl.util import standard_args # noqa E402
|
||||
|
||||
parser = ArgumentParser(description=__doc__)
|
||||
parser.add_argument(
|
||||
"--project_id",
|
||||
"--project-id",
|
||||
default="moz-fx-data-shar-nonprod-efed",
|
||||
help="ID of the project in which to find tables",
|
||||
)
|
||||
parser.add_argument(
|
||||
"--date",
|
||||
required=True,
|
||||
type=lambda d: datetime.strptime(d, "%Y-%m-%d").date(),
|
||||
help="Date partition to process, in format 2019-01-01",
|
||||
)
|
||||
parser.add_argument(
|
||||
"--output_dir",
|
||||
"--output-dir",
|
||||
help="Also write the query text underneath the given sql dir",
|
||||
)
|
||||
parser.add_argument(
|
||||
"--output_only",
|
||||
"--output-only",
|
||||
"--views_only", # Deprecated name
|
||||
"--views-only", # Deprecated name
|
||||
action="store_true",
|
||||
help=(
|
||||
"If set, we only write out sql to --output-dir and we skip"
|
||||
" running the queries"
|
||||
),
|
||||
)
|
||||
standard_args.add_parallelism(parser)
|
||||
standard_args.add_dry_run(parser, debug_log_queries=False)
|
||||
standard_args.add_log_level(parser)
|
||||
standard_args.add_priority(parser)
|
||||
standard_args.add_billing_projects(parser)
|
||||
standard_args.add_table_filter(parser)
|
||||
|
||||
|
||||
TARGET_TABLE_ID = "baseline_clients_first_seen_v1"
|
||||
INIT_FILENAME = f"{TARGET_TABLE_ID}.init.sql"
|
||||
QUERY_FILENAME = f"{TARGET_TABLE_ID}.query.sql"
|
||||
VIEW_FILENAME = f"{TARGET_TABLE_ID[:-3]}.view.sql"
|
||||
VIEW_METADATA_FILENAME = f"{TARGET_TABLE_ID[:-3]}.metadata.yaml"
|
||||
|
||||
|
||||
def main():
|
||||
"""Generate and run queries based on CLI args."""
|
||||
args = parser.parse_args()
|
||||
|
||||
try:
|
||||
logging.basicConfig(level=args.log_level, format="%(levelname)s %(message)s")
|
||||
except ValueError as e:
|
||||
parser.error(f"argument --log-level: {e}")
|
||||
|
||||
baseline_tables = list_baseline_tables(
|
||||
project_id=args.project_id,
|
||||
only_tables=getattr(args, "only_tables", None),
|
||||
table_filter=args.table_filter,
|
||||
)
|
||||
|
||||
with ThreadPool(args.parallelism) as pool:
|
||||
# Do a first pass with dry_run=True so we don't end up with a partial success;
|
||||
# we also write out queries in this pass if so configured.
|
||||
pool.map(
|
||||
partial(
|
||||
run_query,
|
||||
args.project_id,
|
||||
date=args.date,
|
||||
dry_run=True,
|
||||
output_dir=args.output_dir,
|
||||
output_only=args.output_only,
|
||||
),
|
||||
baseline_tables,
|
||||
)
|
||||
if args.output_only:
|
||||
return
|
||||
logging.info(
|
||||
f"Dry runs successful for {len(baseline_tables)}"
|
||||
" baseline_clients_first_seen table(s)"
|
||||
)
|
||||
# Now, actually run the queries.
|
||||
if not args.dry_run:
|
||||
pool.map(
|
||||
partial(run_query, args.project_id, date=args.date, dry_run=False),
|
||||
baseline_tables,
|
||||
)
|
||||
|
||||
|
||||
def run_query(
|
||||
project_id, baseline_table, date, dry_run, output_dir=None, output_only=False
|
||||
):
|
||||
"""Process a single table, potentially also writing out the generated queries."""
|
||||
tables = table_names_from_baseline(baseline_table)
|
||||
|
||||
table_id = tables["first_seen_table"]
|
||||
view_id = tables["first_seen_view"]
|
||||
render_kwargs = dict(
|
||||
header="-- Generated via bigquery_etl.glean_usage\n",
|
||||
fennec_id=any(
|
||||
(app_id in baseline_table)
|
||||
for app_id in [
|
||||
"org_mozilla_firefox",
|
||||
"org_mozilla_fenix_nightly",
|
||||
"org_mozilla_fennec_aurora",
|
||||
"org_mozilla_firefox_beta",
|
||||
"org_mozilla_fenix",
|
||||
]
|
||||
),
|
||||
)
|
||||
render_kwargs.update(
|
||||
# Remove the project from the table name, which is implicit in the
|
||||
# query. It also doesn't play well with tests.
|
||||
{key: ".".join(table_id.split(".")[1:]) for key, table_id in tables.items()}
|
||||
)
|
||||
job_kwargs = dict(use_legacy_sql=False, dry_run=dry_run)
|
||||
|
||||
query_sql = render(QUERY_FILENAME, **render_kwargs)
|
||||
init_sql = render(INIT_FILENAME, **render_kwargs)
|
||||
view_sql = render(VIEW_FILENAME, **render_kwargs)
|
||||
view_metadata = render(VIEW_METADATA_FILENAME, format=False, **render_kwargs)
|
||||
sql = query_sql
|
||||
|
||||
if not (referenced_table_exists(view_sql)):
|
||||
if output_only:
|
||||
logging.info("Skipping view for table which doesn't exist:" f" {table_id}")
|
||||
return
|
||||
elif dry_run:
|
||||
logging.info(f"Table does not yet exist: {table_id}")
|
||||
else:
|
||||
logging.info(f"Creating table: {table_id}")
|
||||
sql = init_sql
|
||||
elif output_only:
|
||||
pass
|
||||
else:
|
||||
# Table exists, so just overwrite the entire table with the day's results
|
||||
job_kwargs.update(
|
||||
destination=table_id,
|
||||
write_disposition=WriteDisposition.WRITE_TRUNCATE,
|
||||
query_parameters=[ScalarQueryParameter("submission_date", "DATE", date)],
|
||||
)
|
||||
if not dry_run:
|
||||
logging.info(f"Running query for: {table_id}")
|
||||
|
||||
if output_dir:
|
||||
write_sql(output_dir, view_id, "metadata.yaml", view_metadata)
|
||||
write_sql(output_dir, view_id, "view.sql", view_sql)
|
||||
write_sql(output_dir, table_id, "query.sql", query_sql)
|
||||
write_sql(output_dir, table_id, "init.sql", init_sql)
|
||||
if output_only:
|
||||
# Return before we initialize the BQ client so that we can generate SQL
|
||||
# without having BQ credentials.
|
||||
return
|
||||
|
||||
client = bigquery.Client(project_id)
|
||||
job_config = bigquery.QueryJobConfig(**job_kwargs)
|
||||
job = client.query(sql, job_config)
|
||||
if not dry_run:
|
||||
job.result()
|
||||
logging.info(f"Recreating view {view_id}")
|
||||
client.query(view_sql, bigquery.QueryJobConfig(use_legacy_sql=False)).result()
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
main()
|
|
@ -81,10 +81,13 @@ def table_names_from_baseline(baseline_table):
|
|||
prefix = re.sub(r"_stable\..+", "", baseline_table)
|
||||
return dict(
|
||||
baseline_table=baseline_table,
|
||||
migration_table=f"{prefix}_stable.migration_v1",
|
||||
daily_table=f"{prefix}_derived.baseline_clients_daily_v1",
|
||||
last_seen_table=f"{prefix}_derived.baseline_clients_last_seen_v1",
|
||||
first_seen_table=f"{prefix}_derived.baseline_clients_first_seen_v1",
|
||||
daily_view=f"{prefix}.baseline_clients_daily",
|
||||
last_seen_view=f"{prefix}.baseline_clients_last_seen",
|
||||
first_seen_view=f"{prefix}.baseline_clients_first_seen",
|
||||
)
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,12 @@
|
|||
---
|
||||
# Generated via bigquery_etl.glean_usage
|
||||
|
||||
friendly_name: Baseline Clients First Seen
|
||||
description: |-
|
||||
Captures the earliest server date that we observe a particular client.
|
||||
|
||||
Clustering fields: `sample_id`
|
||||
|
||||
See also: `baseline_clients_daily` and `baseline_clients_last_seen`.
|
||||
owners:
|
||||
- amiyaguchi@mozilla.com
|
|
@ -0,0 +1,9 @@
|
|||
{{ header }}
|
||||
|
||||
CREATE OR REPLACE VIEW
|
||||
`{{ first_seen_view }}`
|
||||
AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
`{{ first_seen_table }}`
|
|
@ -0,0 +1,43 @@
|
|||
{{ header }}
|
||||
{% from "macros.sql" import core_clients_first_seen %}
|
||||
|
||||
CREATE TABLE IF NOT EXISTS
|
||||
`{{ first_seen_table }}`
|
||||
PARTITION BY
|
||||
first_seen_date
|
||||
CLUSTER BY
|
||||
sample_id,
|
||||
submission_date
|
||||
OPTIONS
|
||||
(require_partition_filter = FALSE)
|
||||
AS
|
||||
WITH
|
||||
baseline AS (
|
||||
SELECT
|
||||
client_info.client_id,
|
||||
sample_id,
|
||||
DATE(MIN(submission_timestamp)) as submission_date,
|
||||
DATE(MIN(submission_timestamp)) as first_seen_date,
|
||||
FROM
|
||||
`{{ baseline_table }}`
|
||||
-- initialize by looking over all of history
|
||||
WHERE
|
||||
DATE(submission_timestamp) > "2010-01-01"
|
||||
GROUP BY
|
||||
client_id,
|
||||
sample_id
|
||||
)
|
||||
{% if fennec_id %}
|
||||
,
|
||||
{{ core_clients_first_seen(migration_table) }}
|
||||
SELECT
|
||||
client_id,
|
||||
submission_date,
|
||||
COALESCE(core.first_seen_date, baseline.first_seen_date) as first_seen_date,
|
||||
sample_id
|
||||
FROM baseline
|
||||
LEFT JOIN _core_clients_first_seen core
|
||||
USING (client_id)
|
||||
{% else %}
|
||||
SELECT * FROM baseline
|
||||
{% endif %}
|
|
@ -0,0 +1,88 @@
|
|||
{{ header }}
|
||||
{% from "macros.sql" import core_clients_first_seen %}
|
||||
|
||||
WITH
|
||||
{% if fennec_id %}
|
||||
{{ core_clients_first_seen(migration_table) }},
|
||||
_baseline AS (
|
||||
-- extract the client_id into the top level for the `USING` clause
|
||||
SELECT DISTINCT
|
||||
sample_id,
|
||||
client_info.client_id
|
||||
FROM
|
||||
`{{ baseline_table }}`
|
||||
WHERE
|
||||
DATE(submission_timestamp) = @submission_date
|
||||
),
|
||||
_current AS (
|
||||
SELECT DISTINCT
|
||||
@submission_date as submission_date,
|
||||
coalesce(first_seen_date, @submission_date) as first_seen_date,
|
||||
sample_id,
|
||||
client_id
|
||||
FROM
|
||||
_baseline
|
||||
LEFT JOIN
|
||||
_core_clients_first_seen
|
||||
USING
|
||||
(client_id)
|
||||
),
|
||||
_previous AS (
|
||||
SELECT
|
||||
fs.submission_date,
|
||||
IF(
|
||||
core IS NOT NULL AND core.first_seen_date <= fs.first_seen_date,
|
||||
core.first_seen_date,
|
||||
fs.first_seen_date
|
||||
) AS first_seen_date,
|
||||
sample_id,
|
||||
client_id
|
||||
FROM
|
||||
`{{ first_seen_table }}` fs
|
||||
LEFT JOIN
|
||||
_core_clients_first_seen core
|
||||
USING
|
||||
(client_id)
|
||||
WHERE
|
||||
fs.first_seen_date > "2010-01-01"
|
||||
)
|
||||
{% else %}
|
||||
_current AS (
|
||||
SELECT DISTINCT
|
||||
@submission_date as submission_date,
|
||||
@submission_date as first_seen_date,
|
||||
sample_id,
|
||||
client_info.client_id
|
||||
FROM
|
||||
`{{ baseline_table }}`
|
||||
WHERE
|
||||
DATE(submission_timestamp) = @submission_date
|
||||
and client_info.client_id IS NOT NULL
|
||||
),
|
||||
-- query over all of history to see whether the client_id has shown up before
|
||||
_previous AS (
|
||||
SELECT
|
||||
submission_date,
|
||||
first_seen_date,
|
||||
sample_id,
|
||||
client_id
|
||||
FROM
|
||||
`{{ first_seen_table }}`
|
||||
WHERE
|
||||
first_seen_date > "2010-01-01"
|
||||
)
|
||||
{% endif %}
|
||||
--
|
||||
SELECT
|
||||
IF(
|
||||
_previous.client_id IS NULL
|
||||
OR _previous.first_seen_date >= _current.first_seen_date,
|
||||
_current,
|
||||
_previous
|
||||
).*
|
||||
FROM
|
||||
_current
|
||||
FULL JOIN
|
||||
_previous
|
||||
USING
|
||||
(client_id)
|
|
@ -0,0 +1,31 @@
|
|||
{% macro core_clients_first_seen(migration_table) %}
|
||||
-- this lookup is ~13GB on release (org_mozilla_firefox) as of 2021-03-31
|
||||
_fennec_id_lookup AS (
|
||||
SELECT
|
||||
client_info.client_id,
|
||||
MIN(metrics.uuid.migration_telemetry_identifiers_fennec_client_id) AS fennec_client_id
|
||||
FROM
|
||||
`{{ migration_table }}`
|
||||
WHERE
|
||||
DATE(submission_timestamp) > "2010-01-01"
|
||||
AND client_info.client_id IS NOT NULL
|
||||
AND metrics.uuid.migration_telemetry_identifiers_fennec_client_id IS NOT NULL
|
||||
GROUP BY 1
|
||||
),
|
||||
_core AS (
|
||||
SELECT *
|
||||
FROM `telemetry_derived.core_clients_first_seen_v1`
|
||||
WHERE first_seen_date > "2010-01-01"
|
||||
),
|
||||
-- scanning this table is ~25GB
|
||||
_core_clients_first_seen AS (
|
||||
SELECT
|
||||
_fennec_id_lookup.client_id,
|
||||
first_seen_date
|
||||
FROM
|
||||
_fennec_id_lookup
|
||||
JOIN
|
||||
_core
|
||||
ON _fennec_id_lookup.fennec_client_id = _core.client_id
|
||||
)
|
||||
{% endmacro %}
|
|
@ -20,6 +20,7 @@ from .sql_test import (
|
|||
load_views,
|
||||
print_and_test,
|
||||
read,
|
||||
default_encoding,
|
||||
)
|
||||
|
||||
expect_names = {f"expect.{ext}" for ext in ("yaml", "json", "ndjson")}
|
||||
|
@ -176,7 +177,22 @@ class SqlTest(pytest.Item, pytest.File):
|
|||
job = bq.query(query, job_config=job_config)
|
||||
|
||||
result = list(coerce_result(*job.result()))
|
||||
result.sort(key=lambda row: json.dumps(row, sort_keys=True))
|
||||
expect.sort(key=lambda row: json.dumps(row, sort_keys=True))
|
||||
result.sort(
|
||||
key=lambda row: json.dumps(
|
||||
row, sort_keys=True, default=default_encoding
|
||||
)
|
||||
)
|
||||
# make sure we encode dates correctly
|
||||
expect = json.loads(
|
||||
json.dumps(
|
||||
sorted(
|
||||
expect,
|
||||
key=lambda row: json.dumps(
|
||||
row, sort_keys=True, default=default_encoding
|
||||
),
|
||||
),
|
||||
default=default_encoding,
|
||||
)
|
||||
)
|
||||
|
||||
print_and_test(expect, result)
|
||||
|
|
|
@ -100,6 +100,13 @@ def dataset(bq: bigquery.Client, dataset_id: str):
|
|||
bq.delete_dataset(dataset_id, delete_contents=True)
|
||||
|
||||
|
||||
def default_encoding(obj):
|
||||
"""Add custom logic for serializing rows into JSON for BigQuery."""
|
||||
if isinstance(obj, date) or isinstance(obj, datetime):
|
||||
return obj.isoformat()
|
||||
return obj
|
||||
|
||||
|
||||
def load_tables(
|
||||
bq: bigquery.Client, dataset: bigquery.Dataset, tables: Iterable[Table]
|
||||
):
|
||||
|
@ -132,13 +139,18 @@ def load_tables(
|
|||
else:
|
||||
file_obj = BytesIO()
|
||||
for row in load(*table.source_path):
|
||||
file_obj.write(json.dumps(row).encode() + b"\n")
|
||||
file_obj.write(
|
||||
json.dumps(row, default=default_encoding).encode() + b"\n"
|
||||
)
|
||||
file_obj.seek(0)
|
||||
job = bq.load_table_from_file(file_obj, destination, job_config=job_config)
|
||||
|
||||
try:
|
||||
job.result()
|
||||
except BadRequest:
|
||||
# print the first 5 rows for debugging errors
|
||||
for row in job.errors[:5]:
|
||||
print(row)
|
||||
raise
|
||||
|
||||
|
||||
|
|
|
@ -52,6 +52,11 @@ PLACEHOLDER_DATE="2000-01-01"
|
|||
--output-dir "${SQL_DIR}/${TARGET_PROJECT}" \
|
||||
--output-only \
|
||||
--date="${PLACEHOLDER_DATE}"
|
||||
./script/run_glean_baseline_clients_first_seen \
|
||||
--project-id "${TARGET_PROJECT}" \
|
||||
--output-dir "${SQL_DIR}/${TARGET_PROJECT}" \
|
||||
--output-only \
|
||||
--date="${PLACEHOLDER_DATE}"
|
||||
|
||||
# Record dependencies in yaml files
|
||||
./script/bqetl dependency record "${SQL_DIR}"
|
||||
|
|
|
@ -0,0 +1,7 @@
|
|||
#!/bin/bash
|
||||
|
||||
## Runs or writes out generated baseline_clients_first_seen queries for all Glean applications.
|
||||
## Example:
|
||||
## ./script/run_glean_baseline_clients_first_seen --project-id moz-fx-data-shared-prod --date 2020-03-01 --only 'org_mozilla_firefox_stable.baseline_v1' --output-dir sql/moz-fx-data-shared-prod/ --dry-run
|
||||
|
||||
exec python3 -m bigquery_etl.glean_usage.baseline_clients_first_seen "$@"
|
|
@ -0,0 +1,23 @@
|
|||
#!/bin/bash
|
||||
|
||||
## Updates example queries that are checked into the default branch for testing.
|
||||
|
||||
cd "$(dirname "$0")/.."
|
||||
|
||||
./script/run_glean_baseline_clients_first_seen \
|
||||
--project-id moz-fx-data-shared-prod \
|
||||
--date 2021-03-01 \
|
||||
--only 'org_mozilla_fenix_stable.baseline_v1' \
|
||||
--output-dir sql/moz-fx-data-shared-prod/ \
|
||||
--dry-run
|
||||
|
||||
./script/run_glean_baseline_clients_first_seen \
|
||||
--project-id moz-fx-data-shared-prod \
|
||||
--date 2021-03-01 \
|
||||
--only 'org_mozilla_ios_firefox_stable.baseline_v1' \
|
||||
--output-dir sql/moz-fx-data-shared-prod/ \
|
||||
--dry-run
|
||||
|
||||
# remove some views that can't be tested and are useless checked in
|
||||
rm -r sql/moz-fx-data-shared-prod/org_mozilla_fenix/baseline_clients_first_seen
|
||||
rm -r sql/moz-fx-data-shared-prod/org_mozilla_ios_firefox/baseline_clients_first_seen
|
|
@ -0,0 +1,71 @@
|
|||
-- Generated via bigquery_etl.glean_usage
|
||||
CREATE TABLE IF NOT EXISTS
|
||||
`org_mozilla_fenix_derived.baseline_clients_first_seen_v1`
|
||||
PARTITION BY
|
||||
first_seen_date
|
||||
CLUSTER BY
|
||||
sample_id,
|
||||
submission_date
|
||||
OPTIONS
|
||||
(require_partition_filter = FALSE)
|
||||
AS
|
||||
WITH baseline AS (
|
||||
SELECT
|
||||
client_info.client_id,
|
||||
sample_id,
|
||||
DATE(MIN(submission_timestamp)) AS submission_date,
|
||||
DATE(MIN(submission_timestamp)) AS first_seen_date,
|
||||
FROM
|
||||
`org_mozilla_fenix_stable.baseline_v1`
|
||||
-- initialize by looking over all of history
|
||||
WHERE
|
||||
DATE(submission_timestamp) > "2010-01-01"
|
||||
GROUP BY
|
||||
client_id,
|
||||
sample_id
|
||||
),
|
||||
-- this lookup is ~13GB on release (org_mozilla_firefox) as of 2021-03-31
|
||||
_fennec_id_lookup AS (
|
||||
SELECT
|
||||
client_info.client_id,
|
||||
MIN(metrics.uuid.migration_telemetry_identifiers_fennec_client_id) AS fennec_client_id
|
||||
FROM
|
||||
`org_mozilla_fenix_stable.migration_v1`
|
||||
WHERE
|
||||
DATE(submission_timestamp) > "2010-01-01"
|
||||
AND client_info.client_id IS NOT NULL
|
||||
AND metrics.uuid.migration_telemetry_identifiers_fennec_client_id IS NOT NULL
|
||||
GROUP BY
|
||||
1
|
||||
),
|
||||
_core AS (
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
`telemetry_derived.core_clients_first_seen_v1`
|
||||
WHERE
|
||||
first_seen_date > "2010-01-01"
|
||||
),
|
||||
-- scanning this table is ~25GB
|
||||
_core_clients_first_seen AS (
|
||||
SELECT
|
||||
_fennec_id_lookup.client_id,
|
||||
first_seen_date
|
||||
FROM
|
||||
_fennec_id_lookup
|
||||
JOIN
|
||||
_core
|
||||
ON
|
||||
_fennec_id_lookup.fennec_client_id = _core.client_id
|
||||
)
|
||||
SELECT
|
||||
client_id,
|
||||
submission_date,
|
||||
COALESCE(core.first_seen_date, baseline.first_seen_date) AS first_seen_date,
|
||||
sample_id
|
||||
FROM
|
||||
baseline
|
||||
LEFT JOIN
|
||||
_core_clients_first_seen core
|
||||
USING
|
||||
(client_id)
|
|
@ -0,0 +1,93 @@
|
|||
-- Generated via bigquery_etl.glean_usage
|
||||
WITH
|
||||
-- this lookup is ~13GB on release (org_mozilla_firefox) as of 2021-03-31
|
||||
_fennec_id_lookup AS (
|
||||
SELECT
|
||||
client_info.client_id,
|
||||
MIN(metrics.uuid.migration_telemetry_identifiers_fennec_client_id) AS fennec_client_id
|
||||
FROM
|
||||
`org_mozilla_fenix_stable.migration_v1`
|
||||
WHERE
|
||||
DATE(submission_timestamp) > "2010-01-01"
|
||||
AND client_info.client_id IS NOT NULL
|
||||
AND metrics.uuid.migration_telemetry_identifiers_fennec_client_id IS NOT NULL
|
||||
GROUP BY
|
||||
1
|
||||
),
|
||||
_core AS (
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
`telemetry_derived.core_clients_first_seen_v1`
|
||||
WHERE
|
||||
first_seen_date > "2010-01-01"
|
||||
),
|
||||
-- scanning this table is ~25GB
|
||||
_core_clients_first_seen AS (
|
||||
SELECT
|
||||
_fennec_id_lookup.client_id,
|
||||
first_seen_date
|
||||
FROM
|
||||
_fennec_id_lookup
|
||||
JOIN
|
||||
_core
|
||||
ON
|
||||
_fennec_id_lookup.fennec_client_id = _core.client_id
|
||||
),
|
||||
_baseline AS (
|
||||
-- extract the client_id into the top level for the `USING` clause
|
||||
SELECT DISTINCT
|
||||
sample_id,
|
||||
client_info.client_id
|
||||
FROM
|
||||
`org_mozilla_fenix_stable.baseline_v1`
|
||||
WHERE
|
||||
DATE(submission_timestamp) = @submission_date
|
||||
),
|
||||
_current AS (
|
||||
SELECT DISTINCT
|
||||
@submission_date AS submission_date,
|
||||
coalesce(first_seen_date, @submission_date) AS first_seen_date,
|
||||
sample_id,
|
||||
client_id
|
||||
FROM
|
||||
_baseline
|
||||
LEFT JOIN
|
||||
_core_clients_first_seen
|
||||
USING
|
||||
(client_id)
|
||||
),
|
||||
_previous AS (
|
||||
SELECT
|
||||
fs.submission_date,
|
||||
IF(
|
||||
core IS NOT NULL
|
||||
AND core.first_seen_date <= fs.first_seen_date,
|
||||
core.first_seen_date,
|
||||
fs.first_seen_date
|
||||
) AS first_seen_date,
|
||||
sample_id,
|
||||
client_id
|
||||
FROM
|
||||
`org_mozilla_fenix_derived.baseline_clients_first_seen_v1` fs
|
||||
LEFT JOIN
|
||||
_core_clients_first_seen core
|
||||
USING
|
||||
(client_id)
|
||||
WHERE
|
||||
fs.first_seen_date > "2010-01-01"
|
||||
)
|
||||
--
|
||||
SELECT
|
||||
IF(
|
||||
_previous.client_id IS NULL
|
||||
OR _previous.first_seen_date >= _current.first_seen_date,
|
||||
_current,
|
||||
_previous
|
||||
).*
|
||||
FROM
|
||||
_current
|
||||
FULL JOIN
|
||||
_previous
|
||||
USING
|
||||
(client_id)
|
|
@ -0,0 +1,30 @@
|
|||
-- Generated via bigquery_etl.glean_usage
|
||||
CREATE TABLE IF NOT EXISTS
|
||||
`org_mozilla_ios_firefox_derived.baseline_clients_first_seen_v1`
|
||||
PARTITION BY
|
||||
first_seen_date
|
||||
CLUSTER BY
|
||||
sample_id,
|
||||
submission_date
|
||||
OPTIONS
|
||||
(require_partition_filter = FALSE)
|
||||
AS
|
||||
WITH baseline AS (
|
||||
SELECT
|
||||
client_info.client_id,
|
||||
sample_id,
|
||||
DATE(MIN(submission_timestamp)) AS submission_date,
|
||||
DATE(MIN(submission_timestamp)) AS first_seen_date,
|
||||
FROM
|
||||
`org_mozilla_ios_firefox_stable.baseline_v1`
|
||||
-- initialize by looking over all of history
|
||||
WHERE
|
||||
DATE(submission_timestamp) > "2010-01-01"
|
||||
GROUP BY
|
||||
client_id,
|
||||
sample_id
|
||||
)
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
baseline
|
|
@ -0,0 +1,39 @@
|
|||
-- Generated via bigquery_etl.glean_usage
|
||||
WITH _current AS (
|
||||
SELECT DISTINCT
|
||||
@submission_date AS submission_date,
|
||||
@submission_date AS first_seen_date,
|
||||
sample_id,
|
||||
client_info.client_id
|
||||
FROM
|
||||
`org_mozilla_ios_firefox_stable.baseline_v1`
|
||||
WHERE
|
||||
DATE(submission_timestamp) = @submission_date
|
||||
AND client_info.client_id IS NOT NULL
|
||||
),
|
||||
-- query over all of history to see whether the client_id has shown up before
|
||||
_previous AS (
|
||||
SELECT
|
||||
submission_date,
|
||||
first_seen_date,
|
||||
sample_id,
|
||||
client_id
|
||||
FROM
|
||||
`org_mozilla_ios_firefox_derived.baseline_clients_first_seen_v1`
|
||||
WHERE
|
||||
first_seen_date > "2010-01-01"
|
||||
)
|
||||
--
|
||||
SELECT
|
||||
IF(
|
||||
_previous.client_id IS NULL
|
||||
OR _previous.first_seen_date >= _current.first_seen_date,
|
||||
_current,
|
||||
_previous
|
||||
).*
|
||||
FROM
|
||||
_current
|
||||
FULL JOIN
|
||||
_previous
|
||||
USING
|
||||
(client_id)
|
|
@ -0,0 +1,13 @@
|
|||
---
|
||||
- client_id: existing-instance
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
||||
- client_id: new-instance
|
||||
submission_date: 2021-03-02
|
||||
first_seen_date: 2021-03-02
|
||||
sample_id: 0
|
||||
- client_id: updated-existing-instance
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-02-02
|
||||
sample_id: 0
|
|
@ -0,0 +1,20 @@
|
|||
[
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "client_id"
|
||||
},
|
||||
{
|
||||
"type": "DATE",
|
||||
"name": "submission_date"
|
||||
},
|
||||
{
|
||||
"type": "DATE",
|
||||
"name": "first_seen_date"
|
||||
},
|
||||
{
|
||||
"description": "Hashed version of client_id (if present) useful for partitioning; ranges from 0 to 99",
|
||||
"type": "INTEGER",
|
||||
"name": "sample_id",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
]
|
|
@ -0,0 +1,9 @@
|
|||
---
|
||||
- client_id: existing-instance
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
||||
- client_id: updated-existing-instance
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
|
@ -0,0 +1,682 @@
|
|||
[
|
||||
{
|
||||
"description": "A JSON string containing any payload properties not present in the schema",
|
||||
"type": "STRING",
|
||||
"name": "additional_properties",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The optional Android specific SDK version of the software running on this hardware device.",
|
||||
"type": "STRING",
|
||||
"name": "android_sdk_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The build identifier generated by the CI system (e.g. \"1234/A\"). For language bindings that provide automatic detection for this value, (e.g. Android/Kotlin), in the unlikely event that the build identifier can not be retrieved from the OS, it is set to \"inaccessible\". For other language bindings, if the value was not provided through configuration, this metric gets set to `Unknown`.",
|
||||
"type": "STRING",
|
||||
"name": "app_build",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The channel the application is being distributed on.",
|
||||
"type": "STRING",
|
||||
"name": "app_channel",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The user visible version string (e.g. \"1.0.3\"). In the unlikely event that the display version can not be retrieved, it is set to \"inaccessible\".",
|
||||
"type": "STRING",
|
||||
"name": "app_display_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The architecture of the device, (e.g. \"arm\", \"x86\").",
|
||||
"type": "STRING",
|
||||
"name": "architecture",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "A UUID uniquely identifying the client.",
|
||||
"type": "STRING",
|
||||
"name": "client_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The manufacturer of the device the application is running on. Not set if the device manufacturer can't be determined (e.g. on Desktop).",
|
||||
"type": "STRING",
|
||||
"name": "device_manufacturer",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The model of the device the application is running on. On Android, this is Build.MODEL, the user-visible marketing name, like \"Pixel 2 XL\". Not set if the device model can't be determined (e.g. on Desktop).",
|
||||
"type": "STRING",
|
||||
"name": "device_model",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The date of the first run of the application.",
|
||||
"type": "STRING",
|
||||
"name": "first_run_date",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of the operating system. Possible values: Android, iOS, Linux, Darwin, Windows, FreeBSD, NetBSD, OpenBSD, Solaris, unknown",
|
||||
"type": "STRING",
|
||||
"name": "os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The user-visible version of the operating system (e.g. \"1.2.3\"). If the version detection fails, this metric gets set to `Unknown`.",
|
||||
"type": "STRING",
|
||||
"name": "os_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The version of the Glean SDK",
|
||||
"type": "STRING",
|
||||
"name": "telemetry_sdk_build",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The locale of the application during initialization (e.g. \"es-ES\"). If the locale can't be determined on the system, the value is [\"und\"](https://unicode.org/reports/tr35/#Unknown_or_Invalid_Identifiers), to indicate \"undetermined\".",
|
||||
"type": "STRING",
|
||||
"name": "locale",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "client_info",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The document ID specified in the URI when the client sent this message",
|
||||
"type": "STRING",
|
||||
"name": "document_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "category",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "extra",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "timestamp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "events",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "city",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "An ISO 3166-1 alpha-2 country code",
|
||||
"type": "STRING",
|
||||
"name": "country",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The specific geo database version used for this lookup",
|
||||
"type": "STRING",
|
||||
"name": "db_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "First major country subdivision, typically a state, province, or county",
|
||||
"type": "STRING",
|
||||
"name": "subdivision1",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Second major country subdivision; not applicable for most countries",
|
||||
"type": "STRING",
|
||||
"name": "subdivision2",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Results of a geographic lookup based on the client's IP address",
|
||||
"type": "RECORD",
|
||||
"name": "geo",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "Date HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "date",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "DNT (Do Not Track) HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "dnt",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-Debug-Id HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_debug_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-PingSender-Version HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_pingsender_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-Source-Tags HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_source_tags",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Headers included in the client's HTTP request",
|
||||
"type": "RECORD",
|
||||
"name": "header",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "browser",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "version",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Parsed components of the client's user agent string",
|
||||
"type": "RECORD",
|
||||
"name": "user_agent",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The specific geo ISP database version used for this lookup",
|
||||
"type": "STRING",
|
||||
"name": "db_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of the ISP associated with the client's IP address",
|
||||
"type": "STRING",
|
||||
"name": "name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of a specific business entity associated with the client's IP address when available; otherwise the ISP name",
|
||||
"type": "STRING",
|
||||
"name": "organization",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Results of ISP lookup based on the client's IP address",
|
||||
"type": "RECORD",
|
||||
"name": "isp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "metadata",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The locale of the application during initialization (e.g. \"es-ES\").\nIf the locale can't be determined on the system, the value is\n[\"und\"](https://unicode.org/reports/tr35/#Unknown_or_Invalid_Identifiers),\nto indicate \"undetermined\".\n",
|
||||
"type": "STRING",
|
||||
"name": "glean_baseline_locale",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "string",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "time_unit",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "The duration of the last foreground session.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_baseline_duration",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "timespan",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "A counter of URIs visited by the user in the current session, including\npage reloads. This does not include background page requests and URIs from\nembedded pages or private browsing but may be incremented without user\ninteraction by website scripts that programmatically redirect to a new\nlocation.\n",
|
||||
"type": "INTEGER",
|
||||
"name": "events_total_uri_count",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The number of metrics pings sent during the lifetime of this baseline ping.",
|
||||
"type": "INTEGER",
|
||||
"name": "glean_validation_metrics_ping_count",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "counter",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set with an invalid label.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_label",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set to an invalid value.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_value",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "The labels for this counter are `<search-engine-name>.<source>`.\n\nIf the search engine is bundled with Fenix `search-engine-name` will be\nthe name of the search engine. If it's a custom search engine (defined:\nhttps://github.com/mozilla-mobile/fenix/issues/1607) the value will be\n`custom`.\n\n`source` will be: `action`, `suggestion`, `widget`, `shortcut`, `topsite`\n(depending on the source from which the search started). Also added the\n`other` option for the source but it should never enter on this case.\n",
|
||||
"type": "RECORD",
|
||||
"name": "metrics_search_count",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a timing metric was used incorrectly.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_state",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set a value that overflowed.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_overflow",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records clicks of adverts on SERP pages.\nThe key format is ‘<provider-name>’.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_ad_clicks",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records the type of interaction a user has on SERP pages.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_in_content",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records counts of SERP pages with adverts displayed.\nThe key format is ‘<provider-name>’.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_with_ads",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "A count of the pings submitted, by ping type.\n\nThis metric appears in both the metrics and baseline pings.\n\n- On the metrics ping, the counts include the number of pings sent since\n the last metrics ping (including the last metrics ping)\n- On the baseline ping, the counts include the number of pings send since\n the last baseline ping (including the last baseline ping)\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_validation_pings_submitted",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "labeled_counter",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "jwe",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The hour of the first run of the application.\n",
|
||||
"type": "STRING",
|
||||
"name": "glean_validation_first_run_hour",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "datetime",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "denominator",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "numerator",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "labeled_rate",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "metrics",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized app name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_app_name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized channel name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_channel",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "An ISO 3166-1 alpha-2 country code",
|
||||
"type": "STRING",
|
||||
"name": "normalized_country_code",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized OS name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "normalized_os_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "end_time",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "branch",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "type",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "extra",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "experiments",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "ping_type",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "seq",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "start_time",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "reason",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "ping_info",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Hashed version of client_id (if present) useful for partitioning; ranges from 0 to 99",
|
||||
"type": "INTEGER",
|
||||
"name": "sample_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Time when the ingestion edge server accepted this message",
|
||||
"type": "TIMESTAMP",
|
||||
"name": "submission_timestamp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
]
|
|
@ -0,0 +1,5 @@
|
|||
---
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: new-instance
|
||||
submission_timestamp: 2021-03-02T00:00:00
|
Разница между файлами не показана из-за своего большого размера
Загрузить разницу
|
@ -0,0 +1,8 @@
|
|||
---
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: updated-existing-instance
|
||||
submission_timestamp: 2021-03-02T00:00:00
|
||||
metrics:
|
||||
uuid:
|
||||
migration_telemetry_identifiers_fennec_client_id: updated-existing-instance
|
|
@ -0,0 +1,4 @@
|
|||
---
|
||||
- name: submission_date
|
||||
type: DATE
|
||||
value: 2021-03-02
|
|
@ -0,0 +1,10 @@
|
|||
[
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "client_id"
|
||||
},
|
||||
{
|
||||
"type": "DATE",
|
||||
"name": "first_seen_date"
|
||||
}
|
||||
]
|
|
@ -0,0 +1,3 @@
|
|||
---
|
||||
- client_id: updated-existing-instance
|
||||
first_seen_date: 2021-02-02
|
|
@ -0,0 +1,13 @@
|
|||
---
|
||||
- client_id: single-instance
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
||||
- client_id: multiple-instances
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
||||
- client_id: core-first-seen
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-02-01
|
||||
sample_id: 0
|
|
@ -0,0 +1,682 @@
|
|||
[
|
||||
{
|
||||
"description": "A JSON string containing any payload properties not present in the schema",
|
||||
"type": "STRING",
|
||||
"name": "additional_properties",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The optional Android specific SDK version of the software running on this hardware device.",
|
||||
"type": "STRING",
|
||||
"name": "android_sdk_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The build identifier generated by the CI system (e.g. \"1234/A\"). For language bindings that provide automatic detection for this value, (e.g. Android/Kotlin), in the unlikely event that the build identifier can not be retrieved from the OS, it is set to \"inaccessible\". For other language bindings, if the value was not provided through configuration, this metric gets set to `Unknown`.",
|
||||
"type": "STRING",
|
||||
"name": "app_build",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The channel the application is being distributed on.",
|
||||
"type": "STRING",
|
||||
"name": "app_channel",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The user visible version string (e.g. \"1.0.3\"). In the unlikely event that the display version can not be retrieved, it is set to \"inaccessible\".",
|
||||
"type": "STRING",
|
||||
"name": "app_display_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The architecture of the device, (e.g. \"arm\", \"x86\").",
|
||||
"type": "STRING",
|
||||
"name": "architecture",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "A UUID uniquely identifying the client.",
|
||||
"type": "STRING",
|
||||
"name": "client_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The manufacturer of the device the application is running on. Not set if the device manufacturer can't be determined (e.g. on Desktop).",
|
||||
"type": "STRING",
|
||||
"name": "device_manufacturer",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The model of the device the application is running on. On Android, this is Build.MODEL, the user-visible marketing name, like \"Pixel 2 XL\". Not set if the device model can't be determined (e.g. on Desktop).",
|
||||
"type": "STRING",
|
||||
"name": "device_model",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The date of the first run of the application.",
|
||||
"type": "STRING",
|
||||
"name": "first_run_date",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of the operating system. Possible values: Android, iOS, Linux, Darwin, Windows, FreeBSD, NetBSD, OpenBSD, Solaris, unknown",
|
||||
"type": "STRING",
|
||||
"name": "os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The user-visible version of the operating system (e.g. \"1.2.3\"). If the version detection fails, this metric gets set to `Unknown`.",
|
||||
"type": "STRING",
|
||||
"name": "os_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The version of the Glean SDK",
|
||||
"type": "STRING",
|
||||
"name": "telemetry_sdk_build",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The locale of the application during initialization (e.g. \"es-ES\"). If the locale can't be determined on the system, the value is [\"und\"](https://unicode.org/reports/tr35/#Unknown_or_Invalid_Identifiers), to indicate \"undetermined\".",
|
||||
"type": "STRING",
|
||||
"name": "locale",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "client_info",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The document ID specified in the URI when the client sent this message",
|
||||
"type": "STRING",
|
||||
"name": "document_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "category",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "extra",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "timestamp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "events",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "city",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "An ISO 3166-1 alpha-2 country code",
|
||||
"type": "STRING",
|
||||
"name": "country",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The specific geo database version used for this lookup",
|
||||
"type": "STRING",
|
||||
"name": "db_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "First major country subdivision, typically a state, province, or county",
|
||||
"type": "STRING",
|
||||
"name": "subdivision1",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Second major country subdivision; not applicable for most countries",
|
||||
"type": "STRING",
|
||||
"name": "subdivision2",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Results of a geographic lookup based on the client's IP address",
|
||||
"type": "RECORD",
|
||||
"name": "geo",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "Date HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "date",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "DNT (Do Not Track) HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "dnt",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-Debug-Id HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_debug_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-PingSender-Version HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_pingsender_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-Source-Tags HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_source_tags",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Headers included in the client's HTTP request",
|
||||
"type": "RECORD",
|
||||
"name": "header",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "browser",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "version",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Parsed components of the client's user agent string",
|
||||
"type": "RECORD",
|
||||
"name": "user_agent",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The specific geo ISP database version used for this lookup",
|
||||
"type": "STRING",
|
||||
"name": "db_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of the ISP associated with the client's IP address",
|
||||
"type": "STRING",
|
||||
"name": "name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of a specific business entity associated with the client's IP address when available; otherwise the ISP name",
|
||||
"type": "STRING",
|
||||
"name": "organization",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Results of ISP lookup based on the client's IP address",
|
||||
"type": "RECORD",
|
||||
"name": "isp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "metadata",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The locale of the application during initialization (e.g. \"es-ES\").\nIf the locale can't be determined on the system, the value is\n[\"und\"](https://unicode.org/reports/tr35/#Unknown_or_Invalid_Identifiers),\nto indicate \"undetermined\".\n",
|
||||
"type": "STRING",
|
||||
"name": "glean_baseline_locale",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "string",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "time_unit",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "The duration of the last foreground session.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_baseline_duration",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "timespan",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "A counter of URIs visited by the user in the current session, including\npage reloads. This does not include background page requests and URIs from\nembedded pages or private browsing but may be incremented without user\ninteraction by website scripts that programmatically redirect to a new\nlocation.\n",
|
||||
"type": "INTEGER",
|
||||
"name": "events_total_uri_count",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The number of metrics pings sent during the lifetime of this baseline ping.",
|
||||
"type": "INTEGER",
|
||||
"name": "glean_validation_metrics_ping_count",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "counter",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set with an invalid label.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_label",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set to an invalid value.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_value",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "The labels for this counter are `<search-engine-name>.<source>`.\n\nIf the search engine is bundled with Fenix `search-engine-name` will be\nthe name of the search engine. If it's a custom search engine (defined:\nhttps://github.com/mozilla-mobile/fenix/issues/1607) the value will be\n`custom`.\n\n`source` will be: `action`, `suggestion`, `widget`, `shortcut`, `topsite`\n(depending on the source from which the search started). Also added the\n`other` option for the source but it should never enter on this case.\n",
|
||||
"type": "RECORD",
|
||||
"name": "metrics_search_count",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a timing metric was used incorrectly.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_state",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set a value that overflowed.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_overflow",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records clicks of adverts on SERP pages.\nThe key format is ‘<provider-name>’.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_ad_clicks",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records the type of interaction a user has on SERP pages.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_in_content",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records counts of SERP pages with adverts displayed.\nThe key format is ‘<provider-name>’.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_with_ads",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "A count of the pings submitted, by ping type.\n\nThis metric appears in both the metrics and baseline pings.\n\n- On the metrics ping, the counts include the number of pings sent since\n the last metrics ping (including the last metrics ping)\n- On the baseline ping, the counts include the number of pings send since\n the last baseline ping (including the last baseline ping)\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_validation_pings_submitted",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "labeled_counter",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "jwe",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The hour of the first run of the application.\n",
|
||||
"type": "STRING",
|
||||
"name": "glean_validation_first_run_hour",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "datetime",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "denominator",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "numerator",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "labeled_rate",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "metrics",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized app name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_app_name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized channel name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_channel",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "An ISO 3166-1 alpha-2 country code",
|
||||
"type": "STRING",
|
||||
"name": "normalized_country_code",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized OS name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "normalized_os_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "end_time",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "branch",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "type",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "extra",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "experiments",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "ping_type",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "seq",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "start_time",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "reason",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "ping_info",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Hashed version of client_id (if present) useful for partitioning; ranges from 0 to 99",
|
||||
"type": "INTEGER",
|
||||
"name": "sample_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Time when the ingestion edge server accepted this message",
|
||||
"type": "TIMESTAMP",
|
||||
"name": "submission_timestamp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
]
|
|
@ -0,0 +1,17 @@
|
|||
---
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: single-instance
|
||||
submission_timestamp: 2021-03-01T00:00:00
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: multiple-instances
|
||||
submission_timestamp: 2021-03-01T00:00:00
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: multiple-instances
|
||||
submission_timestamp: 2021-03-02T00:00:00
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: core-first-seen
|
||||
submission_timestamp: 2021-03-01T00:00:00
|
Разница между файлами не показана из-за своего большого размера
Загрузить разницу
|
@ -0,0 +1,8 @@
|
|||
---
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: core-first-seen
|
||||
submission_timestamp: 2021-03-01T00:00:00
|
||||
metrics:
|
||||
uuid:
|
||||
migration_telemetry_identifiers_fennec_client_id: core-id
|
|
@ -0,0 +1,10 @@
|
|||
[
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "client_id"
|
||||
},
|
||||
{
|
||||
"type": "DATE",
|
||||
"name": "first_seen_date"
|
||||
}
|
||||
]
|
|
@ -0,0 +1,5 @@
|
|||
---
|
||||
- client_id: core-id
|
||||
first_seen_date: 2021-02-01
|
||||
- client_id: some-other-id
|
||||
first_seen_date: 2019-01-01
|
|
@ -0,0 +1,13 @@
|
|||
---
|
||||
- client_id: existing-instance
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
||||
- client_id: existing-instance-with-new-ping
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
||||
- client_id: new-instance
|
||||
submission_date: 2021-03-02
|
||||
first_seen_date: 2021-03-02
|
||||
sample_id: 0
|
|
@ -0,0 +1,20 @@
|
|||
[
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "client_id"
|
||||
},
|
||||
{
|
||||
"type": "DATE",
|
||||
"name": "submission_date"
|
||||
},
|
||||
{
|
||||
"type": "DATE",
|
||||
"name": "first_seen_date"
|
||||
},
|
||||
{
|
||||
"description": "Hashed version of client_id (if present) useful for partitioning; ranges from 0 to 99",
|
||||
"type": "INTEGER",
|
||||
"name": "sample_id",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
]
|
|
@ -0,0 +1,9 @@
|
|||
---
|
||||
- client_id: existing-instance
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
||||
- client_id: existing-instance-with-new-ping
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
|
@ -0,0 +1,682 @@
|
|||
[
|
||||
{
|
||||
"description": "A JSON string containing any payload properties not present in the schema",
|
||||
"type": "STRING",
|
||||
"name": "additional_properties",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The optional Android specific SDK version of the software running on this hardware device.",
|
||||
"type": "STRING",
|
||||
"name": "android_sdk_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The build identifier generated by the CI system (e.g. \"1234/A\"). For language bindings that provide automatic detection for this value, (e.g. Android/Kotlin), in the unlikely event that the build identifier can not be retrieved from the OS, it is set to \"inaccessible\". For other language bindings, if the value was not provided through configuration, this metric gets set to `Unknown`.",
|
||||
"type": "STRING",
|
||||
"name": "app_build",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The channel the application is being distributed on.",
|
||||
"type": "STRING",
|
||||
"name": "app_channel",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The user visible version string (e.g. \"1.0.3\"). In the unlikely event that the display version can not be retrieved, it is set to \"inaccessible\".",
|
||||
"type": "STRING",
|
||||
"name": "app_display_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The architecture of the device, (e.g. \"arm\", \"x86\").",
|
||||
"type": "STRING",
|
||||
"name": "architecture",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "A UUID uniquely identifying the client.",
|
||||
"type": "STRING",
|
||||
"name": "client_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The manufacturer of the device the application is running on. Not set if the device manufacturer can't be determined (e.g. on Desktop).",
|
||||
"type": "STRING",
|
||||
"name": "device_manufacturer",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The model of the device the application is running on. On Android, this is Build.MODEL, the user-visible marketing name, like \"Pixel 2 XL\". Not set if the device model can't be determined (e.g. on Desktop).",
|
||||
"type": "STRING",
|
||||
"name": "device_model",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The date of the first run of the application.",
|
||||
"type": "STRING",
|
||||
"name": "first_run_date",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of the operating system. Possible values: Android, iOS, Linux, Darwin, Windows, FreeBSD, NetBSD, OpenBSD, Solaris, unknown",
|
||||
"type": "STRING",
|
||||
"name": "os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The user-visible version of the operating system (e.g. \"1.2.3\"). If the version detection fails, this metric gets set to `Unknown`.",
|
||||
"type": "STRING",
|
||||
"name": "os_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The version of the Glean SDK",
|
||||
"type": "STRING",
|
||||
"name": "telemetry_sdk_build",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The locale of the application during initialization (e.g. \"es-ES\"). If the locale can't be determined on the system, the value is [\"und\"](https://unicode.org/reports/tr35/#Unknown_or_Invalid_Identifiers), to indicate \"undetermined\".",
|
||||
"type": "STRING",
|
||||
"name": "locale",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "client_info",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The document ID specified in the URI when the client sent this message",
|
||||
"type": "STRING",
|
||||
"name": "document_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "category",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "extra",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "timestamp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "events",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "city",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "An ISO 3166-1 alpha-2 country code",
|
||||
"type": "STRING",
|
||||
"name": "country",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The specific geo database version used for this lookup",
|
||||
"type": "STRING",
|
||||
"name": "db_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "First major country subdivision, typically a state, province, or county",
|
||||
"type": "STRING",
|
||||
"name": "subdivision1",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Second major country subdivision; not applicable for most countries",
|
||||
"type": "STRING",
|
||||
"name": "subdivision2",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Results of a geographic lookup based on the client's IP address",
|
||||
"type": "RECORD",
|
||||
"name": "geo",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "Date HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "date",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "DNT (Do Not Track) HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "dnt",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-Debug-Id HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_debug_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-PingSender-Version HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_pingsender_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-Source-Tags HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_source_tags",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Headers included in the client's HTTP request",
|
||||
"type": "RECORD",
|
||||
"name": "header",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "browser",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "version",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Parsed components of the client's user agent string",
|
||||
"type": "RECORD",
|
||||
"name": "user_agent",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The specific geo ISP database version used for this lookup",
|
||||
"type": "STRING",
|
||||
"name": "db_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of the ISP associated with the client's IP address",
|
||||
"type": "STRING",
|
||||
"name": "name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of a specific business entity associated with the client's IP address when available; otherwise the ISP name",
|
||||
"type": "STRING",
|
||||
"name": "organization",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Results of ISP lookup based on the client's IP address",
|
||||
"type": "RECORD",
|
||||
"name": "isp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "metadata",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The locale of the application during initialization (e.g. \"es-ES\").\nIf the locale can't be determined on the system, the value is\n[\"und\"](https://unicode.org/reports/tr35/#Unknown_or_Invalid_Identifiers),\nto indicate \"undetermined\".\n",
|
||||
"type": "STRING",
|
||||
"name": "glean_baseline_locale",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "string",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "time_unit",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "The duration of the last foreground session.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_baseline_duration",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "timespan",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "A counter of URIs visited by the user in the current session, including\npage reloads. This does not include background page requests and URIs from\nembedded pages or private browsing but may be incremented without user\ninteraction by website scripts that programmatically redirect to a new\nlocation.\n",
|
||||
"type": "INTEGER",
|
||||
"name": "events_total_uri_count",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The number of metrics pings sent during the lifetime of this baseline ping.",
|
||||
"type": "INTEGER",
|
||||
"name": "glean_validation_metrics_ping_count",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "counter",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set with an invalid label.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_label",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set to an invalid value.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_value",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "The labels for this counter are `<search-engine-name>.<source>`.\n\nIf the search engine is bundled with Fenix `search-engine-name` will be\nthe name of the search engine. If it's a custom search engine (defined:\nhttps://github.com/mozilla-mobile/fenix/issues/1607) the value will be\n`custom`.\n\n`source` will be: `action`, `suggestion`, `widget`, `shortcut`, `topsite`\n(depending on the source from which the search started). Also added the\n`other` option for the source but it should never enter on this case.\n",
|
||||
"type": "RECORD",
|
||||
"name": "metrics_search_count",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a timing metric was used incorrectly.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_state",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set a value that overflowed.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_overflow",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records clicks of adverts on SERP pages.\nThe key format is ‘<provider-name>’.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_ad_clicks",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records the type of interaction a user has on SERP pages.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_in_content",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records counts of SERP pages with adverts displayed.\nThe key format is ‘<provider-name>’.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_with_ads",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "A count of the pings submitted, by ping type.\n\nThis metric appears in both the metrics and baseline pings.\n\n- On the metrics ping, the counts include the number of pings sent since\n the last metrics ping (including the last metrics ping)\n- On the baseline ping, the counts include the number of pings send since\n the last baseline ping (including the last baseline ping)\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_validation_pings_submitted",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "labeled_counter",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "jwe",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The hour of the first run of the application.\n",
|
||||
"type": "STRING",
|
||||
"name": "glean_validation_first_run_hour",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "datetime",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "denominator",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "numerator",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "labeled_rate",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "metrics",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized app name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_app_name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized channel name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_channel",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "An ISO 3166-1 alpha-2 country code",
|
||||
"type": "STRING",
|
||||
"name": "normalized_country_code",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized OS name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "normalized_os_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "end_time",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "branch",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "type",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "extra",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "experiments",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "ping_type",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "seq",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "start_time",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "reason",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "ping_info",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Hashed version of client_id (if present) useful for partitioning; ranges from 0 to 99",
|
||||
"type": "INTEGER",
|
||||
"name": "sample_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Time when the ingestion edge server accepted this message",
|
||||
"type": "TIMESTAMP",
|
||||
"name": "submission_timestamp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
]
|
|
@ -0,0 +1,9 @@
|
|||
---
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: new-instance
|
||||
submission_timestamp: 2021-03-02T00:00:00
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: existing-instance-with-new-ping
|
||||
submission_timestamp: 2021-03-02T00:00:00
|
|
@ -0,0 +1,4 @@
|
|||
---
|
||||
- name: submission_date
|
||||
type: DATE
|
||||
value: 2021-03-02
|
|
@ -0,0 +1,9 @@
|
|||
---
|
||||
- client_id: single-instance
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
||||
- client_id: multiple-instances
|
||||
submission_date: 2021-03-01
|
||||
first_seen_date: 2021-03-01
|
||||
sample_id: 0
|
|
@ -0,0 +1,682 @@
|
|||
[
|
||||
{
|
||||
"description": "A JSON string containing any payload properties not present in the schema",
|
||||
"type": "STRING",
|
||||
"name": "additional_properties",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The optional Android specific SDK version of the software running on this hardware device.",
|
||||
"type": "STRING",
|
||||
"name": "android_sdk_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The build identifier generated by the CI system (e.g. \"1234/A\"). For language bindings that provide automatic detection for this value, (e.g. Android/Kotlin), in the unlikely event that the build identifier can not be retrieved from the OS, it is set to \"inaccessible\". For other language bindings, if the value was not provided through configuration, this metric gets set to `Unknown`.",
|
||||
"type": "STRING",
|
||||
"name": "app_build",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The channel the application is being distributed on.",
|
||||
"type": "STRING",
|
||||
"name": "app_channel",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The user visible version string (e.g. \"1.0.3\"). In the unlikely event that the display version can not be retrieved, it is set to \"inaccessible\".",
|
||||
"type": "STRING",
|
||||
"name": "app_display_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The architecture of the device, (e.g. \"arm\", \"x86\").",
|
||||
"type": "STRING",
|
||||
"name": "architecture",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "A UUID uniquely identifying the client.",
|
||||
"type": "STRING",
|
||||
"name": "client_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The manufacturer of the device the application is running on. Not set if the device manufacturer can't be determined (e.g. on Desktop).",
|
||||
"type": "STRING",
|
||||
"name": "device_manufacturer",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The model of the device the application is running on. On Android, this is Build.MODEL, the user-visible marketing name, like \"Pixel 2 XL\". Not set if the device model can't be determined (e.g. on Desktop).",
|
||||
"type": "STRING",
|
||||
"name": "device_model",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The date of the first run of the application.",
|
||||
"type": "STRING",
|
||||
"name": "first_run_date",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of the operating system. Possible values: Android, iOS, Linux, Darwin, Windows, FreeBSD, NetBSD, OpenBSD, Solaris, unknown",
|
||||
"type": "STRING",
|
||||
"name": "os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The user-visible version of the operating system (e.g. \"1.2.3\"). If the version detection fails, this metric gets set to `Unknown`.",
|
||||
"type": "STRING",
|
||||
"name": "os_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The version of the Glean SDK",
|
||||
"type": "STRING",
|
||||
"name": "telemetry_sdk_build",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The locale of the application during initialization (e.g. \"es-ES\"). If the locale can't be determined on the system, the value is [\"und\"](https://unicode.org/reports/tr35/#Unknown_or_Invalid_Identifiers), to indicate \"undetermined\".",
|
||||
"type": "STRING",
|
||||
"name": "locale",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "client_info",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The document ID specified in the URI when the client sent this message",
|
||||
"type": "STRING",
|
||||
"name": "document_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "category",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "extra",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "timestamp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "events",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "city",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "An ISO 3166-1 alpha-2 country code",
|
||||
"type": "STRING",
|
||||
"name": "country",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The specific geo database version used for this lookup",
|
||||
"type": "STRING",
|
||||
"name": "db_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "First major country subdivision, typically a state, province, or county",
|
||||
"type": "STRING",
|
||||
"name": "subdivision1",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Second major country subdivision; not applicable for most countries",
|
||||
"type": "STRING",
|
||||
"name": "subdivision2",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Results of a geographic lookup based on the client's IP address",
|
||||
"type": "RECORD",
|
||||
"name": "geo",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "Date HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "date",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "DNT (Do Not Track) HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "dnt",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-Debug-Id HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_debug_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-PingSender-Version HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_pingsender_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "X-Source-Tags HTTP header",
|
||||
"type": "STRING",
|
||||
"name": "x_source_tags",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Headers included in the client's HTTP request",
|
||||
"type": "RECORD",
|
||||
"name": "header",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "browser",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "version",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Parsed components of the client's user agent string",
|
||||
"type": "RECORD",
|
||||
"name": "user_agent",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The specific geo ISP database version used for this lookup",
|
||||
"type": "STRING",
|
||||
"name": "db_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of the ISP associated with the client's IP address",
|
||||
"type": "STRING",
|
||||
"name": "name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The name of a specific business entity associated with the client's IP address when available; otherwise the ISP name",
|
||||
"type": "STRING",
|
||||
"name": "organization",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Results of ISP lookup based on the client's IP address",
|
||||
"type": "RECORD",
|
||||
"name": "isp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "metadata",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The locale of the application during initialization (e.g. \"es-ES\").\nIf the locale can't be determined on the system, the value is\n[\"und\"](https://unicode.org/reports/tr35/#Unknown_or_Invalid_Identifiers),\nto indicate \"undetermined\".\n",
|
||||
"type": "STRING",
|
||||
"name": "glean_baseline_locale",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "string",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "time_unit",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "The duration of the last foreground session.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_baseline_duration",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "timespan",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "A counter of URIs visited by the user in the current session, including\npage reloads. This does not include background page requests and URIs from\nembedded pages or private browsing but may be incremented without user\ninteraction by website scripts that programmatically redirect to a new\nlocation.\n",
|
||||
"type": "INTEGER",
|
||||
"name": "events_total_uri_count",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "The number of metrics pings sent during the lifetime of this baseline ping.",
|
||||
"type": "INTEGER",
|
||||
"name": "glean_validation_metrics_ping_count",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "counter",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set with an invalid label.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_label",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set to an invalid value.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_value",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "The labels for this counter are `<search-engine-name>.<source>`.\n\nIf the search engine is bundled with Fenix `search-engine-name` will be\nthe name of the search engine. If it's a custom search engine (defined:\nhttps://github.com/mozilla-mobile/fenix/issues/1607) the value will be\n`custom`.\n\n`source` will be: `action`, `suggestion`, `widget`, `shortcut`, `topsite`\n(depending on the source from which the search started). Also added the\n`other` option for the source but it should never enter on this case.\n",
|
||||
"type": "RECORD",
|
||||
"name": "metrics_search_count",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a timing metric was used incorrectly.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_state",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Counts the number of times a metric was set a value that overflowed.\nThe labels are the `category.name` identifier of the metric.\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_error_invalid_overflow",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records clicks of adverts on SERP pages.\nThe key format is ‘<provider-name>’.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_ad_clicks",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records the type of interaction a user has on SERP pages.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_in_content",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "Records counts of SERP pages with adverts displayed.\nThe key format is ‘<provider-name>’.\n",
|
||||
"type": "RECORD",
|
||||
"name": "browser_search_with_ads",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"description": "A count of the pings submitted, by ping type.\n\nThis metric appears in both the metrics and baseline pings.\n\n- On the metrics ping, the counts include the number of pings sent since\n the last metrics ping (including the last metrics ping)\n- On the baseline ping, the counts include the number of pings send since\n the last baseline ping (including the last baseline ping)\n",
|
||||
"type": "RECORD",
|
||||
"name": "glean_validation_pings_submitted",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "labeled_counter",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "jwe",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"description": "The hour of the first run of the application.\n",
|
||||
"type": "STRING",
|
||||
"name": "glean_validation_first_run_hour",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "datetime",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "denominator",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "numerator",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "labeled_rate",
|
||||
"mode": "REPEATED"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "metrics",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized app name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_app_name",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized channel name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_channel",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "An ISO 3166-1 alpha-2 country code",
|
||||
"type": "STRING",
|
||||
"name": "normalized_country_code",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Set to \"Other\" if this message contained an unrecognized OS name",
|
||||
"type": "STRING",
|
||||
"name": "normalized_os",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "normalized_os_version",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "end_time",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "key",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "branch",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"fields": [
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "type",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "extra",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "value",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "experiments",
|
||||
"mode": "REPEATED"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "ping_type",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "INTEGER",
|
||||
"name": "seq",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "start_time",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"type": "STRING",
|
||||
"name": "reason",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
],
|
||||
"type": "RECORD",
|
||||
"name": "ping_info",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Hashed version of client_id (if present) useful for partitioning; ranges from 0 to 99",
|
||||
"type": "INTEGER",
|
||||
"name": "sample_id",
|
||||
"mode": "NULLABLE"
|
||||
},
|
||||
{
|
||||
"description": "Time when the ingestion edge server accepted this message",
|
||||
"type": "TIMESTAMP",
|
||||
"name": "submission_timestamp",
|
||||
"mode": "NULLABLE"
|
||||
}
|
||||
]
|
|
@ -0,0 +1,13 @@
|
|||
---
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: single-instance
|
||||
submission_timestamp: 2021-03-01T00:00:00
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: multiple-instances
|
||||
submission_timestamp: 2021-03-01T00:00:00
|
||||
- sample_id: 0
|
||||
client_info:
|
||||
client_id: multiple-instances
|
||||
submission_timestamp: 2021-03-02T00:00:00
|
Загрузка…
Ссылка в новой задаче