ryucc commented on code in PR #26475: URL: https://github.com/apache/beam/pull/26475#discussion_r1186341900
########## runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableBundleManager.java: ########## @@ -0,0 +1,201 @@ +/* + * 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.runners.samza.runtime; + +import org.apache.beam.runners.core.StateNamespaces; +import org.apache.beam.runners.core.TimerInternals; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.samza.operators.Scheduler; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Bundle management for the {@link DoFnOp} that handles lifecycle of a bundle. It also serves as a + * proxy for the {@link DoFnOp} to process watermark and decides to 1. Hold watermark if there is at + * least one bundle in progress. 2. Propagates the watermark to downstream DAG, if all the previous + * bundles have completed. + * + * <p>This class is not thread safe and the current implementation relies on the assumption that + * messages are dispatched to BundleManager in a single threaded mode. + * + * @param <OutT> output type of the {@link DoFnOp} + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class PortableBundleManager<OutT> implements BundleManager<OutT> { + private static final Logger LOG = LoggerFactory.getLogger(PortableBundleManager.class); + private static final long MIN_BUNDLE_CHECK_TIME_MS = 10L; + + private final long maxBundleSize; + private final long maxBundleTimeMs; + private final BundleProgressListener<OutT> bundleProgressListener; + private final Scheduler<KeyedTimerData<Void>> bundleTimerScheduler; + private final String bundleCheckTimerId; + + // Number elements belonging to the current active bundle + private transient long currentBundleElementCount; + // Number of bundles that are in progress but not yet finished + private transient long pendingBundleCount; + // Denotes the start time of the current active bundle + private transient long bundleStartTime; + // Denotes if there is an active in progress bundle. Note at a given time, we can have multiple + // bundle in progress. + // This flag denotes if there is a bundle that is current and hasn't been closed. + private transient boolean isBundleStarted; + // Holder for watermark which gets propagated when the bundle is finished. + private transient Instant bundleWatermarkHold; Review Comment: I'll remove it. Doesn't look like this class can be serialized. (or not needed at the moment.) I was copying the classic version, but the Scheduler and Listener are probably not serializable as well. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
