aromanenko-dev commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r949194118


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.sdk.io.sparkreceiver;
+
+import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
+
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.range.OffsetRange;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.UnboundedPerElement;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+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.WatermarkEstimator;
+import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.spark.SparkConf;
+import org.apache.spark.streaming.receiver.Receiver;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A SplittableDoFn which reads from {@link Receiver} that implements {@link 
HasOffset}. By default,
+ * a {@link WatermarkEstimators.Manual} watermark estimator is used to track 
watermark.
+ *
+ * <p>Initial range The initial range is {@code [0, Long.MAX_VALUE)}
+ *
+ * <p>Resume Processing Every time the sparkConsumer.hasRecords() returns 
false, {@link
+ * ReadFromSparkReceiverWithOffsetDoFn} will move to process the next element.
+ */
+@UnboundedPerElement
+public class ReadFromSparkReceiverWithOffsetDoFn<V> extends DoFn<byte[], V> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ReadFromSparkReceiverWithOffsetDoFn.class);
+
+  /** Constant waiting time after the {@link Receiver} starts. Required to 
prepare for polling */
+  private static final int START_POLL_TIMEOUT_MS = 1000;
+
+  private final SerializableFunction<Instant, WatermarkEstimator<Instant>>
+      createWatermarkEstimatorFn;
+  private final SerializableFunction<V, Long> getOffsetFn;
+  private final SerializableFunction<V, Instant> getWatermarkFn;
+  private final ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder;
+
+  public ReadFromSparkReceiverWithOffsetDoFn(SparkReceiverIO.Read<V> 
transform) {
+    createWatermarkEstimatorFn = WatermarkEstimators.Manual::new;
+
+    ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder =
+        transform.getSparkReceiverBuilder();
+    checkStateNotNull(sparkReceiverBuilder, "Spark Receiver Builder can't be 
null!");
+    this.sparkReceiverBuilder = sparkReceiverBuilder;
+
+    SerializableFunction<V, Long> getOffsetFn = transform.getGetOffsetFn();
+    checkStateNotNull(getOffsetFn, "Get offset fn can't be null!");
+    this.getOffsetFn = getOffsetFn;
+
+    SerializableFunction<V, Instant> getWatermarkFn = 
transform.getWatermarkFn();
+    checkStateNotNull(getWatermarkFn, "Watermark fn can't be null!");
+    this.getWatermarkFn = getWatermarkFn;
+  }
+
+  @GetInitialRestriction
+  public OffsetRange initialRestriction(@Element byte[] element) {
+    return new OffsetRange(0, Long.MAX_VALUE);
+  }
+
+  @GetInitialWatermarkEstimatorState
+  public Instant getInitialWatermarkEstimatorState(@Timestamp Instant 
currentElementTimestamp) {
+    return currentElementTimestamp;
+  }
+
+  @NewWatermarkEstimator
+  public WatermarkEstimator<Instant> newWatermarkEstimator(
+      @WatermarkEstimatorState Instant watermarkEstimatorState) {
+    return 
createWatermarkEstimatorFn.apply(ensureTimestampWithinBounds(watermarkEstimatorState));
+  }
+
+  @GetSize
+  public double getSize(@Element byte[] element, @Restriction OffsetRange 
offsetRange) {
+    return restrictionTracker(element, 
offsetRange).getProgress().getWorkRemaining();
+    // Before processing elements, we don't have a good estimated size of 
records and offset gap.
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  @Setup
+  public void setup() throws Exception {}
+
+  @Teardown
+  public void teardown() throws Exception {}
+
+  @SuppressWarnings("unchecked")

Review Comment:
   Yes, please. 



-- 
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]

Reply via email to