/** * 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); }
private static PTransformMatcher boundedMatcher() { return PTransformMatchers.urnEqualTo(PTransformTranslation.READ_TRANSFORM_URN) .and( transform -> ReadTranslation.sourceIsBounded(transform) == PCollection.IsBounded.BOUNDED); }
private void checkForMatches(Node node) { for (PTransformOverride override : overrides) { if (override .getMatcher() .matchesDuringValidation(node.toAppliedPTransform(getPipeline()))) { matched.put(node, override); } } } });
@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); } } });
@Override public CompositeBehavior enterCompositeTransform(Node node) { if (!node.isRootNode() && freedNodes.contains(node.getEnclosingNode())) { // This node will be freed because its parent will be freed. freedNodes.add(node); return CompositeBehavior.ENTER_TRANSFORM; } if (!node.isRootNode() && override.getMatcher().matches(node.toAppliedPTransform(getPipeline()))) { matches.add(node); // This node will be freed. When we visit any of its children, they will also be freed freedNodes.add(node); } return CompositeBehavior.ENTER_TRANSFORM; }
/** 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 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)); }
@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 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 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 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 classEqualToDoesNotMatchSubclass() { class MyPTransform extends PTransform<PCollection<KV<String, Integer>>, PCollection<Integer>> { @Override public PCollection<Integer> expand(PCollection<KV<String, Integer>> input) { return PCollection.createPrimitiveOutputInternal( input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), VarIntCoder.of()); } } PTransformMatcher matcher = PTransformMatchers.classEqualTo(MyPTransform.class); MyPTransform subclass = new MyPTransform() {}; assertThat(subclass.getClass(), not(Matchers.<Class<?>>equalTo(MyPTransform.class))); assertThat(subclass, instanceOf(MyPTransform.class)); AppliedPTransform<?, ?, ?> application = getAppliedTransform(subclass); assertThat(matcher.matches(application), is(false)); }
/** * Test significant cases that the matcher should not match against. In this case, this explicitly * tests that any {@link Combine.GroupedValues} with side inputs should not match. */ @Test public void combineValuesWithoutSideInputsSkipsNonmatching() { PTransformMatcher matcher = new DataflowPTransformMatchers.CombineValuesWithoutSideInputsPTransformMatcher(); AppliedPTransform<?, ?, ?> groupedValues; groupedValues = getCombineGroupedValuesFrom(createCombineGroupedValuesWithSideInputsPipeline()); assertThat(matcher.matches(groupedValues), is(false)); groupedValues = getCombineGroupedValuesFrom(createCombinePerKeyWithSideInputsPipeline()); assertThat(matcher.matches(groupedValues), is(false)); }
/** * Test significant cases that the matcher should not match against. In this case, this tests that * any {@link Combine.GroupedValues} with side inputs should not match, and that a {@link * Combine.GroupedValues} without an encompassing {@link Combine.PerKey} will not match. */ @Test public void combineValuesWithParentCheckSkipsNonmatching() { PTransformMatcher matcher = new DataflowPTransformMatchers.CombineValuesWithParentCheckPTransformMatcher(); AppliedPTransform<?, ?, ?> groupedValues; groupedValues = getCombineGroupedValuesFrom(createCombineGroupedValuesPipeline()); assertThat(matcher.matches(groupedValues), is(false)); groupedValues = getCombineGroupedValuesFrom(createCombineGroupedValuesWithSideInputsPipeline()); assertThat(matcher.matches(groupedValues), is(false)); groupedValues = getCombineGroupedValuesFrom(createCombinePerKeyWithSideInputsPipeline()); assertThat(matcher.matches(groupedValues), is(false)); }
/** * Test the cases that the matcher should successfully match against. In this case, it should * match against {@link Combine.GroupedValues} on their own and as part of an expanded {@link * Combine.PerKey} transform. */ @Test public void combineValuesWithoutSideInputsSuccessfulMatches() { PTransformMatcher matcher = new DataflowPTransformMatchers.CombineValuesWithoutSideInputsPTransformMatcher(); AppliedPTransform<?, ?, ?> groupedValues; groupedValues = getCombineGroupedValuesFrom(createCombineGroupedValuesPipeline()); assertThat(matcher.matches(groupedValues), is(true)); groupedValues = getCombineGroupedValuesFrom(createCombinePerKeyPipeline()); assertThat(matcher.matches(groupedValues), is(true)); }