def _write_files_with_auto_sharding(self, destination_data_kv_pc, file_prefix_pcv): clock = self.test_client.test_clock if self.test_client else time.time # Auto-sharding is achieved via GroupIntoBatches.WithShardedKey # transform which shards, groups and at the same time batches the table rows # to be inserted to BigQuery. # Firstly, the keys of tagged_data (table references) are converted to a # hashable format. This is needed to work with the keyed states used by. # GroupIntoBatches. After grouping and batching is done, table references # are restored. destination_files_kv_pc = ( destination_data_kv_pc | 'ToHashableTableRef' >> beam.Map(lambda kv: (bigquery_tools.get_hashable_destination(kv[0]), kv[1])) | 'WithAutoSharding' >> GroupIntoBatches.WithShardedKey( batch_size=_FILE_TRIGGERING_RECORD_COUNT, max_buffering_duration_secs= _FILE_TRIGGERING_BATCHING_DURATION_SECS, clock=clock) | 'FromHashableTableRefAndDropShard' >> beam.Map(lambda kvs: ( bigquery_tools.parse_table_reference(kvs[0].key), kvs[1])) | beam.ParDo( WriteGroupedRecordsToFile(schema=self.schema, file_format=self._temp_file_format), file_prefix_pcv, *self.schema_side_inputs)) return self._maybe_apply_user_trigger(destination_files_kv_pc)
def process(self, element, file_prefix): """Take a tuple with (destination, row) and write to file or spill out. Destination may be a ``TableReference`` or a string, and row is a Python dictionary for a row to be inserted to BigQuery.""" destination = bigquery_tools.get_hashable_destination(element[0]) row = element[1] if destination not in self._destination_to_file_writer: if len(self._destination_to_file_writer ) < self.max_files_per_bundle: self._destination_to_file_writer[ destination] = _make_new_file_writer( file_prefix, destination) else: yield pvalue.TaggedOutput( WriteRecordsToFile.UNWRITTEN_RECORD_TAG, element) return (file_path, writer) = self._destination_to_file_writer[destination] # TODO(pabloem): Is it possible for this to throw exception? writer.write(self.coder.encode(row)) writer.write(b'\n') file_size = writer.tell() if file_size > self.max_file_size: writer.close() self._destination_to_file_writer.pop(destination) yield pvalue.TaggedOutput(WriteRecordsToFile.WRITTEN_FILE_TAG, (element[0], (file_path, file_size)))
def check_many_files(output_pcs): dest_file_pc = output_pcs[bqfl.WriteRecordsToFile.WRITTEN_FILE_TAG] spilled_records_pc = output_pcs[ bqfl.WriteRecordsToFile.UNWRITTEN_RECORD_TAG] spilled_records_count = (spilled_records_pc | beam.combiners.Count.Globally()) assert_that(spilled_records_count, equal_to([3]), label='spilled count') files_per_dest = (dest_file_pc | beam.Map(lambda x: x).with_output_types( beam.typehints.KV[str, str]) | beam.combiners.Count.PerKey()) files_per_dest = ( files_per_dest | "GetDests" >> beam.Map(lambda x: ( bigquery_tools.get_hashable_destination(x[0]), x[1]))) # Only table1 and table3 get files. table2 records get spilled. assert_that(files_per_dest, equal_to([('project1:dataset1.table1', 1), ('project1:dataset1.table3', 1)]), label='file count') # Check that the files exist _ = dest_file_pc | beam.Map(lambda x: x[1]) | beam.Map( lambda x: hamcrest_assert(os.path.exists(x), is_(True)))
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 process(self, element, file_prefix, *schema_side_inputs): destination = bigquery_tools.get_hashable_destination(element[0]) rows = element[1] file_path, writer = None, None for row in rows: if writer is None: (file_path, writer) = _make_new_file_writer( file_prefix, destination, self.file_format, self.schema, schema_side_inputs) writer.write(row) file_size = writer.tell() if file_size > self.max_file_size: writer.close() yield (destination, (file_path, file_size)) file_path, writer = None, None if writer is not None: writer.close() yield (destination, (file_path, file_size))
def check_many_files(output_pcs): dest_file_pc = output_pcs[bqfl.WriteRecordsToFile.WRITTEN_FILE_TAG] spilled_records_pc = output_pcs[ bqfl.WriteRecordsToFile.UNWRITTEN_RECORD_TAG] spilled_records_count = (spilled_records_pc | beam.combiners.Count.Globally()) assert_that(spilled_records_count, equal_to([3]), label='spilled count') files_per_dest = (dest_file_pc | beam.Map(lambda x: x).with_output_types( beam.typehints.KV[str, str]) | beam.combiners.Count.PerKey()) files_per_dest = ( files_per_dest | "GetDests" >> beam.Map( lambda x: (bigquery_tools.get_hashable_destination(x[0]), x[1]))) # Only table1 and table3 get files. table2 records get spilled. assert_that(files_per_dest, equal_to([('project1:dataset1.table1', 1), ('project1:dataset1.table3', 1)]), label='file count') # Check that the files exist _ = dest_file_pc | beam.Map(lambda x: x[1]) | beam.Map( lambda x: hamcrest_assert(os.path.exists(x), is_(True)))
def process(self, element, file_prefix): """Take a tuple with (destination, row) and write to file or spill out. Destination may be a ``TableReference`` or a string, and row is a Python dictionary for a row to be inserted to BigQuery.""" destination = bigquery_tools.get_hashable_destination(element[0]) row = element[1] if destination in self._destination_to_file_writer: writer = self._destination_to_file_writer[destination] elif len(self._destination_to_file_writer) < self.max_files_per_bundle: (file_path, writer) = _make_new_file_writer(file_prefix, destination) self._destination_to_file_writer[destination] = writer yield pvalue.TaggedOutput(WriteRecordsToFile.WRITTEN_FILE_TAG, (element[0], file_path)) else: yield pvalue.TaggedOutput( WriteRecordsToFile.UNWRITTEN_RECORD_TAG, element) return # TODO(pabloem): Is it possible for this to throw exception? writer.write(self.coder.encode(row)) writer.write(b'\n') if writer.tell() > self.max_file_size: writer.close() self._destination_to_file_writer.pop(destination)
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) # 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 check_multiple_files(output_pc): files_per_dest = output_pc | combiners.Count.PerKey() files_per_dest = ( files_per_dest | "GetDests" >> beam.Map( lambda x: (bigquery_tools.get_hashable_destination(x[0]), x[1]))) assert_that(files_per_dest, equal_to([('project1:dataset1.table1', 4), ('project1:dataset1.table2', 2), ('project1:dataset1.table3', 1), ])) # Check that the files exist _ = output_pc | beam.Map(lambda x: x[1][0]) | beam.Map(os.path.exists)
def check_files_created(output_pc): files = output_pc | "GetFiles" >> beam.Map(lambda x: x[1][0]) file_count = files | "CountFiles" >> combiners.Count.Globally() _ = files | "FilesExist" >> beam.Map( lambda x: hamcrest_assert(os.path.exists(x), is_(True))) assert_that(file_count, equal_to([3]), label='check file count') destinations = ( output_pc | "GetDests" >> beam.Map( lambda x: bigquery_tools.get_hashable_destination(x[0]))) assert_that(destinations, equal_to(list(_DISTINCT_DESTINATIONS)), label='check destinations ')
def check_multiple_files(output_pc): files_per_dest = output_pc | beam.combiners.Count.PerKey() files_per_dest = ( files_per_dest | "GetDests" >> beam.Map( lambda x: (bigquery_tools.get_hashable_destination(x[0]), x[1]))) assert_that(files_per_dest, equal_to([('project1:dataset1.table1', 4), ('project1:dataset1.table2', 2), ('project1:dataset1.table3', 1), ])) # Check that the files exist _ = output_pc | beam.Map(lambda x: x[1]) | beam.Map(os.path.exists)
def check_files_created(output_pc): files = output_pc | "GetFiles" >> beam.Map(lambda x: x[1]) file_count = files | "CountFiles" >> beam.combiners.Count.Globally() _ = files | "FilesExist" >> beam.Map( lambda x: hamcrest_assert(os.path.exists(x), is_(True))) assert_that(file_count, equal_to([3]), label='check file count') destinations = ( output_pc | "GetDests" >> beam.Map( lambda x: bigquery_tools.get_hashable_destination(x[0]))) assert_that(destinations, equal_to(list(_DISTINCT_DESTINATIONS)), label='check destinations ')
def _make_new_file_writer(file_prefix, destination): destination = bigquery_tools.get_hashable_destination(destination) # Windows does not allow : on filenames. Replacing with underscore. # Other disallowed characters are: # https://docs.microsoft.com/en-us/windows/desktop/fileio/naming-a-file destination = destination.replace(':', '.') directory = fs.FileSystems.join(file_prefix, destination) if not fs.FileSystems.exists(directory): fs.FileSystems.mkdirs(directory) file_name = str(uuid.uuid4()) file_path = fs.FileSystems.join(file_prefix, destination, file_name) return file_path, fs.FileSystems.create(file_path, 'application/text')
def _make_new_file_writer(file_prefix, destination): destination = bigquery_tools.get_hashable_destination(destination) # Windows does not allow : on filenames. Replacing with underscore. # Other disallowed characters are: # https://docs.microsoft.com/en-us/windows/desktop/fileio/naming-a-file destination = destination.replace(':', '.') directory = fs.FileSystems.join(file_prefix, destination) if not fs.FileSystems.exists(directory): fs.FileSystems.mkdirs(directory) file_name = str(uuid.uuid4()) file_path = fs.FileSystems.join(file_prefix, destination, file_name) return file_path, fs.FileSystems.create(file_path, 'application/text')
def check_many_files(output_pcs): dest_file_pc = output_pcs[bqfl.WriteRecordsToFile.WRITTEN_FILE_TAG] files_per_dest = (dest_file_pc | beam.Map(lambda x: x).with_output_types( beam.typehints.KV[str, str]) | beam.combiners.Count.PerKey()) files_per_dest = ( files_per_dest | "GetDests" >> beam.Map(lambda x: ( bigquery_tools.get_hashable_destination(x[0]), x[1]))) assert_that( files_per_dest, equal_to([('project1:dataset1.table1', 4), ('project1:dataset1.table2', 2), ('project1:dataset1.table3', 1)])) # Check that the files exist _ = dest_file_pc | beam.Map(lambda x: x[1]) | beam.Map( lambda x: hamcrest_assert(os.path.exists(x), is_(True)))
def check_many_files(output_pcs): dest_file_pc = output_pcs[bqfl.WriteRecordsToFile.WRITTEN_FILE_TAG] files_per_dest = (dest_file_pc | beam.Map(lambda x: x).with_output_types( beam.typehints.KV[str, str]) | beam.combiners.Count.PerKey()) files_per_dest = ( files_per_dest | "GetDests" >> beam.Map( lambda x: (bigquery_tools.get_hashable_destination(x[0]), x[1])) ) assert_that(files_per_dest, equal_to([('project1:dataset1.table1', 4), ('project1:dataset1.table2', 2), ('project1:dataset1.table3', 1)])) # Check that the files exist _ = dest_file_pc | beam.Map(lambda x: x[1]) | beam.Map( lambda x: hamcrest_assert(os.path.exists(x), is_(True)))
def process(self, element, unused_create_fn_output=None): destination = element[0] if callable(self.schema): schema = self.schema(destination) elif isinstance(self.schema, vp.ValueProvider): schema = self.schema.get() else: schema = self.schema self._create_table_if_needed( bigquery_tools.parse_table_reference(destination), schema) destination = bigquery_tools.get_hashable_destination(destination) row = element[1] self._rows_buffer[destination].append(row) self._total_buffered_rows += 1 if len(self._rows_buffer[destination]) >= self._max_batch_size: return self._flush_batch(destination) elif self._total_buffered_rows >= self._max_buffered_rows: return self._flush_all_batches()
def _make_new_file_writer( file_prefix, destination, file_format, schema=None, schema_side_inputs=tuple()): destination = bigquery_tools.get_hashable_destination(destination) # Windows does not allow : on filenames. Replacing with underscore. # Other disallowed characters are: # https://docs.microsoft.com/en-us/windows/desktop/fileio/naming-a-file destination = destination.replace(':', '.') directory = fs.FileSystems.join(file_prefix, destination) if not fs.FileSystems.exists(directory): fs.FileSystems.mkdirs(directory) file_name = str(uuid.uuid4()) file_path = fs.FileSystems.join(file_prefix, destination, file_name) if file_format == bigquery_tools.FileFormat.AVRO: if callable(schema): schema = schema(destination, *schema_side_inputs) elif isinstance(schema, vp.ValueProvider): schema = schema.get() writer = bigquery_tools.AvroRowWriter( fs.FileSystems.create(file_path, "application/avro"), schema) elif file_format == bigquery_tools.FileFormat.JSON: writer = bigquery_tools.JsonRowWriter( fs.FileSystems.create(file_path, "application/text")) else: raise ValueError(( 'Only AVRO and JSON are supported as intermediate formats for ' 'BigQuery WriteRecordsToFile, got: {}.').format(file_format)) return file_path, writer
def process(self, element, unused_create_fn_output=None): destination = element[0] if callable(self.schema): schema = self.schema(destination) elif isinstance(self.schema, vp.ValueProvider): schema = self.schema.get() else: schema = self.schema self._create_table_if_needed( bigquery_tools.parse_table_reference(destination), schema) destination = bigquery_tools.get_hashable_destination(destination) row = element[1] self._rows_buffer[destination].append(row) self._total_buffered_rows += 1 if len(self._rows_buffer[destination]) >= self._max_batch_size: return self._flush_batch(destination) elif self._total_buffered_rows >= self._max_buffered_rows: return self._flush_all_batches()
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')
def process(self, element, load_job_name_prefix, *schema_side_inputs): # Each load job is assumed to have files respecting these constraints: # 1. Total size of all files < 15 TB (Max size for load jobs) # 2. Total no. of files in a single load job < 10,000 # This assumption means that there will always be a single load job # triggered for each partition of files. destination = element[0] files = element[1] if callable(self.schema): schema = self.schema(destination, *schema_side_inputs) elif isinstance(self.schema, vp.ValueProvider): schema = self.schema.get() else: schema = self.schema if callable(self.additional_bq_parameters): additional_parameters = self.additional_bq_parameters(destination) elif isinstance(self.additional_bq_parameters, vp.ValueProvider): additional_parameters = self.additional_bq_parameters.get() else: additional_parameters = self.additional_bq_parameters table_reference = bigquery_tools.parse_table_reference(destination) if table_reference.projectId is None: table_reference.projectId = vp.RuntimeValueProvider.get_value( 'project', str, '') # Load jobs for a single destination are always triggered from the same # worker. This means that we can generate a deterministic numbered job id, # and not need to worry. destination_hash = _bq_uuid( '%s:%s.%s' % (table_reference.projectId, table_reference.datasetId, table_reference.tableId)) uid = _bq_uuid() job_name = '%s_%s_%s' % (load_job_name_prefix, destination_hash, uid) _LOGGER.debug('Load job has %s files. Job name is %s.', len(files), job_name) create_disposition = self.create_disposition if self.temporary_tables: # If we are using temporary tables, then we must always create the # temporary tables, so we replace the create_disposition. create_disposition = 'CREATE_IF_NEEDED' # For temporary tables, we create a new table with the name with JobId. table_reference.tableId = job_name yield pvalue.TaggedOutput( TriggerLoadJobs.TEMP_TABLES, bigquery_tools.get_hashable_destination(table_reference)) _LOGGER.info( 'Triggering job %s to load data to BigQuery table %s.' 'Schema: %s. Additional parameters: %s', job_name, table_reference, schema, additional_parameters) if not self.bq_io_metadata: self.bq_io_metadata = create_bigquery_io_metadata(self._step_name) job_reference = self.bq_wrapper.perform_load_job( destination=table_reference, source_uris=files, job_id=job_name, schema=schema, write_disposition=self.write_disposition, create_disposition=create_disposition, additional_load_parameters=additional_parameters, source_format=self.source_format, job_labels=self.bq_io_metadata.add_additional_bq_job_labels()) yield (destination, job_reference)
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')