def main(): project = 'chromeperf' options = PipelineOptions() options.view_as(DebugOptions).add_experiment('use_beam_bq_sink') options.view_as(GoogleCloudOptions).project = project bq_export_options = options.view_as(BqExportOptions) p = beam.Pipeline(options=options) entities_read = Metrics.counter('main', 'entities_read') failed_entity_transforms = Metrics.counter('main', 'failed_entity_transforms') # Read 'Anomaly' entities from datastore. entities = ( p | 'ReadFromDatastore(Anomaly)' >> ReadTimestampRangeFromDatastore( {'project': project, 'kind': 'Anomaly'}, time_range_provider=bq_export_options.GetTimeRangeProvider())) def AnomalyEntityToRowDict(entity): entities_read.inc() try: # We do the iso conversion of the nullable timestamps in isolation. earliest_input_timestamp = entity.get('earliest_input_timestamp') if earliest_input_timestamp: earliest_input_timestamp = earliest_input_timestamp.isoformat() latest_input_timestamp = entity.get('latest_input_timestamp') if latest_input_timestamp: latest_input_timestamp = latest_input_timestamp.isoformat() d = { 'id': entity.key.id, # TODO: 'sheriff' # 'subscriptions' omitted; subscription_names is sufficient 'subscription_names': entity.get('subscription_names', []), 'test': TestPath(entity['test']), 'start_revision': entity['start_revision'], 'end_revision': entity['end_revision'], 'display_start': entity.get('display_start'), 'display_end': entity.get('display_end'), # TODO: 'ownership' 'statistic': entity['statistic'], 'bug_id': entity['bug_id'], 'internal_only': entity['internal_only'], 'timestamp': entity['timestamp'].isoformat(), 'segment_size_before': entity.get('segment_size_before'), 'segment_size_after': entity.get('segment_size_after'), 'median_before_anomaly': entity.get('median_before_anomaly'), 'median_after_anomaly': entity.get('median_after_anomaly'), 'std_dev_before_anomaly': entity.get('std_dev_before_anomaly'), 'window_end_revision': entity.get('window_end_revision'), 't_statistic': FloatHack(entity.get('t_statistic')), 'degrees_of_freedom': entity.get('degrees_of_freedom'), 'p_value': entity.get('p_value'), 'is_improvement': entity.get('is_improvement', False), 'recovered': entity.get('recovered', False), # TODO: 'ref_test' 'units': entity.get('units'), # TODO: 'recipe_bisects' 'pinpoint_bisects': entity.get('pinpoint_bisects', []), # These are critical to "time-to-culprit" calculations. 'earliest_input_timestamp': earliest_input_timestamp, 'latest_input_timestamp': latest_input_timestamp, } if d['statistic'] is None: # Some years-old anomalies lack this. raise UnconvertibleAnomalyError() return [d] except (KeyError, UnconvertibleAnomalyError): failed_entity_transforms.inc() return [] anomaly_dicts = ( entities | 'ConvertEntityToRow(Anomaly)' >> beam.FlatMap(AnomalyEntityToRowDict)) """ CREATE TABLE `chromeperf.chromeperf_dashboard_data.anomalies` (id INT64 NOT NULL, `timestamp` TIMESTAMP NOT NULL, subscription_names ARRAY<STRING>, `test` STRING NOT NULL, start_revision INT64 NOT NULL, end_revision INT64 NOT NULL, display_start INT64, display_end INT64, statistic STRING NOT NULL, bug_id INT64, internal_only BOOLEAN NOT NULL, segment_size_before INT64, segment_size_after INT64, median_before_anomaly FLOAT64, median_after_anomaly FLOAT64, std_dev_before_anomaly FLOAT64, window_end_revision INT64, t_statistic FLOAT64, degrees_of_freedom FLOAT64, p_value FLOAT64, is_improvement BOOLEAN NOT NULL, recovered BOOLEAN NOT NULL, units STRING, pinpoint_bisects ARRAY<STRING>, earliest_input_timestamp TIMESTAMP, latest_input_timestamp TIMESTAMP) PARTITION BY DATE(`timestamp`); """ # pylint: disable=pointless-string-statement bq_anomaly_schema = { 'fields': [ { 'name': 'id', 'type': 'INT64', 'mode': 'REQUIRED' }, { 'name': 'subscription_names', 'type': 'STRING', 'mode': 'REPEATED' }, { 'name': 'test', 'type': 'STRING', 'mode': 'REQUIRED' }, { 'name': 'start_revision', 'type': 'INT64', 'mode': 'REQUIRED' }, { 'name': 'end_revision', 'type': 'INT64', 'mode': 'REQUIRED' }, { 'name': 'display_start', 'type': 'INT64', 'mode': 'NULLABLE' }, { 'name': 'display_end', 'type': 'INT64', 'mode': 'NULLABLE' }, { 'name': 'statistic', 'type': 'STRING', 'mode': 'REQUIRED' }, { 'name': 'bug_id', 'type': 'INT64', 'mode': 'NULLABLE' }, { 'name': 'internal_only', 'type': 'BOOLEAN', 'mode': 'REQUIRED' }, { 'name': 'timestamp', 'type': 'TIMESTAMP', 'mode': 'REQUIRED' }, { 'name': 'segment_size_before', 'type': 'INT64', 'mode': 'NULLABLE' }, { 'name': 'segment_size_after', 'type': 'INT64', 'mode': 'NULLABLE' }, { 'name': 'median_before_anomaly', 'type': 'FLOAT', 'mode': 'NULLABLE' }, { 'name': 'median_after_anomaly', 'type': 'FLOAT', 'mode': 'NULLABLE' }, { 'name': 'std_dev_before_anomaly', 'type': 'FLOAT', 'mode': 'NULLABLE' }, { 'name': 'window_end_revision', 'type': 'INT64', 'mode': 'NULLABLE' }, { 'name': 't_statistic', 'type': 'FLOAT', 'mode': 'NULLABLE' }, { 'name': 'degrees_of_freedom', 'type': 'FLOAT', 'mode': 'NULLABLE' }, { 'name': 'p_value', 'type': 'FLOAT', 'mode': 'NULLABLE' }, { 'name': 'is_improvement', 'type': 'BOOLEAN', 'mode': 'REQUIRED' }, { 'name': 'recovered', 'type': 'BOOLEAN', 'mode': 'REQUIRED' }, { 'name': 'units', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'pinpoint_bisects', 'type': 'STRING', 'mode': 'REPEATED' }, { 'name': 'earliest_input_timestamp', 'type': 'TIMESTAMP', 'mode': 'NULLABLE' }, { 'name': 'latest_input_timestamp', 'type': 'TIMESTAMP', 'mode': 'NULLABLE' }, ] } # 'dataset' may be a RuntimeValueProvider, so we have to defer calculating # the table name until runtime. The simplest way to do this is by passing a # function for the table name rather than a string. def TableNameFn(unused_element): return '{}:{}.anomalies{}'.format(project, bq_export_options.dataset.get(), bq_export_options.table_suffix) _ = ( anomaly_dicts | 'WriteToBigQuery(anomalies)' >> WriteToPartitionedBigQuery(TableNameFn, bq_anomaly_schema)) result = p.run() result.wait_until_finish() PrintCounters(result)
def main(): project = 'chromeperf' options = PipelineOptions() options.view_as(DebugOptions).add_experiment('use_beam_bq_sink') options.view_as(GoogleCloudOptions).project = project bq_export_options = options.view_as(BqExportOptions) p = beam.Pipeline(options=options) entities_read = Metrics.counter('main', 'entities_read') failed_entity_transforms = Metrics.counter('main', 'failed_entity_transforms') """ CREATE TABLE `chromeperf.chromeperf_dashboard_data.rows_test` (revision INT64 NOT NULL, value FLOAT64 NOT NULL, std_error FLOAT64, `timestamp` TIMESTAMP NOT NULL, test STRING NOT NULL, master STRING, bot STRING, properties STRING) PARTITION BY DATE(`timestamp`); """ # pylint: disable=pointless-string-statement bq_row_schema = {'fields': [ {'name': 'revision', 'type': 'INT64', 'mode': 'REQUIRED'}, {'name': 'value', 'type': 'FLOAT', 'mode': 'REQUIRED'}, {'name': 'std_error', 'type': 'FLOAT', 'mode': 'NULLABLE'}, {'name': 'timestamp', 'type': 'TIMESTAMP', 'mode': 'REQUIRED'}, {'name': 'test', 'type': 'STRING', 'mode': 'REQUIRED'}, {'name': 'master', 'type': 'STRING', 'mode': 'NULLABLE'}, {'name': 'bot', 'type': 'STRING', 'mode': 'NULLABLE'}, {'name': 'properties', 'type': 'STRING', 'mode': 'NULLABLE'}, ]} def RowEntityToRowDict(entity): entities_read.inc() try: d = { 'revision': entity.key.id, 'value': FloatHack(entity['value']), 'std_error': FloatHack(entity.get('error')), 'timestamp': entity['timestamp'].isoformat(), 'test': entity.key.parent.name, } # Add the expando properties as a JSON-encoded dict. properties = {} for key, value in entity.items(): if key in d or key in ['parent_test', 'error']: # skip properties with dedicated columns. continue if isinstance(value, float): value = FloatHack(value) properties[key] = value d['properties'] = json.dumps(properties) if properties else None # Add columns derived from test: master, bot. test_path_parts = d['test'].split('/', 2) if len(test_path_parts) >= 3: d['master'] = test_path_parts[0] d['bot'] = test_path_parts[1] return [d] except KeyError: logging.getLogger().exception('Failed to convert Row') failed_entity_transforms.inc() return [] row_query_params = dict(project=project, kind='Row') row_entities = ( p | 'ReadFromDatastore(Row)' >> ReadTimestampRangeFromDatastore( row_query_params, time_range_provider=bq_export_options.GetTimeRangeProvider(), step=datetime.timedelta(minutes=5))) row_dicts = ( row_entities | 'ConvertEntityToRow(Row)' >> FlatMap(RowEntityToRowDict)) table_name = '{}:chromeperf_dashboard_data.rows{}'.format( project, bq_export_options.table_suffix) _ = row_dicts | 'WriteToBigQuery(rows)' >> WriteToPartitionedBigQuery( table_name, bq_row_schema) result = p.run() result.wait_until_finish() PrintCounters(result)
def main(): project = 'chromeperf' options = PipelineOptions() options.view_as(DebugOptions).add_experiment('use_beam_bq_sink') options.view_as(GoogleCloudOptions).project = project bq_export_options = options.view_as(BqExportOptions) p = beam.Pipeline(options=options) entities_read = Metrics.counter('main', 'entities_read') failed_entity_transforms = Metrics.counter('main', 'failed_entity_transforms') # Read 'Job' entities from datastore. job_entities = ( p | 'ReadFromDatastore(Job)' >> ReadTimestampRangeFromDatastore( { 'project': project, 'kind': 'Job' }, time_range_provider=bq_export_options.GetTimeRangeProvider(), timestamp_property='created')) def ConvertEntity(entity): entities_read.inc() try: row_dict = JobEntityToRowDict(entity) except UnconvertibleJobError: logging.getLogger().exception('Failed to convert Job') failed_entity_transforms.inc() return [] return [row_dict] job_dicts = (job_entities | 'ConvertEntityToRow(Job)' >> beam.FlatMap(ConvertEntity)) """ CREATE TABLE `chromeperf.chromeperf_dashboard_data.jobs` (id INT64 NOT NULL, arguments STRING NOT NULL, bug_id INT64, comparison_mode STRING, gerrit STRUCT<server STRING, change_id STRING>, name STRING, tags STRING, user_email STRING, create_time TIMESTAMP NOT NULL, start_time TIMESTAMP, update_time TIMESTAMP NOT NULL, started BOOLEAN NOT NULL, done BOOLEAN NOT NULL, cancelled BOOLEAN NOT NULL, cancel_reason STRING, task STRING, exception STRING, exception_details STRING, difference_count INT64, retry_count INT64 NOT NULL, benchmark_arguments STRUCT<benchmark STRING, story STRING, story_tags STRING, chart STRING, statistic STRING>, use_execution_engine BOOLEAN NOT NULL, completed BOOLEAN NOT NULL, failed BOOLEAN NOT NULL, running BOOLEAN NOT NULL, configuration STRING) PARTITION BY DATE(`create_time`); """ # pylint: disable=pointless-string-statement bq_job_schema = { 'fields': [ { 'name': 'id', 'type': 'INT64', 'mode': 'REQUIRED' }, { 'name': 'arguments', 'type': 'STRING', 'mode': 'REQUIRED' }, { 'name': 'bug_id', 'type': 'INT64', 'mode': 'NULLABLE' }, { 'name': 'comparison_mode', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'gerrit', 'type': 'RECORD', 'mode': 'NULLABLE', 'fields': [ { 'name': 'server', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'change_id', 'type': 'STRING', 'mode': 'NULLABLE' }, ] }, { 'name': 'name', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'tags', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'user_email', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'create_time', 'type': 'TIMESTAMP', 'mode': 'REQUIRED' }, { 'name': 'start_time', 'type': 'TIMESTAMP', 'mode': 'NULLABLE' }, { 'name': 'update_time', 'type': 'TIMESTAMP', 'mode': 'REQUIRED' }, { 'name': 'started', 'type': 'BOOLEAN', 'mode': 'REQUIRED' }, { 'name': 'done', 'type': 'BOOLEAN', 'mode': 'REQUIRED' }, { 'name': 'cancelled', 'type': 'BOOLEAN', 'mode': 'REQUIRED' }, { 'name': 'cancel_reason', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'task', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'exception', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'exception_details', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'difference_count', 'type': 'INT64', 'mode': 'NULLABLE' }, { 'name': 'retry_count', 'type': 'INT64', 'mode': 'REQUIRED' }, { 'name': 'benchmark_arguments', 'type': 'RECORD', 'mode': 'NULLABLE', 'fields': [ { 'name': 'benchmark', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'story', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'story_tags', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'chart', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'statistic', 'type': 'STRING', 'mode': 'NULLABLE' }, ] }, { 'name': 'use_execution_engine', 'type': 'BOOLEAN', 'mode': 'REQUIRED' }, { 'name': 'completed', 'type': 'BOOLEAN', 'mode': 'REQUIRED' }, { 'name': 'failed', 'type': 'BOOLEAN', 'mode': 'REQUIRED' }, { 'name': 'running', 'type': 'BOOLEAN', 'mode': 'REQUIRED' }, { 'name': 'configuration', 'type': 'STRING', 'mode': 'NULLABLE' }, ] } # 'dataset' may be a RuntimeValueProvider, so we have to defer calculating # the table name until runtime. The simplest way to do this is by passing a # function for the table name rather than a string. def TableNameFn(unused_element): return '{}:{}.jobs{}'.format(project, bq_export_options.dataset.get(), bq_export_options.table_suffix) _ = job_dicts | 'WriteToBigQuery(jobs)' >> WriteToPartitionedBigQuery( TableNameFn, bq_job_schema, element_to_yyyymmdd_fn=_JobToYYYYMMDD) result = p.run() result.wait_until_finish() PrintCounters(result)
def main(): project = 'chromeperf' options = PipelineOptions() options.view_as(DebugOptions).add_experiment('use_beam_bq_sink') options.view_as(GoogleCloudOptions).project = project bq_export_options = options.view_as(BqExportOptions) p = beam.Pipeline(options=options) entities_read = Metrics.counter('main', 'entities_read') failed_entity_transforms = Metrics.counter('main', 'failed_entity_transforms') row_conflicts = Metrics.counter('main', 'row_conflicts') multiple_histograms_for_row = Metrics.counter( 'main', 'multiple_histograms_for_row') orphaned_histogram = Metrics.counter('main', 'orphaned_histogram') """ CREATE TABLE `chromeperf.chromeperf_dashboard_rows.<MASTER>` (revision INT64 NOT NULL, value FLOAT64 NOT NULL, std_error FLOAT64, `timestamp` TIMESTAMP NOT NULL, master STRING NOT NULL, bot STRING NOT NULL, measurement STRING, test STRING NOT NULL, properties STRING, sample_values ARRAY<FLOAT64>) PARTITION BY DATE(`timestamp`) CLUSTER BY master, bot, measurement; """ # pylint: disable=pointless-string-statement bq_row_schema = { 'fields': [ { 'name': 'revision', 'type': 'INT64', 'mode': 'REQUIRED' }, { 'name': 'value', 'type': 'FLOAT', 'mode': 'REQUIRED' }, { 'name': 'std_error', 'type': 'FLOAT', 'mode': 'NULLABLE' }, { 'name': 'timestamp', 'type': 'TIMESTAMP', 'mode': 'REQUIRED' }, { 'name': 'master', 'type': 'STRING', 'mode': 'REQUIRED' }, { 'name': 'bot', 'type': 'STRING', 'mode': 'REQUIRED' }, { 'name': 'measurement', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'test', 'type': 'STRING', 'mode': 'REQUIRED' }, { 'name': 'properties', 'type': 'STRING', 'mode': 'NULLABLE' }, { 'name': 'sample_values', 'type': 'FLOAT', 'mode': 'REPEATED' }, ] } def RowEntityToRowDict(entity): entities_read.inc() try: d = { 'revision': entity.key.id, 'value': FloatHack(entity['value']), 'std_error': FloatHack(entity.get('error')), 'timestamp': entity['timestamp'].isoformat(), 'test': entity.key.parent.name, } # Add the expando properties as a JSON-encoded dict. properties = {} for key, value in entity.items(): if key in d or key in ['parent_test', 'error']: # skip properties with dedicated columns. continue if isinstance(value, float): value = FloatHack(value) properties[key] = value d['properties'] = json.dumps(properties) if properties else None # Add columns derived from test: master, bot. test_path_parts = d['test'].split('/', 2) if len(test_path_parts) >= 3: d['master'] = test_path_parts[0] d['bot'] = test_path_parts[1] d['measurement'] = '/'.join(test_path_parts[2:]) return [d] except KeyError: logging.getLogger().exception('Failed to convert Row') failed_entity_transforms.inc() return [] row_query_params = dict(project=project, kind='Row') row_entities = ( p | 'ReadFromDatastore(Row)' >> ReadTimestampRangeFromDatastore( row_query_params, time_range_provider=bq_export_options.GetTimeRangeProvider(), step=datetime.timedelta(minutes=5))) row_dicts = (row_entities | 'ConvertEntityToDict(Row)' >> FlatMap(RowEntityToRowDict)) # The sample_values are not found in the Row entity. So we have to fetch all # the corresponding Histogram entities and join them with our collection of # Rows (by using test + revision as the join key). We also need to unpack the # sample values arrays out of the zlib-compressed JSON stored in the # Histogram's "data" property. def HistogramEntityToDict(entity): """Returns dicts with keys: 'test', 'revision', 'sample_values'.""" entities_read.inc() try: data = entity['data'] except KeyError: logging.getLogger().exception('Histogram missing "data" field') failed_entity_transforms.inc() return [] try: json_str = zlib.decompress(data) except zlib.error: logging.getLogger().exception('Histogram data not valid zlib: %r', data) failed_entity_transforms.inc() return [] try: data_dict = json.loads(json_str) except json.JSONDecodeError: logging.getLogger().exception('Histogram data not valid json.') failed_entity_transforms.inc() return [] sample_values = data_dict.get('sampleValues', []) if not isinstance(sample_values, list): logging.getLogger().exception( 'Histogram data.sampleValues not valid list.') failed_entity_transforms.inc() return [] count = len(sample_values) sample_values = [v for v in sample_values if v is not None] if len(sample_values) != count: logging.getLogger().warn( 'Histogram data.sampleValues contains null: %r', entity.key) for v in sample_values: if not isinstance(v, (int, float)): logging.getLogger().exception( 'Histogram data.sampleValues contains non-numeric: %r', v) failed_entity_transforms.inc() return [] try: return [{ 'test': entity['test'].name, 'revision': entity['revision'], 'sample_values': sample_values, }] except KeyError: logging.getLogger().exception( 'Histogram missing test or revision field/s') failed_entity_transforms.inc() return [] histogram_query_params = dict(project=project, kind='Histogram') histogram_entities = ( p | 'ReadFromDatastore(Histogram)' >> ReadTimestampRangeFromDatastore( histogram_query_params, time_range_provider=bq_export_options.GetTimeRangeProvider(), step=datetime.timedelta(minutes=5))) histogram_dicts = ( histogram_entities | 'ConvertEntityToDict(Histogram)' >> FlatMap(HistogramEntityToDict)) def TestRevision(element): return (element['test'], element['revision']) rows_with_key = (row_dicts | 'WithKeys(Row)' >> beam.WithKeys(TestRevision)) histograms_with_key = ( histogram_dicts | 'WithKeys(Histogram)' >> beam.WithKeys(TestRevision)) def MergeRowAndSampleValues(element): group_key, join_values = element rows, histograms = join_values if len(rows) == 0: orphaned_histogram.inc() logging.getLogger().error("No Row for Histogram(s) (%r)", group_key) return [] elif len(rows) > 1: row_conflicts.inc() logging.getLogger().error("Multiple rows (%d) for %r", len(rows), group_key) return rows row = rows[0] if len(histograms) > 1: # We'll merge these, so this isn't an error. multiple_histograms_for_row.inc() elif len(histograms) == 0: # No sample values to annotate the row with. This is common. return [row] # Merge multiple histogram's values into a single row. row['sample_values'] = list( itertools.chain.from_iterable(h['sample_values'] for h in histograms)) return [row] joined_and_annotated = ((rows_with_key, histograms_with_key) | beam.CoGroupByKey() | beam.FlatMap(MergeRowAndSampleValues)) def TableNameFn(unused_element): return '{project}:{dataset}.rows{suffix}'.format( project=project, dataset=bq_export_options.dataset.get(), suffix=bq_export_options.table_suffix) _ = (joined_and_annotated | 'WriteToBigQuery(rows)' >> WriteToPartitionedBigQuery( TableNameFn, bq_row_schema, additional_bq_parameters={ 'clustering': { 'fields': ['master', 'bot', 'measurement'] } })) result = p.run() result.wait_until_finish() PrintCounters(result)