def put_metadata(staging_token, name, sha256=None): return beam_artifact_api_pb2.PutArtifactRequest( metadata=beam_artifact_api_pb2.PutArtifactMetadata( staging_session_token=staging_token, metadata=beam_artifact_api_pb2.ArtifactMetadata( name=name, sha256=sha256)))
def put(index): artifacts[session(index)].append( beam_artifact_api_pb2.ArtifactMetadata(name=name(index))) self._service.PutArtifact([ self.put_metadata(session(index), name(index)), self.put_data(delayed_data('a', index)), self.put_data(delayed_data('b' * 20, index, 2))]) return session(index)
def artifact_request_generator(): artifact_metadata = beam_artifact_api_pb2.ArtifactMetadata( name=artifact_name, sha256=_get_file_hash(local_path_to_artifact)) metadata = beam_artifact_api_pb2.PutArtifactMetadata( staging_session_token=self._staging_session_token, metadata=artifact_metadata) request = beam_artifact_api_pb2.PutArtifactRequest(metadata=metadata) yield request with open(local_path_to_artifact, 'rb') as f: while True: chunk = f.read(1 << 21) # 2MB if not chunk: break request = beam_artifact_api_pb2.PutArtifactRequest( data=beam_artifact_api_pb2.ArtifactChunk(data=chunk)) yield request self._artifacts.append(artifact_metadata)
def _run_staging(self, staging_service, retrieval_service): staging_session_token = '/session_staging_token \n\0*' # First stage some files. staging_service.PutArtifact( iter([ self.put_metadata(staging_session_token, 'name'), self.put_data(b'data') ])) staging_service.PutArtifact( iter([ self.put_metadata(staging_session_token, 'many_chunks'), self.put_data(b'a'), self.put_data(b'b'), self.put_data(b'c') ])) staging_service.PutArtifact( iter([ self.put_metadata(staging_session_token, 'long'), self.put_data(b'a' * 1000) ])) staging_service.PutArtifact( iter([ self.put_metadata(staging_session_token, 'with_hash', hashlib.sha256(b'data...').hexdigest()), self.put_data(b'data'), self.put_data(b'...') ])) with self.assertRaises(Exception): staging_service.PutArtifact( iter([ self.put_metadata(staging_session_token, 'bad_hash', 'bad_hash'), self.put_data(b'data') ])) manifest = beam_artifact_api_pb2.Manifest(artifact=[ beam_artifact_api_pb2.ArtifactMetadata(name='name'), beam_artifact_api_pb2.ArtifactMetadata(name='many_chunks'), beam_artifact_api_pb2.ArtifactMetadata(name='long'), beam_artifact_api_pb2.ArtifactMetadata(name='with_hash'), ]) retrieval_token = staging_service.CommitManifest( beam_artifact_api_pb2.CommitManifestRequest( staging_session_token=staging_session_token, manifest=manifest)).retrieval_token # Now attempt to retrieve them. retrieved_manifest = retrieval_service.GetManifest( beam_artifact_api_pb2.GetManifestRequest( retrieval_token=retrieval_token)).manifest self.assertEqual(manifest, retrieved_manifest) self.assertEqual( b'data', self.retrieve_artifact(retrieval_service, retrieval_token, 'name')) self.assertEqual( b'abc', self.retrieve_artifact(retrieval_service, retrieval_token, 'many_chunks')) self.assertEqual( b'a' * 1000, self.retrieve_artifact(retrieval_service, retrieval_token, 'long')) self.assertEqual( b'data...', self.retrieve_artifact(retrieval_service, retrieval_token, 'with_hash')) with self.assertRaises(Exception): self.retrieve_artifact(retrieval_service, retrieval_token, 'bad_hash') with self.assertRaises(Exception): self.retrieve_artifact(retrieval_service, retrieval_token, 'missing')