def test(self): if self.get_option_or_default('use_stateful_load_generator', False): source = ( self.pipeline | 'LoadGenerator' >> StatefulLoadGenerator(self.input_options) | beam.ParDo(AssignTimestamps()) | beam.WindowInto(window.FixedWindows(20))) else: source = ( self.pipeline | 'Read synthetic' >> beam.io.Read( SyntheticSource(self.parse_synthetic_source_options()))) pc = (source | 'Measure time: Start' >> beam.ParDo( MeasureTime(self.metrics_namespace))) for branch in range(self.fanout): ( # pylint: disable=expression-not-assigned pc | 'Combine with Top %i' % branch >> beam.CombineGlobally( beam.combiners.TopCombineFn( self.top_count)).without_defaults() | 'Consume %i' % branch >> beam.ParDo(self._GetElement()) | 'Measure time: End %i' % branch >> beam.ParDo( MeasureTime(self.metrics_namespace)))
def testCoGroupByKey(self): pc1 = (self.pipeline | 'Read ' + INPUT_TAG >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions(self.input_options))) | 'Make ' + INPUT_TAG + ' iterable' >> beam.Map(lambda x: (x, x)) | 'Measure time: Start pc1' >> beam.ParDo( MeasureTime(self.metrics_namespace))) pc2 = ( self.pipeline | 'Read ' + CO_INPUT_TAG >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions(self.co_input_options))) | 'Make ' + CO_INPUT_TAG + ' iterable' >> beam.Map(lambda x: (x, x)) | 'Measure time: Start pc2' >> beam.ParDo( MeasureTime(self.metrics_namespace))) # pylint: disable=expression-not-assigned ({ INPUT_TAG: pc1, CO_INPUT_TAG: pc2 } | 'CoGroupByKey: ' >> beam.CoGroupByKey() | 'Consume Joined Collections' >> beam.ParDo(self._Ungroup()) | 'Measure time: End' >> beam.ParDo( MeasureTime(self.metrics_namespace))) result = self.pipeline.run() result.wait_until_finish() if self.metrics_monitor is not None: self.metrics_monitor.send_metrics(result)
def testSideInput(self): def join_fn(element, side_input, iterations): list = [] for i in range(iterations): for key, value in side_input: if i == iterations - 1: list.append({key: element[1] + value}) yield list with self.pipeline as p: main_input = (p | "Read pcoll 1" >> beam.io.Read( synthetic_pipeline.SyntheticSource( self._parseTestPipelineOptions())) | 'Measure time: Start pcoll 1' >> beam.ParDo( MeasureTime(self.metrics_namespace))) side_input = ( p | "Read pcoll 2" >> beam.io.Read( synthetic_pipeline.SyntheticSource(self._getSideInput())) | 'Measure time: Start pcoll 2' >> beam.ParDo( MeasureTime(self.metrics_namespace))) # pylint: disable=expression-not-assigned (main_input | "Merge" >> beam.ParDo(join_fn, AsIter(side_input), self.iterations) | 'Measure time' >> beam.ParDo(MeasureTime(self.metrics_namespace))) result = p.run() result.wait_until_finish() if self.metrics_monitor is not None: self.metrics_monitor.send_metrics(result)
def test(self): pc1 = (self.pipeline | 'Read ' + self.INPUT_TAG >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parse_synthetic_source_options())) | 'Measure time: Start pc1' >> beam.ParDo( MeasureTime(self.metrics_namespace))) pc2 = (self.pipeline | 'Read ' + self.CO_INPUT_TAG >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parse_synthetic_source_options( self.co_input_options))) | 'Measure time: Start pc2' >> beam.ParDo( MeasureTime(self.metrics_namespace))) # pylint: disable=expression-not-assigned ({ self.INPUT_TAG: pc1, self.CO_INPUT_TAG: pc2 } | 'CoGroupByKey ' >> beam.CoGroupByKey() | 'Consume Joined Collections' >> beam.ParDo( self._UngroupAndReiterate(self.INPUT_TAG, self.CO_INPUT_TAG), self.iterations) | 'Measure time: End' >> beam.ParDo( MeasureTime(self.metrics_namespace)))
def testParDo(self): class CounterOperation(beam.DoFn): def __init__(self, number_of_counters, number_of_operations): self.number_of_operations = number_of_operations self.counters = [] for i in range(number_of_counters): self.counters.append(Metrics.counter('do-not-publish', 'name-{}'.format(i))) def process(self, element): for _ in range(self.number_of_operations): for counter in self.counters: counter.inc() yield element pc = (self.pipeline | 'Read synthetic' >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions() )) | 'Measure time: Start' >> beam.ParDo( MeasureTime(self.metrics_namespace)) ) for i in range(self.iterations): pc = (pc | 'Step: %d' % i >> beam.ParDo( CounterOperation(self.number_of_counters, self.number_of_operations)) ) # pylint: disable=expression-not-assigned (pc | 'Measure time: End' >> beam.ParDo(MeasureTime(self.metrics_namespace)) )
def test(self): def join_fn(element, side_input, iterations): result = [] for i in range(iterations): for key, value in side_input: if i == iterations - 1: result.append({key: element[1] + value}) yield result main_input = ( self.pipeline | "Read pcoll 1" >> beam.io.Read( SyntheticSource(self.parse_synthetic_source_options())) | 'Measure time: Start pcoll 1' >> beam.ParDo( MeasureTime(self.metrics_namespace))) side_input = ( self.pipeline | "Read pcoll 2" >> beam.io.Read( SyntheticSource(self.parse_synthetic_source_options())) | 'Measure time: Start pcoll 2' >> beam.ParDo( MeasureTime(self.metrics_namespace))) # pylint: disable=expression-not-assigned ( main_input | "Merge" >> beam.ParDo(join_fn, AsIter(side_input), self.iterations) | 'Measure time' >> beam.ParDo(MeasureTime(self.metrics_namespace)))
def testCoGroupByKey(self): pc1 = (self.pipeline | 'Read ' + INPUT_TAG >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions(self.input_options))) | 'Make ' + INPUT_TAG + ' iterable' >> beam.Map(lambda x: (x, x)) | 'Measure time: Start pc1' >> beam.ParDo( MeasureTime(self.metrics_namespace)) ) pc2 = (self.pipeline | 'Read ' + CO_INPUT_TAG >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions(self.co_input_options))) | 'Make ' + CO_INPUT_TAG + ' iterable' >> beam.Map( lambda x: (x, x)) | 'Measure time: Start pc2' >> beam.ParDo( MeasureTime(self.metrics_namespace)) ) # pylint: disable=expression-not-assigned ({INPUT_TAG: pc1, CO_INPUT_TAG: pc2} | 'CoGroupByKey ' >> beam.CoGroupByKey() | 'Consume Joined Collections' >> beam.ParDo(self._UngroupAndReiterate(), self.iterations) | 'Measure time: End' >> beam.ParDo(MeasureTime(self.metrics_namespace)) )
def testParDo(self): class _GetElement(beam.DoFn): from apache_beam.testing.load_tests.load_test_metrics_utils import count_bytes @count_bytes def process(self, element, namespace, is_returning): if is_returning: yield element if not self.iterations: num_runs = 1 else: num_runs = int(self.iterations) pc = (self.pipeline | 'Read synthetic' >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions())) | 'Measure time: Start' >> beam.ParDo( MeasureTime(self.metrics_namespace))) for i in range(num_runs): is_returning = (i == (num_runs - 1)) pc = (pc | 'Step: %d' % i >> beam.ParDo( _GetElement(), self.metrics_namespace, is_returning)) if self.output: pc = (pc | "Write" >> beam.io.WriteToText(self.output)) # pylint: disable=expression-not-assigned (pc | 'Measure time: End' >> beam.ParDo( MeasureTime(self.metrics_namespace)))
def compute_stats( input_handle, stats_path, max_rows=None, for_eval=False, pipeline_args=None, publish_to_bq=None, metrics_dataset=None, metrics_table=None, project=None): """Computes statistics on the input data. Args: input_handle: BigQuery table name to process specified as DATASET.TABLE or path to csv file with input data. stats_path: Directory in which stats are materialized. max_rows: Number of rows to query from BigQuery for_eval: Query for eval set rows from BigQuery pipeline_args: additional DataflowRunner or DirectRunner args passed to the beam pipeline. """ namespace = metrics_table pipeline = beam.Pipeline(argv=pipeline_args) metrics_monitor = None if publish_to_bq: metrics_monitor = MetricsReader( publish_to_bq=publish_to_bq, project_name=project, bq_table=metrics_table, bq_dataset=metrics_dataset, namespace=namespace, filters=MetricsFilter().with_namespace(namespace), ) query = taxi.make_sql( table_name=input_handle, max_rows=max_rows, for_eval=for_eval) raw_data = ( pipeline | 'ReadBigQuery' >> ReadFromBigQuery( query=query, project=project, use_standard_sql=True) | 'Measure time: Start' >> beam.ParDo(MeasureTime(namespace)) | 'ConvertToTFDVInput' >> beam.Map( lambda x: {key: np.asarray([x[key]]) for key in x if x[key] is not None})) _ = ( raw_data | 'GenerateStatistics' >> tfdv.GenerateStatistics() | 'Measure time: End' >> beam.ParDo(MeasureTime(namespace)) | 'WriteStatsOutput' >> beam.io.WriteToTFRecord( stats_path, shard_name_template='', coder=beam.coders.ProtoCoder( statistics_pb2.DatasetFeatureStatisticsList))) result = pipeline.run() result.wait_until_finish() if metrics_monitor: metrics_monitor.publish_metrics(result)
def run(argv=None): class MessageParser(beam.DoFn): # It is required to parse messages for GBK operation. # Otherwise there are encoding problems. def process(self, item): if item.attributes: k, v = item.attributes.popitem() yield (str(k), str(v)) class ParserToBytes(beam.DoFn): # Parsing to bytes is required for saving in PubSub. def process(self, item): _, v = item yield bytes(v, encoding='utf8') parser = argparse.ArgumentParser() parser.add_argument('--output_topic', required=True, help=('Output PubSub topic of the form ' '"projects/<PROJECT>/topic/<TOPIC>".')) parser.add_argument( '--input_subscription', help=('Input PubSub subscription of the form ' '"projects/<PROJECT>/subscriptions/<SUBSCRIPTION>."')) parser.add_argument('--metrics_namespace', help=('Namespace of metrics ' '"string".')) known_args, pipeline_args = parser.parse_known_args(argv) pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = True pipeline_options.view_as(StandardOptions).streaming = True p = beam.Pipeline(options=pipeline_options) # pylint: disable=expression-not-assigned (p | ReadFromPubSub(subscription=known_args.input_subscription, with_attributes=True) | 'Window' >> beam.WindowInto(window.FixedWindows(1000, 0)) | 'Measure time: Start' >> beam.ParDo( MeasureTime(known_args.metrics_namespace)) | 'Count messages' >> beam.ParDo( CountMessages(known_args.metrics_namespace)) | 'Parse' >> beam.ParDo(MessageParser()) | 'GroupByKey' >> beam.GroupByKey() | 'Ungroup' >> beam.FlatMap(lambda elm: [(elm[0], v) for v in elm[1]]) | 'Measure time: End' >> beam.ParDo( MeasureTime(known_args.metrics_namespace)) | 'Parse to bytes' >> beam.ParDo(ParserToBytes()) | 'Write' >> beam.io.WriteToPubSub(topic=known_args.output_topic)) result = p.run() result.wait_until_finish() logging.error(result) return result
def test(self): class CounterOperation(beam.DoFn): def __init__(self, number_of_counters, number_of_operations): self.number_of_operations = number_of_operations self.counters = [] for i in range(number_of_counters): self.counters.append( Metrics.counter('do-not-publish', 'name-{}'.format(i))) state_param = beam.DoFn.StateParam( userstate.CombiningValueStateSpec( 'count', beam.coders.IterableCoder(beam.coders.VarIntCoder()), sum)) if self.stateful else None def process(self, element, state=state_param): for _ in range(self.number_of_operations): for counter in self.counters: counter.inc() if state: state.add(1) yield element if self.get_option_or_default('streaming', False): source = ( self.pipeline | 'LoadGenerator' >> StatefulLoadGenerator(self.input_options)) else: source = ( self.pipeline | 'Read synthetic' >> beam.io.Read( SyntheticSource(self.parse_synthetic_source_options()))) pc = ( source | 'Measure time: Start' >> beam.ParDo( MeasureTime(self.metrics_namespace)) | 'Assign timestamps' >> beam.ParDo(AssignTimestamps())) for i in range(self.iterations): pc = ( pc | 'Step: %d' % i >> beam.ParDo( CounterOperation( self.number_of_counters, self.number_of_operations))) # pylint: disable=expression-not-assigned ( pc | 'Measure latency' >> beam.ParDo(MeasureLatency(self.metrics_namespace)) | 'Measure time: End' >> beam.ParDo(MeasureTime(self.metrics_namespace)))
def test(self): pc = (self.pipeline | beam.io.Read( SyntheticSource(self.parse_synthetic_source_options())) | 'Measure time: Start' >> beam.ParDo( MeasureTime(self.metrics_namespace))) for branch in range(self.fanout): ( # pylint: disable=expression-not-assigned pc | 'GroupByKey %i' % branch >> beam.GroupByKey() | 'Ungroup %i' % branch >> beam.ParDo( self._UngroupAndReiterate(), self.iterations) | 'Measure time: End %i' % branch >> beam.ParDo( MeasureTime(self.metrics_namespace)))
def test(self): input = (self.pipeline | beam.io.Read( SyntheticSource(self.parse_synthetic_source_options())) | 'Measure time: Start' >> beam.ParDo( MeasureTime(self.metrics_namespace))) for branch in range(self.fanout): ( # pylint: disable=expression-not-assigned input | 'Combine with Top %i' % branch >> beam.CombineGlobally( beam.combiners.TopCombineFn(self.top_count)) | 'Consume %i' % branch >> beam.ParDo(self._GetElement()) | 'Measure time: End %i' % branch >> beam.ParDo( MeasureTime(self.metrics_namespace)))
def testCoGroupByKey(self): with self.pipeline as p: pc1 = (p | 'Read ' + INPUT_TAG >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions(self.inputOptions))) | 'Make ' + INPUT_TAG + ' iterable' >> beam.Map(lambda x: (x, x)) ) pc2 = (p | 'Read ' + CO_INPUT_TAG >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions(self.coInputOptions))) | 'Make ' + CO_INPUT_TAG + ' iterable' >> beam.Map( lambda x: (x, x)) ) # pylint: disable=expression-not-assigned ({INPUT_TAG: pc1, CO_INPUT_TAG: pc2} | 'CoGroupByKey: ' >> beam.CoGroupByKey() | 'Consume Joined Collections' >> beam.ParDo(self._Ungroup()) | 'Measure time' >> beam.ParDo(MeasureTime()) ) result = p.run() result.wait_until_finish() metrics = result.metrics().query() for dist in metrics['distributions']: logging.info("Distribution: %s", dist)
def test(self): def format_record(record): import base64 return base64.b64encode(record[1]) def make_insert_mutations(element): import uuid # pylint: disable=reimported from apache_beam.io.gcp.experimental.spannerio import WriteMutation ins_mutation = WriteMutation.insert(table='test', columns=('id', 'data'), values=[(str(uuid.uuid1()), element)]) return [ins_mutation] ( # pylint: disable=expression-not-assigned self.pipeline | 'Produce rows' >> Read( SyntheticSource(self.parse_synthetic_source_options())) | 'Count messages' >> ParDo(CountMessages(self.metrics_namespace)) | 'Format' >> Map(format_record) | 'Make mutations' >> FlatMap(make_insert_mutations) | 'Measure time' >> ParDo(MeasureTime(self.metrics_namespace)) | 'Write to Spanner' >> WriteToSpanner( project_id=self.project, instance_id=self.spanner_instance, database_id=self.TEST_DATABASE, max_batch_size_bytes=5120))
def testParDo(self): if self.iterations is None: num_runs = 1 else: num_runs = int(self.iterations) with self.pipeline as p: pc = (p | 'Read synthetic' >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions())) | 'Measure time' >> beam.ParDo(MeasureTime())) for i in range(num_runs): label = 'Step: %d' % i pc = (pc | label >> beam.ParDo(self._GetElement())) if self.output is not None: # pylint: disable=expression-not-assigned (pc | "Write" >> beam.io.WriteToText(self.output)) result = p.run() result.wait_until_finish() metrics = result.metrics().query() for counter in metrics['counters']: logging.info("Counter: %s", counter) for dist in metrics['distributions']: logging.info("Distribution: %s", dist)
def testParDo(self): class _GetElement(beam.DoFn): from apache_beam.testing.load_tests.load_test_metrics_utils import count_bytes @count_bytes(COUNTER_LABEL) def process(self, element, namespace, is_returning): if is_returning: yield element if self.iterations is None: num_runs = 1 else: num_runs = int(self.iterations) with self.pipeline as p: pc = (p | 'Read synthetic' >> beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions() )) | 'Measure time: Start' >> beam.ParDo( MeasureTime(self.metrics_namespace)) ) for i in range(num_runs): is_returning = (i == (num_runs-1)) pc = (pc | 'Step: %d' % i >> beam.ParDo( _GetElement(), self.metrics_namespace, is_returning) ) if self.output is not None: pc = (pc | "Write" >> beam.io.WriteToText(self.output) ) # pylint: disable=expression-not-assigned (pc | 'Measure time: End' >> beam.ParDo(MeasureTime(self.metrics_namespace)) ) result = p.run() result.wait_until_finish() if self.metrics_monitor is not None: self.metrics_monitor.send_metrics(result)
def testGroupByKey(self): # pylint: disable=expression-not-assigned (self.pipeline | beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions())) | 'Measure time: Start' >> beam.ParDo( MeasureTime(self.metrics_namespace)) | 'GroupByKey' >> beam.GroupByKey() | 'Ungroup' >> beam.FlatMap(lambda elm: [(elm[0], v) for v in elm[1]]) | 'Measure time: End' >> beam.ParDo( MeasureTime(self.metrics_namespace))) result = self.pipeline.run() result.wait_until_finish() if self.metrics_monitor is not None: self.metrics_monitor.send_metrics(result)
def testCombineGlobally(self): # pylint: disable=expression-not-assigned (self.pipeline | beam.io.Read(synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions())) | 'Measure time: Start' >> beam.ParDo( MeasureTime(self.metrics_namespace)) | 'Combine with Top' >> beam.CombineGlobally( beam.combiners.TopCombineFn(1000)) | 'Consume' >> beam.ParDo(self._GetElement()) | 'Measure time: End' >> beam.ParDo(MeasureTime(self.metrics_namespace)) ) result = self.pipeline.run() result.wait_until_finish() if self.metrics_monitor is not None: self.metrics_monitor.send_metrics(result)
def test(self): self.result = (self.pipeline | 'Read from BigQuery' >> Read(BigQuerySource( dataset=self.input_dataset, table=self.input_table)) | 'Count messages' >> ParDo(CountMessages( self.metrics_namespace)) | 'Measure time' >> ParDo(MeasureTime( self.metrics_namespace)) | 'Count' >> Count.Globally())
def test(self): output = ( self.pipeline | 'Read from BigQuery' >> Read( BigQuerySource(dataset=self.input_dataset, table=self.input_table)) | 'Count messages' >> ParDo(CountMessages(self.metrics_namespace)) | 'Measure time' >> ParDo(MeasureTime(self.metrics_namespace)) | 'Count' >> Count.Globally()) assert_that(output, equal_to([self.input_options['num_records']]))
def test(self): output = ( self.pipeline | 'Read from Spanner' >> ReadFromSpanner( self.project, self.spanner_instance, self.spanner_database, sql="select data from test_data") | 'Count messages' >> ParDo(CountMessages(self.metrics_namespace)) | 'Measure time' >> ParDo(MeasureTime(self.metrics_namespace)) | 'Count' >> Count.Globally()) assert_that(output, equal_to([self.input_options['num_records']]))
def testGroupByKey(self): with self.pipeline as p: # pylint: disable=expression-not-assigned (p | beam.io.Read( synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions())) | 'Measure time' >> beam.ParDo(MeasureTime()) | 'GroupByKey' >> beam.GroupByKey() | 'Ungroup' >> beam.FlatMap(lambda elm: [(elm[0], v) for v in elm[1]])) result = p.run() result.wait_until_finish() metrics = result.metrics().query() for dist in metrics['distributions']: logging.info("Distribution: %s", dist)
def testCombineGlobally(self): with self.pipeline as p: # pylint: disable=expression-not-assigned (p | beam.io.Read(synthetic_pipeline.SyntheticSource( self.parseTestPipelineOptions())) | 'Measure time' >> beam.ParDo(MeasureTime()) | 'Combine with Top' >> beam.CombineGlobally( beam.combiners.TopCombineFn(1000)) | 'Consume' >> beam.ParDo(self._GetElement()) ) result = p.run() result.wait_until_finish() metrics = result.metrics().query() for dist in metrics['distributions']: logging.info("Distribution: %s", dist)
def test(self): def to_pubsub_message(element): import uuid from apache_beam.io import PubsubMessage return PubsubMessage( data=element[1], attributes={'id': str(uuid.uuid1()).encode('utf-8')}, ) _ = ( self.pipeline | 'Create input' >> Read( SyntheticSource(self.parse_synthetic_source_options())) | 'Format to pubsub message in bytes' >> beam.Map(to_pubsub_message) | 'Measure time' >> beam.ParDo(MeasureTime(self.metrics_namespace)) | 'Write to Pubsub' >> beam.io.WriteToPubSub( self.topic_name, with_attributes=True, id_label='id', ))
def test(self): _ = ( self.pipeline | 'Read from pubsub' >> ReadFromPubSub( subscription=self.read_sub_name, with_attributes=True, id_label='id', ) | beam.Map(lambda x: bytes(1)).with_output_types(bytes) | 'Measure time' >> beam.ParDo(MeasureTime(self.metrics_namespace)) | 'Window' >> beam.WindowInto( window.GlobalWindows(), trigger=trigger.Repeatedly( trigger.AfterCount(self.num_of_messages)), accumulation_mode=trigger.AccumulationMode.DISCARDING) | 'Count messages' >> beam.CombineGlobally( beam.combiners.CountCombineFn()).without_defaults(). with_output_types(int) | 'Convert to bytes' >> beam.Map(lambda count: str(count).encode('utf-8')) | 'Write to Pubsub' >> beam.io.WriteToPubSub(self.matcher_topic_name))
def test(self): SCHEMA = parse_table_schema_from_json( '{"fields": [{"name": "data", "type": "BYTES"}]}') def format_record(record): # Since Synthetic Source returns data as a dictionary, we should skip one # of the part return {'data': base64.b64encode(record[1])} ( # pylint: disable=expression-not-assigned self.pipeline | 'Produce rows' >> Read( SyntheticSource(self.parse_synthetic_source_options())) | 'Count messages' >> ParDo(CountMessages(self.metrics_namespace)) | 'Format' >> Map(format_record) | 'Measure time' >> ParDo(MeasureTime(self.metrics_namespace)) | 'Write to BigQuery' >> WriteToBigQuery( dataset=self.output_dataset, table=self.output_table, schema=SCHEMA, create_disposition=BigQueryDisposition.CREATE_IF_NEEDED, write_disposition=BigQueryDisposition.WRITE_TRUNCATE))
def process_tfma(schema_file, big_query_table=None, eval_model_dir=None, max_eval_rows=None, pipeline_args=None, publish_to_bq=False, project=None, metrics_table=None, metrics_dataset=None): """Runs a batch job to evaluate the eval_model against the given input. Args: schema_file: A file containing a text-serialized Schema that describes the eval data. big_query_table: A BigQuery table name specified as DATASET.TABLE which should be the input for evaluation. This can only be set if input_csv is None. eval_model_dir: A directory where the eval model is located. max_eval_rows: Number of rows to query from BigQuery. pipeline_args: additional DataflowRunner or DirectRunner args passed to the beam pipeline. publish_to_bq: project: metrics_dataset: metrics_table: Raises: ValueError: if input_csv and big_query_table are not specified correctly. """ if big_query_table is None: raise ValueError( '--big_query_table should be provided.') slice_spec = [ tfma.slicer.SingleSliceSpec(), tfma.slicer.SingleSliceSpec(columns=['trip_start_hour']) ] metrics_namespace = metrics_table schema = taxi.read_schema(schema_file) eval_shared_model = tfma.default_eval_shared_model( eval_saved_model_path=eval_model_dir, add_metrics_callbacks=[ tfma.post_export_metrics.calibration_plot_and_prediction_histogram(), tfma.post_export_metrics.auc_plots() ]) metrics_monitor = None if publish_to_bq: metrics_monitor = MetricsReader( publish_to_bq=publish_to_bq, project_name=project, bq_table=metrics_table, bq_dataset=metrics_dataset, filters=MetricsFilter().with_namespace(metrics_namespace) ) pipeline = beam.Pipeline(argv=pipeline_args) query = taxi.make_sql(big_query_table, max_eval_rows, for_eval=True) raw_feature_spec = taxi.get_raw_feature_spec(schema) raw_data = ( pipeline | 'ReadBigQuery' >> ReadFromBigQuery(query=query, project=project, use_standard_sql=True) | 'Measure time: Start' >> beam.ParDo(MeasureTime(metrics_namespace)) | 'CleanData' >> beam.Map(lambda x: ( taxi.clean_raw_data_dict(x, raw_feature_spec)))) # Examples must be in clean tf-example format. coder = taxi.make_proto_coder(schema) # Prepare arguments for Extract, Evaluate and Write steps extractors = tfma.default_extractors( eval_shared_model=eval_shared_model, slice_spec=slice_spec, desired_batch_size=None, materialize=False) evaluators = tfma.default_evaluators( eval_shared_model=eval_shared_model, desired_batch_size=None, num_bootstrap_samples=1) _ = ( raw_data | 'ToSerializedTFExample' >> beam.Map(coder.encode) | 'Extract Results' >> tfma.InputsToExtracts() | 'Extract and evaluate' >> tfma.ExtractAndEvaluate( extractors=extractors, evaluators=evaluators) | 'Map Evaluations to PCollection' >> MapEvalToPCollection() | 'Measure time: End' >> beam.ParDo( MeasureTime(metrics_namespace)) ) result = pipeline.run() result.wait_until_finish() if metrics_monitor: metrics_monitor.publish_metrics(result)
def test(self): class SequenceSideInputTestDoFn(beam.DoFn): """Iterate over first n side_input elements.""" def __init__(self, first_n: int): self._first_n = first_n def process( # type: ignore[override] self, element: Any, side_input: Iterable[Tuple[bytes, bytes]]) -> None: i = 0 it = iter(side_input) while i < self._first_n: i += 1 try: # No-op. We only make sure that the element is accessed. next(it) except StopIteration: break class MappingSideInputTestDoFn(beam.DoFn): """Iterates over first n keys in the dictionary and checks the value.""" def __init__(self, first_n: int): self._first_n = first_n def process( # type: ignore[override] self, element: Any, dict_side_input: Dict[bytes, bytes]) -> None: i = 0 for key in dict_side_input: if i == self._first_n: break # No-op. We only make sure that the element is accessed. dict_side_input[key] i += 1 class AssignTimestamps(beam.DoFn): """Produces timestamped values. Timestamps are equal to the value of the element.""" def __init__(self): # Avoid having to use save_main_session self.window = window def process( self, element: int ) -> Iterable[window.TimestampedValue]: # type: ignore[override] yield self.window.TimestampedValue(element, element) class GetSyntheticSDFOptions(beam.DoFn): def __init__(self, elements_per_record: int, key_size: int, value_size: int): self.elements_per_record = elements_per_record self.key_size = key_size self.value_size = value_size def process( self, element: Any ) -> Iterable[Dict[str, Union[int, str]]]: # type: ignore[override] yield { 'num_records': self.elements_per_record, 'key_size': self.key_size, 'value_size': self.value_size, 'initial_splitting_num_bundles': 0, 'initial_splitting_desired_bundle_size': 0, 'sleep_per_input_record_sec': 0, 'initial_splitting': 'const' } main_input = self.pipeline | 'Create' >> beam.Create( range(self.windows)) initial_elements = self.SDF_INITIAL_ELEMENTS if self.windows > 1: main_input = ( main_input | 'Assign timestamps' >> beam.ParDo(AssignTimestamps()) | 'Apply windows' >> beam.WindowInto(window.FixedWindows(1))) side_input = main_input initial_elements = self.windows else: side_input = self.pipeline | 'Side input: create' >> beam.Create( range(initial_elements)) side_input = (side_input | 'Get synthetic SDF options' >> beam.ParDo( GetSyntheticSDFOptions( self.input_options['num_records'] // initial_elements, self.input_options['key_size'], self.input_options['value_size'])) | 'Generate input' >> beam.ParDo(SyntheticSDFAsSource())) main_input |= 'Collect start time metrics' >> beam.ParDo( MeasureTime(self.metrics_namespace)) side_input_type = self.materialize_as() elements_to_access = self.elements_per_window * \ self.access_percentage // 100 logging.info( '%s out of %s total elements in each window will be accessed.', elements_to_access, self.elements_per_window) if side_input_type is beam.pvalue.AsDict: dofn = MappingSideInputTestDoFn(elements_to_access) else: dofn = SequenceSideInputTestDoFn(elements_to_access) _ = (main_input | beam.ParDo(dofn, side_input_type(side_input)) | 'Collect end time metrics' >> beam.ParDo( MeasureTime(self.metrics_namespace)))
def transform_data(input_handle, outfile_prefix, working_dir, schema_file, transform_dir=None, max_rows=None, pipeline_args=None, publish_to_bq=False, project=None, metrics_table=None, metrics_dataset=None): """The main tf.transform method which analyzes and transforms data. Args: input_handle: BigQuery table name to process specified as DATASET.TABLE or path to csv file with input data. outfile_prefix: Filename prefix for emitted transformed examples working_dir: Directory in which transformed examples and transform function will be emitted. schema_file: An file path that contains a text-serialized TensorFlow metadata schema of the input data. transform_dir: Directory in which the transform output is located. If provided, this will load the transform_fn from disk instead of computing it over the data. Hint: this is useful for transforming eval data. max_rows: Number of rows to query from BigQuery pipeline_args: additional DataflowRunner or DirectRunner args passed to the beam pipeline. """ def preprocessing_fn(inputs): """tf.transform's callback function for preprocessing inputs. Args: inputs: map from feature keys to raw not-yet-transformed features. Returns: Map from string feature key to transformed feature operations. """ outputs = {} for key in taxi.DENSE_FLOAT_FEATURE_KEYS: # Preserve this feature as a dense float, setting nan's to the mean. outputs[taxi.transformed_name(key)] = transform.scale_to_z_score( _fill_in_missing(inputs[key])) for key in taxi.VOCAB_FEATURE_KEYS: # Build a vocabulary for this feature. outputs[taxi.transformed_name( key)] = transform.compute_and_apply_vocabulary( _fill_in_missing(inputs[key]), top_k=taxi.VOCAB_SIZE, num_oov_buckets=taxi.OOV_SIZE) for key in taxi.BUCKET_FEATURE_KEYS: outputs[taxi.transformed_name(key)] = transform.bucketize( _fill_in_missing(inputs[key]), taxi.FEATURE_BUCKET_COUNT) for key in taxi.CATEGORICAL_FEATURE_KEYS: outputs[taxi.transformed_name(key)] = _fill_in_missing(inputs[key]) # Was this passenger a big tipper? taxi_fare = _fill_in_missing(inputs[taxi.FARE_KEY]) tips = _fill_in_missing(inputs[taxi.LABEL_KEY]) outputs[taxi.transformed_name(taxi.LABEL_KEY)] = tf.where( tf.is_nan(taxi_fare), tf.cast(tf.zeros_like(taxi_fare), tf.int64), # Test if the tip was > 20% of the fare. tf.cast(tf.greater(tips, tf.multiply(taxi_fare, tf.constant(0.2))), tf.int64)) return outputs namespace = metrics_table metrics_monitor = None if publish_to_bq: metrics_monitor = MetricsReader( publish_to_bq=publish_to_bq, project_name=project, bq_table=metrics_table, bq_dataset=metrics_dataset, namespace=namespace, filters=MetricsFilter().with_namespace(namespace)) schema = taxi.read_schema(schema_file) raw_feature_spec = taxi.get_raw_feature_spec(schema) raw_schema = dataset_schema.from_feature_spec(raw_feature_spec) raw_data_metadata = dataset_metadata.DatasetMetadata(raw_schema) pipeline = beam.Pipeline(argv=pipeline_args) with tft_beam.Context(temp_dir=working_dir): query = taxi.make_sql(input_handle, max_rows, for_eval=False) raw_data = ( pipeline | 'ReadBigQuery' >> ReadFromBigQuery( query=query, project=project, use_standard_sql=True) | 'Measure time: start' >> beam.ParDo(MeasureTime(namespace))) decode_transform = beam.Map(taxi.clean_raw_data_dict, raw_feature_spec=raw_feature_spec) if transform_dir is None: decoded_data = raw_data | 'DecodeForAnalyze' >> decode_transform transform_fn = ( (decoded_data, raw_data_metadata) | ('Analyze' >> tft_beam.AnalyzeDataset(preprocessing_fn))) _ = ( transform_fn | ('WriteTransformFn' >> tft_beam.WriteTransformFn(working_dir))) else: transform_fn = pipeline | tft_beam.ReadTransformFn(transform_dir) # Shuffling the data before materialization will improve Training # effectiveness downstream. Here we shuffle the raw_data (as opposed to # decoded data) since it has a compact representation. shuffled_data = raw_data | 'RandomizeData' >> beam.transforms.Reshuffle( ) decoded_data = shuffled_data | 'DecodeForTransform' >> decode_transform (transformed_data, transformed_metadata) = ( ((decoded_data, raw_data_metadata), transform_fn) | 'Transform' >> tft_beam.TransformDataset()) coder = example_proto_coder.ExampleProtoCoder( transformed_metadata.schema) _ = (transformed_data | 'SerializeExamples' >> beam.Map(coder.encode) | 'Measure time: end' >> beam.ParDo(MeasureTime(namespace)) | 'WriteExamples' >> beam.io.WriteToTFRecord( os.path.join(working_dir, outfile_prefix), file_name_suffix='.gz')) result = pipeline.run() result.wait_until_finish() if metrics_monitor: metrics_monitor.publish_metrics(result)