@Override public void onNext(ArtifactChunk value) { try { target.write(value.getData().toByteArray()); } catch (IOException e) { // This should never happen throw new AssertionError(e); } }
@VisibleForTesting static FileBasedSink<?, ?, ?> sinkFromProto(SdkFunctionSpec sinkProto) throws IOException { checkArgument( sinkProto.getSpec().getUrn().equals(CUSTOM_JAVA_FILE_BASED_SINK_URN), "Cannot extract %s instance from %s with URN %s", FileBasedSink.class.getSimpleName(), FunctionSpec.class.getSimpleName(), sinkProto.getSpec().getUrn()); byte[] serializedSink = sinkProto.getSpec().getPayload().toByteArray(); return (FileBasedSink<?, ?, ?>) SerializableUtils.deserializeFromByteArray( serializedSink, FileBasedSink.class.getSimpleName()); }
private static Coder<?> fromCustomCoder(RunnerApi.Coder protoCoder) throws IOException { return (Coder<?>) SerializableUtils.deserializeFromByteArray( protoCoder.getSpec().getSpec().getPayload().toByteArray(), "Custom Coder Bytes"); } }
public static DoFnAndMainOutput doFnAndMainOutputTagFromProto(SdkFunctionSpec fnSpec) { checkArgument( fnSpec.getSpec().getUrn().equals(CUSTOM_JAVA_DO_FN_URN), "Expected %s to be %s with URN %s, but URN was %s", DoFn.class.getSimpleName(), FunctionSpec.class.getSimpleName(), CUSTOM_JAVA_DO_FN_URN, fnSpec.getSpec().getUrn()); byte[] serializedFn = fnSpec.getSpec().getPayload().toByteArray(); return (DoFnAndMainOutput) SerializableUtils.deserializeFromByteArray(serializedFn, "Custom DoFn And Main Output tag"); }
/** * Converts a {@link org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec} into a {@link * ViewFn} using the URN. */ public static ViewFn<?, ?> viewFnFromProto(RunnerApi.SdkFunctionSpec viewFn) throws InvalidProtocolBufferException { RunnerApi.FunctionSpec spec = viewFn.getSpec(); checkArgument( spec.getUrn().equals(ParDoTranslation.CUSTOM_JAVA_VIEW_FN_URN), "Can't deserialize unknown %s type %s", ViewFn.class.getSimpleName(), spec.getUrn()); return (ViewFn<?, ?>) SerializableUtils.deserializeFromByteArray( spec.getPayload().toByteArray(), "Custom ViewFn"); }
@Override public void onNext(PutArtifactRequest value) { try { stream.write(value.getData().getData().toByteArray()); } catch (IOException e) { throw new RuntimeException(e); } }
/** * Converts a {@link org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec} into a {@link * WindowMappingFn} using the URN. */ public static WindowMappingFn<?> windowMappingFnFromProto( RunnerApi.SdkFunctionSpec windowMappingFn) throws InvalidProtocolBufferException { RunnerApi.FunctionSpec spec = windowMappingFn.getSpec(); checkArgument( spec.getUrn().equals(ParDoTranslation.CUSTOM_JAVA_WINDOW_MAPPING_FN_URN), "Can't deserialize unknown %s type %s", WindowMappingFn.class.getSimpleName(), spec.getUrn()); return (WindowMappingFn<?>) SerializableUtils.deserializeFromByteArray( spec.getPayload().toByteArray(), "Custom WinodwMappingFn"); } }
private void translateStreamingImpulse( String id, RunnerApi.Pipeline pipeline, StreamingTranslationContext context) { RunnerApi.PTransform pTransform = pipeline.getComponents().getTransformsOrThrow(id); TypeInformation<WindowedValue<byte[]>> typeInfo = new CoderTypeInformation<>( WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE)); ObjectMapper objectMapper = new ObjectMapper(); final int intervalMillis; final int messageCount; try { JsonNode config = objectMapper.readTree(pTransform.getSpec().getPayload().toByteArray()); intervalMillis = config.path("interval_ms").asInt(100); messageCount = config.path("message_count").asInt(0); } catch (IOException e) { throw new RuntimeException("Failed to parse configuration for streaming impulse", e); } SingleOutputStreamOperator<WindowedValue<byte[]>> source = context .getExecutionEnvironment() .addSource(new StreamingImpulseSource(intervalMillis, messageCount)) .returns(typeInfo); context.addDataStream(Iterables.getOnlyElement(pTransform.getOutputsMap().values()), source); }
public static UnboundedSource<?, ?> unboundedSourceFromProto(ReadPayload payload) throws InvalidProtocolBufferException { checkArgument(payload.getIsBounded().equals(IsBounded.Enum.UNBOUNDED)); return (UnboundedSource<?, ?>) SerializableUtils.deserializeFromByteArray( payload.getSource().getSpec().getPayload().toByteArray(), "UnboundedSource"); }
/** * @deprecated Since {@link CreatePCollectionView} is not a part of the Beam model, there is no * SDK-agnostic specification. Using this method means your runner is tied to Java. */ @Deprecated public static <ElemT, ViewT> PCollectionView<ViewT> getView( AppliedPTransform< PCollection<ElemT>, PCollection<ElemT>, PTransform<PCollection<ElemT>, PCollection<ElemT>>> application) throws IOException { RunnerApi.PTransform transformProto = PTransformTranslation.toProto( application, Collections.emptyList(), SdkComponents.create(application.getPipeline().getOptions())); checkArgument( PTransformTranslation.CREATE_VIEW_TRANSFORM_URN.equals(transformProto.getSpec().getUrn()), "Illegal attempt to extract %s from transform %s with name \"%s\" and URN \"%s\"", PCollectionView.class.getSimpleName(), application.getTransform(), application.getFullName(), transformProto.getSpec().getUrn()); return (PCollectionView<ViewT>) SerializableUtils.deserializeFromByteArray( transformProto.getSpec().getPayload().toByteArray(), PCollectionView.class.getSimpleName()); }
public static BoundedSource<?> boundedSourceFromProto(ReadPayload payload) throws InvalidProtocolBufferException { checkArgument(payload.getIsBounded().equals(IsBounded.Enum.BOUNDED)); return (BoundedSource<?>) SerializableUtils.deserializeFromByteArray( payload.getSource().getSpec().getPayload().toByteArray(), "BoundedSource"); }
private static Coder<?> fromKnownCoder(RunnerApi.Coder coder, RehydratedComponents components) throws IOException { String coderUrn = coder.getSpec().getSpec().getUrn(); List<Coder<?>> coderComponents = new ArrayList<>(); for (String componentId : coder.getComponentCoderIdsList()) { Coder<?> innerCoder = components.getCoder(componentId); coderComponents.add(innerCoder); } Class<? extends Coder> coderType = KNOWN_CODER_URNS.inverse().get(coderUrn); CoderTranslator<?> translator = KNOWN_TRANSLATORS.get(coderType); checkArgument( translator != null, "Unknown Coder URN %s. Known URNs: %s", coderUrn, KNOWN_CODER_URNS.values()); return translator.fromComponents( coderComponents, coder.getSpec().getSpec().getPayload().toByteArray()); }
static <T> TestStream.Event<T> eventFromProto( RunnerApi.TestStreamPayload.Event protoEvent, Coder<T> coder) throws IOException { switch (protoEvent.getEventCase()) { case WATERMARK_EVENT: return TestStream.WatermarkEvent.advanceTo( new Instant(protoEvent.getWatermarkEvent().getNewWatermark())); case PROCESSING_TIME_EVENT: return TestStream.ProcessingTimeEvent.advanceBy( Duration.millis(protoEvent.getProcessingTimeEvent().getAdvanceDuration())); case ELEMENT_EVENT: List<TimestampedValue<T>> decodedElements = new ArrayList<>(); for (RunnerApi.TestStreamPayload.TimestampedElement element : protoEvent.getElementEvent().getElementsList()) { decodedElements.add( TimestampedValue.of( CoderUtils.decodeFromByteArray(coder, element.getEncodedElement().toByteArray()), new Instant(element.getTimestamp()))); } return TestStream.ElementEvent.add(decodedElements); case EVENT_NOT_SET: default: throw new IllegalArgumentException( String.format( "Unsupported type of %s: %s", RunnerApi.TestStreamPayload.Event.class.getCanonicalName(), protoEvent.getEventCase())); } }
ByteString chunk = ByteString.copyFrom(readBuffer); hasher.putBytes(chunk.toByteArray()); readBuffer.rewind(); PutArtifactRequest request =
(Combine.CombineFn<?, ?, ?>) SerializableUtils.deserializeFromByteArray( combineFnSpec.getPayload().toByteArray(), Combine.CombineFn.class.getSimpleName());
return (WindowFn<?, ?>) SerializableUtils.deserializeFromByteArray( windowFnSpec.getSpec().getPayload().toByteArray(), "WindowFn"); } else { throw new IllegalArgumentException(
@Test public void testEncodedProto() throws Exception { SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); components.registerPCollection(testPCollection); AppliedPTransform<?, ?, ?> appliedPTransform = AppliedPTransform.of( "foo", testPCollection.expand(), createViewTransform.getView().expand(), createViewTransform, p); FunctionSpec payload = PTransformTranslation.toProto(appliedPTransform, components).getSpec(); // Checks that the payload is what it should be PCollectionView<?> deserializedView = (PCollectionView<?>) SerializableUtils.deserializeFromByteArray( payload.getPayload().toByteArray(), PCollectionView.class.getSimpleName()); assertThat(deserializedView, Matchers.equalTo(createViewTransform.getView())); }
@Test public void testExtractionDirectFromTransform() throws Exception { SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); components.registerPCollection(testPCollection); AppliedPTransform<?, ?, ?> appliedPTransform = AppliedPTransform.of( "foo", testPCollection.expand(), createViewTransform.getView().expand(), createViewTransform, p); CreatePCollectionViewTranslation.getView((AppliedPTransform) appliedPTransform); FunctionSpec payload = PTransformTranslation.toProto(appliedPTransform, components).getSpec(); // Checks that the payload is what it should be PCollectionView<?> deserializedView = (PCollectionView<?>) SerializableUtils.deserializeFromByteArray( payload.getPayload().toByteArray(), PCollectionView.class.getSimpleName()); assertThat(deserializedView, Matchers.equalTo(createViewTransform.getView())); } }
combineFn, SerializableUtils.deserializeFromByteArray( combineProto.getCombineFn().getSpec().getPayload().toByteArray(), "CombineFn"));
combineFn, SerializableUtils.deserializeFromByteArray( combineProto.getCombineFn().getSpec().getPayload().toByteArray(), "CombineFn"));