def test_records_traverse_transform_with_mocks(self): destination = 'project1:dataset1.table1' job_reference = bigquery_api.JobReference() job_reference.projectId = 'project1' job_reference.jobId = 'job_name1' result_job = bigquery_api.Job() result_job.jobReference = job_reference mock_job = mock.Mock() mock_job.status.state = 'DONE' mock_job.status.errorResult = None mock_job.jobReference = job_reference bq_client = mock.Mock() bq_client.jobs.Get.return_value = mock_job bq_client.jobs.Insert.return_value = result_job transform = bqfl.BigQueryBatchFileLoads( destination, custom_gcs_temp_location=self._new_tempdir(), test_client=bq_client, validate=False, coder=CustomRowCoder()) # Need to test this with the DirectRunner to avoid serializing mocks with TestPipeline('DirectRunner') as p: outputs = p | beam.Create(_ELEMENTS) | transform dest_files = outputs[ bqfl.BigQueryBatchFileLoads.DESTINATION_FILE_PAIRS] dest_job = outputs[ bqfl.BigQueryBatchFileLoads.DESTINATION_JOBID_PAIRS] jobs = dest_job | "GetJobs" >> beam.Map(lambda x: x[1]) files = dest_files | "GetFiles" >> beam.Map(lambda x: x[1]) destinations = ( dest_files | "GetDests" >> beam.Map(lambda x: (bigquery_tools.get_hashable_destination(x[0]), x[1])) | "GetUniques" >> beam.combiners.Count.PerKey() | "GetFinalDests" >> beam.Keys()) # All files exist _ = (files | beam.Map( lambda x: hamcrest_assert(os.path.exists(x), is_(True)))) # One file per destination assert_that(files | beam.combiners.Count.Globally(), equal_to([1]), label='CountFiles') assert_that(destinations, equal_to([destination]), label='CheckDestinations') assert_that(jobs, equal_to([job_reference]), label='CheckJobs')
def expand(self, pcoll): p = pcoll.pipeline # TODO(pabloem): Use a different method to determine if streaming or batch. standard_options = p.options.view_as(StandardOptions) if (not callable(self.table_reference) and self.table_reference.projectId is None): self.table_reference.projectId = pcoll.pipeline.options.view_as( GoogleCloudOptions).project if standard_options.streaming: # TODO: Support load jobs for streaming pipelines. bigquery_write_fn = BigQueryWriteFn( table_id=self.table_reference.tableId, dataset_id=self.table_reference.datasetId, project_id=self.table_reference.projectId, batch_size=self.batch_size, schema=self.get_dict_table_schema(self.schema), create_disposition=self.create_disposition, write_disposition=self.write_disposition, kms_key=self.kms_key, test_client=self.test_client) return pcoll | 'WriteToBigQuery' >> ParDo(bigquery_write_fn) else: from apache_beam.io.gcp import bigquery_file_loads return pcoll | bigquery_file_loads.BigQueryBatchFileLoads( destination=self.table_reference, schema=self.get_dict_table_schema(self.schema), create_disposition=self.create_disposition, write_disposition=self.write_disposition, max_file_size=self.max_file_size, max_files_per_bundle=self.max_files_per_bundle, gs_location=self.gs_location, test_client=self.test_client)
def test_load_job_id_used(self): job_reference = bigquery_api.JobReference() job_reference.projectId = 'loadJobProject' job_reference.jobId = 'job_name1' result_job = bigquery_api.Job() result_job.jobReference = job_reference mock_job = mock.Mock() mock_job.status.state = 'DONE' mock_job.status.errorResult = None mock_job.jobReference = job_reference bq_client = mock.Mock() bq_client.jobs.Get.return_value = mock_job bq_client.jobs.Insert.return_value = result_job transform = bqfl.BigQueryBatchFileLoads( 'project1:dataset1.table1', custom_gcs_temp_location=self._new_tempdir(), test_client=bq_client, validate=False, load_job_project_id='loadJobProject') with TestPipeline('DirectRunner') as p: outputs = p | beam.Create(_ELEMENTS) | transform jobs = outputs[bqfl.BigQueryBatchFileLoads.DESTINATION_JOBID_PAIRS] \ | "GetJobs" >> beam.Map(lambda x: x[1]) assert_that(jobs, equal_to([job_reference]), label='CheckJobProjectIds')
def expand(self, pcoll): p = pcoll.pipeline # TODO(pabloem): Use a different method to determine if streaming or batch. standard_options = p.options.view_as(StandardOptions) if (not callable(self.table_reference) and self.table_reference.projectId is None): self.table_reference.projectId = pcoll.pipeline.options.view_as( GoogleCloudOptions).project if (standard_options.streaming or self.method == WriteToBigQuery.Method.STREAMING_INSERTS): # TODO: Support load jobs for streaming pipelines. bigquery_write_fn = BigQueryWriteFn( batch_size=self.batch_size, create_disposition=self.create_disposition, write_disposition=self.write_disposition, kms_key=self.kms_key, retry_strategy=self.insert_retry_strategy, test_client=self.test_client) # TODO: Use utility functions from BQTools table_fn = self._get_table_fn() outputs = ( pcoll | 'AppendDestination' >> beam.Map(lambda x: (table_fn(x), x)) | 'StreamInsertRows' >> ParDo(bigquery_write_fn).with_outputs( BigQueryWriteFn.FAILED_ROWS, main='main')) return { BigQueryWriteFn.FAILED_ROWS: outputs[BigQueryWriteFn.FAILED_ROWS] } else: if standard_options.streaming: raise NotImplementedError( 'File Loads to BigQuery are only supported on Batch pipelines.' ) from apache_beam.io.gcp import bigquery_file_loads return pcoll | bigquery_file_loads.BigQueryBatchFileLoads( destination=self.table_reference, schema=self.get_dict_table_schema(self.schema), create_disposition=self.create_disposition, write_disposition=self.write_disposition, max_file_size=self.max_file_size, max_files_per_bundle=self.max_files_per_bundle, gs_location=self.gs_location, test_client=self.test_client)
def expand(self, pcoll): p = pcoll.pipeline if (isinstance(self.table_reference, bigquery.TableReference) and self.table_reference.projectId is None): self.table_reference.projectId = pcoll.pipeline.options.view_as( GoogleCloudOptions).project method_to_use = self._compute_method(p, p.options) if method_to_use == WriteToBigQuery.Method.STREAMING_INSERTS: # TODO: Support load jobs for streaming pipelines. bigquery_write_fn = BigQueryWriteFn( schema=self.schema, batch_size=self.batch_size, create_disposition=self.create_disposition, write_disposition=self.write_disposition, kms_key=self.kms_key, retry_strategy=self.insert_retry_strategy, test_client=self.test_client) outputs = ( pcoll | 'AppendDestination' >> beam.ParDo( bigquery_tools.AppendDestinationsFn(self.table_reference)) | 'StreamInsertRows' >> ParDo(bigquery_write_fn).with_outputs( BigQueryWriteFn.FAILED_ROWS, main='main')) return { BigQueryWriteFn.FAILED_ROWS: outputs[BigQueryWriteFn.FAILED_ROWS] } else: if p.options.view_as(StandardOptions).streaming: raise NotImplementedError( 'File Loads to BigQuery are only supported on Batch pipelines.' ) from apache_beam.io.gcp import bigquery_file_loads return (pcoll | bigquery_file_loads.BigQueryBatchFileLoads( destination=self.table_reference, schema=self.schema, create_disposition=self.create_disposition, write_disposition=self.write_disposition, max_file_size=self.max_file_size, max_files_per_bundle=self.max_files_per_bundle, custom_gcs_temp_location=self.custom_gcs_temp_location, test_client=self.test_client, validate=self._validate))
def test_load_job_id_use_for_copy_job(self): destination = 'project1:dataset1.table1' job_reference = bigquery_api.JobReference() job_reference.projectId = 'loadJobProject' job_reference.jobId = 'job_name1' result_job = mock.Mock() result_job.jobReference = job_reference mock_job = mock.Mock() mock_job.status.state = 'DONE' mock_job.status.errorResult = None mock_job.jobReference = job_reference bq_client = mock.Mock() bq_client.jobs.Get.return_value = mock_job bq_client.jobs.Insert.return_value = result_job bq_client.tables.Delete.return_value = None with TestPipeline('DirectRunner') as p: outputs = (p | beam.Create(_ELEMENTS, reshuffle=False) | bqfl.BigQueryBatchFileLoads( destination, custom_gcs_temp_location=self._new_tempdir(), test_client=bq_client, validate=False, temp_file_format=bigquery_tools.FileFormat.JSON, max_file_size=45, max_partition_size=80, max_files_per_partition=2, load_job_project_id='loadJobProject')) dest_copy_jobs = outputs[ bqfl.BigQueryBatchFileLoads.DESTINATION_COPY_JOBID_PAIRS] copy_jobs = dest_copy_jobs | "GetCopyJobs" >> beam.Map( lambda x: x[1]) assert_that(copy_jobs, equal_to([ job_reference, job_reference, job_reference, job_reference, job_reference, job_reference ]), label='CheckCopyJobProjectIds')
def test_multiple_partition_files(self): destination = 'project1:dataset1.table1' job_reference = bigquery_api.JobReference() job_reference.projectId = 'project1' job_reference.jobId = 'job_name1' result_job = mock.Mock() result_job.jobReference = job_reference mock_job = mock.Mock() mock_job.status.state = 'DONE' mock_job.status.errorResult = None mock_job.jobReference = job_reference bq_client = mock.Mock() bq_client.jobs.Get.return_value = mock_job bq_client.jobs.Insert.return_value = result_job bq_client.tables.Delete.return_value = None with TestPipeline('DirectRunner') as p: outputs = (p | beam.Create(_ELEMENTS) | bqfl.BigQueryBatchFileLoads( destination, custom_gcs_temp_location=self._new_tempdir(), test_client=bq_client, validate=False, coder=CustomRowCoder(), max_file_size=45, max_partition_size=80, max_files_per_partition=2)) dest_files = outputs[ bqfl.BigQueryBatchFileLoads.DESTINATION_FILE_PAIRS] dest_load_jobs = outputs[ bqfl.BigQueryBatchFileLoads.DESTINATION_JOBID_PAIRS] dest_copy_jobs = outputs[ bqfl.BigQueryBatchFileLoads.DESTINATION_COPY_JOBID_PAIRS] load_jobs = dest_load_jobs | "GetLoadJobs" >> beam.Map( lambda x: x[1]) copy_jobs = dest_copy_jobs | "GetCopyJobs" >> beam.Map( lambda x: x[1]) files = dest_files | "GetFiles" >> beam.Map(lambda x: x[1][0]) destinations = ( dest_files | "GetDests" >> beam.Map(lambda x: (bigquery_tools.get_hashable_destination(x[0]), x[1])) | "GetUniques" >> combiners.Count.PerKey() | "GetFinalDests" >> beam.Keys()) # All files exist _ = (files | beam.Map( lambda x: hamcrest_assert(os.path.exists(x), is_(True)))) # One file per destination assert_that(files | "CountFiles" >> combiners.Count.Globally(), equal_to([6]), label='CheckFileCount') assert_that(destinations, equal_to([destination]), label='CheckDestinations') assert_that(load_jobs | "CountLoadJobs" >> combiners.Count.Globally(), equal_to([6]), label='CheckLoadJobCount') assert_that(copy_jobs | "CountCopyJobs" >> combiners.Count.Globally(), equal_to([6]), label='CheckCopyJobCount')
def test_triggering_frequency(self, is_streaming, with_auto_sharding): destination = 'project1:dataset1.table1' job_reference = bigquery_api.JobReference() job_reference.projectId = 'project1' job_reference.jobId = 'job_name1' result_job = bigquery_api.Job() result_job.jobReference = job_reference mock_job = mock.Mock() mock_job.status.state = 'DONE' mock_job.status.errorResult = None mock_job.jobReference = job_reference bq_client = mock.Mock() bq_client.jobs.Get.return_value = mock_job bq_client.jobs.Insert.return_value = result_job # Insert a fake clock to work with auto-sharding which needs a processing # time timer. class _FakeClock(object): def __init__(self, now=time.time()): self._now = now def __call__(self): return self._now start_time = timestamp.Timestamp(0) bq_client.test_clock = _FakeClock(now=start_time) triggering_frequency = 20 if is_streaming else None transform = bqfl.BigQueryBatchFileLoads( destination, custom_gcs_temp_location=self._new_tempdir(), test_client=bq_client, validate=False, temp_file_format=bigquery_tools.FileFormat.JSON, is_streaming_pipeline=is_streaming, triggering_frequency=triggering_frequency, with_auto_sharding=with_auto_sharding) # Need to test this with the DirectRunner to avoid serializing mocks with TestPipeline( runner='BundleBasedDirectRunner', options=StandardOptions(streaming=is_streaming)) as p: if is_streaming: _SIZE = len(_ELEMENTS) fisrt_batch = [ TimestampedValue(value, start_time + i + 1) for i, value in enumerate(_ELEMENTS[:_SIZE // 2]) ] second_batch = [ TimestampedValue(value, start_time + _SIZE // 2 + i + 1) for i, value in enumerate(_ELEMENTS[_SIZE // 2:]) ] # Advance processing time between batches of input elements to fire the # user triggers. Intentionally advance the processing time twice for the # auto-sharding case since we need to first fire the timer and then # fire the trigger. test_stream = ( TestStream().advance_watermark_to(start_time).add_elements( fisrt_batch).advance_processing_time(30). advance_processing_time(30).add_elements(second_batch). advance_processing_time(30).advance_processing_time( 30).advance_watermark_to_infinity()) input = p | test_stream else: input = p | beam.Create(_ELEMENTS) outputs = input | transform dest_files = outputs[ bqfl.BigQueryBatchFileLoads.DESTINATION_FILE_PAIRS] dest_job = outputs[ bqfl.BigQueryBatchFileLoads.DESTINATION_JOBID_PAIRS] files = dest_files | "GetFiles" >> beam.Map(lambda x: x[1][0]) destinations = ( dest_files | "GetDests" >> beam.Map(lambda x: (bigquery_tools.get_hashable_destination(x[0]), x[1])) | "GetUniques" >> combiners.Count.PerKey() | "GetFinalDests" >> beam.Keys()) jobs = dest_job | "GetJobs" >> beam.Map(lambda x: x[1]) # Check that all files exist. _ = (files | beam.Map( lambda x: hamcrest_assert(os.path.exists(x), is_(True)))) # Expect two load jobs are generated in the streaming case due to the # triggering frequency. Grouping is per trigger so we expect two entries # in the output as opposed to one. file_count = files | combiners.Count.Globally().without_defaults() expected_file_count = [1, 1] if is_streaming else [1] expected_destinations = [destination, destination ] if is_streaming else [destination] expected_jobs = [job_reference, job_reference ] if is_streaming else [job_reference] assert_that(file_count, equal_to(expected_file_count), label='CountFiles') assert_that(destinations, equal_to(expected_destinations), label='CheckDestinations') assert_that(jobs, equal_to(expected_jobs), label='CheckJobs')