/** * Converts from a {@link RunnerApi.WindowingStrategy} accompanied by {@link Components} to the * SDK's {@link WindowingStrategy}. */ public static WindowingStrategy<?, ?> fromProto(RunnerApi.MessageWithComponents proto) throws InvalidProtocolBufferException { switch (proto.getRootCase()) { case WINDOWING_STRATEGY: return fromProto( proto.getWindowingStrategy(), RehydratedComponents.forComponents(proto.getComponents())); default: throw new IllegalArgumentException( String.format( "Expected a %s with components but received %s", RunnerApi.WindowingStrategy.class.getCanonicalName(), proto)); } }
private Coder<T> getCoder() throws IOException { if (coder == null) { coder = (Coder) CoderTranslation.fromProto( coderSpec.getCoder(), RehydratedComponents.forComponents(coderSpec.getComponents())); } return coder; }
public static WindowingStrategy getWindowingStrategy( String pCollectionId, RunnerApi.Components components) { RunnerApi.WindowingStrategy windowingStrategyProto = components.getWindowingStrategiesOrThrow( components.getPcollectionsOrThrow(pCollectionId).getWindowingStrategyId()); final WindowingStrategy<?, ?> windowingStrategy; try { return WindowingStrategyTranslation.fromProto( windowingStrategyProto, RehydratedComponents.forComponents(components)); } catch (InvalidProtocolBufferException e) { throw new IllegalStateException( String.format( "Unable to hydrate windowing strategy %s for %s.", windowingStrategyProto, pCollectionId), e); } } }
public static WindowingStrategy getWindowingStrategy( String pCollectionId, RunnerApi.Components components) { RunnerApi.WindowingStrategy windowingStrategyProto = components.getWindowingStrategiesOrThrow( components.getPcollectionsOrThrow(pCollectionId).getWindowingStrategyId()); final WindowingStrategy<?, ?> windowingStrategy; try { return WindowingStrategyTranslation.fromProto( windowingStrategyProto, RehydratedComponents.forComponents(components)); } catch (InvalidProtocolBufferException e) { throw new IllegalStateException( String.format( "Unable to hydrate windowing strategy %s for %s.", windowingStrategyProto, pCollectionId), e); } } }
public static List<PCollectionView<?>> getSideInputs(AppliedPTransform<?, ?, ?> application) throws IOException { PTransform<?, ?> transform = application.getTransform(); if (transform instanceof ParDo.MultiOutput) { return ((ParDo.MultiOutput<?, ?>) transform).getSideInputs(); } SdkComponents sdkComponents = SdkComponents.create(application.getPipeline().getOptions()); RunnerApi.PTransform parDoProto = PTransformTranslation.toProto(application, sdkComponents); ParDoPayload payload = ParDoPayload.parseFrom(parDoProto.getSpec().getPayload()); List<PCollectionView<?>> views = new ArrayList<>(); RehydratedComponents components = RehydratedComponents.forComponents(sdkComponents.toComponents()); for (Map.Entry<String, SideInput> sideInputEntry : payload.getSideInputsMap().entrySet()) { String sideInputTag = sideInputEntry.getKey(); RunnerApi.SideInput sideInput = sideInputEntry.getValue(); PCollection<?> originalPCollection = checkNotNull( (PCollection<?>) application.getInputs().get(new TupleTag<>(sideInputTag)), "no input with tag %s", sideInputTag); views.add( PCollectionViewTranslation.viewFromProto( sideInput, sideInputTag, originalPCollection, parDoProto, components)); } return views; }
private static Coder<?> getAccumulatorCoder(AppliedPTransform<?, ?, ?> transform) throws IOException { SdkComponents sdkComponents = SdkComponents.create(transform.getPipeline().getOptions()); String id = getCombinePayload(transform, sdkComponents) .map(CombinePayload::getAccumulatorCoderId) .orElseThrow(() -> new IOException("Transform does not contain an AccumulatorCoder")); Components components = sdkComponents.toComponents(); return CoderTranslation.fromProto( components.getCodersOrThrow(id), RehydratedComponents.forComponents(components)); }
/** * Converts an {@link AppliedPTransform}, which may be a rehydrated transform or an original * {@link TestStream}, to a {@link TestStream}. */ public static <T> TestStream<T> getTestStream( AppliedPTransform<PBegin, PCollection<T>, PTransform<PBegin, PCollection<T>>> application) throws IOException { // For robustness, we don't take this shortcut: // if (application.getTransform() instanceof TestStream) { // return application.getTransform() // } SdkComponents sdkComponents = SdkComponents.create(application.getPipeline().getOptions()); RunnerApi.PTransform transformProto = PTransformTranslation.toProto(application, sdkComponents); checkArgument( TEST_STREAM_TRANSFORM_URN.equals(transformProto.getSpec().getUrn()), "Attempt to get %s from a transform with wrong URN %s", TestStream.class.getSimpleName(), transformProto.getSpec().getUrn()); RunnerApi.TestStreamPayload testStreamPayload = RunnerApi.TestStreamPayload.parseFrom(transformProto.getSpec().getPayload()); return (TestStream<T>) testStreamFromProtoPayload( testStreamPayload, RehydratedComponents.forComponents(sdkComponents.toComponents())); }
@Test public void toAndFromProto() throws Exception { SdkComponents sdkComponents = SdkComponents.create(); sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java")); RunnerApi.Coder coderProto = CoderTranslation.toProto(coder, sdkComponents); Components encodedComponents = sdkComponents.toComponents(); Coder<?> decodedCoder = CoderTranslation.fromProto( coderProto, RehydratedComponents.forComponents(encodedComponents)); assertThat(decodedCoder, equalTo(coder)); if (KNOWN_CODERS.contains(coder)) { for (RunnerApi.Coder encodedCoder : encodedComponents.getCodersMap().values()) { assertThat( encodedCoder.getSpec().getSpec().getUrn(), not(equalTo(CoderTranslation.JAVA_SERIALIZED_CODER_URN))); } } }
@Test public void testStateSpecToFromProto() throws Exception { // Encode SdkComponents sdkComponents = SdkComponents.create(); sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java")); RunnerApi.StateSpec stateSpecProto = ParDoTranslation.translateStateSpec(stateSpec, sdkComponents); // Decode RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(sdkComponents.toComponents()); StateSpec<?> deserializedStateSpec = ParDoTranslation.fromProto(stateSpecProto, rehydratedComponents); assertThat(stateSpec, equalTo(deserializedStateSpec)); } }
@Test public void testEncodedProto() throws Exception { SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); RunnerApi.TestStreamPayload payload = TestStreamTranslation.payloadForTestStream(testStream, components); verifyTestStreamEncoding( testStream, payload, RehydratedComponents.forComponents(components.toComponents())); }
@Test public void testSimpleCoder() throws Exception { SdkComponents sdkComponents = SdkComponents.create(); sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java")); Coder<?> coder = VarIntCoder.of(); String id = sdkComponents.registerCoder(coder); RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(sdkComponents.toComponents()); Coder<?> rehydratedCoder = rehydratedComponents.getCoder(id); assertThat(rehydratedCoder, equalTo((Coder) coder)); assertThat(rehydratedComponents.getCoder(id), theInstance((Coder) rehydratedCoder)); }
@Test public void getEnvironmentUnknownFnType() throws IOException { SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(components.toComponents()); PTransform builder = PTransform.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN) .build()) .build(); Optional<Environment> env = Environments.getEnvironment(builder, rehydratedComponents); assertThat(env.isPresent(), is(false)); }
@Test public void testEnvironment() { SdkComponents sdkComponents = SdkComponents.create(); sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java")); Environment env = Environments.createDockerEnvironment("java_test"); String id = sdkComponents.registerEnvironment(env); RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(sdkComponents.toComponents()); Environment rehydratedEnv = rehydratedComponents.getEnvironment(id); assertThat(rehydratedEnv, equalTo(env)); assertThat(rehydratedComponents.getEnvironment(id), theInstance(rehydratedEnv)); } }
@Test public void testEncodeDecodeFields() throws Exception { SdkComponents sdkComponents = SdkComponents.create(); sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java")); RunnerApi.PCollection protoCollection = PCollectionTranslation.toProto(testCollection, sdkComponents); RehydratedComponents protoComponents = RehydratedComponents.forComponents(sdkComponents.toComponents()); Coder<?> decodedCoder = protoComponents.getCoder(protoCollection.getCoderId()); WindowingStrategy<?, ?> decodedStrategy = protoComponents.getWindowingStrategy(protoCollection.getWindowingStrategyId()); IsBounded decodedIsBounded = PCollectionTranslation.isBounded(protoCollection); assertThat(decodedCoder, equalTo(testCollection.getCoder())); assertThat(decodedStrategy, equalTo(testCollection.getWindowingStrategy().fixDefaults())); assertThat(decodedIsBounded, equalTo(testCollection.isBounded())); }
@Test public void testEncodeDecodeCycle() throws Exception { // Encode SdkComponents sdkComponents = SdkComponents.create(); sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java")); RunnerApi.PCollection protoCollection = PCollectionTranslation.toProto(testCollection, sdkComponents); RehydratedComponents protoComponents = RehydratedComponents.forComponents(sdkComponents.toComponents()); // Decode Pipeline pipeline = Pipeline.create(); PCollection<?> decodedCollection = PCollectionTranslation.fromProto(protoCollection, pipeline, protoComponents); // Verify assertThat(decodedCollection.getCoder(), equalTo(testCollection.getCoder())); assertThat( decodedCollection.getWindowingStrategy(), equalTo(testCollection.getWindowingStrategy().fixDefaults())); assertThat(decodedCollection.isBounded(), equalTo(testCollection.isBounded())); }
@Test public void testToProtoAndBackWithComponents() throws Exception { WindowingStrategy<?, ?> windowingStrategy = toProtoAndBackSpec.getWindowingStrategy(); SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); RunnerApi.WindowingStrategy proto = WindowingStrategyTranslation.toProto(windowingStrategy, components); RehydratedComponents protoComponents = RehydratedComponents.forComponents(components.toComponents()); assertThat( WindowingStrategyTranslation.fromProto(proto, protoComponents).fixDefaults(), equalTo(windowingStrategy.fixDefaults())); protoComponents.getCoder( components.registerCoder(windowingStrategy.getWindowFn().windowCoder())); assertThat( proto.getAssignsToOneWindow(), equalTo(windowingStrategy.getWindowFn().assignsToOneWindow())); } }
@Test public void testCompoundCoder() throws Exception { SdkComponents sdkComponents = SdkComponents.create(); sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java")); Coder<?> coder = VarIntCoder.of(); Coder<?> compoundCoder = NullableCoder.of(coder); String compoundCoderId = sdkComponents.registerCoder(compoundCoder); String coderId = sdkComponents.registerCoder(coder); RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(sdkComponents.toComponents()); Coder<?> rehydratedCoder = rehydratedComponents.getCoder(coderId); Coder<?> rehydratedCompoundCoder = rehydratedComponents.getCoder(compoundCoderId); assertThat(rehydratedCoder, equalTo((Coder) coder)); assertThat(rehydratedCompoundCoder, equalTo((Coder) compoundCoder)); assertThat(rehydratedComponents.getCoder(coderId), theInstance((Coder) rehydratedCoder)); assertThat( rehydratedComponents.getCoder(compoundCoderId), theInstance((Coder) rehydratedCompoundCoder)); }
@Test public void testRegistrarEncodedProto() throws Exception { PCollection<String> output = p.apply(testStream); AppliedPTransform<PBegin, PCollection<String>, TestStream<String>> appliedTestStream = AppliedPTransform.of("fakeName", PBegin.in(p).expand(), output.expand(), testStream, p); SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); RunnerApi.FunctionSpec spec = PTransformTranslation.toProto(appliedTestStream, components).getSpec(); assertThat(spec.getUrn(), equalTo(TEST_STREAM_TRANSFORM_URN)); RunnerApi.TestStreamPayload payload = TestStreamPayload.parseFrom(spec.getPayload()); verifyTestStreamEncoding( testStream, payload, RehydratedComponents.forComponents(components.toComponents())); }
@Test public void testWindowingStrategy() throws Exception { SdkComponents sdkComponents = SdkComponents.create(); sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java")); WindowingStrategy windowingStrategy = WindowingStrategy.of(FixedWindows.of(Duration.millis(1))) .withAllowedLateness(Duration.standardSeconds(4)); String id = sdkComponents.registerWindowingStrategy(windowingStrategy); RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(sdkComponents.toComponents()); WindowingStrategy<?, ?> rehydratedStrategy = rehydratedComponents.getWindowingStrategy(id); assertThat(rehydratedStrategy, equalTo((WindowingStrategy) windowingStrategy.fixDefaults())); assertThat( rehydratedComponents.getWindowingStrategy(id), theInstance((WindowingStrategy) rehydratedStrategy)); }
@Test public void getEnvironmentRead() throws IOException { SdkComponents components = SdkComponents.create(); components.registerEnvironment(Environments.createDockerEnvironment("java")); ReadPayload payload = ReadTranslation.toProto(Read.from(CountingSource.unbounded()), components); RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(components.toComponents()); PTransform builder = PTransform.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN) .setPayload(payload.toByteString()) .build()) .build(); Environment env = Environments.getEnvironment(builder, rehydratedComponents).get(); assertThat( env, equalTo( components .toComponents() .getEnvironmentsOrThrow(payload.getSource().getEnvironmentId()))); }