Skip to content

Commit

Permalink
fix: Use BlockingPullSubscriber instead of BufferingPullSubscriber in…
Browse files Browse the repository at this point in the history
… kafka client (#150)

This also makes polling reactive, and not have a minimum 100ms latency.
  • Loading branch information
dpcollins-google committed Jun 15, 2021
1 parent 59a8986 commit a3cdd44
Show file tree
Hide file tree
Showing 5 changed files with 197 additions and 92 deletions.
@@ -0,0 +1,34 @@
/*
* Copyright 2021 Google LLC
*
* Licensed 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 com.google.cloud.pubsublite.kafka;

import com.google.api.core.ApiFuture;
import com.google.api.core.SettableApiFuture;
import com.google.common.util.concurrent.MoreExecutors;
import java.util.Collection;

final class ApiFuturesExtensions {
private ApiFuturesExtensions() {}

public static <T> ApiFuture<Void> whenFirstDone(Collection<ApiFuture<T>> futures) {
SettableApiFuture<Void> someFutureDone = SettableApiFuture.create();
futures.forEach(
future ->
future.addListener(() -> someFutureDone.set(null), MoreExecutors.directExecutor()));
return someFutureDone;
}
}
Expand Up @@ -26,7 +26,7 @@
import com.google.cloud.pubsublite.SubscriptionPath;
import com.google.cloud.pubsublite.TopicPath;
import com.google.cloud.pubsublite.cloudpubsub.FlowControlSettings;
import com.google.cloud.pubsublite.internal.BufferingPullSubscriber;
import com.google.cloud.pubsublite.internal.BlockingPullSubscriberImpl;
import com.google.cloud.pubsublite.internal.CursorClient;
import com.google.cloud.pubsublite.internal.CursorClientSettings;
import com.google.cloud.pubsublite.internal.TopicStatsClient;
Expand Down Expand Up @@ -124,7 +124,7 @@ public Consumer<byte[], byte[]> instantiate() throws ApiException {
throw toCanonical(t).underlying;
}
};
return new BufferingPullSubscriber(
return new BlockingPullSubscriberImpl(
subscriberFactory, perPartitionFlowControlSettings());
};
CommitterFactory committerFactory =
Expand Down
Expand Up @@ -17,13 +17,12 @@
package com.google.cloud.pubsublite.kafka;

import com.google.cloud.pubsublite.Partition;
import com.google.cloud.pubsublite.SequencedMessage;
import com.google.cloud.pubsublite.internal.BlockingPullSubscriber;
import com.google.cloud.pubsublite.internal.CheckedApiException;
import com.google.cloud.pubsublite.internal.PullSubscriber;
import com.google.cloud.pubsublite.proto.SeekRequest;

/** A factory for making new PullSubscribers for a given partition of a subscription. */
interface PullSubscriberFactory {
PullSubscriber<SequencedMessage> newPullSubscriber(Partition partition, SeekRequest initial)
BlockingPullSubscriber newPullSubscriber(Partition partition, SeekRequest initial)
throws CheckedApiException;
}
Expand Up @@ -17,17 +17,20 @@
package com.google.cloud.pubsublite.kafka;

import static com.google.cloud.pubsublite.kafka.KafkaExceptionUtils.toKafka;
import static com.google.common.base.Preconditions.checkState;
import static java.util.concurrent.TimeUnit.MILLISECONDS;

import com.google.api.core.ApiFuture;
import com.google.api.core.ApiFutureCallback;
import com.google.api.core.ApiFutures;
import com.google.api.core.SettableApiFuture;
import com.google.cloud.pubsublite.Offset;
import com.google.cloud.pubsublite.Partition;
import com.google.cloud.pubsublite.SequencedMessage;
import com.google.cloud.pubsublite.TopicPath;
import com.google.cloud.pubsublite.internal.BlockingPullSubscriber;
import com.google.cloud.pubsublite.internal.CloseableMonitor;
import com.google.cloud.pubsublite.internal.ExtractStatus;
import com.google.cloud.pubsublite.internal.PullSubscriber;
import com.google.cloud.pubsublite.internal.wire.Committer;
import com.google.cloud.pubsublite.proto.SeekRequest;
import com.google.cloud.pubsublite.proto.SeekRequest.NamedTarget;
Expand All @@ -40,13 +43,13 @@
import com.google.errorprone.annotations.concurrent.GuardedBy;
import java.time.Duration;
import java.util.ArrayDeque;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.kafka.clients.consumer.CommitFailedException;
import org.apache.kafka.clients.consumer.ConsumerRecord;
Expand All @@ -67,17 +70,21 @@ class SingleSubscriptionConsumerImpl implements SingleSubscriptionConsumer {
private final CloseableMonitor monitor = new CloseableMonitor();

static class SubscriberState {
PullSubscriber<SequencedMessage> subscriber;
BlockingPullSubscriber subscriber;
Committer committer;
Optional<Offset> lastUncommitted = Optional.empty();
boolean needsCommitting = false;
Optional<Offset> lastReceived = Optional.empty();
}

@GuardedBy("monitor.monitor")
private Map<Partition, SubscriberState> partitions = new HashMap<>();

// Set to true when wakeup() has been called once.
private final Map<Partition, SubscriberState> partitions = new HashMap<>();
// When the set of assignments changes, this future will be set and swapped with a new future to
// let ongoing pollers know that they should pick up new assignments.
@GuardedBy("monitor.monitor")
private boolean wakeupTriggered = false;
private SettableApiFuture<Void> assignmentChanged = SettableApiFuture.create();

// Set when wakeup() has been called once.
private final SettableApiFuture<Void> wakeupTriggered = SettableApiFuture.create();

SingleSubscriptionConsumerImpl(
TopicPath topic,
Expand Down Expand Up @@ -118,6 +125,8 @@ public void setAssignment(Set<Partition> assignment) {
s.committer.startAsync().awaitRunning();
partitions.put(partition, s);
}));
assignmentChanged.set(null);
assignmentChanged = SettableApiFuture.create();
} catch (Throwable t) {
throw ExtractStatus.toCanonical(t).underlying;
}
Expand All @@ -136,27 +145,33 @@ private Map<Partition, Queue<SequencedMessage>> fetchAll() {
partitions.forEach(
ExtractStatus.rethrowAsRuntime(
(partition, state) -> {
List<SequencedMessage> messages = state.subscriber.pull();
if (messages.isEmpty()) return;
partitionQueues.computeIfAbsent(partition, x -> new ArrayDeque<>()).addAll(messages);
ArrayDeque<SequencedMessage> messages = new ArrayDeque<>();
for (Optional<SequencedMessage> message = state.subscriber.messageIfAvailable();
message.isPresent();
message = state.subscriber.messageIfAvailable()) {
messages.add(message.get());
}
partitionQueues.put(partition, messages);
}));
return partitionQueues;
}

private Map<Partition, Queue<SequencedMessage>> doPoll(Duration duration) {
try {
while (!duration.isZero()) {
try (CloseableMonitor.Hold h = monitor.enter()) {
if (wakeupTriggered) throw new WakeupException();
Map<Partition, Queue<SequencedMessage>> partitionQueues = fetchAll();
if (!partitionQueues.isEmpty()) return partitionQueues;
}
Duration sleepFor = Collections.min(ImmutableList.of(duration, Duration.ofMillis(10)));
Thread.sleep(sleepFor.toMillis());
duration = duration.minus(sleepFor);
ImmutableList.Builder<ApiFuture<Void>> stopSleepingSignals = ImmutableList.builder();
try (CloseableMonitor.Hold h = monitor.enter()) {
stopSleepingSignals.add(wakeupTriggered);
stopSleepingSignals.add(assignmentChanged);
partitions.values().forEach(state -> stopSleepingSignals.add(state.subscriber.onData()));
}
try {
ApiFuturesExtensions.whenFirstDone(stopSleepingSignals.build())
.get(duration.toMillis(), MILLISECONDS);
} catch (TimeoutException e) {
return ImmutableMap.of();
}
// Last fetch to handle duration originally being 0 and last time window sleep.
try (CloseableMonitor.Hold h = monitor.enter()) {
if (wakeupTriggered.isDone()) throw new WakeupException();
return fetchAll();
}
} catch (Throwable t) {
Expand All @@ -166,8 +181,6 @@ private Map<Partition, Queue<SequencedMessage>> doPoll(Duration duration) {

@Override
public ConsumerRecords<byte[], byte[]> poll(Duration duration) {
Map<Partition, Queue<SequencedMessage>> partitionQueues = doPoll(duration);
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> records = new HashMap<>();
if (autocommit) {
ApiFuture<?> future = commitAll();
ApiFutures.addCallback(
Expand All @@ -183,14 +196,16 @@ public void onSuccess(Object result) {}
},
MoreExecutors.directExecutor());
}
Map<Partition, Queue<SequencedMessage>> partitionQueues = doPoll(duration);
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> records = new HashMap<>();
partitionQueues.forEach(
(partition, queue) -> {
if (queue.isEmpty()) return;
try (CloseableMonitor.Hold h = monitor.enter()) {
SubscriberState state = partitions.getOrDefault(partition, null);
if (state != null) {
state.lastUncommitted = Optional.of(Iterables.getLast(queue).offset());
}
if (state == null) return;
state.lastReceived = Optional.of(Iterables.getLast(queue).offset());
state.needsCommitting = true;
}
List<ConsumerRecord<byte[], byte[]>> partitionRecords =
queue.stream()
Expand All @@ -207,17 +222,16 @@ public ApiFuture<Map<Partition, Offset>> commitAll() {
try (CloseableMonitor.Hold h = monitor.enter()) {
ImmutableMap.Builder<Partition, Offset> builder = ImmutableMap.builder();
ImmutableList.Builder<ApiFuture<?>> commitFutures = ImmutableList.builder();
partitions.entrySet().stream()
.filter(entry -> entry.getValue().lastUncommitted.isPresent())
.forEach(
entry -> {
// The Pub/Sub Lite commit offset is the next to be received.
Offset lastUncommitted = entry.getValue().lastUncommitted.get();
entry.getValue().lastUncommitted = Optional.empty();
Offset toCommit = Offset.of(lastUncommitted.value() + 1);
builder.put(entry.getKey(), toCommit);
commitFutures.add(entry.getValue().committer.commitOffset(toCommit));
});
partitions.forEach(
(partition, state) -> {
if (!state.needsCommitting) return;
checkState(state.lastReceived.isPresent());
state.needsCommitting = false;
// The Pub/Sub Lite commit offset is one more than the last received.
Offset toCommit = Offset.of(state.lastReceived.get().value() + 1);
builder.put(partition, toCommit);
commitFutures.add(state.committer.commitOffset(toCommit));
});
Map<Partition, Offset> map = builder.build();
return ApiFutures.transform(
ApiFutures.allAsList(commitFutures.build()),
Expand Down Expand Up @@ -269,7 +283,7 @@ public void doSeek(Partition partition, SeekRequest request) throws KafkaExcepti
@Override
public Optional<Long> position(Partition partition) {
if (!partitions.containsKey(partition)) return Optional.empty();
return partitions.get(partition).subscriber.nextOffset().map(Offset::value);
return partitions.get(partition).lastReceived.map(lastReceived -> lastReceived.value() + 1);
}

@Override
Expand All @@ -286,8 +300,6 @@ public void close(Duration duration) {

@Override
public void wakeup() {
try (CloseableMonitor.Hold h = monitor.enter()) {
wakeupTriggered = true;
}
wakeupTriggered.set(null);
}
}

0 comments on commit a3cdd44

Please sign in to comment.