def python_sdk_dependencies(options, tmp_dir=None): if tmp_dir is None: tmp_dir = tempfile.mkdtemp() skip_prestaged_dependencies = options.view_as( SetupOptions).prebuild_sdk_container_engine is not None return tuple( beam_runner_api_pb2.ArtifactInformation( type_urn=common_urns.artifact_types.FILE.urn, type_payload=beam_runner_api_pb2.ArtifactFilePayload( path=local_path).SerializeToString(), role_urn=common_urns.artifact_roles.STAGING_TO.urn, role_payload=beam_runner_api_pb2.ArtifactStagingToRolePayload( staged_name=staged_name).SerializeToString()) for local_path, staged_name in stager.Stager.create_job_resources( options, tmp_dir, skip_prestaged_dependencies=skip_prestaged_dependencies))
def resolve_as_files(retrieval_service, file_writer, dependencies): """Translates a set of dependencies into file-based dependencies.""" # Resolve until nothing changes. This ensures that they can be fetched. resolution = retrieval_service.ResolveArtifactss( beam_artifact_api_pb2.ResolveArtifactsRequest( artifacts=dependencies, # Anything fetchable will do. # TODO(robertwb): Take advantage of shared filesystems, urls. preferred_urns=[], )) dependencies = resolution.replacements # Fetch each of the dependencies, using file_writer to store them as # file-based artifacts. # TODO(robertwb): Consider parallelizing the actual writes. for dep in dependencies: if dep.role_urn == common_urns.artifact_roles.STAGING_TO.urn: base_name = os.path.basename( proto_utils.parse_Bytes( dep.role_payload, beam_runner_api_pb2.ArtifactStagingToRolePayload).staged_name) else: base_name = None unique_name = '-'.join( filter( None, [hashlib.sha256(dep.SerializeToString()).hexdigest(), base_name])) file_handle, path = file_writer(unique_name) with file_handle as fout: for chunk in retrieval_service.GetArtifact( beam_artifact_api_pb2.GetArtifactRequest(artifact=dep)): fout.write(chunk.data) yield beam_runner_api_pb2.ArtifactInformation( type_urn=common_urns.artifact_types.FILE.urn, type_payload=beam_runner_api_pb2.ArtifactFilePayload( path=path).SerializeToString(), role_urn=dep.role_urn, role_payload=dep.role_payload)
def test_stage_resources(self): pipeline_options = PipelineOptions([ '--temp_location', 'gs://test-location/temp', '--staging_location', 'gs://test-location/staging' ]) pipeline = beam_runner_api_pb2.Pipeline( components=beam_runner_api_pb2.Components( environments={ 'env1': beam_runner_api_pb2.Environment( dependencies=[ beam_runner_api_pb2.ArtifactInformation( type_urn=common_urns.artifact_types.FILE.urn, type_payload=beam_runner_api_pb2. ArtifactFilePayload( path='/tmp/foo1').SerializeToString(), role_urn=common_urns.artifact_roles.STAGING_TO.urn, role_payload=beam_runner_api_pb2. ArtifactStagingToRolePayload( staged_name='foo1').SerializeToString()), beam_runner_api_pb2.ArtifactInformation( type_urn=common_urns.artifact_types.FILE.urn, type_payload=beam_runner_api_pb2. ArtifactFilePayload( path='/tmp/bar1').SerializeToString(), role_urn=common_urns.artifact_roles.STAGING_TO.urn, role_payload=beam_runner_api_pb2. ArtifactStagingToRolePayload( staged_name='bar1').SerializeToString()) ]), 'env2': beam_runner_api_pb2.Environment( dependencies=[ beam_runner_api_pb2.ArtifactInformation( type_urn=common_urns.artifact_types.FILE.urn, type_payload=beam_runner_api_pb2. ArtifactFilePayload( path='/tmp/foo2').SerializeToString(), role_urn=common_urns.artifact_roles.STAGING_TO.urn, role_payload=beam_runner_api_pb2. ArtifactStagingToRolePayload( staged_name='foo2').SerializeToString()), beam_runner_api_pb2.ArtifactInformation( type_urn=common_urns.artifact_types.FILE.urn, type_payload=beam_runner_api_pb2. ArtifactFilePayload( path='/tmp/bar2').SerializeToString(), role_urn=common_urns.artifact_roles.STAGING_TO.urn, role_payload=beam_runner_api_pb2. ArtifactStagingToRolePayload( staged_name='bar2').SerializeToString()) ]) })) client = apiclient.DataflowApplicationClient(pipeline_options) with mock.patch.object(apiclient._LegacyDataflowStager, 'stage_job_resources') as mock_stager: client._stage_resources(pipeline, pipeline_options) mock_stager.assert_called_once_with( [('/tmp/foo1', 'foo1'), ('/tmp/bar1', 'bar1'), ('/tmp/foo2', 'foo2'), ('/tmp/bar2', 'bar2')], staging_location='gs://test-location/staging')
def file_artifact(self, path): return beam_runner_api_pb2.ArtifactInformation( type_urn=common_urns.artifact_types.FILE.urn, type_payload=beam_runner_api_pb2.ArtifactFilePayload( path=path).SerializeToString())
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, {})
def _create_file_pip_requirements_artifact(local_path): return beam_runner_api_pb2.ArtifactInformation( type_urn=common_urns.artifact_types.FILE.urn, type_payload=beam_runner_api_pb2.ArtifactFilePayload( path=local_path).SerializeToString(), role_urn=common_urns.artifact_roles.PIP_REQUIREMENTS_FILE.urn)