private static <T> void verifyTestStreamEncoding( TestStream<T> testStream, RunnerApi.TestStreamPayload payload, RehydratedComponents protoComponents) throws Exception { // This reverse direction is only valid for Java-based coders assertThat(protoComponents.getCoder(payload.getCoderId()), equalTo(testStream.getValueCoder())); assertThat(payload.getEventsList().size(), equalTo(testStream.getEvents().size())); for (int i = 0; i < payload.getEventsList().size(); ++i) { assertThat( TestStreamTranslation.eventFromProto(payload.getEvents(i), testStream.getValueCoder()), equalTo(testStream.getEvents().get(i))); } } }
/** Produces a {@link RunnerApi.TestStreamPayload} from a {@link TestStream}. */ static <T> RunnerApi.TestStreamPayload payloadForTestStream( final TestStream<T> transform, SdkComponents components) throws IOException { List<RunnerApi.TestStreamPayload.Event> protoEvents = new ArrayList<>(); try { for (TestStream.Event<T> event : transform.getEvents()) { protoEvents.add(eventToProto(event, transform.getValueCoder())); } } catch (IOException e) { throw new RuntimeException(e); } return RunnerApi.TestStreamPayload.newBuilder() .setCoderId(components.registerCoder(transform.getValueCoder())) .addAllEvents(protoEvents) .build(); } }
/** * 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()); }
private static TestStream<?> testStreamFromProtoPayload( RunnerApi.TestStreamPayload testStreamPayload, RehydratedComponents components) throws IOException { Coder<Object> coder = (Coder<Object>) components.getCoder(testStreamPayload.getCoderId()); List<TestStream.Event<Object>> events = new ArrayList<>(); for (RunnerApi.TestStreamPayload.Event event : testStreamPayload.getEventsList()) { events.add(eventFromProto(event, coder)); } return TestStream.fromRawEvents(coder, events); }
private <T> RunnerApi.FunctionSpec translateTyped( final TestStream<T> testStream, SdkComponents components) throws IOException { return RunnerApi.FunctionSpec.newBuilder() .setUrn(TEST_STREAM_TRANSFORM_URN) .setPayload(payloadForTestStream(testStream, components).toByteString()) .build(); }