Abacn commented on code in PR #34883: URL: https://github.com/apache/beam/pull/34883#discussion_r2085617820
########## sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java: ########## @@ -0,0 +1,805 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.fn.harness; + +import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.ServiceLoader; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.fn.harness.HandlesSplits.SplitResult; +import org.apache.beam.fn.harness.control.BundleProgressReporter; +import org.apache.beam.fn.harness.state.FakeBeamFnStateClient; +import org.apache.beam.model.fnexecution.v1.BeamFnApi.BundleApplication; +import org.apache.beam.model.fnexecution.v1.BeamFnApi.DelayedBundleApplication; +import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.DoubleCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.fn.data.FnDataReceiver; +import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.testing.ResetDateTimeProvider; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator; +import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker.TruncateResult; +import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.transforms.windowing.SlidingWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.util.construction.graph.ProtoOverrides; +import org.apache.beam.sdk.util.construction.graph.SplittableParDoExpander; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.hamcrest.Matchers; +import org.hamcrest.collection.IsMapContaining; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.runners.Enclosed; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link SplittablePairWithRestrictionDoFnRunner}. */ +@RunWith(Enclosed.class) +@SuppressWarnings({ + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) + // TODO(https://github.com/apache/beam/issues/21230): Remove when new version of + // errorprone is released (2.11.0) + "unused" +}) +public class SplittablePairWithRestrictionDoFnRunnerTest implements Serializable { + + @RunWith(JUnit4.class) + public static class ExecutionTest implements Serializable { + @Rule public transient ResetDateTimeProvider dateTimeProvider = new ResetDateTimeProvider(); + + public static final String TEST_TRANSFORM_ID = "pTransformId"; + + private <K> org.apache.beam.sdk.util.construction.Timer<K> timerInGlobalWindow( + K userKey, Instant holdTimestamp, Instant fireTimestamp) { + return dynamicTimerInGlobalWindow(userKey, "", holdTimestamp, fireTimestamp); + } + + private <K> org.apache.beam.sdk.util.construction.Timer<K> clearedTimerInGlobalWindow( + K userKey) { + return clearedTimerInGlobalWindow(userKey, ""); + } + + private <K> org.apache.beam.sdk.util.construction.Timer<K> clearedTimerInGlobalWindow( + K userKey, String dynamicTimerTag) { + return org.apache.beam.sdk.util.construction.Timer.cleared( + userKey, dynamicTimerTag, Collections.singletonList(GlobalWindow.INSTANCE)); + } + + private <K> org.apache.beam.sdk.util.construction.Timer<K> dynamicTimerInGlobalWindow( + K userKey, String dynamicTimerTag, Instant holdTimestamp, Instant fireTimestamp) { + return org.apache.beam.sdk.util.construction.Timer.of( + userKey, + dynamicTimerTag, + Collections.singletonList(GlobalWindow.INSTANCE), + fireTimestamp, + holdTimestamp, + PaneInfo.NO_FIRING); + } + + private <T> WindowedValue<T> valueInWindows( + T value, BoundedWindow window, BoundedWindow... windows) { + return WindowedValue.of( + value, + window.maxTimestamp(), + ImmutableList.<BoundedWindow>builder().add(window).add(windows).build(), + PaneInfo.NO_FIRING); + } + + /** + * Produces an iterable side input {@link StateKey} for the test PTransform id in the global + * window. + */ + private StateKey iterableSideInputKey(String sideInputId) throws IOException { + return iterableSideInputKey( + sideInputId, + ByteString.copyFrom( + CoderUtils.encodeToByteArray(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE))); + } + + /** + * Produces an iterable side input {@link StateKey} for the test PTransform id in the supplied + * window. + */ + private StateKey iterableSideInputKey(String sideInputId, ByteString windowKey) { + return StateKey.newBuilder() + .setIterableSideInput( + StateKey.IterableSideInput.newBuilder() + .setTransformId(TEST_TRANSFORM_ID) + .setSideInputId(sideInputId) + .setWindow(windowKey)) + .build(); + } + + private ByteString encode(String... values) throws IOException { + ByteStringOutputStream out = new ByteStringOutputStream(); + for (String value : values) { + StringUtf8Coder.of().encode(value, out); + } + return out.toByteString(); + } + + @Test + public void testRegistration() { + for (PTransformRunnerFactory.Registrar registrar : + ServiceLoader.load(PTransformRunnerFactory.Registrar.class)) { + if (registrar instanceof SplittablePairWithRestrictionDoFnRunner.Registrar) { + assertThat( + registrar.getPTransformRunnerFactories(), + IsMapContaining.hasKey(PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN)); + return; + } + } + fail("Expected registrar not found."); + } + + /** + * The trySplit testing of this splittable DoFn is done when processing the {@link + * NonWindowObservingTestSplittableDoFn#SPLIT_ELEMENT}. Always checkpoints at element {@link + * NonWindowObservingTestSplittableDoFn#CHECKPOINT_UPPER_BOUND}. + * + * <p>The expected thread flow is: + * + * <ul> + * <li>splitting thread: {@link + * NonWindowObservingTestSplittableDoFn#waitForSplitElementToBeProcessed()} + * <li>process element thread: {@link + * NonWindowObservingTestSplittableDoFn#splitElementProcessed()} + * <li>splitting thread: perform try split + * <li>splitting thread: {@link NonWindowObservingTestSplittableDoFn#trySplitPerformed()} * + * <li>process element thread: {@link + * NonWindowObservingTestSplittableDoFn#waitForTrySplitPerformed()} + * </ul> + */ + static class NonWindowObservingTestSplittableDoFn extends DoFn<String, String> { + private static final ConcurrentMap<String, Latches> DOFN_INSTANCE_TO_LATCHES = + new ConcurrentHashMap<>(); + private static final long SPLIT_ELEMENT = 3; + private static final long CHECKPOINT_UPPER_BOUND = 8; + + static class Latches { + public Latches() {} + + CountDownLatch blockProcessLatch = new CountDownLatch(0); + CountDownLatch processEnteredLatch = new CountDownLatch(1); + CountDownLatch splitElementProcessedLatch = new CountDownLatch(1); + CountDownLatch trySplitPerformedLatch = new CountDownLatch(1); + AtomicBoolean abortProcessing = new AtomicBoolean(); + } + + private Latches getLatches() { + return DOFN_INSTANCE_TO_LATCHES.computeIfAbsent(this.uuid, (uuid) -> new Latches()); + } + + public void splitElementProcessed() { + getLatches().splitElementProcessedLatch.countDown(); + } + + public void waitForSplitElementToBeProcessed() throws InterruptedException { + if (!getLatches().splitElementProcessedLatch.await(30, TimeUnit.SECONDS)) { + fail("Failed to wait for trySplit to occur."); + } + } + + public void trySplitPerformed() { + getLatches().trySplitPerformedLatch.countDown(); + } + + public void waitForTrySplitPerformed() throws InterruptedException { + if (!getLatches().trySplitPerformedLatch.await(30, TimeUnit.SECONDS)) { + fail("Failed to wait for trySplit to occur."); + } + } + + // Must be called before process is invoked. Will prevent process from doing anything until + // unblockProcess is + // called. + public void setupBlockProcess() { + getLatches().blockProcessLatch = new CountDownLatch(1); + } + + public void enterProcessAndBlockIfEnabled() throws InterruptedException { + getLatches().processEnteredLatch.countDown(); + if (!getLatches().blockProcessLatch.await(30, TimeUnit.SECONDS)) { + fail("Failed to wait for unblockProcess to occur."); + } + } + + public void waitForProcessEntered() throws InterruptedException { + if (!getLatches().processEnteredLatch.await(5, TimeUnit.SECONDS)) { + fail("Failed to wait for process to begin."); + } + } + + public void unblockProcess() throws InterruptedException { + getLatches().blockProcessLatch.countDown(); + } + + public void setAbortProcessing() { + getLatches().abortProcessing.set(true); + } + + public boolean shouldAbortProcessing() { + return getLatches().abortProcessing.get(); + } + + private final String uuid; + + private NonWindowObservingTestSplittableDoFn() { + this.uuid = UUID.randomUUID().toString(); + } + + @ProcessElement + public ProcessContinuation processElement( + ProcessContext context, + RestrictionTracker<OffsetRange, Long> tracker, + ManualWatermarkEstimator<Instant> watermarkEstimator) + throws Exception { + long checkpointUpperBound = CHECKPOINT_UPPER_BOUND; + long position = tracker.currentRestriction().getFrom(); + boolean claimStatus = true; + while (!shouldAbortProcessing()) { + claimStatus = tracker.tryClaim(position); + if (!claimStatus) { + break; + } else if (position == SPLIT_ELEMENT) { + splitElementProcessed(); + waitForTrySplitPerformed(); + } + context.outputWithTimestamp( + context.element() + ":" + position, + GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(position))); + watermarkEstimator.setWatermark( + GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(position))); + position += 1L; + if (position == checkpointUpperBound) { + break; + } + } + if (!claimStatus) { + return ProcessContinuation.stop(); + } else { + return ProcessContinuation.resume().withResumeDelay(Duration.millis(54321L)); + } + } + + @GetInitialRestriction + public OffsetRange restriction(@Element String element) { + return new OffsetRange(0, Integer.parseInt(element)); + } + + @NewTracker + public RestrictionTracker<OffsetRange, Long> newTracker( + @Restriction OffsetRange restriction) { + return new OffsetRangeTracker(restriction); + } + + @SplitRestriction + public void splitRange(@Restriction OffsetRange range, OutputReceiver<OffsetRange> receiver) { + receiver.output(new OffsetRange(range.getFrom(), (range.getFrom() + range.getTo()) / 2)); + receiver.output(new OffsetRange((range.getFrom() + range.getTo()) / 2, range.getTo())); + } + + @TruncateRestriction + public TruncateResult<OffsetRange> truncateRestriction(@Restriction OffsetRange range) + throws Exception { + return TruncateResult.of(new OffsetRange(range.getFrom(), range.getTo() / 2)); + } + + @GetInitialWatermarkEstimatorState + public Instant getInitialWatermarkEstimatorState() { + return GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)); + } + + @NewWatermarkEstimator + public WatermarkEstimators.Manual newWatermarkEstimator( + @WatermarkEstimatorState Instant watermark) { + return new WatermarkEstimators.Manual(watermark); + } + } + + /** + * A window observing variant of {@link NonWindowObservingTestSplittableDoFn} which uses the + * side inputs to choose the checkpoint upper bound. + */ + static class WindowObservingTestSplittableDoFn extends NonWindowObservingTestSplittableDoFn { Review Comment: There are static classes copy-pasted from FnApiDoFnRunnerTest, contributing to most of the positive diff. Would it be preferable to simply use the one in FnApiDoFnRunnerTest or moved to a test fixture class? ########## sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java: ########## @@ -0,0 +1,384 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.fn.harness; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; +import org.apache.beam.fn.harness.state.BeamFnStateClient; +import org.apache.beam.fn.harness.state.FnApiStateAccessor; +import org.apache.beam.fn.harness.state.SideInputSpec; +import org.apache.beam.model.fnexecution.v1.BeamFnApi; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; +import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; +import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.fn.data.FnDataReceiver; +import org.apache.beam.sdk.function.ThrowingRunnable; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnSchemaInformation; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; +import org.apache.beam.sdk.transforms.reflect.DoFnSignature; +import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; +import org.apache.beam.sdk.util.UserCodeException; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.construction.PCollectionViewTranslation; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.ParDoTranslation; +import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; + +/** + * A runner for the PTransform step that invokes GetInitialRestriction and pairs it with the + * incoming element. + * + * <p>The DoFn will have type DoFn<InputT, OutputT> but this transform's Fn API type is from {@code + * WindowedValue<InputT>} to {@code WindowedValue<KV<InputT, KV<RestrictionT, + * WatermarkEstimatorStateT>>>} + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class SplittablePairWithRestrictionDoFnRunner< + InputT, WindowT extends BoundedWindow, RestrictionT, WatermarkEstimatorStateT, OutputT> { + /** A registrar which provides a factory to handle Java {@link DoFn}s. */ + @AutoService(PTransformRunnerFactory.Registrar.class) + public static class Registrar implements PTransformRunnerFactory.Registrar { + @Override + public Map<String, PTransformRunnerFactory> getPTransformRunnerFactories() { + Factory factory = new Factory(); + return ImmutableMap.<String, PTransformRunnerFactory>builder() + .put(PTransformTranslation.SPLITTABLE_PAIR_WITH_RESTRICTION_URN, factory) + .build(); + } + } + + static class Factory implements PTransformRunnerFactory { + + @Override + public final void addRunnerForPTransform(Context context) throws IOException { + // The constructor itself registers consumption + new SplittablePairWithRestrictionDoFnRunner<>( + context.getPipelineOptions(), + context.getRunnerCapabilities(), + context.getBeamFnStateClient(), + context.getPTransformId(), + context.getPTransform(), + context.getProcessBundleInstructionIdSupplier(), + context.getCacheTokensSupplier(), + context.getBundleCacheSupplier(), + context.getProcessWideCache(), + context.getComponents(), + context::addTearDownFunction, + context::getPCollectionConsumer, + context::addPCollectionConsumer); + } + } + + ////////////////////////////////////////////////////////////////////////////////////////////////// + + private final PipelineOptions pipelineOptions; + + private final FnDataReceiver< + WindowedValue<KV<InputT, KV<RestrictionT, WatermarkEstimatorStateT>>>> + mainOutputConsumer; + + private final FnApiStateAccessor<?> stateAccessor; + private final DoFnInvoker<InputT, OutputT> doFnInvoker; + private final PairWithRestrictionArgumentProvider mutableArgumentProvider; + + private final DoFnSchemaInformation doFnSchemaInformation; + private final Map<String, PCollectionView<?>> sideInputMapping; + + SplittablePairWithRestrictionDoFnRunner( + PipelineOptions pipelineOptions, + Set<String> runnerCapabilities, + BeamFnStateClient beamFnStateClient, + String pTransformId, + PTransform pTransform, + Supplier<String> processBundleInstructionId, + Supplier<List<BeamFnApi.ProcessBundleRequest.CacheToken>> cacheTokens, + Supplier<Cache<?, ?>> bundleCache, + Cache<?, ?> processWideCache, + RunnerApi.Components components, + Consumer<ThrowingRunnable> addTearDownFunction, + Function<String, FnDataReceiver<WindowedValue<?>>> getPCollectionConsumer, + BiConsumer<String, FnDataReceiver<WindowedValue<InputT>>> addPCollectionConsumer) + throws IOException { + this.pipelineOptions = pipelineOptions; + + RehydratedComponents rehydratedComponents = + RehydratedComponents.forComponents(components).withPipeline(Pipeline.create()); + ParDoPayload parDoPayload = ParDoPayload.parseFrom(pTransform.getSpec().getPayload()); + + // DoFn and metadata + DoFn<InputT, OutputT> doFn = (DoFn<InputT, OutputT>) ParDoTranslation.getDoFn(parDoPayload); + DoFnSignature doFnSignature = DoFnSignatures.signatureForDoFn(doFn); + this.doFnInvoker = DoFnInvokers.tryInvokeSetupFor(doFn, pipelineOptions); + this.doFnSchemaInformation = ParDoTranslation.getSchemaInformation(parDoPayload); + + // Main output + checkArgument( + pTransform.getOutputsMap().size() == 1, + "PairWithRestriction expects exact one output, but got: ", + pTransform.getOutputsMap().size()); + TupleTag<KV<InputT, KV<RestrictionT, WatermarkEstimatorStateT>>> mainOutputTag = + new TupleTag<>(Iterables.getOnlyElement(pTransform.getOutputsMap().keySet())); + @SuppressWarnings("rawtypes") // cannot do this multi-level cast without rawtypes + FnDataReceiver<WindowedValue<KV<InputT, KV<RestrictionT, WatermarkEstimatorStateT>>>> + mainOutputConsumer = + (FnDataReceiver) + getPCollectionConsumer.apply(pTransform.getOutputsMap().get(mainOutputTag.getId())); + this.mainOutputConsumer = mainOutputConsumer; + + // Main input + String mainInputTag = + Iterables.getOnlyElement( + Sets.difference( + pTransform.getInputsMap().keySet(), parDoPayload.getSideInputsMap().keySet())); + String mainInputName = pTransform.getInputsOrThrow(mainInputTag); + PCollection mainInput = + components.getPcollectionsMap().get(pTransform.getInputsOrThrow(mainInputTag)); + + // Side inputs + this.sideInputMapping = ParDoTranslation.getSideInputMapping(parDoPayload); + @SuppressWarnings("rawtypes") // passed to FnApiStateAccessor which uses rawtypes + ImmutableMap.Builder<TupleTag<?>, SideInputSpec> tagToSideInputSpecMapBuilder = + ImmutableMap.builder(); + for (Map.Entry<String, RunnerApi.SideInput> entry : + parDoPayload.getSideInputsMap().entrySet()) { + String sideInputTag = entry.getKey(); + RunnerApi.SideInput sideInput = entry.getValue(); + PCollection sideInputPCollection = + components.getPcollectionsMap().get(pTransform.getInputsOrThrow(sideInputTag)); + WindowingStrategy<?, ?> sideInputWindowingStrategy = + rehydratedComponents.getWindowingStrategy(sideInputPCollection.getWindowingStrategyId()); + tagToSideInputSpecMapBuilder.put( + new TupleTag<>(entry.getKey()), + SideInputSpec.create( + sideInput.getAccessPattern().getUrn(), + rehydratedComponents.getCoder(sideInputPCollection.getCoderId()), + (Coder<WindowT>) sideInputWindowingStrategy.getWindowFn().windowCoder(), + PCollectionViewTranslation.viewFnFromProto(entry.getValue().getViewFn()), + (WindowMappingFn<WindowT>) + PCollectionViewTranslation.windowMappingFnFromProto( + entry.getValue().getWindowMappingFn()))); + } + @SuppressWarnings("rawtypes") // passed to FnApiStateAccessor which uses rawtypes + Map<TupleTag<?>, SideInputSpec> tagToSideInputSpecMap = tagToSideInputSpecMapBuilder.build(); + + // Register processing methods + if (doFnSignature.getInitialRestriction().observesWindow() || !sideInputMapping.isEmpty()) { + addPCollectionConsumer.accept( + mainInputName, this::processElementForWindowObservingPairWithRestriction); + this.mutableArgumentProvider = new WindowObservingProcessBundleContext(); + } else { + addPCollectionConsumer.accept(mainInputName, this::processElementForPairWithRestriction); + this.mutableArgumentProvider = new NonWindowObservingProcessBundleContext(); + } + addTearDownFunction.accept(this::tearDown); Review Comment: Not directly related to this PR, but notice that it does something for DoFn.teardown here, but there is a long standing issue DoFn.teardown isn't honored in Dataflow runner (both legacy and v2): go/beam-issue/18592 . Wondering if there is any gap in SDK harness -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: github-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org