def run(input_pcol, config): """Main entrypoint in running a job's transform(s). Run any Beam transforms that need to happen after a message is consumed from PubSub from an upstream job (if not an apex job), and before publishing a message to any downstream job (if needed/configured). Args: input_pcol: A Beam PCollection returned from ``beam.io.ReadFromPubSub``. config (klio.KlioConfig): Job-related configuration as defined in ``klio-job.yaml``. Returns: A Beam PCollection that will be passed to ``beam.io.WriteToPubSub``. """ output_data = (input_pcol | beam.ParDo( transforms.CatVDogOutputCheck()).with_outputs()) output_force = output_data.found | helpers.KlioFilterForce() to_input_check = ((output_data.not_found, output_force.process) | beam.Flatten()) to_process = to_input_check | helpers.KlioGcsCheckInputExists() return to_process.found | beam.ParDo(transforms.CatVDog())
def test_trigger_upstream_job(mock_config, mocker, caplog): mock_gcs_client = mocker.patch("klio.transforms._helpers.gcsio.GcsIO") mock_gcs_client.return_value.exists.return_value = False mock_pubsub_client = mocker.patch("google.cloud.pubsub.PublisherClient") kmsg = klio_pb2.KlioMessage() kmsg.data.element = b"does_not_exist" exp_current_job = klio_pb2.KlioJob() exp_current_job.job_name = "a-job" exp_current_job.gcp_project = "not-a-real-project" exp_upstream_job = klio_pb2.KlioJob() exp_upstream_job.job_name = "upstream-job" exp_upstream_job.gcp_project = "upstream-project" exp_kmsg = klio_pb2.KlioMessage() exp_kmsg.version = klio_pb2.Version.V2 exp_kmsg.data.element = b"does_not_exist" exp_lmtd = exp_kmsg.metadata.intended_recipients.limited exp_lmtd.recipients.extend([exp_upstream_job, exp_current_job]) exp_lmtd.trigger_children_of.CopyFrom(exp_current_job) options = pipeline_options.PipelineOptions([]) options.view_as(pipeline_options.StandardOptions).streaming = True with test_pipeline.TestPipeline(options=options) as p: in_pcol = p | beam.Create([kmsg.SerializeToString()]) input_data = in_pcol | helpers.KlioGcsCheckInputExists() _ = input_data.not_found | helpers.KlioTriggerUpstream( upstream_job_name="upstream-job", upstream_topic="projects/upstream-project/topics/does-not-exist", ) mock_gcs_client.return_value.exists.assert_called_once_with( "gs://hopefully-this-bucket-doesnt-exist/does_not_exist") mock_pubsub_client.return_value.publish.assert_called_once_with( mock_pubsub_client.return_value.topic_path.return_value, exp_kmsg.SerializeToString(), ) actual_counters = p.result.metrics().query()["counters"] assert 2 == len(actual_counters) data_not_found_ctr = actual_counters[0] trigger_upstream_ctr = actual_counters[1] assert 1 == data_not_found_ctr.committed assert "KlioGcsCheckInputExists" == data_not_found_ctr.key.metric.namespace assert "kmsg-data-not-found-input" == data_not_found_ctr.key.metric.name assert 1 == trigger_upstream_ctr.committed assert "KlioTriggerUpstream" == trigger_upstream_ctr.key.metric.namespace assert "kmsg-trigger-upstream" == trigger_upstream_ctr.key.metric.name expected_log_msg = "Triggering upstream upstream-job for does_not_exist" for record in caplog.records: if expected_log_msg in record.message: assert True break else: assert False, "Expected log message not found"
def test_trigger_upstream_job(mock_config, mocker, capsys): mock_gcs_client = mocker.patch("klio.transforms._helpers.gcsio.GcsIO") mock_gcs_client.return_value.exists.return_value = False mock_pubsub_client = mocker.patch("google.cloud.pubsub.PublisherClient") kmsg = klio_pb2.KlioMessage() kmsg.data.element = b"does_not_exist" exp_current_job = klio_pb2.KlioJob() exp_current_job.job_name = "a-job" exp_current_job.gcp_project = "not-a-real-project" exp_upstream_job = klio_pb2.KlioJob() exp_upstream_job.job_name = "upstream-job" exp_upstream_job.gcp_project = "upstream-project" exp_kmsg = klio_pb2.KlioMessage() exp_kmsg.version = klio_pb2.Version.V2 exp_kmsg.data.element = b"does_not_exist" exp_lmtd = exp_kmsg.metadata.intended_recipients.limited exp_lmtd.recipients.extend([exp_upstream_job, exp_current_job]) exp_lmtd.trigger_children_of.CopyFrom(exp_current_job) options = pipeline_options.PipelineOptions([]) options.view_as(pipeline_options.StandardOptions).streaming = True with test_pipeline.TestPipeline(options=options) as p: in_pcol = p | beam.Create([kmsg.SerializeToString()]) input_data = in_pcol | helpers.KlioGcsCheckInputExists() _ = input_data.not_found | helpers.KlioTriggerUpstream( upstream_job_name="upstream-job", upstream_topic="projects/upstream-project/topics/does-not-exist", ) mock_gcs_client.return_value.exists.assert_called_once_with( "gs://hopefully-this-bucket-doesnt-exist/does_not_exist") mock_pubsub_client.return_value.publish.assert_called_once_with( mock_pubsub_client.return_value.topic_path.return_value, exp_kmsg.SerializeToString(), )
def _setup_data_io_filters(self, in_pcol, label_prefix=None): # label prefixes are required for multiple inputs (to avoid label # name collisions in Beam) if self._has_multi_data_inputs or self._has_multi_data_outputs: logging.error( "Klio does not (yet) support multiple data inputs and outputs." ) raise SystemExit(1) data_in_config, data_out_config = None, None if self._has_data_inputs: data_in_config = self.config.job_config.data.inputs[0] if self._has_data_outputs: data_out_config = self.config.job_config.data.outputs[0] pfx = "" if label_prefix is not None: pfx = "[{}] ".format(label_prefix) def lbl(label): return "{}{}".format(pfx, label) to_process_output = in_pcol pass_thru = None if data_in_config: pings = in_pcol | lbl("Ping Filter") >> helpers.KlioFilterPing() to_process_output = pings.process pass_thru = pings.pass_thru if data_out_config and not data_out_config.skip_klio_existence_check: output_exists = (to_process_output | lbl("Output Exists Filter") >> helpers.KlioGcsCheckOutputExists()) output_force = ( output_exists.found | lbl("Output Force Filter") >> helpers.KlioFilterForce()) to_pass_thru_tuple = (pass_thru, output_force.pass_thru) to_pass_thru = (to_pass_thru_tuple | lbl("Flatten to Pass Thru") >> beam.Flatten()) to_filter_input_tuple = ( output_exists.not_found, output_force.process, ) to_filter_input = (to_filter_input_tuple | lbl("Flatten to Process") >> beam.Flatten()) else: to_pass_thru = pass_thru to_filter_input = to_process_output if data_in_config and not data_in_config.skip_klio_existence_check: input_exists = (to_filter_input | lbl("Input Exists Filter") >> helpers.KlioGcsCheckInputExists()) _ = (input_exists.not_found | lbl("Drop Not Found Data") >> helpers.KlioDrop()) to_process = input_exists.found else: to_process = to_filter_input return to_process, to_pass_thru
def _setup_data_io_filters(self, in_pcol, label_prefix=None): # label prefixes are required for multiple inputs (to avoid label # name collisions in Beam) if self._has_multi_data_inputs or self._has_multi_data_outputs: logging.error( "Klio does not (yet) support multiple data inputs and outputs." ) raise SystemExit(1) data_in_config, data_out_config = None, None if self._has_data_inputs: data_in_config = self.config.job_config.data.inputs[0] if self._has_data_outputs: data_out_config = self.config.job_config.data.outputs[0] pfx = "" if label_prefix is not None: pfx = "[{}] ".format(label_prefix) def lbl(label): return "{}{}".format(pfx, label) to_process_output = in_pcol pass_thru = None if data_in_config: pings = in_pcol | lbl("Ping Filter") >> helpers.KlioFilterPing() to_process_output = pings.process pass_thru = pings.pass_thru if data_out_config and not data_out_config.skip_klio_existence_check: output_exists = (to_process_output | lbl("Output Exists Filter") >> helpers.KlioGcsCheckOutputExists()) output_force = ( output_exists.found | lbl("Output Force Filter") >> helpers.KlioFilterForce()) if pass_thru is not None: to_pass_thru_tuple = (pass_thru, output_force.pass_thru) to_pass_thru = (to_pass_thru_tuple | lbl("Flatten to Pass Thru") >> beam.Flatten()) else: to_pass_thru = output_force.pass_thru to_filter_input_tuple = ( output_exists.not_found, output_force.process, ) to_filter_input = (to_filter_input_tuple | lbl("Flatten to Process") >> beam.Flatten()) else: to_pass_thru = pass_thru to_filter_input = to_process_output if data_in_config and not data_in_config.skip_klio_existence_check: input_exists = (to_filter_input | lbl("Input Exists Filter") >> helpers.KlioGcsCheckInputExists()) # TODO: update me to `var.KlioRunner.DIRECT_GKE_RUNNER` once # direct_on_gke_runner_clean is merged if self.config.pipeline_options.runner == "DirectGKERunner": ack_inp_lbl = lbl("Ack Input Message from No Data Input Found") _ = input_exists.not_found | ack_inp_lbl >> beam.ParDo( helpers.KlioAckInputMessage()) _ = (input_exists.not_found | lbl("Drop Not Found Data") >> helpers.KlioDrop()) to_process = input_exists.found else: to_process = to_filter_input return to_process, to_pass_thru