pipeline.replaceAll( Collections.singletonList( PTransformOverride.of(new OriginalMatcher(), new ReplacementOverrideFactory())));
@Test @Category(NeedsRunner.class) public void testReadToImpulseOverride() { BoundedSource<Long> source = CountingSource.upTo(10L); // Use an explicit read transform to ensure the override is exercised. PCollection<Long> input = p.apply(Read.from(source)); PAssert.that(input).containsInAnyOrder(0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L); p.replaceAll(Collections.singletonList(JavaReadViaImpulse.boundedOverride())); p.traverseTopologically( new Pipeline.PipelineVisitor() { @Override public void enterPipeline(Pipeline p) {} @Override public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { assertNotReadTransform(node.getTransform()); return CompositeBehavior.ENTER_TRANSFORM; } @Override public void leaveCompositeTransform(TransformHierarchy.Node node) {} @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { assertNotReadTransform(node.getTransform()); } @Override public void visitValue(PValue value, TransformHierarchy.Node producer) {} @Override public void leavePipeline(Pipeline pipeline) {} }); p.run(); }
/** * Tests that {@link Pipeline#replaceAll(List)} throws when one of the PTransformOverride still * matches. */ @Test public void testReplaceAllIncomplete() { pipeline.enableAbandonedNodeEnforcement(false); pipeline.apply(GenerateSequence.from(0)); // The order is such that the output of the second will match the first, which is not permitted thrown.expect(IllegalStateException.class); pipeline.replaceAll( ImmutableList.of( PTransformOverride.of( application -> application.getTransform() instanceof Create.Values, new CreateValuesToEmptyFlattenOverride()), PTransformOverride.of( application -> application.getTransform() instanceof GenerateSequence, new GenerateSequenceToCreateOverride()))); }
@Test public void getViewsReturnsViews() { PCollectionView<List<String>> listView = p.apply("listCreate", Create.of("foo", "bar")) .apply( ParDo.of( new DoFn<String, String>() { @ProcessElement public void processElement(DoFn<String, String>.ProcessContext c) throws Exception { c.output(Integer.toString(c.element().length())); } })) .apply(View.asList()); PCollectionView<Object> singletonView = p.apply("singletonCreate", Create.<Object>of(1, 2, 3)).apply(View.asSingleton()); p.replaceAll( DirectRunner.fromOptions(TestPipeline.testingPipelineOptions()) .defaultTransformOverrides()); p.traverseTopologically(visitor); assertThat(visitor.getGraph().getViews(), Matchers.containsInAnyOrder(listView, singletonView)); }
@Test public void testReplaceAll() { pipeline.enableAbandonedNodeEnforcement(false); pipeline.apply("unbounded", GenerateSequence.from(0)); pipeline.apply("bounded", GenerateSequence.from(0).to(100)); pipeline.replaceAll( ImmutableList.of( PTransformOverride.of( application -> application.getTransform() instanceof GenerateSequence, new GenerateSequenceToCreateOverride()), PTransformOverride.of( application -> application.getTransform() instanceof Create.Values, new CreateValuesToEmptyFlattenOverride()))); pipeline.traverseTopologically( new PipelineVisitor.Defaults() { @Override public CompositeBehavior enterCompositeTransform(Node node) { if (!node.isRootNode()) { assertThat( node.getTransform().getClass(), not( anyOf( Matchers.equalTo(GenerateSequence.class), Matchers.equalTo(Create.Values.class)))); } return CompositeBehavior.ENTER_TRANSFORM; } }); }
@Before public void setup() { DirectRunner runner = DirectRunner.fromOptions(PipelineOptionsFactory.create()); created = p.apply(Create.of(1, 2, 3)); downstream = created.apply(WithKeys.of("foo")); view = created.apply(View.asIterable()); unbounded = p.apply(GenerateSequence.from(0)); p.replaceAll(runner.defaultTransformOverrides()); KeyedPValueTrackingVisitor keyedPValueTrackingVisitor = KeyedPValueTrackingVisitor.create(); p.traverseTopologically(keyedPValueTrackingVisitor); BundleFactory bundleFactory = ImmutableListBundleFactory.create(); DirectGraphs.performDirectOverrides(p); graph = DirectGraphs.getGraph(p); context = EvaluationContext.create( NanosOffsetClock.create(), bundleFactory, graph, keyedPValueTrackingVisitor.getKeyedPValues(), Executors.newSingleThreadExecutor()); createdProducer = graph.getProducer(created); downstreamProducer = graph.getProducer(downstream); viewProducer = graph.getProducer(view); unboundedProducer = graph.getProducer(unbounded); }