chamikaramj commented on code in PR #22612: URL: https://github.com/apache/beam/pull/22612#discussion_r939738263
########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedReaderImpl.java: ########## @@ -0,0 +1,136 @@ +/* + * 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) { + SequencedMessage peeked = subscriber.peek().get(); + Offset nextOffset = Offset.of(peeked.getCursor().getOffset() + 1); + checkState(nextOffset.value() > fetchOffset.value()); + fetchOffset = nextOffset; + lastMessageTimestamp = Optional.of(getTimestamp(peeked)); Review Comment: Seems like lastMessageTimestamp and fetchOffset will not be set for the first record ? ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedReaderImpl.java: ########## @@ -0,0 +1,136 @@ +/* + * 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(); Review Comment: Probably check subscriber state and fail if not State.RUNNING ? Otherwise if the reader doesn't get to the correct state here (State.RUNNING) the reader will never be able to recover. ########## 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) { Review Comment: Instead of this, we should introduce a TransformOverride that overrides the SDF-based source transform by the UnboundedSource-based source transform. We do exactly that for Kafka: https://github.com/apache/beam/blob/17fb9c0342064cd4375b0d7f2c37e12a175d03ef/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java#L537 https://github.com/apache/beam/blob/67e6726ffeb47d2ada0122369fa230833ce0f026/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L1418 This way Dataflow Runner v2 and other runners will continue to use SDF by default. I think a direct SDF implementation should be more powerful when it comes to capabilities (and potentially more performant) over a SDF wrapped UnboundedSource implementation. The downside is that we'll have to maintain two PSL source implementations but I think this is OK given that the goal is to fully migrate to the SDF version in the future while supporting the current Dataflow production runner well using an UnboundedSource-based source implementation. WDYT ? ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/UnboundedReaderImpl.java: ########## @@ -0,0 +1,136 @@ +/* + * 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) { Review Comment: Is the subscriber already set to the first record the first time the start() is called ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
