Skip to content
This repository was archived by the owner on Jan 24, 2024. It is now read-only.

Commit 141536b

Browse files
authored
Issue 134: produce optimize (#150)
optimize for #134 including the following aspects: recordToEntry paralleled publish message to bookie asynchronous In our environment, produce delay is about one-fifth of the previous
1 parent 17ddbc2 commit 141536b

File tree

4 files changed

+297
-147
lines changed

4 files changed

+297
-147
lines changed

kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/KafkaRequestHandler.java

Lines changed: 117 additions & 78 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,8 @@
1919
import static io.streamnative.pulsar.handlers.kop.KafkaProtocolHandler.ListenerType.SSL;
2020
import static io.streamnative.pulsar.handlers.kop.KafkaProtocolHandler.getKopBrokerUrl;
2121
import static io.streamnative.pulsar.handlers.kop.KafkaProtocolHandler.getListenerPort;
22-
import static io.streamnative.pulsar.handlers.kop.MessagePublishContext.publishMessages;
22+
import static io.streamnative.pulsar.handlers.kop.MessagePublishContext.MESSAGE_BATCHED;
23+
import static io.streamnative.pulsar.handlers.kop.utils.MessageRecordUtils.recordsToByteBuf;
2324
import static io.streamnative.pulsar.handlers.kop.utils.TopicNameUtils.getKafkaTopicNameFromPulsarTopicname;
2425
import static io.streamnative.pulsar.handlers.kop.utils.TopicNameUtils.pulsarTopicName;
2526
import static java.nio.charset.StandardCharsets.UTF_8;
@@ -29,6 +30,7 @@
2930
import com.google.common.collect.Lists;
3031
import com.google.common.collect.Maps;
3132
import com.google.common.collect.Queues;
33+
import io.netty.buffer.ByteBuf;
3234
import io.netty.channel.ChannelHandlerContext;
3335
import io.streamnative.pulsar.handlers.kop.coordinator.group.GroupCoordinator;
3436
import io.streamnative.pulsar.handlers.kop.coordinator.group.GroupMetadata.GroupOverview;
@@ -38,6 +40,7 @@
3840
import io.streamnative.pulsar.handlers.kop.utils.MessageIdUtils;
3941
import io.streamnative.pulsar.handlers.kop.utils.OffsetFinder;
4042
import io.streamnative.pulsar.handlers.kop.utils.SaslUtils;
43+
4144
import java.io.IOException;
4245
import java.net.InetSocketAddress;
4346
import java.net.URI;
@@ -60,6 +63,7 @@
6063
import java.util.stream.Collectors;
6164
import java.util.stream.IntStream;
6265
import javax.naming.AuthenticationException;
66+
6367
import lombok.Getter;
6468
import lombok.extern.slf4j.Slf4j;
6569
import org.apache.bookkeeper.mledger.AsyncCallbacks;
@@ -496,92 +500,81 @@ protected void handleTopicMetadataRequest(KafkaHeaderAndRequest metadataHar,
496500
// whether the head of queue is running.
497501
private AtomicBoolean isHeadRequestRun = new AtomicBoolean(false);
498502

499-
private void handleProducerRequestInternal() {
500-
// the first request that success set to running, get running.
501-
if (produceRequestsQueue.isEmpty() || !isHeadRequestRun.compareAndSet(false, true)) {
502-
// the head of queue is already running, when head complete, it will peek the following request to run.
503-
if (log.isDebugEnabled()) {
504-
log.debug(" Produce messages not entered. queue.size: {}, head isHeadRequestRun: {}",
505-
produceRequestsQueue.size(), isHeadRequestRun.get());
506-
}
507-
return;
508-
}
509-
510-
Pair<KafkaHeaderAndRequest, CompletableFuture<AbstractResponse>> head = produceRequestsQueue.peek();
511-
KafkaHeaderAndRequest produceHar = head.getKey();
512-
CompletableFuture<AbstractResponse> resultFuture = head.getValue();
513-
ProduceRequest produceRequest = (ProduceRequest) produceHar.getRequest();
514-
515-
// Ignore request.acks() and request.timeout(), which related to kafka replication in this broker.
516-
Map<TopicPartition, CompletableFuture<PartitionResponse>> responsesFutures = new HashMap<>();
503+
private ConcurrentHashMap<TopicName, Queue<Pair<CompletableFuture<ByteBuf>, CompletableFuture<PartitionResponse>>>>
504+
transQueue = new ConcurrentHashMap<>();
517505

518-
final int responsesSize = produceRequest.partitionRecordsOrFail().size();
519-
520-
// TODO: handle un-exist topic:
521-
// nonExistingTopicResponses += topicPartition -> new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION)
522-
for (Map.Entry<TopicPartition, ? extends Records> entry : produceRequest.partitionRecordsOrFail().entrySet()) {
523-
TopicPartition topicPartition = entry.getKey();
524-
525-
CompletableFuture<PartitionResponse> partitionResponse = new CompletableFuture<>();
526-
responsesFutures.put(topicPartition, partitionResponse);
506+
private void publishMessages(MemoryRecords records,
507+
TopicName topic,
508+
CompletableFuture<PartitionResponse> future,
509+
CompletableFuture<ByteBuf> transFuture) {
510+
// get records size.
511+
AtomicInteger size = new AtomicInteger(0);
512+
records.records().forEach(record -> size.incrementAndGet());
513+
int rec = size.get();
527514

528-
if (log.isDebugEnabled()) {
529-
log.debug("[{}] Request {}: Produce messages for topic {} partition {}, request size: {} ",
530-
ctx.channel(), produceHar.getHeader(),
531-
topicPartition.topic(), topicPartition.partition(), responsesSize);
532-
}
515+
if (log.isDebugEnabled()) {
516+
log.debug("publishMessages for topic partition: {} , records size is {} ", topic.toString(), size.get());
517+
}
533518

534-
TopicName topicName = pulsarTopicName(topicPartition, namespace);
519+
if (MESSAGE_BATCHED) {
520+
pulsarService.getExecutor().submit(() -> {
521+
ByteBuf buf = recordsToByteBuf(records, rec);
522+
transFuture.complete(buf);
523+
});
535524

536-
topicManager.getTopic(topicName.toString()).whenComplete((persistentTopic, exception) -> {
537-
if (exception != null || persistentTopic == null) {
538-
log.warn("[{}] Request {}: Failed to getOrCreateTopic {}. "
539-
+ "Topic is in loading status, return LEADER_NOT_AVAILABLE. exception:",
540-
ctx.channel(), produceHar.getHeader(), topicName, exception);
541-
partitionResponse.complete(new PartitionResponse(Errors.LEADER_NOT_AVAILABLE));
525+
transFuture.whenComplete((headerAndPayload, ex) -> {
526+
if (ex != null) {
527+
log.error("record to bytebuf error: ", ex);
528+
future.complete(new PartitionResponse(Errors.KAFKA_STORAGE_ERROR));
542529
} else {
543-
CompletableFuture<PersistentTopic> topicFuture = new CompletableFuture<>();
544-
topicFuture.complete(persistentTopic);
545-
publishMessages((MemoryRecords) entry.getValue(), persistentTopic, partitionResponse);
530+
doPublishMessages(topic);
546531
}
547532
});
548533
}
534+
}
549535

550-
CompletableFuture.allOf(responsesFutures.values().toArray(new CompletableFuture<?>[responsesSize]))
551-
.whenComplete((ignore, ex) -> {
552-
// all ex has translated to PartitionResponse with Errors.KAFKA_STORAGE_ERROR
553-
Map<TopicPartition, PartitionResponse> responses = new ConcurrentHashMap<>();
554-
for (Map.Entry<TopicPartition, CompletableFuture<PartitionResponse>> entry:
555-
responsesFutures.entrySet()) {
556-
responses.put(entry.getKey(), entry.getValue().join());
557-
}
536+
private void doPublishMessages(TopicName topic) {
537+
Queue<Pair<CompletableFuture<ByteBuf>, CompletableFuture<PartitionResponse>>> topicQueue =
538+
transQueue.get(topic);
558539

540+
// loop from first responseFuture.
541+
while (topicQueue != null && topicQueue.peek() != null
542+
&& topicQueue.peek().getLeft().isDone() && isActive.get()) {
543+
CompletableFuture<Long> offsetFuture = new CompletableFuture<>();
544+
Pair<CompletableFuture<ByteBuf>, CompletableFuture<PartitionResponse>> result = topicQueue.remove();
545+
try {
559546
if (log.isDebugEnabled()) {
560-
log.debug("[{}] Request {}: Complete handle produce.",
561-
ctx.channel(), produceHar.toString());
547+
log.debug("[{}] topic", topic.toString());
562548
}
563-
resultFuture.complete(new ProduceResponse(responses));
564-
});
565-
566-
// trigger following request to run.
567-
resultFuture.whenComplete((response, throwable) -> {
568-
if (throwable != null) {
569-
log.warn("Error produce message for {}.", produceHar.getHeader(), throwable);
570-
}
549+
ByteBuf headerAndPayload = result.getLeft().get();
550+
topicManager.getTopic(topic.toString()).whenComplete((persistentTopic, throwable) -> {
551+
if (throwable != null || persistentTopic == null) {
552+
log.warn("[{}] Request {}: Failed to getOrCreateTopic {}. "
553+
+ "Topic is in loading status, return LEADER_NOT_AVAILABLE. exception:",
554+
ctx.channel(), topic.toString(), throwable);
555+
result.getRight().complete(new PartitionResponse(Errors.LEADER_NOT_AVAILABLE));
556+
} else {
557+
persistentTopic.publishMessage(
558+
headerAndPayload,
559+
MessagePublishContext.get(
560+
offsetFuture, persistentTopic, System.nanoTime()));
561+
}
562+
});
571563

572-
if (log.isDebugEnabled()) {
573-
log.debug("Produce messages complete. trigger next. queue.size: {}, head isHeadRequestRun: {}",
574-
produceRequestsQueue.size(), isHeadRequestRun.get());
564+
offsetFuture.whenComplete((offset, ex) -> {
565+
if (ex != null) {
566+
log.error("publishMessages for topic partition: {} failed when write.",
567+
topic.toString(), ex);
568+
result.getRight().complete(new PartitionResponse(Errors.KAFKA_STORAGE_ERROR));
569+
} else {
570+
result.getRight().complete(new PartitionResponse(Errors.NONE));
571+
}
572+
});
573+
} catch (Exception e) {
574+
// should not comes here.
575+
log.error("error to get Response ByteBuf:", e);
575576
}
576-
577-
boolean compare = isHeadRequestRun.compareAndSet(true, false);
578-
checkState(compare, "Head should be running when completed head.");
579-
// remove completed request.
580-
produceRequestsQueue.remove();
581-
582-
// trigger another run.
583-
handleProducerRequestInternal();
584-
});
577+
}
585578
}
586579

587580
protected void handleProduceRequest(KafkaHeaderAndRequest produceHar,
@@ -596,13 +589,59 @@ protected void handleProduceRequest(KafkaHeaderAndRequest produceHar,
596589
return;
597590
}
598591

599-
if (log.isDebugEnabled()) {
600-
log.debug(" new produce request comes: {}, isHeadRequestRun: {}",
601-
produceRequestsQueue.size(), isHeadRequestRun.get());
592+
Map<TopicPartition, CompletableFuture<PartitionResponse>> responsesFutures = new HashMap<>();
593+
594+
final int responsesSize = produceRequest.partitionRecordsOrFail().size();
595+
596+
// TODO: handle un-exist topic:
597+
// nonExistingTopicResponses += topicPartition -> new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION)
598+
for (Map.Entry<TopicPartition, ? extends Records> entry : produceRequest.partitionRecordsOrFail().entrySet()) {
599+
TopicPartition topicPartition = entry.getKey();
600+
601+
CompletableFuture<PartitionResponse> partitionResponse = new CompletableFuture<>();
602+
responsesFutures.put(topicPartition, partitionResponse);
603+
604+
if (log.isDebugEnabled()) {
605+
log.debug("[{}] Request {}: Produce messages for topic {} partition {}, request size: {} ",
606+
ctx.channel(), produceHar.getHeader(),
607+
topicPartition.topic(), topicPartition.partition(), responsesSize);
608+
}
609+
610+
TopicName topicName = pulsarTopicName(topicPartition, namespace);
611+
612+
CompletableFuture<ByteBuf> transFuture = new CompletableFuture<>();
613+
//put queue
614+
transQueue.compute(topicName, (key, queue) -> {
615+
if (queue == null) {
616+
Queue<Pair<CompletableFuture<ByteBuf>, CompletableFuture<PartitionResponse>>> newQueue =
617+
Queues.newConcurrentLinkedQueue();
618+
newQueue.add(Pair.of(transFuture, partitionResponse));
619+
return newQueue;
620+
} else {
621+
queue.add(Pair.of(transFuture, partitionResponse));
622+
return queue;
623+
}
624+
});
625+
626+
topicManager.getTopic(topicName.toString());
627+
publishMessages((MemoryRecords) entry.getValue(), topicName, partitionResponse, transFuture);
602628
}
603-
produceRequestsQueue.add(Pair.of(produceHar, resultFuture));
604629

605-
handleProducerRequestInternal();
630+
CompletableFuture.allOf(responsesFutures.values().toArray(new CompletableFuture<?>[responsesSize]))
631+
.whenComplete((ignore, ex) -> {
632+
// all ex has translated to PartitionResponse with Errors.KAFKA_STORAGE_ERROR
633+
Map<TopicPartition, PartitionResponse> responses = new ConcurrentHashMap<>();
634+
for (Map.Entry<TopicPartition, CompletableFuture<PartitionResponse>> entry :
635+
responsesFutures.entrySet()) {
636+
responses.put(entry.getKey(), entry.getValue().join());
637+
}
638+
639+
if (log.isDebugEnabled()) {
640+
log.debug("[{}] Request {}: Complete handle produce.",
641+
ctx.channel(), produceHar.toString());
642+
}
643+
resultFuture.complete(new ProduceResponse(responses));
644+
});
606645
}
607646

608647
protected void handleFindCoordinatorRequest(KafkaHeaderAndRequest findCoordinator,

kafka-impl/src/main/java/io/streamnative/pulsar/handlers/kop/KafkaTopicManager.java

Lines changed: 42 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import java.util.concurrent.ScheduledFuture;
2323
import java.util.concurrent.TimeUnit;
2424
import java.util.concurrent.locks.ReentrantReadWriteLock;
25+
2526
import lombok.Getter;
2627
import lombok.extern.slf4j.Slf4j;
2728
import org.apache.pulsar.broker.PulsarServerException;
@@ -218,8 +219,8 @@ private void lookupBroker(String topicName,
218219
long waitTimeMs = backoff.next();
219220

220221
if (backoff.isMandatoryStopMade()) {
221-
log.warn("[{}] getBroker for topic failed, retried too many times, return null. throwable: ",
222-
requestHandler.ctx.channel(), topicName, waitTimeMs, th);
222+
log.warn("[{}] getBroker for topic {} failed, retried too many times {}, return null."
223+
+ " throwable: ", requestHandler.ctx.channel(), topicName, waitTimeMs, th);
223224
retFuture.complete(null);
224225
} else {
225226
log.warn("[{}] getBroker for topic failed, will retry in {} ms. throwable: ",
@@ -260,58 +261,55 @@ public CompletableFuture<PersistentTopic> getTopic(String topicName) {
260261
rwLock.readLock().unlock();
261262
}
262263

263-
return topics.computeIfAbsent(topicName,
264-
t -> {
265-
getTopicBroker(t).whenCompleteAsync((ignore, th) -> {
266-
if (th != null || ignore == null) {
267-
log.warn("[{}] Failed getTopicBroker {}, return null PersistentTopic. throwable: ",
264+
return topics.computeIfAbsent(topicName, t -> {
265+
getTopicBroker(t).whenCompleteAsync((ignore, th) -> {
266+
if (th != null || ignore == null) {
267+
log.warn("[{}] Failed getTopicBroker {}, return null PersistentTopic. throwable: ",
268268
requestHandler.ctx.channel(), t, th);
269269

270-
// get topic broker returns null. topic should be removed from LookupCache.
271-
if (ignore == null) {
272-
removeLookupCache(topicName);
273-
}
270+
// get topic broker returns null. topic should be removed from LookupCache.
271+
if (ignore == null) {
272+
removeLookupCache(topicName);
273+
}
274+
275+
topicCompletableFuture.complete(null);
276+
return;
277+
}
278+
279+
if (log.isDebugEnabled()) {
280+
log.debug("[{}] getTopicBroker for {} in KafkaTopicManager. brokerAddress: {}",
281+
requestHandler.ctx.channel(), t, ignore);
282+
}
274283

284+
brokerService.getTopic(t, true).whenComplete((t2, throwable) -> {
285+
if (throwable != null) {
286+
log.error("[{}] Failed to getTopic {}. exception:",
287+
requestHandler.ctx.channel(), t, throwable);
288+
// failed to getTopic from current broker, remove cache, which added in getTopicBroker.
289+
removeLookupCache(t);
275290
topicCompletableFuture.complete(null);
276291
return;
277292
}
278293

279-
if (log.isDebugEnabled()) {
280-
log.debug("[{}] getTopicBroker for {} in KafkaTopicManager. brokerAddress: {}",
281-
requestHandler.ctx.channel(), t, ignore);
294+
try {
295+
if (t2.isPresent()) {
296+
PersistentTopic persistentTopic = (PersistentTopic) t2.get();
297+
references.putIfAbsent(t, registerInPersistentTopic(persistentTopic));
298+
topicCompletableFuture.complete(persistentTopic);
299+
} else {
300+
log.error("[{}]Get empty topic for name {}",
301+
requestHandler.ctx.channel(), t);
302+
topicCompletableFuture.complete(null);
303+
}
304+
} catch (Exception e) {
305+
log.error("[{}] Failed to get client in registerInPersistentTopic {}. exception:",
306+
requestHandler.ctx.channel(), t, e);
307+
topicCompletableFuture.complete(null);
282308
}
283-
284-
brokerService
285-
.getTopic(t, true)
286-
.whenComplete((t2, throwable) -> {
287-
if (throwable != null) {
288-
log.error("[{}] Failed to getTopic {}. exception:",
289-
requestHandler.ctx.channel(), t, throwable);
290-
// failed to getTopic from current broker, remove cache, which added in getTopicBroker.
291-
removeLookupCache(t);
292-
topicCompletableFuture.complete(null);
293-
return;
294-
}
295-
296-
try {
297-
if (t2.isPresent()) {
298-
PersistentTopic persistentTopic = (PersistentTopic) t2.get();
299-
references.putIfAbsent(t, registerInPersistentTopic(persistentTopic));
300-
topicCompletableFuture.complete(persistentTopic);
301-
} else {
302-
log.error("[{}]Get empty topic for name {}",
303-
requestHandler.ctx.channel(), t);
304-
topicCompletableFuture.complete(null);
305-
}
306-
} catch (Exception e) {
307-
log.error("[{}] Failed to get client in registerInPersistentTopic {}. exception:",
308-
requestHandler.ctx.channel(), t, e);
309-
topicCompletableFuture.complete(null);
310-
}
311-
});
312309
});
313-
return topicCompletableFuture;
314310
});
311+
return topicCompletableFuture;
312+
});
315313
}
316314

317315
// when channel close, release all the topics reference in persistentTopic

0 commit comments

Comments
 (0)