[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:
Anthony Miyaguchi 2021-04-05 11:36:39 -07:00 коммит произвёл GitHub
Родитель 420a1bfdba
Коммит 1503a7fa89
Не найден ключ, соответствующий данной подписи
Идентификатор ключа GPG: 4AEE18F83AFDEB23
44 изменённых файлов: 5995 добавлений и 8 удалений

Просмотреть файл

@ -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