def _create_file_stage_to_artifact(local_path, staged_name): 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.STAGING_TO.urn, role_payload=beam_runner_api_pb2.ArtifactStagingToRolePayload( staged_name=staged_name).SerializeToString())
def _generate_artifacts_manifests_json_file(file_names, temp_dir): infos = [] for name in file_names: info = beam_runner_api_pb2.ArtifactInformation( type_urn=common_urns.StandardArtifacts.Types.FILE.urn, type_payload=beam_runner_api_pb2.ArtifactFilePayload( path=name).SerializeToString(), ) infos.append(json.dumps(MessageToJson(info))) with open(os.path.join(temp_dir, ARTIFACTS_MANIFEST_FILE), 'w') as file: file.write('[\n' + ',\n'.join(infos) + '\n]')
def python_sdk_dependencies(options, tmp_dir=None): if tmp_dir is None: tmp_dir = tempfile.mkdtemp() 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))
def store_artifact(artifact, service, dest_dir): hasher = hashlib.sha256() with tempfile.NamedTemporaryFile(dir=dest_dir, delete=False) as fout: for block in service.GetArtifact( beam_artifact_api_pb2.GetArtifactRequest(artifact=artifact)): hasher.update(block.data) fout.write(block.data) return beam_runner_api_pb2.ArtifactInformation( type_urn=common_urns.artifact_types.FILE.urn, type_payload=beam_runner_api_pb2.ArtifactFilePayload( path=fout.name, sha256=hasher.hexdigest()).SerializeToString(), role_urn=artifact.role_urn, role_payload=artifact.role_payload)
def extract_staging_tuple_iter( artifacts: List[beam_runner_api_pb2.ArtifactInformation]): for artifact in artifacts: if artifact.type_urn == common_urns.artifact_types.FILE.urn: file_payload = beam_runner_api_pb2.ArtifactFilePayload() file_payload.ParseFromString(artifact.type_payload) src = file_payload.path if artifact.role_urn == common_urns.artifact_roles.STAGING_TO.urn: role_payload = beam_runner_api_pb2.ArtifactStagingToRolePayload() role_payload.ParseFromString(artifact.role_payload) dst = role_payload.staged_name elif (artifact.role_urn == common_urns.artifact_roles.PIP_REQUIREMENTS_FILE.urn): dst = hashlib.sha256(artifact.SerializeToString()).hexdigest() else: raise RuntimeError("unknown role type: %s" % artifact.role_urn) yield (src, dst) else: raise RuntimeError("unknown artifact type: %s" % artifact.type_urn)
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 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 _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)