You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/06/03 10:06:21 UTC

[GitHub] [beam] Amar3tto opened a new pull request, #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Amar3tto opened a new pull request, #17828:
URL: https://github.com/apache/beam/pull/17828

   ![SparkReceiverViaSDF drawio (2)](https://user-images.githubusercontent.com/17278361/171834141-ce5d753a-ca37-4715-95b3-e08591e30d97.png)
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Add a link to the appropriate issue in your description, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1191067651

   Run Java PreCommit


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1157355902

   Run Java PreCommit


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1186799689

   Run SQL_Java17 PreCommit


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1192203425

   Run Java PreCommit


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aromanenko-dev commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
aromanenko-dev commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1217804946

   Please, avoid `merge` commits and use `git rebase` instead


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] chamikaramj commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1220712731

   Ack. Will take a look. Thanks @aromanenko-dev 


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r971690946


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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
+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> getTimestampFn;
+  private final ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder;
+
+  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> getTimestampFn = transform.getTimestampFn();
+    if (getTimestampFn == null) {
+      getTimestampFn = input -> Instant.now();
+    }
+    this.getTimestampFn = getTimestampFn;
+  }
+
+  @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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+        throw new IllegalStateException("Spark Receiver was not initialized");
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("SparkReceiver was interrupted before polling started", e);
+        throw new IllegalStateException("Spark Receiver was interrupted before polling started");
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (sparkReceiver != null) {
+        sparkReceiver.stop("SparkReceiver is stopped.");
+      }
+      recordsQueue.clear();
+    }
+  }
+
+  @ProcessElement
+  public ProcessContinuation processElement(
+      @Element byte[] element,
+      RestrictionTracker<OffsetRange, Long> tracker,
+      WatermarkEstimator<Instant> watermarkEstimator,
+      OutputReceiver<V> receiver) {
+
+    SparkConsumer<V> sparkConsumer;
+    Receiver<V> sparkReceiver;
+    try {
+      sparkReceiver = sparkReceiverBuilder.build();
+    } catch (Exception e) {
+      LOG.error("Can not build Spark Receiver", e);
+      throw new IllegalStateException("Spark Receiver was not built!");
+    }
+    sparkConsumer = new SparkConsumerWithOffset<>(tracker.currentRestriction().getFrom());
+    sparkConsumer.start(sparkReceiver);
+
+    while (sparkConsumer.hasRecords()) {
+      V record = sparkConsumer.poll();
+      if (record != null) {
+        Long offset = getOffsetFn.apply(record);
+        if (!tracker.tryClaim(offset)) {

Review Comment:
   Start reading from the correct position ("seek") implemented here:
   `sparkConsumer = new SparkConsumerWithOffset<>(tracker.currentRestriction().getFrom());`



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r950150524


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/** Streaming sources for Spark {@link Receiver}. */
+public class SparkReceiverIO {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkReceiverIO.class);
+
+  public static <V> Read<V> read() {
+    return new AutoValue_SparkReceiverIO_Read.Builder<V>().build();
+  }
+
+  /** A {@link PTransform} to read from Spark {@link Receiver}. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<V> extends PTransform<PBegin, PCollection<V>> {
+
+    abstract @Nullable ReceiverBuilder<V, ? extends Receiver<V>> getSparkReceiverBuilder();
+
+    abstract @Nullable Class<V> getValueClass();
+
+    abstract @Nullable SerializableFunction<V, Long> getGetOffsetFn();
+
+    abstract @Nullable SerializableFunction<V, Instant> getWatermarkFn();
+
+    abstract Builder<V> toBuilder();
+
+    @Experimental(Experimental.Kind.PORTABILITY)

Review Comment:
   No, removed annotation



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r949019150


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/** Streaming sources for Spark {@link Receiver}. */

Review Comment:
   We expect to add more detailed documentation and README in a separate PR.



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aromanenko-dev commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
aromanenko-dev commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r949173195


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/** Streaming sources for Spark {@link Receiver}. */

Review Comment:
   Is there any reason why not to add it along with this PR? Seems logical for me...



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Lizzfox commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Lizzfox commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r958733870


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/**
+ * Streaming sources for Spark {@link Receiver}.
+ *
+ * <h3>Reading using {@link SparkReceiverIO}</h3>
+ *
+ * <p>You will need to pass a {@link ReceiverBuilder} which is responsible for instantiating new
+ * {@link Receiver} objects.
+ *
+ * <p>{@link Receiver} that will be used should implement {@link HasOffset} interface. You will need
+ * to pass {@code getOffsetFn} which is a {@link SerializableFunction} that defines how to get
+ * {@code Long offset} from {@code V record}.
+ *
+ * <p>Optionally you can pass {@code watermarkFn} which is a {@link SerializableFunction} that
+ * defines how to get {@code Instant watermark} from {@code V record}.
+ *
+ * <p>Example of {@link SparkReceiverIO#read()} usage:
+ *
+ * <pre>{@code
+ * Pipeline p = ...; // Create pipeline.
+ *
+ * // Create ReceiverBuilder for CustomReceiver
+ * ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+ *         new ReceiverBuilder<>(CustomReceiver.class).withConstructorArgs();
+ *
+ * //Read from CustomReceiver
+ * p.apply("Spark Receiver Read",
+ *  SparkReceiverIO.Read<String> reader =
+ *    SparkReceiverIO.<String>read()
+ *      .withGetOffsetFn(Long::valueOf)
+ *      .withWatermarkFn(Instant::parse)
+ *      .withSparkReceiverBuilder(receiverBuilder);
+ * }</pre>
+ */
+public class SparkReceiverIO {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkReceiverIO.class);
+
+  public static <V> Read<V> read() {
+    return new AutoValue_SparkReceiverIO_Read.Builder<V>().build();
+  }
+
+  /** A {@link PTransform} to read from Spark {@link Receiver}. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<V> extends PTransform<PBegin, PCollection<V>> {
+
+    abstract @Nullable ReceiverBuilder<V, ? extends Receiver<V>> getSparkReceiverBuilder();
+
+    abstract @Nullable SerializableFunction<V, Long> getGetOffsetFn();
+
+    abstract @Nullable SerializableFunction<V, Instant> getWatermarkFn();
+
+    abstract Builder<V> toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder<V> {
+
+      abstract Builder<V> setSparkReceiverBuilder(
+          ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder);
+
+      abstract Builder<V> setGetOffsetFn(SerializableFunction<V, Long> getOffsetFn);
+
+      abstract Builder<V> setWatermarkFn(SerializableFunction<V, Instant> watermarkFn);
+
+      abstract Read<V> build();
+    }
+
+    /** Sets {@link ReceiverBuilder} with value and custom Spark {@link Receiver} class. */
+    public Read<V> withSparkReceiverBuilder(
+        ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder) {
+      checkArgument(sparkReceiverBuilder != null, "Spark receiver builder can not be null");
+      return toBuilder().setSparkReceiverBuilder(sparkReceiverBuilder).build();
+    }
+
+    /** A function to get offset in order to start {@link Receiver} from it. */
+    public Read<V> withGetOffsetFn(SerializableFunction<V, Long> getOffsetFn) {
+      checkArgument(getOffsetFn != null, "Get offset function can not be null");
+      return toBuilder().setGetOffsetFn(getOffsetFn).build();
+    }
+
+    /** A function to calculate watermark after a record. */
+    public Read<V> withWatermarkFn(SerializableFunction<V, Instant> watermarkFn) {
+      checkArgument(watermarkFn != null, "Watermark function can not be null");
+      return toBuilder().setWatermarkFn(watermarkFn).build();
+    }
+
+    @Override
+    public PCollection<V> expand(PBegin input) {
+      validateTransform();
+      return input.apply(new ReadFromSparkReceiverViaSdf<>(this));
+    }
+
+    public void validateTransform() {
+      ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder = getSparkReceiverBuilder();
+      checkStateNotNull(sparkReceiverBuilder, "withSparkReceiverBuilder() is required");
+      checkStateNotNull(getGetOffsetFn(), "withGetOffsetFn() is required");
+    }
+  }
+
+  static class ReadFromSparkReceiverViaSdf<V> extends PTransform<PBegin, PCollection<V>> {
+
+    private final Read<V> sparkReceiverRead;
+
+    ReadFromSparkReceiverViaSdf(Read<V> sparkReceiverRead) {
+      this.sparkReceiverRead = sparkReceiverRead;
+    }
+
+    @Override
+    public PCollection<V> expand(PBegin input) {
+      final ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder =
+          sparkReceiverRead.getSparkReceiverBuilder();
+      checkStateNotNull(sparkReceiverBuilder, "withSparkReceiverBuilder() is required");
+      if (!HasOffset.class.isAssignableFrom(sparkReceiverBuilder.getSparkReceiverClass())) {
+        throw new UnsupportedOperationException(
+            String.format(
+                "Given Spark Receiver class %s doesn't implement HasOffset interface,"
+                    + " therefore it is not supported!",
+                sparkReceiverBuilder.getSparkReceiverClass().getName()));
+      } else {
+        LOG.info("{} started reading", ReadFromSparkReceiverWithOffsetDoFn.class.getSimpleName());
+        return input
+            .apply(Impulse.create())
+            .apply(ParDo.of(new ReadFromSparkReceiverWithOffsetDoFn<>(sparkReceiverRead)));

Review Comment:
   Could you please merge this implementation to start with, to unblock the work on PRs with Spark Recevier integration tests and integration with Cdap IO?
   While we haven't found a way to split the Spark receivers, we continue to work in this direction



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aromanenko-dev commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
aromanenko-dev commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r950085123


##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.util.List;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test class for {@link SparkReceiverIO}. */
+@RunWith(JUnit4.class)
+public class SparkReceiverIOTest {
+
+  public static final TestPipelineOptions OPTIONS =

Review Comment:
   What is a reason for that?



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r949017467


##########
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")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    public SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("Interrupted", e);
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (sparkReceiver != null) {
+        sparkReceiver.stop("Stopped");

Review Comment:
   Done



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r954753500


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+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;
+
+  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();
+    if (getWatermarkFn == null) {
+      getWatermarkFn = input -> Instant.now();
+    }
+    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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;

Review Comment:
   Yes it was intended, because it is `SerializableFunction<?, Void>` and it should return something. The common way is to return null here.



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r954777841


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+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;
+
+  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();
+    if (getWatermarkFn == null) {
+      getWatermarkFn = input -> Instant.now();
+    }
+    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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);

Review Comment:
   Done



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+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;
+
+  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();
+    if (getWatermarkFn == null) {
+      getWatermarkFn = input -> Instant.now();
+    }
+    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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("SparkReceiver was interrupted before polling started", e);

Review Comment:
   Done



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1210952239

   @aromanenko-dev Kind ping on this. 
   Did you have time to review this PR?


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r971694255


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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
+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> getTimestampFn;
+  private final ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder;
+
+  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> getTimestampFn = transform.getTimestampFn();
+    if (getTimestampFn == null) {
+      getTimestampFn = input -> Instant.now();
+    }
+    this.getTimestampFn = getTimestampFn;
+  }
+
+  @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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+        throw new IllegalStateException("Spark Receiver was not initialized");
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("SparkReceiver was interrupted before polling started", e);
+        throw new IllegalStateException("Spark Receiver was interrupted before polling started");
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (sparkReceiver != null) {
+        sparkReceiver.stop("SparkReceiver is stopped.");
+      }
+      recordsQueue.clear();
+    }
+  }
+
+  @ProcessElement
+  public ProcessContinuation processElement(
+      @Element byte[] element,
+      RestrictionTracker<OffsetRange, Long> tracker,
+      WatermarkEstimator<Instant> watermarkEstimator,
+      OutputReceiver<V> receiver) {
+
+    SparkConsumer<V> sparkConsumer;
+    Receiver<V> sparkReceiver;
+    try {
+      sparkReceiver = sparkReceiverBuilder.build();
+    } catch (Exception e) {
+      LOG.error("Can not build Spark Receiver", e);
+      throw new IllegalStateException("Spark Receiver was not built!");
+    }
+    sparkConsumer = new SparkConsumerWithOffset<>(tracker.currentRestriction().getFrom());
+    sparkConsumer.start(sparkReceiver);
+
+    while (sparkConsumer.hasRecords()) {
+      V record = sparkConsumer.poll();
+      if (record != null) {
+        Long offset = getOffsetFn.apply(record);
+        if (!tracker.tryClaim(offset)) {
+          sparkConsumer.stop();
+          LOG.debug("Stop for restriction: {}", tracker.currentRestriction().toString());

Review Comment:
   Discussed at the meeting  that an additional ack is not needed, since Spark Consumer is created inside DoFn. As soon as element is processed it’s deleted from queue (not from the source).



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aromanenko-dev commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
aromanenko-dev commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r948150634


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/** Streaming sources for Spark {@link Receiver}. */

Review Comment:
   Please, add more details about this connector and examples of how to use it.



##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.util.List;
+import org.apache.beam.runners.direct.DirectOptions;
+import org.apache.beam.runners.direct.DirectRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test class for {@link SparkReceiverIO}. */
+@RunWith(JUnit4.class)
+public class SparkReceiverIOTest {
+
+  @Test
+  public void testReadBuildsCorrectly() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SerializableFunction<String, Long> offsetFn = Long::valueOf;
+    SerializableFunction<String, Instant> watermarkFn = Instant::parse;
+
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read()
+            .withValueClass(String.class)
+            .withGetOffsetFn(offsetFn)
+            .withWatermarkFn(watermarkFn)
+            .withSparkReceiverBuilder(receiverBuilder);
+
+    assertEquals(offsetFn, read.getGetOffsetFn());
+    assertEquals(receiverBuilder, read.getSparkReceiverBuilder());
+    assertEquals(String.class, read.getValueClass());
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfReceiverBuilderIsNull() {
+    assertThrows(
+        IllegalArgumentException.class,
+        () -> SparkReceiverIO.<String>read().withSparkReceiverBuilder(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfGetOffsetFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withGetOffsetFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfWatermarkFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withWatermarkFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfValueClassIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withValueClass(null));
+  }
+
+  @Test
+  public void testReadValidationFailsMissingReceiverBuilder() {
+    SparkReceiverIO.Read<String> read = SparkReceiverIO.read();
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadValidationFailsMissingSparkConsumer() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read().withSparkReceiverBuilder(receiverBuilder);
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadFromCustomReceiverWithOffset() {
+    DirectOptions options = PipelineOptionsFactory.as(DirectOptions.class);
+    options.setBlockOnRun(false);
+    options.setRunner(DirectRunner.class);

Review Comment:
   No need to specify a runner - unit tests are running by default on DirectRunner



##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.util.List;
+import org.apache.beam.runners.direct.DirectOptions;
+import org.apache.beam.runners.direct.DirectRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test class for {@link SparkReceiverIO}. */
+@RunWith(JUnit4.class)
+public class SparkReceiverIOTest {
+
+  @Test
+  public void testReadBuildsCorrectly() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SerializableFunction<String, Long> offsetFn = Long::valueOf;
+    SerializableFunction<String, Instant> watermarkFn = Instant::parse;
+
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read()
+            .withValueClass(String.class)
+            .withGetOffsetFn(offsetFn)
+            .withWatermarkFn(watermarkFn)
+            .withSparkReceiverBuilder(receiverBuilder);
+
+    assertEquals(offsetFn, read.getGetOffsetFn());
+    assertEquals(receiverBuilder, read.getSparkReceiverBuilder());
+    assertEquals(String.class, read.getValueClass());
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfReceiverBuilderIsNull() {
+    assertThrows(
+        IllegalArgumentException.class,
+        () -> SparkReceiverIO.<String>read().withSparkReceiverBuilder(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfGetOffsetFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withGetOffsetFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfWatermarkFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withWatermarkFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfValueClassIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withValueClass(null));
+  }
+
+  @Test
+  public void testReadValidationFailsMissingReceiverBuilder() {
+    SparkReceiverIO.Read<String> read = SparkReceiverIO.read();
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadValidationFailsMissingSparkConsumer() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read().withSparkReceiverBuilder(receiverBuilder);
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadFromCustomReceiverWithOffset() {
+    DirectOptions options = PipelineOptionsFactory.as(DirectOptions.class);
+    options.setBlockOnRun(false);
+    options.setRunner(DirectRunner.class);
+    Pipeline p = Pipeline.create(options);

Review Comment:
   Use `TestPipeline` for tests (see examples for other IO connectors).



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/** Streaming sources for Spark {@link Receiver}. */
+public class SparkReceiverIO {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkReceiverIO.class);
+
+  public static <V> Read<V> read() {
+    return new AutoValue_SparkReceiverIO_Read.Builder<V>().build();
+  }
+
+  /** A {@link PTransform} to read from Spark {@link Receiver}. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<V> extends PTransform<PBegin, PCollection<V>> {
+
+    abstract @Nullable ReceiverBuilder<V, ? extends Receiver<V>> getSparkReceiverBuilder();
+
+    abstract @Nullable Class<V> getValueClass();
+
+    abstract @Nullable SerializableFunction<V, Long> getGetOffsetFn();
+
+    abstract @Nullable SerializableFunction<V, Instant> getWatermarkFn();
+
+    abstract Builder<V> toBuilder();
+
+    @Experimental(Experimental.Kind.PORTABILITY)

Review Comment:
   Does this connector support portability?



##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.util.List;
+import org.apache.beam.runners.direct.DirectOptions;
+import org.apache.beam.runners.direct.DirectRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test class for {@link SparkReceiverIO}. */
+@RunWith(JUnit4.class)
+public class SparkReceiverIOTest {
+
+  @Test
+  public void testReadBuildsCorrectly() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SerializableFunction<String, Long> offsetFn = Long::valueOf;
+    SerializableFunction<String, Instant> watermarkFn = Instant::parse;
+
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read()
+            .withValueClass(String.class)
+            .withGetOffsetFn(offsetFn)
+            .withWatermarkFn(watermarkFn)
+            .withSparkReceiverBuilder(receiverBuilder);
+
+    assertEquals(offsetFn, read.getGetOffsetFn());
+    assertEquals(receiverBuilder, read.getSparkReceiverBuilder());
+    assertEquals(String.class, read.getValueClass());
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfReceiverBuilderIsNull() {
+    assertThrows(
+        IllegalArgumentException.class,
+        () -> SparkReceiverIO.<String>read().withSparkReceiverBuilder(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfGetOffsetFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withGetOffsetFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfWatermarkFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withWatermarkFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfValueClassIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withValueClass(null));
+  }
+
+  @Test
+  public void testReadValidationFailsMissingReceiverBuilder() {
+    SparkReceiverIO.Read<String> read = SparkReceiverIO.read();
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadValidationFailsMissingSparkConsumer() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read().withSparkReceiverBuilder(receiverBuilder);
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadFromCustomReceiverWithOffset() {
+    DirectOptions options = PipelineOptionsFactory.as(DirectOptions.class);
+    options.setBlockOnRun(false);
+    options.setRunner(DirectRunner.class);
+    Pipeline p = Pipeline.create(options);
+
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SparkReceiverIO.Read<String> reader =
+        SparkReceiverIO.<String>read()
+            .withValueClass(String.class)
+            .withGetOffsetFn(Long::valueOf)
+            .withWatermarkFn(Instant::parse)
+            .withSparkReceiverBuilder(receiverBuilder);
+
+    List<String> storedRecords = CustomReceiverWithOffset.getStoredRecords();
+    List<String> outputRecords = TestOutputDoFn.getRecords();
+    outputRecords.clear();
+
+    p.apply(reader).setCoder(StringUtf8Coder.of()).apply(ParDo.of(new TestOutputDoFn()));
+    p.run().waitUntilFinish(Duration.standardSeconds(15));
+
+    assertEquals(outputRecords, storedRecords);

Review Comment:
   Use 
   ```
   PCollection<String> output = p.apply(...)...
   
   PAssert.that(output).containsInAnyOrder(expectedRecords);
   ```



##########
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 {}

Review Comment:
   No need to specify the empty methods.



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkConsumer.java:
##########
@@ -0,0 +1,38 @@
+/*
+ * 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 java.io.Serializable;
+import org.apache.spark.streaming.receiver.Receiver;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Interface for start/stop reading from some Spark {@link Receiver} into some place and poll from
+ * it.
+ */
+public interface SparkConsumer<V> extends Serializable {

Review Comment:
   Do we need this `public`?



##########
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:
   Better to avoid `@SuppressWarnings` if possible



##########
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")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    public SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("Interrupted", e);
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (sparkReceiver != null) {
+        sparkReceiver.stop("Stopped");

Review Comment:
   nit: `SparkReceiver is stopped.`



##########
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")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    public SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("Interrupted", e);

Review Comment:
   Please, add more detailed error message, like `SparkReceiver was interrupted because of <>`.



##########
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> {

Review Comment:
   Do we need this  class and its methods be `public`?



##########
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")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    public SparkConsumerWithOffset(Long startOffset) {

Review Comment:
   Please, restrict methods visibility where it's possible.



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r949016993


##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.util.List;
+import org.apache.beam.runners.direct.DirectOptions;
+import org.apache.beam.runners.direct.DirectRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test class for {@link SparkReceiverIO}. */
+@RunWith(JUnit4.class)
+public class SparkReceiverIOTest {
+
+  @Test
+  public void testReadBuildsCorrectly() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SerializableFunction<String, Long> offsetFn = Long::valueOf;
+    SerializableFunction<String, Instant> watermarkFn = Instant::parse;
+
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read()
+            .withValueClass(String.class)
+            .withGetOffsetFn(offsetFn)
+            .withWatermarkFn(watermarkFn)
+            .withSparkReceiverBuilder(receiverBuilder);
+
+    assertEquals(offsetFn, read.getGetOffsetFn());
+    assertEquals(receiverBuilder, read.getSparkReceiverBuilder());
+    assertEquals(String.class, read.getValueClass());
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfReceiverBuilderIsNull() {
+    assertThrows(
+        IllegalArgumentException.class,
+        () -> SparkReceiverIO.<String>read().withSparkReceiverBuilder(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfGetOffsetFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withGetOffsetFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfWatermarkFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withWatermarkFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfValueClassIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withValueClass(null));
+  }
+
+  @Test
+  public void testReadValidationFailsMissingReceiverBuilder() {
+    SparkReceiverIO.Read<String> read = SparkReceiverIO.read();
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadValidationFailsMissingSparkConsumer() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read().withSparkReceiverBuilder(receiverBuilder);
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadFromCustomReceiverWithOffset() {
+    DirectOptions options = PipelineOptionsFactory.as(DirectOptions.class);
+    options.setBlockOnRun(false);
+    options.setRunner(DirectRunner.class);

Review Comment:
   Removed



##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.util.List;
+import org.apache.beam.runners.direct.DirectOptions;
+import org.apache.beam.runners.direct.DirectRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test class for {@link SparkReceiverIO}. */
+@RunWith(JUnit4.class)
+public class SparkReceiverIOTest {
+
+  @Test
+  public void testReadBuildsCorrectly() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SerializableFunction<String, Long> offsetFn = Long::valueOf;
+    SerializableFunction<String, Instant> watermarkFn = Instant::parse;
+
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read()
+            .withValueClass(String.class)
+            .withGetOffsetFn(offsetFn)
+            .withWatermarkFn(watermarkFn)
+            .withSparkReceiverBuilder(receiverBuilder);
+
+    assertEquals(offsetFn, read.getGetOffsetFn());
+    assertEquals(receiverBuilder, read.getSparkReceiverBuilder());
+    assertEquals(String.class, read.getValueClass());
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfReceiverBuilderIsNull() {
+    assertThrows(
+        IllegalArgumentException.class,
+        () -> SparkReceiverIO.<String>read().withSparkReceiverBuilder(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfGetOffsetFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withGetOffsetFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfWatermarkFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withWatermarkFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfValueClassIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withValueClass(null));
+  }
+
+  @Test
+  public void testReadValidationFailsMissingReceiverBuilder() {
+    SparkReceiverIO.Read<String> read = SparkReceiverIO.read();
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadValidationFailsMissingSparkConsumer() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read().withSparkReceiverBuilder(receiverBuilder);
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadFromCustomReceiverWithOffset() {
+    DirectOptions options = PipelineOptionsFactory.as(DirectOptions.class);
+    options.setBlockOnRun(false);
+    options.setRunner(DirectRunner.class);
+    Pipeline p = Pipeline.create(options);

Review Comment:
   Done



##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.util.List;
+import org.apache.beam.runners.direct.DirectOptions;
+import org.apache.beam.runners.direct.DirectRunner;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test class for {@link SparkReceiverIO}. */
+@RunWith(JUnit4.class)
+public class SparkReceiverIOTest {
+
+  @Test
+  public void testReadBuildsCorrectly() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SerializableFunction<String, Long> offsetFn = Long::valueOf;
+    SerializableFunction<String, Instant> watermarkFn = Instant::parse;
+
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read()
+            .withValueClass(String.class)
+            .withGetOffsetFn(offsetFn)
+            .withWatermarkFn(watermarkFn)
+            .withSparkReceiverBuilder(receiverBuilder);
+
+    assertEquals(offsetFn, read.getGetOffsetFn());
+    assertEquals(receiverBuilder, read.getSparkReceiverBuilder());
+    assertEquals(String.class, read.getValueClass());
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfReceiverBuilderIsNull() {
+    assertThrows(
+        IllegalArgumentException.class,
+        () -> SparkReceiverIO.<String>read().withSparkReceiverBuilder(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfGetOffsetFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withGetOffsetFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfWatermarkFnIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withWatermarkFn(null));
+  }
+
+  @Test
+  public void testReadObjectCreationFailsIfValueClassIsNull() {
+    assertThrows(
+        IllegalArgumentException.class, () -> SparkReceiverIO.<String>read().withValueClass(null));
+  }
+
+  @Test
+  public void testReadValidationFailsMissingReceiverBuilder() {
+    SparkReceiverIO.Read<String> read = SparkReceiverIO.read();
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadValidationFailsMissingSparkConsumer() {
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SparkReceiverIO.Read<String> read =
+        SparkReceiverIO.<String>read().withSparkReceiverBuilder(receiverBuilder);
+    assertThrows(IllegalStateException.class, read::validateTransform);
+  }
+
+  @Test
+  public void testReadFromCustomReceiverWithOffset() {
+    DirectOptions options = PipelineOptionsFactory.as(DirectOptions.class);
+    options.setBlockOnRun(false);
+    options.setRunner(DirectRunner.class);
+    Pipeline p = Pipeline.create(options);
+
+    ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+        new ReceiverBuilder<>(CustomReceiverWithOffset.class).withConstructorArgs();
+    SparkReceiverIO.Read<String> reader =
+        SparkReceiverIO.<String>read()
+            .withValueClass(String.class)
+            .withGetOffsetFn(Long::valueOf)
+            .withWatermarkFn(Instant::parse)
+            .withSparkReceiverBuilder(receiverBuilder);
+
+    List<String> storedRecords = CustomReceiverWithOffset.getStoredRecords();
+    List<String> outputRecords = TestOutputDoFn.getRecords();
+    outputRecords.clear();
+
+    p.apply(reader).setCoder(StringUtf8Coder.of()).apply(ParDo.of(new TestOutputDoFn()));
+    p.run().waitUntilFinish(Duration.standardSeconds(15));
+
+    assertEquals(outputRecords, storedRecords);

Review Comment:
   Done



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aromanenko-dev commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r950119782


##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.util.List;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test class for {@link SparkReceiverIO}. */
+@RunWith(JUnit4.class)
+public class SparkReceiverIOTest {
+
+  public static final TestPipelineOptions OPTIONS =

Review Comment:
   We use `PipelineResult.waitUntilFinish()` following the example of others IO tests, and setBlockOnRun(false) is required in this case.



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] chamikaramj commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r951479298


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+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;
+
+  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();
+    if (getWatermarkFn == null) {
+      getWatermarkFn = input -> Instant.now();
+    }
+    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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("SparkReceiver was interrupted before polling started", e);
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (sparkReceiver != null) {
+        sparkReceiver.stop("SparkReceiver is stopped.");
+      }
+      recordsQueue.clear();

Review Comment:
   What happens to existing records in the queue ?



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+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;
+
+  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();
+    if (getWatermarkFn == null) {
+      getWatermarkFn = input -> Instant.now();
+    }
+    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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);

Review Comment:
   We should be raising an exception here (or retry) instead of logging ?



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/**
+ * Streaming sources for Spark {@link Receiver}.
+ *
+ * <h3>Reading using {@link SparkReceiverIO}</h3>
+ *
+ * <p>You will need to pass a {@link ReceiverBuilder} which is responsible for instantiating new
+ * {@link Receiver} objects.
+ *
+ * <p>{@link Receiver} that will be used should implement {@link HasOffset} interface. You will need
+ * to pass {@code getOffsetFn} which is a {@link SerializableFunction} that defines how to get
+ * {@code Long offset} from {@code V record}.
+ *
+ * <p>Optionally you can pass {@code watermarkFn} which is a {@link SerializableFunction} that
+ * defines how to get {@code Instant watermark} from {@code V record}.
+ *
+ * <p>Example of {@link SparkReceiverIO#read()} usage:
+ *
+ * <pre>{@code
+ * Pipeline p = ...; // Create pipeline.
+ *
+ * // Create ReceiverBuilder for CustomReceiver
+ * ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+ *         new ReceiverBuilder<>(CustomReceiver.class).withConstructorArgs();
+ *
+ * //Read from CustomReceiver
+ * p.apply("Spark Receiver Read",
+ *  SparkReceiverIO.Read<String> reader =
+ *    SparkReceiverIO.<String>read()
+ *      .withGetOffsetFn(Long::valueOf)
+ *      .withWatermarkFn(Instant::parse)
+ *      .withSparkReceiverBuilder(receiverBuilder);
+ * }</pre>
+ */
+public class SparkReceiverIO {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkReceiverIO.class);
+
+  public static <V> Read<V> read() {
+    return new AutoValue_SparkReceiverIO_Read.Builder<V>().build();
+  }
+
+  /** A {@link PTransform} to read from Spark {@link Receiver}. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<V> extends PTransform<PBegin, PCollection<V>> {
+
+    abstract @Nullable ReceiverBuilder<V, ? extends Receiver<V>> getSparkReceiverBuilder();
+
+    abstract @Nullable SerializableFunction<V, Long> getGetOffsetFn();
+
+    abstract @Nullable SerializableFunction<V, Instant> getWatermarkFn();
+
+    abstract Builder<V> toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder<V> {
+
+      abstract Builder<V> setSparkReceiverBuilder(
+          ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder);
+
+      abstract Builder<V> setGetOffsetFn(SerializableFunction<V, Long> getOffsetFn);
+
+      abstract Builder<V> setWatermarkFn(SerializableFunction<V, Instant> watermarkFn);
+
+      abstract Read<V> build();
+    }
+
+    /** Sets {@link ReceiverBuilder} with value and custom Spark {@link Receiver} class. */
+    public Read<V> withSparkReceiverBuilder(
+        ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder) {
+      checkArgument(sparkReceiverBuilder != null, "Spark receiver builder can not be null");
+      return toBuilder().setSparkReceiverBuilder(sparkReceiverBuilder).build();
+    }
+
+    /** A function to get offset in order to start {@link Receiver} from it. */
+    public Read<V> withGetOffsetFn(SerializableFunction<V, Long> getOffsetFn) {
+      checkArgument(getOffsetFn != null, "Get offset function can not be null");
+      return toBuilder().setGetOffsetFn(getOffsetFn).build();
+    }
+
+    /** A function to calculate watermark after a record. */
+    public Read<V> withWatermarkFn(SerializableFunction<V, Instant> watermarkFn) {

Review Comment:
   Probably should be renamed to getTimestampFn or similar (and also update the Javadoc).



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/**
+ * Streaming sources for Spark {@link Receiver}.
+ *
+ * <h3>Reading using {@link SparkReceiverIO}</h3>
+ *
+ * <p>You will need to pass a {@link ReceiverBuilder} which is responsible for instantiating new
+ * {@link Receiver} objects.
+ *
+ * <p>{@link Receiver} that will be used should implement {@link HasOffset} interface. You will need
+ * to pass {@code getOffsetFn} which is a {@link SerializableFunction} that defines how to get
+ * {@code Long offset} from {@code V record}.
+ *
+ * <p>Optionally you can pass {@code watermarkFn} which is a {@link SerializableFunction} that
+ * defines how to get {@code Instant watermark} from {@code V record}.
+ *
+ * <p>Example of {@link SparkReceiverIO#read()} usage:
+ *
+ * <pre>{@code
+ * Pipeline p = ...; // Create pipeline.
+ *
+ * // Create ReceiverBuilder for CustomReceiver
+ * ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+ *         new ReceiverBuilder<>(CustomReceiver.class).withConstructorArgs();
+ *
+ * //Read from CustomReceiver
+ * p.apply("Spark Receiver Read",
+ *  SparkReceiverIO.Read<String> reader =
+ *    SparkReceiverIO.<String>read()
+ *      .withGetOffsetFn(Long::valueOf)
+ *      .withWatermarkFn(Instant::parse)
+ *      .withSparkReceiverBuilder(receiverBuilder);
+ * }</pre>
+ */
+public class SparkReceiverIO {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkReceiverIO.class);
+
+  public static <V> Read<V> read() {
+    return new AutoValue_SparkReceiverIO_Read.Builder<V>().build();
+  }
+
+  /** A {@link PTransform} to read from Spark {@link Receiver}. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<V> extends PTransform<PBegin, PCollection<V>> {
+
+    abstract @Nullable ReceiverBuilder<V, ? extends Receiver<V>> getSparkReceiverBuilder();
+
+    abstract @Nullable SerializableFunction<V, Long> getGetOffsetFn();
+
+    abstract @Nullable SerializableFunction<V, Instant> getWatermarkFn();
+
+    abstract Builder<V> toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder<V> {
+
+      abstract Builder<V> setSparkReceiverBuilder(
+          ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder);
+
+      abstract Builder<V> setGetOffsetFn(SerializableFunction<V, Long> getOffsetFn);
+
+      abstract Builder<V> setWatermarkFn(SerializableFunction<V, Instant> watermarkFn);
+
+      abstract Read<V> build();
+    }
+
+    /** Sets {@link ReceiverBuilder} with value and custom Spark {@link Receiver} class. */
+    public Read<V> withSparkReceiverBuilder(
+        ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder) {
+      checkArgument(sparkReceiverBuilder != null, "Spark receiver builder can not be null");
+      return toBuilder().setSparkReceiverBuilder(sparkReceiverBuilder).build();
+    }
+
+    /** A function to get offset in order to start {@link Receiver} from it. */
+    public Read<V> withGetOffsetFn(SerializableFunction<V, Long> getOffsetFn) {
+      checkArgument(getOffsetFn != null, "Get offset function can not be null");
+      return toBuilder().setGetOffsetFn(getOffsetFn).build();
+    }
+
+    /** A function to calculate watermark after a record. */
+    public Read<V> withWatermarkFn(SerializableFunction<V, Instant> watermarkFn) {
+      checkArgument(watermarkFn != null, "Watermark function can not be null");
+      return toBuilder().setWatermarkFn(watermarkFn).build();
+    }
+
+    @Override
+    public PCollection<V> expand(PBegin input) {
+      validateTransform();
+      return input.apply(new ReadFromSparkReceiverViaSdf<>(this));
+    }
+
+    public void validateTransform() {
+      ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder = getSparkReceiverBuilder();
+      checkStateNotNull(sparkReceiverBuilder, "withSparkReceiverBuilder() is required");
+      checkStateNotNull(getGetOffsetFn(), "withGetOffsetFn() is required");
+    }
+  }
+
+  static class ReadFromSparkReceiverViaSdf<V> extends PTransform<PBegin, PCollection<V>> {
+
+    private final Read<V> sparkReceiverRead;
+
+    ReadFromSparkReceiverViaSdf(Read<V> sparkReceiverRead) {
+      this.sparkReceiverRead = sparkReceiverRead;
+    }
+
+    @Override
+    public PCollection<V> expand(PBegin input) {
+      final ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder =
+          sparkReceiverRead.getSparkReceiverBuilder();
+      checkStateNotNull(sparkReceiverBuilder, "withSparkReceiverBuilder() is required");
+      if (!HasOffset.class.isAssignableFrom(sparkReceiverBuilder.getSparkReceiverClass())) {
+        throw new UnsupportedOperationException(
+            String.format(
+                "Given Spark Receiver class %s doesn't implement HasOffset interface,"
+                    + " therefore it is not supported!",
+                sparkReceiverBuilder.getSparkReceiverClass().getName()));
+      } else {
+        LOG.info("{} started reading", ReadFromSparkReceiverWithOffsetDoFn.class.getSimpleName());
+        return input
+            .apply(Impulse.create())
+            .apply(ParDo.of(new ReadFromSparkReceiverWithOffsetDoFn<>(sparkReceiverRead)));

Review Comment:
   Seems like we don't do any kind of splitting here which will limit the reading to a single worker. Is it possible to split data from SparkReceiver into multiple workers (for example, Kafka does this by creating splits for different partitions).



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+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;
+
+  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();
+    if (getWatermarkFn == null) {
+      getWatermarkFn = input -> Instant.now();
+    }
+    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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;

Review Comment:
   "return null" was intended here ?



##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/CustomReceiverWithOffset.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.receiver.Receiver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Imitation of Spark {@link Receiver} that implements {@link HasOffset} interface. Used to test
+ * {@link SparkReceiverIO#read()}.
+ */
+public class CustomReceiverWithOffset extends Receiver<String> implements HasOffset {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CustomReceiverWithOffset.class);
+  private static final int TIMEOUT_MS = 500;
+  private static final List<String> STORED_RECORDS = new ArrayList<>();
+  private static final int RECORDS_COUNT = 20;
+  private Long startOffset;
+
+  CustomReceiverWithOffset() {
+    super(StorageLevel.MEMORY_AND_DISK_2());
+  }
+
+  @Override
+  public void setStartOffset(Long startOffset) {
+    if (startOffset != null) {
+      this.startOffset = startOffset;
+    }
+  }
+
+  @Override
+  @SuppressWarnings("FutureReturnValueIgnored")
+  public void onStart() {
+    Executors.newSingleThreadExecutor(new ThreadFactoryBuilder().build()).submit(this::receive);
+  }
+
+  @Override
+  public void onStop() {}
+
+  @Override
+  public Long getEndOffset() {
+    return Long.MAX_VALUE;
+  }
+
+  private void receive() {
+    Long currentOffset = startOffset;
+    while (!isStopped()) {

Review Comment:
   This operation of the test does not seem to be thread safe currently  (source might read while a batch is being loaded).



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+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;
+
+  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();

Review Comment:
   This probably should be renamed to timestampFn since it basically provides the timestamp for a given record.



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+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;
+
+  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();
+    if (getWatermarkFn == null) {
+      getWatermarkFn = input -> Instant.now();
+    }
+    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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("SparkReceiver was interrupted before polling started", e);

Review Comment:
   Ditto.



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Abacn commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1263773594

   New test flake seen on SparkReceiverIO test: #23449


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aromanenko-dev commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
aromanenko-dev commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r949191944


##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.util.List;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test class for {@link SparkReceiverIO}. */
+@RunWith(JUnit4.class)
+public class SparkReceiverIOTest {
+
+  public static final TestPipelineOptions OPTIONS =

Review Comment:
   Do you really need `PipelineOptions`? If not then it would be better create a pipeline in this way:
   ```
   @Rule public TestPipeline readPipeline = TestPipeline.create();
   ```



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Lizzfox commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Lizzfox commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1251103756

   @chamikaramj 
   All your comments have been answered. The PR is ready to be merged.
   Thank you.


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aromanenko-dev commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
aromanenko-dev commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1201482012

   Thanks!
   I'm sorry for a delay with review, I'll try to take a look as soon as I can.


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Lizzfox commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Lizzfox commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1203912927

   Run Java CdapIO Performance Test
   


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r954750796


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+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;
+
+  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();
+    if (getWatermarkFn == null) {
+      getWatermarkFn = input -> Instant.now();
+    }
+    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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("SparkReceiver was interrupted before polling started", e);
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (sparkReceiver != null) {
+        sparkReceiver.stop("SparkReceiver is stopped.");
+      }
+      recordsQueue.clear();

Review Comment:
   At this moment all records for the current restriction should be already read from the queue.
   Here we are stopping the Receiver and clearing the queue because it is not needed anymore.



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] chamikaramj commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r959038624


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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
+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> getTimestampFn;
+  private final ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder;
+
+  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> getTimestampFn = transform.getTimestampFn();
+    if (getTimestampFn == null) {
+      getTimestampFn = input -> Instant.now();
+    }
+    this.getTimestampFn = getTimestampFn;
+  }
+
+  @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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+        throw new IllegalStateException("Spark Receiver was not initialized");
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("SparkReceiver was interrupted before polling started", e);
+        throw new IllegalStateException("Spark Receiver was interrupted before polling started");
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (sparkReceiver != null) {
+        sparkReceiver.stop("SparkReceiver is stopped.");
+      }
+      recordsQueue.clear();
+    }
+  }
+
+  @ProcessElement
+  public ProcessContinuation processElement(
+      @Element byte[] element,
+      RestrictionTracker<OffsetRange, Long> tracker,
+      WatermarkEstimator<Instant> watermarkEstimator,
+      OutputReceiver<V> receiver) {
+
+    SparkConsumer<V> sparkConsumer;
+    Receiver<V> sparkReceiver;
+    try {
+      sparkReceiver = sparkReceiverBuilder.build();
+    } catch (Exception e) {
+      LOG.error("Can not build Spark Receiver", e);
+      throw new IllegalStateException("Spark Receiver was not built!");
+    }
+    sparkConsumer = new SparkConsumerWithOffset<>(tracker.currentRestriction().getFrom());
+    sparkConsumer.start(sparkReceiver);
+
+    while (sparkConsumer.hasRecords()) {
+      V record = sparkConsumer.poll();
+      if (record != null) {
+        Long offset = getOffsetFn.apply(record);
+        if (!tracker.tryClaim(offset)) {

Review Comment:
   When the runner resume an element from a checkpoint, how will the source start reading from the correct position ?
   
   I think there should be logic here to seek to the correct position when "currentRestriction().getFrom()" is not zero. 
   
   Note that this has to be a seek (not read all elements from the beginning) for reading to be efficient. 



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/**
+ * Streaming sources for Spark {@link Receiver}.
+ *
+ * <h3>Reading using {@link SparkReceiverIO}</h3>
+ *
+ * <p>You will need to pass a {@link ReceiverBuilder} which is responsible for instantiating new
+ * {@link Receiver} objects.
+ *
+ * <p>{@link Receiver} that will be used should implement {@link HasOffset} interface. You will need
+ * to pass {@code getOffsetFn} which is a {@link SerializableFunction} that defines how to get
+ * {@code Long offset} from {@code V record}.
+ *
+ * <p>Optionally you can pass {@code watermarkFn} which is a {@link SerializableFunction} that
+ * defines how to get {@code Instant watermark} from {@code V record}.
+ *
+ * <p>Example of {@link SparkReceiverIO#read()} usage:
+ *
+ * <pre>{@code
+ * Pipeline p = ...; // Create pipeline.
+ *
+ * // Create ReceiverBuilder for CustomReceiver
+ * ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+ *         new ReceiverBuilder<>(CustomReceiver.class).withConstructorArgs();
+ *
+ * //Read from CustomReceiver
+ * p.apply("Spark Receiver Read",
+ *  SparkReceiverIO.Read<String> reader =
+ *    SparkReceiverIO.<String>read()
+ *      .withGetOffsetFn(Long::valueOf)
+ *      .withWatermarkFn(Instant::parse)
+ *      .withSparkReceiverBuilder(receiverBuilder);
+ * }</pre>
+ */
+public class SparkReceiverIO {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkReceiverIO.class);
+
+  public static <V> Read<V> read() {
+    return new AutoValue_SparkReceiverIO_Read.Builder<V>().build();
+  }
+
+  /** A {@link PTransform} to read from Spark {@link Receiver}. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<V> extends PTransform<PBegin, PCollection<V>> {
+
+    abstract @Nullable ReceiverBuilder<V, ? extends Receiver<V>> getSparkReceiverBuilder();
+
+    abstract @Nullable SerializableFunction<V, Long> getGetOffsetFn();
+
+    abstract @Nullable SerializableFunction<V, Instant> getWatermarkFn();
+
+    abstract Builder<V> toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder<V> {
+
+      abstract Builder<V> setSparkReceiverBuilder(
+          ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder);
+
+      abstract Builder<V> setGetOffsetFn(SerializableFunction<V, Long> getOffsetFn);
+
+      abstract Builder<V> setWatermarkFn(SerializableFunction<V, Instant> watermarkFn);
+
+      abstract Read<V> build();
+    }
+
+    /** Sets {@link ReceiverBuilder} with value and custom Spark {@link Receiver} class. */
+    public Read<V> withSparkReceiverBuilder(
+        ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder) {
+      checkArgument(sparkReceiverBuilder != null, "Spark receiver builder can not be null");
+      return toBuilder().setSparkReceiverBuilder(sparkReceiverBuilder).build();
+    }
+
+    /** A function to get offset in order to start {@link Receiver} from it. */
+    public Read<V> withGetOffsetFn(SerializableFunction<V, Long> getOffsetFn) {
+      checkArgument(getOffsetFn != null, "Get offset function can not be null");
+      return toBuilder().setGetOffsetFn(getOffsetFn).build();
+    }
+
+    /** A function to calculate watermark after a record. */
+    public Read<V> withWatermarkFn(SerializableFunction<V, Instant> watermarkFn) {
+      checkArgument(watermarkFn != null, "Watermark function can not be null");
+      return toBuilder().setWatermarkFn(watermarkFn).build();
+    }
+
+    @Override
+    public PCollection<V> expand(PBegin input) {
+      validateTransform();
+      return input.apply(new ReadFromSparkReceiverViaSdf<>(this));
+    }
+
+    public void validateTransform() {
+      ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder = getSparkReceiverBuilder();
+      checkStateNotNull(sparkReceiverBuilder, "withSparkReceiverBuilder() is required");
+      checkStateNotNull(getGetOffsetFn(), "withGetOffsetFn() is required");
+    }
+  }
+
+  static class ReadFromSparkReceiverViaSdf<V> extends PTransform<PBegin, PCollection<V>> {
+
+    private final Read<V> sparkReceiverRead;
+
+    ReadFromSparkReceiverViaSdf(Read<V> sparkReceiverRead) {
+      this.sparkReceiverRead = sparkReceiverRead;
+    }
+
+    @Override
+    public PCollection<V> expand(PBegin input) {
+      final ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder =
+          sparkReceiverRead.getSparkReceiverBuilder();
+      checkStateNotNull(sparkReceiverBuilder, "withSparkReceiverBuilder() is required");
+      if (!HasOffset.class.isAssignableFrom(sparkReceiverBuilder.getSparkReceiverClass())) {
+        throw new UnsupportedOperationException(
+            String.format(
+                "Given Spark Receiver class %s doesn't implement HasOffset interface,"
+                    + " therefore it is not supported!",
+                sparkReceiverBuilder.getSparkReceiverClass().getName()));
+      } else {
+        LOG.info("{} started reading", ReadFromSparkReceiverWithOffsetDoFn.class.getSimpleName());
+        return input
+            .apply(Impulse.create())
+            .apply(ParDo.of(new ReadFromSparkReceiverWithOffsetDoFn<>(sparkReceiverRead)));

Review Comment:
   Yeah, this can be pushed to a future PR. Let's add a TODO though.



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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
+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> getTimestampFn;
+  private final ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder;
+
+  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> getTimestampFn = transform.getTimestampFn();
+    if (getTimestampFn == null) {
+      getTimestampFn = input -> Instant.now();
+    }
+    this.getTimestampFn = getTimestampFn;
+  }
+
+  @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();
+  }
+
+  @NewTracker
+  public OffsetRangeTracker restrictionTracker(
+      @Element byte[] element, @Restriction OffsetRange restriction) {
+    return new OffsetRangeTracker(restriction);
+  }
+
+  @GetRestrictionCoder
+  public Coder<OffsetRange> restrictionCoder() {
+    return new OffsetRange.Coder();
+  }
+
+  // Need to do an unchecked cast from Object
+  // because org.apache.spark.streaming.receiver.ReceiverSupervisor accepts Object in push methods
+  @SuppressWarnings("unchecked")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+        throw new IllegalStateException("Spark Receiver was not initialized");
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("SparkReceiver was interrupted before polling started", e);
+        throw new IllegalStateException("Spark Receiver was interrupted before polling started");
+      }
+    }
+
+    @Override
+    public void stop() {
+      if (sparkReceiver != null) {
+        sparkReceiver.stop("SparkReceiver is stopped.");
+      }
+      recordsQueue.clear();
+    }
+  }
+
+  @ProcessElement
+  public ProcessContinuation processElement(
+      @Element byte[] element,
+      RestrictionTracker<OffsetRange, Long> tracker,
+      WatermarkEstimator<Instant> watermarkEstimator,
+      OutputReceiver<V> receiver) {
+
+    SparkConsumer<V> sparkConsumer;
+    Receiver<V> sparkReceiver;
+    try {
+      sparkReceiver = sparkReceiverBuilder.build();
+    } catch (Exception e) {
+      LOG.error("Can not build Spark Receiver", e);
+      throw new IllegalStateException("Spark Receiver was not built!");
+    }
+    sparkConsumer = new SparkConsumerWithOffset<>(tracker.currentRestriction().getFrom());
+    sparkConsumer.start(sparkReceiver);
+
+    while (sparkConsumer.hasRecords()) {
+      V record = sparkConsumer.poll();
+      if (record != null) {
+        Long offset = getOffsetFn.apply(record);
+        if (!tracker.tryClaim(offset)) {
+          sparkConsumer.stop();
+          LOG.debug("Stop for restriction: {}", tracker.currentRestriction().toString());

Review Comment:
   I don't see any sort of acking here for messages that are already read. How does the SparkConsumer finalize messages so that it does not have to hold off to data forever ? Usually this is implemented via a bundle finalizer: https://beam.apache.org/documentation/programming-guide/#bundle-finalization



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/**
+ * Streaming sources for Spark {@link Receiver}.
+ *
+ * <h3>Reading using {@link SparkReceiverIO}</h3>
+ *
+ * <p>You will need to pass a {@link ReceiverBuilder} which is responsible for instantiating new
+ * {@link Receiver} objects.
+ *
+ * <p>{@link Receiver} that will be used should implement {@link HasOffset} interface. You will need
+ * to pass {@code getOffsetFn} which is a {@link SerializableFunction} that defines how to get
+ * {@code Long offset} from {@code V record}.
+ *
+ * <p>Optionally you can pass {@code timestampFn} which is a {@link SerializableFunction} that
+ * defines how to get {@code Instant timestamp} from {@code V record}.
+ *
+ * <p>Example of {@link SparkReceiverIO#read()} usage:
+ *
+ * <pre>{@code
+ * Pipeline p = ...; // Create pipeline.
+ *
+ * // Create ReceiverBuilder for CustomReceiver
+ * ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+ *         new ReceiverBuilder<>(CustomReceiver.class).withConstructorArgs();
+ *
+ * //Read from CustomReceiver
+ * p.apply("Spark Receiver Read",
+ *  SparkReceiverIO.Read<String> reader =
+ *    SparkReceiverIO.<String>read()
+ *      .withGetOffsetFn(Long::valueOf)
+ *      .withTimestampFn(Instant::parse)
+ *      .withSparkReceiverBuilder(receiverBuilder);
+ * }</pre>
+ */
+public class SparkReceiverIO {

Review Comment:
   Please add an experimental tag (see [1] for an example) since this is a new connector and the API can change.
   
   [1] https://github.com/apache/beam/blob/f5435c0575870062f39575271c0f483117908403/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java#L322



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r949017937


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkConsumer.java:
##########
@@ -0,0 +1,38 @@
+/*
+ * 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 java.io.Serializable;
+import org.apache.spark.streaming.receiver.Receiver;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Interface for start/stop reading from some Spark {@link Receiver} into some place and poll from
+ * it.
+ */
+public interface SparkConsumer<V> extends Serializable {

Review Comment:
   Changed to package-private



##########
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> {

Review Comment:
   Changed to package-private



##########
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")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    public SparkConsumerWithOffset(Long startOffset) {

Review Comment:
   Done



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r949826794


##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+import java.util.List;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Test class for {@link SparkReceiverIO}. */
+@RunWith(JUnit4.class)
+public class SparkReceiverIOTest {
+
+  public static final TestPipelineOptions OPTIONS =

Review Comment:
   Yes, we need to set `blockOnRun(false)`.



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1190596164

   Run Java PreCommit


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aromanenko-dev commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
aromanenko-dev commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1220703371

   Thanks for addressing review comments. It looks ok in general for me. 
   
   I'll be on PTO for the next 2 weeks, so I'd kindly ask @chamikaramj or @lukecwik to take a look on SDF part.


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r949017261


##########
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 {}

Review Comment:
   Removed



##########
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")
+  private static class SparkConsumerWithOffset<V> implements SparkConsumer<V> {
+    private final Queue<V> recordsQueue;
+    private @Nullable Receiver<V> sparkReceiver;
+    private final Long startOffset;
+
+    public SparkConsumerWithOffset(Long startOffset) {
+      this.startOffset = startOffset;
+      this.recordsQueue = new ConcurrentLinkedQueue<>();
+    }
+
+    @Override
+    public boolean hasRecords() {
+      return !recordsQueue.isEmpty();
+    }
+
+    @Override
+    public @Nullable V poll() {
+      return recordsQueue.poll();
+    }
+
+    @Override
+    public void start(Receiver<V> sparkReceiver) {
+      this.sparkReceiver = sparkReceiver;
+      try {
+        new WrappedSupervisor(
+            sparkReceiver,
+            new SparkConf(),
+            objects -> {
+              V record = (V) objects[0];
+              recordsQueue.offer(record);
+              return null;
+            });
+      } catch (Exception e) {
+        LOG.error("Can not init Spark Receiver!", e);
+      }
+      ((HasOffset) sparkReceiver).setStartOffset(startOffset);
+      sparkReceiver.supervisor().startReceiver();
+      try {
+        TimeUnit.MILLISECONDS.sleep(START_POLL_TIMEOUT_MS);
+      } catch (InterruptedException e) {
+        LOG.error("Interrupted", e);

Review Comment:
   Done



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r954777509


##########
sdks/java/io/sparkreceiver/src/test/java/org/apache/beam/sdk/io/sparkreceiver/CustomReceiverWithOffset.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.receiver.Receiver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Imitation of Spark {@link Receiver} that implements {@link HasOffset} interface. Used to test
+ * {@link SparkReceiverIO#read()}.
+ */
+public class CustomReceiverWithOffset extends Receiver<String> implements HasOffset {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CustomReceiverWithOffset.class);
+  private static final int TIMEOUT_MS = 500;
+  private static final List<String> STORED_RECORDS = new ArrayList<>();
+  private static final int RECORDS_COUNT = 20;
+  private Long startOffset;
+
+  CustomReceiverWithOffset() {
+    super(StorageLevel.MEMORY_AND_DISK_2());
+  }
+
+  @Override
+  public void setStartOffset(Long startOffset) {
+    if (startOffset != null) {
+      this.startOffset = startOffset;
+    }
+  }
+
+  @Override
+  @SuppressWarnings("FutureReturnValueIgnored")
+  public void onStart() {
+    Executors.newSingleThreadExecutor(new ThreadFactoryBuilder().build()).submit(this::receive);
+  }
+
+  @Override
+  public void onStop() {}
+
+  @Override
+  public Long getEndOffset() {
+    return Long.MAX_VALUE;
+  }
+
+  private void receive() {
+    Long currentOffset = startOffset;
+    while (!isStopped()) {

Review Comment:
   Fixed



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/**
+ * Streaming sources for Spark {@link Receiver}.
+ *
+ * <h3>Reading using {@link SparkReceiverIO}</h3>
+ *
+ * <p>You will need to pass a {@link ReceiverBuilder} which is responsible for instantiating new
+ * {@link Receiver} objects.
+ *
+ * <p>{@link Receiver} that will be used should implement {@link HasOffset} interface. You will need
+ * to pass {@code getOffsetFn} which is a {@link SerializableFunction} that defines how to get
+ * {@code Long offset} from {@code V record}.
+ *
+ * <p>Optionally you can pass {@code watermarkFn} which is a {@link SerializableFunction} that
+ * defines how to get {@code Instant watermark} from {@code V record}.
+ *
+ * <p>Example of {@link SparkReceiverIO#read()} usage:
+ *
+ * <pre>{@code
+ * Pipeline p = ...; // Create pipeline.
+ *
+ * // Create ReceiverBuilder for CustomReceiver
+ * ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+ *         new ReceiverBuilder<>(CustomReceiver.class).withConstructorArgs();
+ *
+ * //Read from CustomReceiver
+ * p.apply("Spark Receiver Read",
+ *  SparkReceiverIO.Read<String> reader =
+ *    SparkReceiverIO.<String>read()
+ *      .withGetOffsetFn(Long::valueOf)
+ *      .withWatermarkFn(Instant::parse)
+ *      .withSparkReceiverBuilder(receiverBuilder);
+ * }</pre>
+ */
+public class SparkReceiverIO {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkReceiverIO.class);
+
+  public static <V> Read<V> read() {
+    return new AutoValue_SparkReceiverIO_Read.Builder<V>().build();
+  }
+
+  /** A {@link PTransform} to read from Spark {@link Receiver}. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<V> extends PTransform<PBegin, PCollection<V>> {
+
+    abstract @Nullable ReceiverBuilder<V, ? extends Receiver<V>> getSparkReceiverBuilder();
+
+    abstract @Nullable SerializableFunction<V, Long> getGetOffsetFn();
+
+    abstract @Nullable SerializableFunction<V, Instant> getWatermarkFn();
+
+    abstract Builder<V> toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder<V> {
+
+      abstract Builder<V> setSparkReceiverBuilder(
+          ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder);
+
+      abstract Builder<V> setGetOffsetFn(SerializableFunction<V, Long> getOffsetFn);
+
+      abstract Builder<V> setWatermarkFn(SerializableFunction<V, Instant> watermarkFn);
+
+      abstract Read<V> build();
+    }
+
+    /** Sets {@link ReceiverBuilder} with value and custom Spark {@link Receiver} class. */
+    public Read<V> withSparkReceiverBuilder(
+        ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder) {
+      checkArgument(sparkReceiverBuilder != null, "Spark receiver builder can not be null");
+      return toBuilder().setSparkReceiverBuilder(sparkReceiverBuilder).build();
+    }
+
+    /** A function to get offset in order to start {@link Receiver} from it. */
+    public Read<V> withGetOffsetFn(SerializableFunction<V, Long> getOffsetFn) {
+      checkArgument(getOffsetFn != null, "Get offset function can not be null");
+      return toBuilder().setGetOffsetFn(getOffsetFn).build();
+    }
+
+    /** A function to calculate watermark after a record. */
+    public Read<V> withWatermarkFn(SerializableFunction<V, Instant> watermarkFn) {

Review Comment:
   Renamed



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+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;
+
+  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();

Review Comment:
   Renamed



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Lizzfox commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Lizzfox commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1195628803

   @aromanenko-dev 
   Alexey, we've prepared the SparkReceiverIO connector Read interface. This PR is ready to be reviewed.
   Thanks for your attention!


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r971688522


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/**
+ * Streaming sources for Spark {@link Receiver}.
+ *
+ * <h3>Reading using {@link SparkReceiverIO}</h3>
+ *
+ * <p>You will need to pass a {@link ReceiverBuilder} which is responsible for instantiating new
+ * {@link Receiver} objects.
+ *
+ * <p>{@link Receiver} that will be used should implement {@link HasOffset} interface. You will need
+ * to pass {@code getOffsetFn} which is a {@link SerializableFunction} that defines how to get
+ * {@code Long offset} from {@code V record}.
+ *
+ * <p>Optionally you can pass {@code watermarkFn} which is a {@link SerializableFunction} that
+ * defines how to get {@code Instant watermark} from {@code V record}.
+ *
+ * <p>Example of {@link SparkReceiverIO#read()} usage:
+ *
+ * <pre>{@code
+ * Pipeline p = ...; // Create pipeline.
+ *
+ * // Create ReceiverBuilder for CustomReceiver
+ * ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+ *         new ReceiverBuilder<>(CustomReceiver.class).withConstructorArgs();
+ *
+ * //Read from CustomReceiver
+ * p.apply("Spark Receiver Read",
+ *  SparkReceiverIO.Read<String> reader =
+ *    SparkReceiverIO.<String>read()
+ *      .withGetOffsetFn(Long::valueOf)
+ *      .withWatermarkFn(Instant::parse)
+ *      .withSparkReceiverBuilder(receiverBuilder);
+ * }</pre>
+ */
+public class SparkReceiverIO {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkReceiverIO.class);
+
+  public static <V> Read<V> read() {
+    return new AutoValue_SparkReceiverIO_Read.Builder<V>().build();
+  }
+
+  /** A {@link PTransform} to read from Spark {@link Receiver}. */
+  @AutoValue
+  @AutoValue.CopyAnnotations
+  public abstract static class Read<V> extends PTransform<PBegin, PCollection<V>> {
+
+    abstract @Nullable ReceiverBuilder<V, ? extends Receiver<V>> getSparkReceiverBuilder();
+
+    abstract @Nullable SerializableFunction<V, Long> getGetOffsetFn();
+
+    abstract @Nullable SerializableFunction<V, Instant> getWatermarkFn();
+
+    abstract Builder<V> toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder<V> {
+
+      abstract Builder<V> setSparkReceiverBuilder(
+          ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder);
+
+      abstract Builder<V> setGetOffsetFn(SerializableFunction<V, Long> getOffsetFn);
+
+      abstract Builder<V> setWatermarkFn(SerializableFunction<V, Instant> watermarkFn);
+
+      abstract Read<V> build();
+    }
+
+    /** Sets {@link ReceiverBuilder} with value and custom Spark {@link Receiver} class. */
+    public Read<V> withSparkReceiverBuilder(
+        ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder) {
+      checkArgument(sparkReceiverBuilder != null, "Spark receiver builder can not be null");
+      return toBuilder().setSparkReceiverBuilder(sparkReceiverBuilder).build();
+    }
+
+    /** A function to get offset in order to start {@link Receiver} from it. */
+    public Read<V> withGetOffsetFn(SerializableFunction<V, Long> getOffsetFn) {
+      checkArgument(getOffsetFn != null, "Get offset function can not be null");
+      return toBuilder().setGetOffsetFn(getOffsetFn).build();
+    }
+
+    /** A function to calculate watermark after a record. */
+    public Read<V> withWatermarkFn(SerializableFunction<V, Instant> watermarkFn) {
+      checkArgument(watermarkFn != null, "Watermark function can not be null");
+      return toBuilder().setWatermarkFn(watermarkFn).build();
+    }
+
+    @Override
+    public PCollection<V> expand(PBegin input) {
+      validateTransform();
+      return input.apply(new ReadFromSparkReceiverViaSdf<>(this));
+    }
+
+    public void validateTransform() {
+      ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder = getSparkReceiverBuilder();
+      checkStateNotNull(sparkReceiverBuilder, "withSparkReceiverBuilder() is required");
+      checkStateNotNull(getGetOffsetFn(), "withGetOffsetFn() is required");
+    }
+  }
+
+  static class ReadFromSparkReceiverViaSdf<V> extends PTransform<PBegin, PCollection<V>> {
+
+    private final Read<V> sparkReceiverRead;
+
+    ReadFromSparkReceiverViaSdf(Read<V> sparkReceiverRead) {
+      this.sparkReceiverRead = sparkReceiverRead;
+    }
+
+    @Override
+    public PCollection<V> expand(PBegin input) {
+      final ReceiverBuilder<V, ? extends Receiver<V>> sparkReceiverBuilder =
+          sparkReceiverRead.getSparkReceiverBuilder();
+      checkStateNotNull(sparkReceiverBuilder, "withSparkReceiverBuilder() is required");
+      if (!HasOffset.class.isAssignableFrom(sparkReceiverBuilder.getSparkReceiverClass())) {
+        throw new UnsupportedOperationException(
+            String.format(
+                "Given Spark Receiver class %s doesn't implement HasOffset interface,"
+                    + " therefore it is not supported!",
+                sparkReceiverBuilder.getSparkReceiverClass().getName()));
+      } else {
+        LOG.info("{} started reading", ReadFromSparkReceiverWithOffsetDoFn.class.getSimpleName());
+        return input
+            .apply(Impulse.create())
+            .apply(ParDo.of(new ReadFromSparkReceiverWithOffsetDoFn<>(sparkReceiverRead)));

Review Comment:
   Added TODO



##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/**
+ * Streaming sources for Spark {@link Receiver}.
+ *
+ * <h3>Reading using {@link SparkReceiverIO}</h3>
+ *
+ * <p>You will need to pass a {@link ReceiverBuilder} which is responsible for instantiating new
+ * {@link Receiver} objects.
+ *
+ * <p>{@link Receiver} that will be used should implement {@link HasOffset} interface. You will need
+ * to pass {@code getOffsetFn} which is a {@link SerializableFunction} that defines how to get
+ * {@code Long offset} from {@code V record}.
+ *
+ * <p>Optionally you can pass {@code timestampFn} which is a {@link SerializableFunction} that
+ * defines how to get {@code Instant timestamp} from {@code V record}.
+ *
+ * <p>Example of {@link SparkReceiverIO#read()} usage:
+ *
+ * <pre>{@code
+ * Pipeline p = ...; // Create pipeline.
+ *
+ * // Create ReceiverBuilder for CustomReceiver
+ * ReceiverBuilder<String, CustomReceiverWithOffset> receiverBuilder =
+ *         new ReceiverBuilder<>(CustomReceiver.class).withConstructorArgs();
+ *
+ * //Read from CustomReceiver
+ * p.apply("Spark Receiver Read",
+ *  SparkReceiverIO.Read<String> reader =
+ *    SparkReceiverIO.<String>read()
+ *      .withGetOffsetFn(Long::valueOf)
+ *      .withTimestampFn(Instant::parse)
+ *      .withSparkReceiverBuilder(receiverBuilder);
+ * }</pre>
+ */
+public class SparkReceiverIO {

Review Comment:
   Done



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] chamikaramj merged pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
chamikaramj merged PR #17828:
URL: https://github.com/apache/beam/pull/17828


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] asf-ci commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1145808018

   Can one of the admins verify this patch?


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] asf-ci commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1145808019

   Can one of the admins verify this patch?


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] codecov[bot] commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1145827401

   # [Codecov](https://codecov.io/gh/apache/beam/pull/17828?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#17828](https://codecov.io/gh/apache/beam/pull/17828?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0e8321e) into [master](https://codecov.io/gh/apache/beam/commit/30f1a0c26b199c172bbc563d3c481c1148841a03?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (30f1a0c) will **increase** coverage by `0.01%`.
   > The diff coverage is `n/a`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17828      +/-   ##
   ==========================================
   + Coverage   73.95%   73.97%   +0.01%     
   ==========================================
     Files         693      695       +2     
     Lines       91721    91838     +117     
   ==========================================
   + Hits        67836    67939     +103     
   - Misses      22636    22650      +14     
     Partials     1249     1249              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.72% <ø> (+<0.01%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/17828?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...s/interactive/dataproc/dataproc\_cluster\_manager.py](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kYXRhcHJvYy9kYXRhcHJvY19jbHVzdGVyX21hbmFnZXIucHk=) | `77.41% <0.00%> (-6.80%)` | :arrow_down: |
   | [.../apache\_beam/runners/interactive/dataproc/types.py](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kYXRhcHJvYy90eXBlcy5weQ==) | `96.55% <0.00%> (-3.45%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `95.12% <0.00%> (-2.44%)` | :arrow_down: |
   | [sdks/python/apache\_beam/internal/gcp/auth.py](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW50ZXJuYWwvZ2NwL2F1dGgucHk=) | `78.66% <0.00%> (-2.37%)` | :arrow_down: |
   | [sdks/python/apache\_beam/internal/dill\_pickler.py](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW50ZXJuYWwvZGlsbF9waWNrbGVyLnB5) | `84.89% <0.00%> (-1.44%)` | :arrow_down: |
   | [...eam/runners/portability/fn\_api\_runner/execution.py](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9mbl9hcGlfcnVubmVyL2V4ZWN1dGlvbi5weQ==) | `92.44% <0.00%> (-0.65%)` | :arrow_down: |
   | [sdks/python/apache\_beam/typehints/schemas.py](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHlwZWhpbnRzL3NjaGVtYXMucHk=) | `94.16% <0.00%> (-0.42%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `81.70% <0.00%> (-0.39%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/common.py](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9jb21tb24ucHk=) | `87.94% <0.00%> (-0.25%)` | :arrow_down: |
   | [sdks/python/apache\_beam/coders/coders.py](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL2NvZGVycy5weQ==) | `88.22% <0.00%> (-0.23%)` | :arrow_down: |
   | ... and [31 more](https://codecov.io/gh/apache/beam/pull/17828/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17828?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17828?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [30f1a0c...0e8321e](https://codecov.io/gh/apache/beam/pull/17828?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] asf-ci commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
asf-ci commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1145808020

   Can one of the admins verify this patch?


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1191572723

   Run Java PreCommit


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1190231700

   Run Java PreCommit


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] chamikaramj commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
chamikaramj commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1244754992

   Any updates ?


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on PR #17828:
URL: https://github.com/apache/beam/pull/17828#issuecomment-1221237448

   Run Java PreCommit


-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r950142618


##########
sdks/java/io/sparkreceiver/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+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;
+
+/** Streaming sources for Spark {@link Receiver}. */

Review Comment:
   Added javadoc



##########
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:
   Done



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Amar3tto commented on a diff in pull request #17828: [BEAM-14378] [CdapIO] SparkReceiverIO Read via SDF

Posted by GitBox <gi...@apache.org>.
Amar3tto commented on code in PR #17828:
URL: https://github.com/apache/beam/pull/17828#discussion_r949024612


##########
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:
   We are dealing with `org.apache.spark.streaming.receiver.ReceiverSupervisor` class which has `pushSingle(Object o)` method. So we need to do an unchecked cast from Object.
   Should we add an explanation as a comment here?



-- 
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-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org