/** * An {@link AppliedPTransform} matched by a {@link PTransformMatcher} will be replaced during * pipeline surgery, and is often expected to be gone the new pipeline. For the {@link * AppliedPTransform} that is expected to remain in the pipeline after surgery, the corresponding * {@link PTransformMatcher} should override this method, such that it will not be matched during * the validation. */ default boolean matchesDuringValidation(AppliedPTransform<?, ?, ?> application) { return matches(application); }
@Override public void visitPrimitiveTransform(Node node) { if (freedNodes.contains(node.getEnclosingNode())) { freedNodes.add(node); } else if (override.getMatcher().matches(node.toAppliedPTransform(getPipeline()))) { matches.add(node); freedNodes.add(node); } } });
@Test public void parDoSingleWithState() { AppliedPTransform<?, ?, ?> parDoApplication = getAppliedTransform(ParDo.of(doFnWithState)); assertThat(PTransformMatchers.stateOrTimerParDoSingle().matches(parDoApplication), is(true)); assertThat(PTransformMatchers.splittableParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.splittableParDoSingle().matches(parDoApplication), is(false)); }
/** Demonstrates that a {@link ParDo.SingleOutput} does not match any ParDo matcher. */ @Test public void parDoSingle() { AppliedPTransform<?, ?, ?> parDoApplication = getAppliedTransform(ParDo.of(doFn)); assertThat(PTransformMatchers.splittableParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.splittableParDoSingle().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoSingle().matches(parDoApplication), is(false)); }
@Test public void parDoSingleSplittable() { AppliedPTransform<?, ?, ?> parDoApplication = getAppliedTransform(ParDo.of(splittableDoFn)); assertThat(PTransformMatchers.splittableParDoSingle().matches(parDoApplication), is(true)); assertThat(PTransformMatchers.splittableParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoSingle().matches(parDoApplication), is(false)); }
@Test public void parDoSingleWithTimers() { AppliedPTransform<?, ?, ?> parDoApplication = getAppliedTransform(ParDo.of(doFnWithTimers)); assertThat(PTransformMatchers.stateOrTimerParDoSingle().matches(parDoApplication), is(true)); assertThat(PTransformMatchers.splittableParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.splittableParDoSingle().matches(parDoApplication), is(false)); }
@Test public void parDoMulti() { AppliedPTransform<?, ?, ?> parDoApplication = getAppliedTransform(ParDo.of(doFn).withOutputTags(new TupleTag<>(), TupleTagList.empty())); assertThat(PTransformMatchers.splittableParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.splittableParDoSingle().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoSingle().matches(parDoApplication), is(false)); }
@Test public void parDoMultiWithState() { AppliedPTransform<?, ?, ?> parDoApplication = getAppliedTransform( ParDo.of(doFnWithState).withOutputTags(new TupleTag<>(), TupleTagList.empty())); assertThat(PTransformMatchers.stateOrTimerParDoMulti().matches(parDoApplication), is(true)); assertThat(PTransformMatchers.splittableParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.splittableParDoSingle().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoSingle().matches(parDoApplication), is(false)); }
@Test public void parDoMultiSplittable() { AppliedPTransform<?, ?, ?> parDoApplication = getAppliedTransform( ParDo.of(splittableDoFn).withOutputTags(new TupleTag<>(), TupleTagList.empty())); assertThat(PTransformMatchers.splittableParDoMulti().matches(parDoApplication), is(true)); assertThat(PTransformMatchers.stateOrTimerParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.splittableParDoSingle().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoSingle().matches(parDoApplication), is(false)); }
@Test public void parDoMultiWithTimers() { AppliedPTransform<?, ?, ?> parDoApplication = getAppliedTransform( ParDo.of(doFnWithTimers).withOutputTags(new TupleTag<>(), TupleTagList.empty())); assertThat(PTransformMatchers.stateOrTimerParDoMulti().matches(parDoApplication), is(true)); assertThat(PTransformMatchers.splittableParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.splittableParDoSingle().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoSingle().matches(parDoApplication), is(false)); }
@Test public void parDoSplittable() { AppliedPTransform<?, ?, ?> parDoApplication = getAppliedTransform( ParDo.of(splittableDoFn).withOutputTags(new TupleTag<>(), TupleTagList.empty())); assertThat(PTransformMatchers.splittableParDo().matches(parDoApplication), is(true)); assertThat(PTransformMatchers.stateOrTimerParDoMulti().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.splittableParDoSingle().matches(parDoApplication), is(false)); assertThat(PTransformMatchers.stateOrTimerParDoSingle().matches(parDoApplication), is(false)); }
@Test public void parDoWithFnTypeNotParDo() { AppliedPTransform<?, ?, ?> notParDo = getAppliedTransform(Create.empty(VoidCoder.of())); PTransformMatcher matcher = PTransformMatchers.parDoWithFnType(doFnWithState.getClass()); assertThat(matcher.matches(notParDo), is(false)); }
@Test public void classEqualToMatchesSameClass() { PTransformMatcher matcher = PTransformMatchers.classEqualTo(ParDo.SingleOutput.class); AppliedPTransform<?, ?, ?> application = getAppliedTransform( ParDo.of( new DoFn<KV<String, Integer>, Integer>() { @ProcessElement public void doStuff(ProcessContext ctxt) {} })); assertThat(matcher.matches(application), is(true)); }
/** * Test the cases that the matcher should successfully match against. In this case, it should * match against {@link Combine.GroupedValues} that are part of an expanded {@link Combine.PerKey} * transform. */ @Test public void combineValuesWithParentCheckSuccessfulMatches() { PTransformMatcher matcher = new DataflowPTransformMatchers.CombineValuesWithParentCheckPTransformMatcher(); AppliedPTransform<?, ?, ?> groupedValues; groupedValues = getCombineGroupedValuesFrom(createCombinePerKeyPipeline()); assertThat(matcher.matches(groupedValues), is(true)); }
@Test public void classEqualToDoesNotMatchUnrelatedClass() { PTransformMatcher matcher = PTransformMatchers.classEqualTo(ParDo.SingleOutput.class); AppliedPTransform<?, ?, ?> application = getAppliedTransform(Window.<KV<String, Integer>>into(new GlobalWindows())); assertThat(matcher.matches(application), is(false)); }
@Test public void parDoWithState() { AppliedPTransform<?, ?, ?> statefulApplication = getAppliedTransform( ParDo.of(doFnWithState).withOutputTags(new TupleTag<>(), TupleTagList.empty())); assertThat(PTransformMatchers.stateOrTimerParDo().matches(statefulApplication), is(true)); AppliedPTransform<?, ?, ?> splittableApplication = getAppliedTransform( ParDo.of(splittableDoFn).withOutputTags(new TupleTag<>(), TupleTagList.empty())); assertThat(PTransformMatchers.stateOrTimerParDo().matches(splittableApplication), is(false)); }
@Test public void createViewWithViewFnDifferentViewFn() { PCollection<Integer> input = p.apply(Create.of(1)); PCollectionView<Iterable<Integer>> view = input.apply(View.asIterable()); // Purposely create a subclass to get a different class then what was expected. ViewFn<?, ?> viewFn = new PCollectionViews.IterableViewFn() {}; CreatePCollectionView<?, ?> createView = CreatePCollectionView.of(view); PTransformMatcher matcher = PTransformMatchers.createViewWithViewFn(viewFn.getClass()); assertThat(matcher.matches(getAppliedTransform(createView)), is(false)); }
@Test public void createViewWithViewFn() { PCollection<Integer> input = p.apply(Create.of(1)); PCollectionView<Iterable<Integer>> view = input.apply(View.asIterable()); ViewFn<?, ?> viewFn = view.getViewFn(); CreatePCollectionView<?, ?> createView = CreatePCollectionView.of(view); PTransformMatcher matcher = PTransformMatchers.createViewWithViewFn(viewFn.getClass()); assertThat(matcher.matches(getAppliedTransform(createView)), is(true)); }
@Test public void createViewWithViewFnNotCreatePCollectionView() { PCollection<Integer> input = p.apply(Create.of(1)); PCollectionView<Iterable<Integer>> view = input.apply(View.asIterable()); PTransformMatcher matcher = PTransformMatchers.createViewWithViewFn(view.getViewFn().getClass()); assertThat(matcher.matches(getAppliedTransform(View.asIterable())), is(false)); }