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/08/09 13:57:35 UTC

[GitHub] [beam] scwhittle commented on a diff in pull request #22612: Reimplement Pub/Sub Lite's I/O using UnboundedSource.

scwhittle commented on code in PR #22612:
URL: https://github.com/apache/beam/pull/22612#discussion_r941358993


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java:
##########
@@ -101,6 +91,42 @@ private MemoryBufferedSubscriber newBufferedSubscriber(
         consumer -> newSubscriber(subscriptionPartition.partition(), startOffset, consumer));
   }
 
+  private MemoryBufferedSubscriber getCachedSubscriber(
+      SubscriptionPartition subscriptionPartition, Offset startOffset) {
+    Supplier<MemoryBufferedSubscriber> getOrCreate =
+        () ->
+            PerServerSubscriberCache.CACHE.get(
+                subscriptionPartition,
+                () -> newBufferedSubscriber(subscriptionPartition, startOffset));
+    while (true) {
+      MemoryBufferedSubscriber subscriber = getOrCreate.get();
+      Offset fetchOffset = subscriber.fetchOffset();
+      if (startOffset.equals(fetchOffset)) {
+        return subscriber;
+      }
+      LOG.info(
+          "Discarding subscriber due to mismatch, this should be rare. {}, start: {} fetch: {}",
+          subscriptionPartition,
+          startOffset,
+          fetchOffset);
+      try {
+        subscriber.stopAsync().awaitTerminated();

Review Comment:
   does the get above remove it from the cache? Or do you have to do so here so a new one is returned next time?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/SubscribeTransform.java:
##########
@@ -134,19 +160,28 @@ private TopicPath getTopicPath() {
     }
   }
 
-  @Override
-  public PCollection<SequencedMessage> expand(PBegin input) {
-    PCollection<SubscriptionPartition> subscriptionPartitions;
-    subscriptionPartitions =
+  @SuppressWarnings("unused")
+  private PCollection<SequencedMessage> expandSdf(PBegin input) {
+    PCollection<SubscriptionPartition> subscriptionPartitions =
         input.apply(new SubscriptionPartitionLoader(getTopicPath(), options.subscriptionPath()));
-
     return subscriptionPartitions.apply(
         ParDo.of(
             new PerSubscriptionPartitionSdf(
-                new ManagedBacklogReaderFactoryImpl(this::newBacklogReader),
+                new ManagedFactoryImpl<>(this::newBacklogReader),
+                new ManagedFactoryImpl<>(this::newCommitter),
                 this::newInitialOffsetReader,
                 this::newRestrictionTracker,
-                this::newPartitionProcessor,
-                this::newCommitter)));
+                this::newPartitionProcessor)));
+  }
+
+  private PCollection<SequencedMessage> expandSource(PBegin input) {
+    return input.apply(
+        Read.from(
+            new UnboundedSourceImpl(options, this::newBufferedSubscriber, this::newBacklogReader)));
+  }
+
+  @Override
+  public PCollection<SequencedMessage> expand(PBegin input) {
+    return expandSource(input);

Review Comment:
   option to use SDF or Source? Or can SDF be used elsewhere by configuration?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/ApiServices.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.gcp.pubsublite.internal;
+
+import com.google.api.core.ApiService;
+import java.util.concurrent.TimeUnit;
+
+class ApiServices {
+
+  private ApiServices() {}
+
+  static AutoCloseable asCloseable(ApiService service) {
+    return () -> service.stopAsync().awaitTerminated(1, TimeUnit.MINUTES);

Review Comment:
   Is exception thrown if this is timed out? Should 1 minute be adjustable?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedReaderImpl.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.gcp.pubsublite.internal;
+
+import static org.apache.beam.sdk.io.gcp.pubsublite.internal.ApiServices.asCloseable;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+
+import com.google.api.core.ApiService.State;
+import com.google.cloud.pubsublite.Offset;
+import com.google.cloud.pubsublite.proto.SequencedMessage;
+import com.google.protobuf.util.Timestamps;
+import java.io.IOException;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.joda.time.Instant;
+
+public class UnboundedReaderImpl extends UnboundedReader<SequencedMessage> {
+
+  private final UnboundedSource<SequencedMessage, CheckpointMarkImpl> source;
+  private final MemoryBufferedSubscriber subscriber;
+  private final TopicBacklogReader backlogReader;
+  private final BlockingCommitter committer;
+
+  private Offset fetchOffset;
+  private Optional<Instant> lastMessageTimestamp = Optional.empty();
+  private boolean advanced = false;
+
+  UnboundedReaderImpl(
+      UnboundedSource<SequencedMessage, CheckpointMarkImpl> source,
+      MemoryBufferedSubscriber subscriber,
+      TopicBacklogReader backlogReader,
+      BlockingCommitter committer,
+      Offset initialOffset) {
+    checkArgument(initialOffset.equals(subscriber.fetchOffset()));
+    this.source = source;
+    this.subscriber = subscriber;
+    this.backlogReader = backlogReader;
+    this.committer = committer;
+    this.fetchOffset = initialOffset;
+  }
+
+  @Override
+  public SequencedMessage getCurrent() throws NoSuchElementException {
+    if (!advanced) {
+      throw new NoSuchElementException();
+    }
+    return subscriber.peek().get();
+  }
+
+  @Override
+  public Instant getCurrentTimestamp() throws NoSuchElementException {
+    return getTimestamp(getCurrent());
+  }
+
+  private static Instant getTimestamp(SequencedMessage message) {
+    return Instant.ofEpochMilli(Timestamps.toMillis(message.getPublishTime()));
+  }
+
+  @Override
+  public void close() throws IOException {
+    try (AutoCloseable c1 = backlogReader;
+        AutoCloseable c2 = committer;
+        AutoCloseable c3 = asCloseable(subscriber)) {
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public boolean start() throws IOException {
+    try {
+      subscriber.startAsync().awaitRunning(1, TimeUnit.MINUTES);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    return advance();
+  }
+
+  @Override
+  public boolean advance() throws IOException {
+    if (!subscriber.state().equals(State.RUNNING)) {
+      throw new IOException("Subscriber failed: ", subscriber.failureCause());
+    }
+    if (advanced) {
+      subscriber.pop();
+    }
+    Optional<SequencedMessage> next = subscriber.peek();
+    advanced = next.isPresent();
+    if (!advanced) {
+      return false;
+    }
+    Offset nextOffset = Offset.of(next.get().getCursor().getOffset() + 1);
+    checkState(nextOffset.value() > fetchOffset.value());
+    fetchOffset = nextOffset;
+    lastMessageTimestamp = Optional.of(getTimestamp(next.get()));
+    return true;
+  }
+
+  @Override
+  public Instant getWatermark() {
+    return lastMessageTimestamp.orElse(BoundedWindow.TIMESTAMP_MIN_VALUE);

Review Comment:
   are timestamps increasing in PubsubLite?  Or does this need to be based upon last messages and some controllable skew option etc



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/BlockingCommitterImpl.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.gcp.pubsublite.internal;
+
+import static com.google.cloud.pubsublite.internal.ExtractStatus.toCanonical;
+
+import com.google.cloud.pubsublite.Offset;
+import com.google.cloud.pubsublite.internal.wire.Committer;
+import java.util.concurrent.TimeUnit;
+
+public class BlockingCommitterImpl implements BlockingCommitter {
+
+  private final Committer committer;
+
+  BlockingCommitterImpl(Committer committer) {
+    if (!committer.isRunning()) {
+      throw new RuntimeException(committer.failureCause());
+    }
+    this.committer = committer;
+  }
+
+  @Override
+  public void commitOffset(Offset offset) {
+    try {
+      committer.commitOffset(offset).get(1, TimeUnit.MINUTES);

Review Comment:
   ditto should this timeout come from option?



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedSourceImpl.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.gcp.pubsublite.internal;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+
+import com.google.cloud.pubsublite.Offset;
+import com.google.cloud.pubsublite.Partition;
+import com.google.cloud.pubsublite.PartitionLookupUtils;
+import com.google.cloud.pubsublite.proto.SequencedMessage;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.extensions.protobuf.ProtoCoder;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.gcp.pubsublite.SubscriberOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Optional;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+public class UnboundedSourceImpl extends UnboundedSource<SequencedMessage, CheckpointMarkImpl> {
+
+  private final SubscriberOptions subscriberOptions;
+  private final SubscriberFactory subscriberFactory;
+  private final BacklogReaderFactory readerFactory;
+  private final Optional<Partition> partition;
+
+  interface SubscriberFactory extends Serializable {
+
+    MemoryBufferedSubscriber create(SubscriptionPartition subscriptionPartition, Offset offset);
+  }
+
+  interface BacklogReaderFactory extends Serializable {
+
+    TopicBacklogReader create(SubscriptionPartition subscriptionPartition);
+  }
+
+  UnboundedSourceImpl(
+      SubscriberOptions subscriberOptions,
+      SubscriberFactory subscriberFactory,
+      BacklogReaderFactory readerFactory) {
+    this.subscriberOptions = subscriberOptions;
+    this.subscriberFactory = subscriberFactory;
+    this.readerFactory = readerFactory;
+    this.partition = Optional.absent();
+  }
+
+  private UnboundedSourceImpl(
+      SubscriberOptions subscriberOptions,
+      SubscriberFactory subscriberFactory,
+      BacklogReaderFactory readerFactory,
+      Partition partition) {
+    this.subscriberOptions = subscriberOptions;
+    this.subscriberFactory = subscriberFactory;
+    this.readerFactory = readerFactory;
+    this.partition = Optional.of(partition);
+  }
+
+  @Override
+  public List<? extends UnboundedSource<SequencedMessage, CheckpointMarkImpl>> split(
+      int desiredNumSplits, PipelineOptions options) throws Exception {
+    checkState(!partition.isPresent());
+    int numPartitions = PartitionLookupUtils.numPartitions(subscriberOptions.subscriptionPath());

Review Comment:
   Is there ever a crazy # of partitions? Would we want multiple partitions to a single source shard.  I believe Kafka did this.  I think one concern is per-shard information building up in the graph submitted to dataflow.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedReaderImpl.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.gcp.pubsublite.internal;
+
+import static org.apache.beam.sdk.io.gcp.pubsublite.internal.ApiServices.asCloseable;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+
+import com.google.api.core.ApiService.State;
+import com.google.cloud.pubsublite.Offset;
+import com.google.cloud.pubsublite.proto.SequencedMessage;
+import com.google.protobuf.util.Timestamps;
+import java.io.IOException;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.joda.time.Instant;
+
+public class UnboundedReaderImpl extends UnboundedReader<SequencedMessage> {
+
+  private final UnboundedSource<SequencedMessage, CheckpointMarkImpl> source;
+  private final MemoryBufferedSubscriber subscriber;
+  private final TopicBacklogReader backlogReader;
+  private final BlockingCommitter committer;
+
+  private Offset fetchOffset;
+  private Optional<Instant> lastMessageTimestamp = Optional.empty();
+  private boolean advanced = false;
+
+  UnboundedReaderImpl(
+      UnboundedSource<SequencedMessage, CheckpointMarkImpl> source,
+      MemoryBufferedSubscriber subscriber,
+      TopicBacklogReader backlogReader,
+      BlockingCommitter committer,
+      Offset initialOffset) {
+    checkArgument(initialOffset.equals(subscriber.fetchOffset()));
+    this.source = source;
+    this.subscriber = subscriber;
+    this.backlogReader = backlogReader;
+    this.committer = committer;
+    this.fetchOffset = initialOffset;
+  }
+
+  @Override
+  public SequencedMessage getCurrent() throws NoSuchElementException {
+    if (!advanced) {
+      throw new NoSuchElementException();
+    }
+    return subscriber.peek().get();
+  }
+
+  @Override
+  public Instant getCurrentTimestamp() throws NoSuchElementException {
+    return getTimestamp(getCurrent());
+  }
+
+  private static Instant getTimestamp(SequencedMessage message) {
+    return Instant.ofEpochMilli(Timestamps.toMillis(message.getPublishTime()));
+  }
+
+  @Override
+  public void close() throws IOException {
+    try (AutoCloseable c1 = backlogReader;
+        AutoCloseable c2 = committer;
+        AutoCloseable c3 = asCloseable(subscriber)) {
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public boolean start() throws IOException {
+    try {
+      subscriber.startAsync().awaitRunning(1, TimeUnit.MINUTES);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+    return advance();
+  }
+
+  @Override
+  public boolean advance() throws IOException {
+    if (!subscriber.state().equals(State.RUNNING)) {
+      throw new IOException("Subscriber failed: ", subscriber.failureCause());
+    }
+    if (advanced) {
+      subscriber.pop();
+    }
+    Optional<SequencedMessage> next = subscriber.peek();
+    advanced = next.isPresent();
+    if (!advanced) {
+      return false;
+    }
+    Offset nextOffset = Offset.of(next.get().getCursor().getOffset() + 1);
+    checkState(nextOffset.value() > fetchOffset.value());
+    fetchOffset = nextOffset;
+    lastMessageTimestamp = Optional.of(getTimestamp(next.get()));
+    return true;
+  }
+
+  @Override
+  public Instant getWatermark() {
+    return lastMessageTimestamp.orElse(BoundedWindow.TIMESTAMP_MIN_VALUE);
+  }
+
+  @Override
+  public CheckpointMarkImpl getCheckpointMark() {
+    subscriber.rebuffer();

Review Comment:
   add comment on why you rebuffer here



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/BlockingCommitterImpl.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.gcp.pubsublite.internal;
+
+import static com.google.cloud.pubsublite.internal.ExtractStatus.toCanonical;
+
+import com.google.cloud.pubsublite.Offset;
+import com.google.cloud.pubsublite.internal.wire.Committer;
+import java.util.concurrent.TimeUnit;
+
+public class BlockingCommitterImpl implements BlockingCommitter {
+
+  private final Committer committer;
+
+  BlockingCommitterImpl(Committer committer) {
+    if (!committer.isRunning()) {
+      throw new RuntimeException(committer.failureCause());

Review Comment:
   would it be easier to debug if the exception was wrapped?



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