queue consumer: implemented nonblocking subscribe method to prevent locks on repartition event. moved message decoding out of the lock to reduce locking time. using monotonic time instead wall-clock. added debug messages

This commit is contained in:
Sergey Matvienko 2021-03-31 22:01:08 +03:00 committed by Andrew Shvayka
parent 3362ab37da
commit b35ce062a7
2 changed files with 87 additions and 60 deletions

View File

@ -21,22 +21,28 @@ import org.thingsboard.server.common.msg.queue.TopicPartitionInfo;
import org.thingsboard.server.queue.TbQueueConsumer; import org.thingsboard.server.queue.TbQueueConsumer;
import org.thingsboard.server.queue.TbQueueMsg; import org.thingsboard.server.queue.TbQueueMsg;
import javax.annotation.Nonnull;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Queue;
import java.util.Set; import java.util.Set;
import java.util.concurrent.locks.Lock; import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import static java.util.Collections.emptyList;
@Slf4j @Slf4j
public abstract class AbstractTbQueueConsumerTemplate<R, T extends TbQueueMsg> implements TbQueueConsumer<T> { public abstract class AbstractTbQueueConsumerTemplate<R, T extends TbQueueMsg> implements TbQueueConsumer<T> {
private volatile boolean subscribed; private volatile boolean subscribed;
protected volatile boolean stopped = false; protected volatile boolean stopped = false;
protected volatile Set<TopicPartitionInfo> partitions; protected volatile Set<TopicPartitionInfo> partitions;
protected final Lock consumerLock = new ReentrantLock(); protected final ReentrantLock consumerLock = new ReentrantLock(); //NonfairSync
final Queue<Set<TopicPartitionInfo>> subscribeQueue = new ConcurrentLinkedQueue<>();
@Getter @Getter
private final String topic; private final String topic;
@ -47,84 +53,101 @@ public abstract class AbstractTbQueueConsumerTemplate<R, T extends TbQueueMsg> i
@Override @Override
public void subscribe() { public void subscribe() {
consumerLock.lock(); log.info("enqueue topic subscribe {} ", topic);
try { if (stopped) {
partitions = Collections.singleton(new TopicPartitionInfo(topic, null, null, true)); log.error("trying subscribe, but consumer stopped for topic {}", topic);
subscribed = false; return;
} finally {
consumerLock.unlock();
} }
subscribeQueue.add(Collections.singleton(new TopicPartitionInfo(topic, null, null, true)));
} }
@Override @Override
public void subscribe(Set<TopicPartitionInfo> partitions) { public void subscribe(Set<TopicPartitionInfo> partitions) {
consumerLock.lock(); log.info("enqueue topics subscribe {} ", partitions);
try { if (stopped) {
this.partitions = partitions; log.error("trying subscribe, but consumer stopped for topic {}", topic);
subscribed = false; return;
} finally {
consumerLock.unlock();
} }
subscribeQueue.add(partitions);
} }
@Override @Override
public List<T> poll(long durationInMillis) { public List<T> poll(long durationInMillis) {
List<R> records;
long startNanos = System.nanoTime();
if (stopped) {
return errorAndReturnEmpty();
}
if (!subscribed && partitions == null) { if (!subscribed && partitions == null) {
try { return sleepAndReturnEmpty(startNanos, durationInMillis);
Thread.sleep(durationInMillis); }
} catch (InterruptedException e) {
log.debug("Failed to await subscription", e);
}
} else {
long pollStartTs = System.currentTimeMillis();
consumerLock.lock();
try {
if (!subscribed) {
List<String> topicNames = partitions.stream().map(TopicPartitionInfo::getFullTopicName).collect(Collectors.toList());
doSubscribe(topicNames);
subscribed = true;
}
List<R> records; if (consumerLock.isLocked()) {
if (partitions.isEmpty()) { log.error("poll. consumerLock is locked. will wait with no timeout. it looks like a race conditions or deadlock", new RuntimeException("stacktrace"));
records = Collections.emptyList(); }
} else {
records = doPoll(durationInMillis); consumerLock.lock();
try {
while (!subscribeQueue.isEmpty()) {
subscribed = false;
partitions = subscribeQueue.poll();
}
if (!subscribed) {
List<String> topicNames = partitions.stream().map(TopicPartitionInfo::getFullTopicName).collect(Collectors.toList());
doSubscribe(topicNames);
subscribed = true;
}
records = partitions.isEmpty() ? emptyList() : doPoll(durationInMillis);
} finally {
consumerLock.unlock();
}
if (records.isEmpty()) { return sleepAndReturnEmpty(startNanos, durationInMillis); }
return decodeRecords(records);
}
@Nonnull
List<T> decodeRecords(@Nonnull List<R> records) {
List<T> result = new ArrayList<>(records.size());
records.forEach(record -> {
try {
if (record != null) {
result.add(decode(record));
} }
if (!records.isEmpty()) { } catch (IOException e) {
List<T> result = new ArrayList<>(records.size()); log.error("Failed decode record: [{}]", record);
records.forEach(record -> { throw new RuntimeException("Failed to decode record: ", e);
try { }
if (record != null) { });
result.add(decode(record)); return result;
} }
} catch (IOException e) {
log.error("Failed decode record: [{}]", record); List<T> errorAndReturnEmpty() {
throw new RuntimeException("Failed to decode record: ", e); log.error("poll invoked but consumer stopped for topic" + topic, new RuntimeException("stacktrace"));
} return emptyList();
}); }
return result;
} else { List<T> sleepAndReturnEmpty(final long startNanos, final long durationInMillis) {
long pollDuration = System.currentTimeMillis() - pollStartTs; long durationNanos = TimeUnit.MILLISECONDS.toNanos(durationInMillis);
if (pollDuration < durationInMillis) { long spentNanos = System.nanoTime() - startNanos;
try { if (spentNanos < durationNanos) {
Thread.sleep(durationInMillis - pollDuration); try {
} catch (InterruptedException e) { Thread.sleep(Math.max(TimeUnit.NANOSECONDS.toMillis(durationNanos - spentNanos), 1));
if (!stopped) { } catch (InterruptedException e) {
log.error("Failed to wait.", e); if (!stopped) {
} log.error("Failed to wait", e);
}
}
} }
} finally {
consumerLock.unlock();
} }
} }
return Collections.emptyList(); return emptyList();
} }
@Override @Override
public void commit() { public void commit() {
if (consumerLock.isLocked()) {
log.error("commit. consumerLock is locked. will wait with no timeout. it looks like a race conditions or deadlock", new RuntimeException("stacktrace"));
}
consumerLock.lock(); consumerLock.lock();
try { try {
doCommit(); doCommit();
@ -135,6 +158,7 @@ public abstract class AbstractTbQueueConsumerTemplate<R, T extends TbQueueMsg> i
@Override @Override
public void unsubscribe() { public void unsubscribe() {
log.info("unsubscribe topic and stop consumer {}", getTopic());
stopped = true; stopped = true;
consumerLock.lock(); consumerLock.lock();
try { try {

View File

@ -72,8 +72,10 @@ public class TbKafkaConsumerTemplate<T extends TbQueueMsg> extends AbstractTbQue
protected void doSubscribe(List<String> topicNames) { protected void doSubscribe(List<String> topicNames) {
if (!topicNames.isEmpty()) { if (!topicNames.isEmpty()) {
topicNames.forEach(admin::createTopicIfNotExists); topicNames.forEach(admin::createTopicIfNotExists);
log.info("subscribe topics {}", topicNames);
consumer.subscribe(topicNames); consumer.subscribe(topicNames);
} else { } else {
log.info("unsubscribe due to empty topic list");
consumer.unsubscribe(); consumer.unsubscribe();
} }
} }
@ -102,6 +104,7 @@ public class TbKafkaConsumerTemplate<T extends TbQueueMsg> extends AbstractTbQue
@Override @Override
protected void doUnsubscribe() { protected void doUnsubscribe() {
log.info("unsubscribe topic and close consumer for topic {}", getTopic());
if (consumer != null) { if (consumer != null) {
consumer.unsubscribe(); consumer.unsubscribe();
consumer.close(); consumer.close();