kennknowles commented on code in PR #34919: URL: https://github.com/apache/beam/pull/34919#discussion_r2093222874
########## sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunner.java: ########## @@ -0,0 +1,481 @@ +/* + * 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.Iterator; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import org.apache.beam.fn.harness.state.FnApiStateAccessor; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; +import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload; +import org.apache.beam.sdk.fn.data.FnDataReceiver; +import org.apache.beam.sdk.fn.splittabledofn.RestrictionTrackers; +import org.apache.beam.sdk.fn.splittabledofn.RestrictionTrackers.ClaimObserver; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.BundleFinalizer; +import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; +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.DoFnInvoker.DelegatingArgumentProvider; +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.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker.Progress; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.UserCodeException; +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.values.KV; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +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.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; + +/** + * A runner for the PTransform that takes restrictions and splits them, outputting + * + * <p>This is meant to consume the output of {@link SplittablePairWithRestrictionDoFnRunner}. + * + * <p>The DoFn will have type DoFn<InputT, OutputT> but this transform's Fn API input and output + * types are: + * + * <ul> + * <li>Input: {@code WindowedValue<KV<InputT, KV<RestrictionT, WatermarkEstimatorStateT>>>} + * <li>Output: {@code WindowedValue<KV<KV<InputT, KV<RestrictionT, WatermarkEstimatorStateT>>, + * Double>>} + * </ul> + * + * <p>In addition to this, it passes {@Code OutputReceiver<RestrictionT>} to the DoFn GetRestriction + * method. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class SplittableSplitAndSizeRestrictionsDoFnRunner< + InputT, RestrictionT, PositionT, WatermarkEstimatorStateT, OutputT> + implements FnApiStateAccessor.MutatingStateContext<Object, BoundedWindow> { + + /** 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() { + return ImmutableMap.<String, PTransformRunnerFactory>builder() + .put(PTransformTranslation.SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN, new Factory()) + .build(); + } + } + + static class Factory implements PTransformRunnerFactory { + + @Override + public final void addRunnerForPTransform(Context context) throws IOException { + addRunnerForSplitAndSizeRestriction(context); + } + + private <InputT, RestrictionT, PositionT, WatermarkEstimatorStateT, OutputT> + void addRunnerForSplitAndSizeRestriction(Context context) throws IOException { + SplittableSplitAndSizeRestrictionsDoFnRunner< + InputT, RestrictionT, PositionT, WatermarkEstimatorStateT, OutputT> + runner = + new SplittableSplitAndSizeRestrictionsDoFnRunner<>( + context.getPipelineOptions(), + context.getPTransform(), + context.getComponents(), + context::getPCollectionConsumer, + context.getBundleFinalizer(), + FnApiStateAccessor.Factory.factoryForPTransformContext(context)); + + context.addPCollectionConsumer( + context + .getPTransform() + .getInputsOrThrow(ParDoTranslation.getMainInputName(context.getPTransform())), + runner::processElement); + context.addTearDownFunction(runner::tearDown); + } + } + + ////////////////////////////////////////////////////////////////////////////////////////////////// + + private final boolean observesWindow; + private final PipelineOptions pipelineOptions; + + private final DoFnInvoker<InputT, OutputT> doFnInvoker; + + private final FnDataReceiver< + WindowedValue<KV<KV<InputT, KV<RestrictionT, WatermarkEstimatorStateT>>, Double>>> + mainOutputConsumer; + + private final FnApiStateAccessor<?> stateAccessor; + private final SplitRestrictionArgumentProvider mutableArgumentProvider; + + private final DoFnSchemaInformation doFnSchemaInformation; + private final Map<String, PCollectionView<?>> sideInputMapping; + + private RestrictionT currentRestriction; + private WatermarkEstimatorStateT currentWatermarkEstimatorState; + + private RestrictionTracker<RestrictionT, PositionT> currentTracker; + + /** + * If non-null, set to true after currentTracker has had a tryClaim issued on it. Used to ignore + * checkpoint split requests if no progress was made. + */ + private @Nullable AtomicBoolean currentTrackerClaimed; Review Comment: Ah I see what you mean. Done. Based on that, does it even make sense to set `currentTracker = new tracker`. We ignore it afterwards, and never react to touching it. -- 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