/** * 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 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())); }
private static void validateTestStream(String id, PTransform transform, Components components) throws Exception { TestStreamPayload.parseFrom(transform.getSpec().getPayload()); }