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

fix: Use BlockingPullSubscriber instead of BufferingPullSubscriber in kafka client #150

Merged
merged 1 commit into from Jun 15, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
@@ -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);
}
}