def _stage_via_portable_service(self, artifact_staging_channel, staging_session_token): artifact_service.offer_artifacts( beam_artifact_api_pb2_grpc.ArtifactStagingServiceStub( channel=artifact_staging_channel), artifact_service.ArtifactRetrievalService( artifact_service.BeamFilesystemHandler(None).file_reader), staging_session_token)
def stage(self, pipeline, artifact_staging_endpoint, staging_session_token): # type: (...) -> Optional[Any] """Stage artifacts""" if artifact_staging_endpoint: artifact_service.offer_artifacts( beam_artifact_api_pb2_grpc.ArtifactStagingServiceStub( channel=grpc.insecure_channel(artifact_staging_endpoint)), artifact_service.ArtifactRetrievalService( artifact_service.BeamFilesystemHandler(None).file_reader), staging_session_token)
def test_push_artifacts(self): unresolved = beam_runner_api_pb2.ArtifactInformation( type_urn='unresolved') resolved_a = self.embedded_artifact(data=b'a', name='a.txt') resolved_b = self.embedded_artifact(data=b'bb', name='b.txt') dep_big = self.embedded_artifact(data=b'big ' * 100, name='big.txt') class TestArtifacts(object): def ResolveArtifacts(self, request): replacements = [] for artifact in request.artifacts: if artifact.type_urn == 'unresolved': replacements += [resolved_a, resolved_b] else: replacements.append(artifact) return beam_artifact_api_pb2.ResolveArtifactsResponse( replacements=replacements) def GetArtifact(self, request): if request.artifact.type_urn == common_urns.artifact_types.EMBEDDED.urn: content = proto_utils.parse_Bytes( request.artifact.type_payload, beam_runner_api_pb2.EmbeddedFilePayload).data for k in range(0, len(content), 13): yield beam_artifact_api_pb2.GetArtifactResponse( data=content[k:k + 13]) else: raise NotImplementedError file_manager = InMemoryFileManager() server = artifact_service.ArtifactStagingService( file_manager.file_writer) server.register_job('staging_token', {'env': [unresolved, dep_big]}) # "Push" artifacts as if from a client. t = threading.Thread(target=lambda: artifact_service.offer_artifacts( server, TestArtifacts(), 'staging_token')) t.daemon = True t.start() resolved_deps = server.resolved_deps('staging_token', timeout=5)['env'] expected = { 'a.txt': b'a', 'b.txt': b'bb', 'big.txt': b'big ' * 100, } for dep in resolved_deps: self.assertEqual(dep.type_urn, common_urns.artifact_types.FILE.urn) self.assertEqual(dep.role_urn, common_urns.artifact_roles.STAGING_TO.urn) type_payload = proto_utils.parse_Bytes( dep.type_payload, beam_runner_api_pb2.ArtifactFilePayload) role_payload = proto_utils.parse_Bytes( dep.role_payload, beam_runner_api_pb2.ArtifactStagingToRolePayload) self.assertTrue( type_payload.path.endswith(role_payload.staged_name), type_payload.path) self.assertEqual(file_manager.get(type_payload.path), expected.pop(role_payload.staged_name)) self.assertEqual(expected, {})