Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Reimplement Pub/Sub Lite's I/O using UnboundedSource. #22612

Merged
merged 8 commits into from
Aug 10, 2022
Next Next commit
Reimplement Pub/Sub Lite's I/O using UnboundedSource.
SDF is not well supported by the default mode of dataflow, and UnboundedSource is supported by all modes.
  • Loading branch information
dpcollins-google committed Aug 5, 2022
commit 718094666f2567a60f12dcd4068d007b0c92b1ef
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,9 @@ private PubsubLiteIO() {}
* retried, which the PubsubLiteIO write method will do. Use the dedupe transform to remove these
* duplicates.
*
* <p>Note that this will not pick up newly added partitions. To pick up new partitions, drain and
* restart the pipeline.
*
* <pre>{@code
* Pipeline p = ...;
*
Expand Down
Original file line number Diff line number Diff line change
@@ -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
*
* https://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);
dpcollins-google marked this conversation as resolved.
Show resolved Hide resolved
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import com.google.cloud.pubsublite.Offset;

interface BlockingCommitter {
interface BlockingCommitter extends AutoCloseable {

void commitOffset(Offset offset);
}
Original file line number Diff line number Diff line change
@@ -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
*
* https://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());
dpcollins-google marked this conversation as resolved.
Show resolved Hide resolved
}
this.committer = committer;
}

@Override
public void commitOffset(Offset offset) {
try {
committer.commitOffset(offset).get(1, TimeUnit.MINUTES);
dpcollins-google marked this conversation as resolved.
Show resolved Hide resolved
} catch (Exception e) {
throw toCanonical(e).underlying;
}
}

@Override
public void close() throws Exception {
committer.stopAsync().awaitTerminated(1, TimeUnit.MINUTES);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* 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
*
* https://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 java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Optional;
import org.apache.beam.sdk.coders.AtomicCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.VarLongCoder;
import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;

public class CheckpointMarkImpl implements CheckpointMark {

final Offset offset;

private final Optional<BlockingCommitter> committer;

CheckpointMarkImpl(Offset offset, BlockingCommitter committer) {
this.offset = offset;
this.committer = Optional.of(committer);
}

/** Internal-only constructor for deserialization. */
private CheckpointMarkImpl(Offset offset) {
this.offset = offset;
this.committer = Optional.empty();
}

static Coder<CheckpointMarkImpl> coder() {
return new AtomicCoder<CheckpointMarkImpl>() {
@Override
public void encode(CheckpointMarkImpl value, OutputStream outStream) throws IOException {
VarLongCoder.of().encode(value.offset.value(), outStream);
}

@Override
public CheckpointMarkImpl decode(InputStream inStream) throws IOException {
return new CheckpointMarkImpl(Offset.of(VarLongCoder.of().decode(inStream)));
}
};
}

@Override
public void finalizeCheckpoint() {
checkState(committer.isPresent());
committer.get().commitOffset(offset);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ public synchronized ComputeMessageStatsResponse computeMessageStats(Offset offse
}

@Override
public void close() {
public void close() throws Exception {
underlying.close();
}
}

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,12 @@
import java.io.Serializable;

/**
* A ManagedBacklogReaderFactory produces TopicBacklogReaders and tears down any produced readers
* when it is itself closed.
* A ManagedFactory produces instances and tears down any produced instances when it is itself
* closed.
*
* <p>close() should never be called on produced readers.
* <p>close() should never be called on produced instances.
*/
public interface ManagedBacklogReaderFactory extends AutoCloseable, Serializable {
TopicBacklogReader newReader(SubscriptionPartition subscriptionPartition);
public interface ManagedFactory<T extends AutoCloseable> extends AutoCloseable, Serializable {

@Override
void close();
T create(SubscriptionPartition subscriptionPartition);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* 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
*
* https://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 java.util.HashMap;
import java.util.Map;
import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;
import org.apache.beam.sdk.transforms.SerializableFunction;

public class ManagedFactoryImpl<T extends AutoCloseable> implements ManagedFactory<T> {

private final SerializableFunction<SubscriptionPartition, T> newInstance;

@GuardedBy("this")
private final Map<SubscriptionPartition, T> instances = new HashMap<>();

ManagedFactoryImpl(SerializableFunction<SubscriptionPartition, T> newInstance) {
this.newInstance = newInstance;
}

@Override
public synchronized T create(SubscriptionPartition subscriptionPartition) {
return instances.computeIfAbsent(subscriptionPartition, newInstance::apply);
}

@Override
public synchronized void close() throws Exception {
@Nullable Exception e = null;
for (AutoCloseable c : instances.values()) {
try {
c.close();
} catch (Exception e2) {
if (e == null) {
e = e2;
} else {
e.addSuppressed(e2);
}
}
}
if (e != null) {
throw e;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,32 +31,34 @@
import org.slf4j.LoggerFactory;

class PerSubscriptionPartitionSdf extends DoFn<SubscriptionPartition, SequencedMessage> {

private static final Logger LOG = LoggerFactory.getLogger(PerSubscriptionPartitionSdf.class);
private final ManagedBacklogReaderFactory backlogReaderFactory;
private final ManagedFactory<TopicBacklogReader> backlogReaderFactory;
private final ManagedFactory<BlockingCommitter> committerFactory;
private final SubscriptionPartitionProcessorFactory processorFactory;
private final SerializableFunction<SubscriptionPartition, InitialOffsetReader>
offsetReaderFactory;
private final SerializableBiFunction<TopicBacklogReader, OffsetByteRange, TrackerWithProgress>
trackerFactory;
private final SerializableFunction<SubscriptionPartition, BlockingCommitter> committerFactory;

PerSubscriptionPartitionSdf(
ManagedBacklogReaderFactory backlogReaderFactory,
ManagedFactory<TopicBacklogReader> backlogReaderFactory,
ManagedFactory<BlockingCommitter> committerFactory,
SerializableFunction<SubscriptionPartition, InitialOffsetReader> offsetReaderFactory,
SerializableBiFunction<TopicBacklogReader, OffsetByteRange, TrackerWithProgress>
trackerFactory,
SubscriptionPartitionProcessorFactory processorFactory,
SerializableFunction<SubscriptionPartition, BlockingCommitter> committerFactory) {
SubscriptionPartitionProcessorFactory processorFactory) {
this.backlogReaderFactory = backlogReaderFactory;
this.committerFactory = committerFactory;
this.processorFactory = processorFactory;
this.offsetReaderFactory = offsetReaderFactory;
this.trackerFactory = trackerFactory;
this.committerFactory = committerFactory;
}

@Teardown
public void teardown() {
backlogReaderFactory.close();
public void teardown() throws Exception {
try (AutoCloseable c1 = committerFactory;
AutoCloseable c2 = backlogReaderFactory) {}
}

/**
Expand Down Expand Up @@ -94,7 +96,7 @@ public ProcessContinuation processElement(
lastClaimed -> {
try {
committerFactory
.apply(subscriptionPartition)
.create(subscriptionPartition)
.commitOffset(Offset.of(lastClaimed.value() + 1));
} catch (Exception e) {
throw ExtractStatus.toCanonical(e).underlying;
Expand All @@ -114,7 +116,7 @@ public OffsetByteRange getInitialRestriction(
@NewTracker
public TrackerWithProgress newTracker(
@Element SubscriptionPartition subscriptionPartition, @Restriction OffsetByteRange range) {
return trackerFactory.apply(backlogReaderFactory.newReader(subscriptionPartition), range);
return trackerFactory.apply(backlogReaderFactory.create(subscriptionPartition), range);
}

@GetSize
Expand Down
Loading