From 36d95cd4ff5ca4b2347628d31a9b8b2d8d5f3775 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Fri, 15 Dec 2023 10:01:16 -0700 Subject: [PATCH 01/18] Add experimental Kafka batch producer --- .../kaldb/server/OpenSearchBulkIngestApi.java | 97 ++++++++++++++++--- 1 file changed, 81 insertions(+), 16 deletions(-) diff --git a/kaldb/src/main/java/com/slack/kaldb/server/OpenSearchBulkIngestApi.java b/kaldb/src/main/java/com/slack/kaldb/server/OpenSearchBulkIngestApi.java index 32aaf4b760..d8079e1c81 100644 --- a/kaldb/src/main/java/com/slack/kaldb/server/OpenSearchBulkIngestApi.java +++ b/kaldb/src/main/java/com/slack/kaldb/server/OpenSearchBulkIngestApi.java @@ -28,12 +28,16 @@ import io.micrometer.core.instrument.Timer; import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics; import io.micrometer.prometheus.PrometheusMeterRegistry; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.UUID; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiPredicate; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -69,7 +73,7 @@ public class OpenSearchBulkIngestApi extends AbstractService { private final KafkaProducer kafkaProducer; private final KafkaClientMetrics kafkaMetrics; - private final ReentrantLock lockTransactionalProducer = new ReentrantLock(); + private Thread producerThread; @Override protected void doStart() { @@ -78,6 +82,7 @@ protected void doStart() { load(); datasetMetadataStore.addListener(datasetListener); LOG.info("OpenSearchBulkIngestAPI service started"); + producerThread = Thread.ofVirtual().start(this::run); notifyStarted(); } catch (Throwable t) { notifyFailed(t); @@ -89,6 +94,9 @@ protected void doStop() { try { LOG.info("Stopping OpenSearchBulkIngestApi service"); datasetMetadataStore.removeListener(datasetListener); + if (producerThread != null) { + producerThread.interrupt(); + } kafkaProducer.close(); if (kafkaMetrics != null) { kafkaMetrics.close(); @@ -193,7 +201,7 @@ public HttpResponse addDocument(String bulkRequest) { return HttpResponse.ofJson(TOO_MANY_REQUESTS, response); } } - BulkIngestResponse response = produceDocuments(docs); + BulkIngestResponse response = createRequest(docs).getResponse(); return HttpResponse.ofJson(response); } catch (Exception e) { LOG.error("Request failed ", e); @@ -202,6 +210,76 @@ public HttpResponse addDocument(String bulkRequest) { } } + static class BatchRequest { + private final Map> inputDocs; + private final SynchronousQueue internalResponse = new SynchronousQueue<>(); + + public BatchRequest(Map> inputDocs) { + this.inputDocs = inputDocs; + } + + public Map> getInputDocs() { + return inputDocs; + } + + public void setResponse(BulkIngestResponse response) { + internalResponse.add(response); + } + + public BulkIngestResponse getResponse() throws InterruptedException { + return internalResponse.take(); + } + } + + BlockingQueue pendingRequests = new ArrayBlockingQueue<>(500); + + public BatchRequest createRequest(Map> inputDocs) { + BatchRequest request = new BatchRequest(inputDocs); + // todo - add can throw exceptions + pendingRequests.add(request); + return request; + } + + public void run() { + while (true) { + List requests = new ArrayList<>(); + pendingRequests.drainTo(requests); + + if (requests.isEmpty()) { + try { + Thread.sleep(2000); + } catch (InterruptedException e) { + return; + } + } else { + try { + Map responseMap = new HashMap<>(); + kafkaProducer.beginTransaction(); + for (BatchRequest request : requests) { + responseMap.put(request, produceDocuments(request.getInputDocs())); + } + kafkaProducer.commitTransaction(); + responseMap.forEach(BatchRequest::setResponse); + } catch (Exception e) { + LOG.warn("failed transaction with error", e); + try { + kafkaProducer.abortTransaction(); + } catch (ProducerFencedException err) { + LOG.error("Could not abort transaction", err); + } + + for (BatchRequest request : requests) { + request.setResponse( + new BulkIngestResponse( + 0, + request.inputDocs.values().stream().mapToInt(List::size).sum(), + e.getMessage())); + } + } + } + } + } + @SuppressWarnings("FutureReturnValueIgnored") public BulkIngestResponse produceDocuments(Map> indexDocs) { int totalDocs = indexDocs.values().stream().mapToInt(List::size).sum(); @@ -225,9 +303,7 @@ public BulkIngestResponse produceDocuments(Map> indexDo // Till we fix the producer design to allow for multiple /_bulk requests to be able to // write to the same txn // we will limit producing documents 1 thread at a time - lockTransactionalProducer.lock(); try { - kafkaProducer.beginTransaction(); for (Trace.Span doc : indexDoc.getValue()) { ProducerRecord producerRecord = new ProducerRecord<>( @@ -238,7 +314,6 @@ public BulkIngestResponse produceDocuments(Map> indexDo // messages kafkaProducer.send(producerRecord); } - kafkaProducer.commitTransaction(); } catch (TimeoutException te) { LOG.error("Commit transaction timeout", te); // the commitTransaction waits till "max.block.ms" after which it will time out @@ -255,16 +330,6 @@ public BulkIngestResponse produceDocuments(Map> indexDo // We can't recover from these exceptions, so our only option is to close the producer and // exit. new RuntimeHalterImpl().handleFatal(new Throwable("KafkaProducer needs to shutdown ", e)); - } catch (Exception e) { - LOG.warn("failed transaction with error", e); - try { - kafkaProducer.abortTransaction(); - } catch (ProducerFencedException err) { - LOG.error("Could not abort transaction", err); - } - return new BulkIngestResponse(0, totalDocs, e.getMessage()); - } finally { - lockTransactionalProducer.unlock(); } } From 3119f04c6ebb77adad89ecebd2a8ae3b9a7d4fdd Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Fri, 15 Dec 2023 12:41:14 -0700 Subject: [PATCH 02/18] Initial refactor of experimental batch producer --- .../kaldb/bulkIngestApi/BatchRequest.java | 27 ++ .../kaldb/bulkIngestApi/BulkIngestApi.java | 68 ++++ .../BulkIngestResponse.java | 2 +- .../DatasetRateLimitingService.java | 70 ++++ .../TransactionBatchingKafkaProducer.java | 247 ++++++++++++ .../java/com/slack/kaldb/server/Kaldb.java | 18 +- .../kaldb/server/OpenSearchBulkIngestApi.java | 365 ------------------ .../TransactionBatchingKafkaProducerTest.java | 262 +++++++++++++ ...estApiTest.java => BulkIngestApiTest.java} | 213 +++++----- 9 files changed, 776 insertions(+), 496 deletions(-) create mode 100644 kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BatchRequest.java create mode 100644 kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java rename kaldb/src/main/java/com/slack/kaldb/{elasticsearchApi => bulkIngestApi}/BulkIngestResponse.java (67%) create mode 100644 kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java create mode 100644 kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducer.java delete mode 100644 kaldb/src/main/java/com/slack/kaldb/server/OpenSearchBulkIngestApi.java create mode 100644 kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducerTest.java rename kaldb/src/test/java/com/slack/kaldb/server/{OpenSearchBulkIngestApiTest.java => BulkIngestApiTest.java} (64%) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BatchRequest.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BatchRequest.java new file mode 100644 index 0000000000..d1ff7b24ce --- /dev/null +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BatchRequest.java @@ -0,0 +1,27 @@ +package com.slack.kaldb.bulkIngestApi; + +import com.slack.service.murron.trace.Trace; +import java.util.List; +import java.util.Map; +import java.util.concurrent.SynchronousQueue; + +public class BatchRequest { + private final Map> inputDocs; + private final SynchronousQueue internalResponse = new SynchronousQueue<>(); + + protected BatchRequest(Map> inputDocs) { + this.inputDocs = inputDocs; + } + + Map> getInputDocs() { + return inputDocs; + } + + void setResponse(BulkIngestResponse response) { + internalResponse.add(response); + } + + public BulkIngestResponse getResponse() throws InterruptedException { + return internalResponse.take(); + } +} diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java new file mode 100644 index 0000000000..bdc41d3f67 --- /dev/null +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java @@ -0,0 +1,68 @@ +package com.slack.kaldb.bulkIngestApi; + +import static com.linecorp.armeria.common.HttpStatus.INTERNAL_SERVER_ERROR; +import static com.linecorp.armeria.common.HttpStatus.TOO_MANY_REQUESTS; + +import com.linecorp.armeria.common.HttpResponse; +import com.linecorp.armeria.server.annotation.Blocking; +import com.linecorp.armeria.server.annotation.Post; +import com.slack.kaldb.preprocessor.ingest.OpenSearchBulkApiRequestParser; +import com.slack.service.murron.trace.Trace; +import java.util.List; +import java.util.Map; +import org.opensearch.action.index.IndexRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BulkIngestApi { + private static final Logger LOG = LoggerFactory.getLogger(BulkIngestApi.class); + private final TransactionBatchingKafkaProducer transactionBatchingKafkaProducer; + private final DatasetRateLimitingService datasetRateLimitingService; + + public BulkIngestApi( + TransactionBatchingKafkaProducer transactionBatchingKafkaProducer, + DatasetRateLimitingService datasetRateLimitingService) { + + this.transactionBatchingKafkaProducer = transactionBatchingKafkaProducer; + this.datasetRateLimitingService = datasetRateLimitingService; + } + + /** + * 1. Kaldb does not support the concept of "updates". It's always an add 2. The "index" is used + * as the span name + */ + @Blocking + @Post("/_bulk") + public HttpResponse addDocument(String bulkRequest) { + try { + List indexRequests = + OpenSearchBulkApiRequestParser.parseBulkRequest(bulkRequest); + Map> docs = + OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + + // todo - our rate limiter doesn't have a way to acquire permits across multiple datasets + // so today as a limitation we reject any request that has documents against multiple indexes + // We think most indexing requests will be against 1 index + if (docs.keySet().size() > 1) { + BulkIngestResponse response = + new BulkIngestResponse(0, 0, "request must contain only 1 unique index"); + return HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response); + } + + for (Map.Entry> indexDocs : docs.entrySet()) { + final String index = indexDocs.getKey(); + if (!datasetRateLimitingService.test(index, indexDocs.getValue())) { + BulkIngestResponse response = new BulkIngestResponse(0, 0, "rate limit exceeded"); + return HttpResponse.ofJson(TOO_MANY_REQUESTS, response); + } + } + BulkIngestResponse response = + transactionBatchingKafkaProducer.createRequest(docs).getResponse(); + return HttpResponse.ofJson(response); + } catch (Exception e) { + LOG.error("Request failed ", e); + BulkIngestResponse response = new BulkIngestResponse(0, 0, e.getMessage()); + return HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response); + } + } +} diff --git a/kaldb/src/main/java/com/slack/kaldb/elasticsearchApi/BulkIngestResponse.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestResponse.java similarity index 67% rename from kaldb/src/main/java/com/slack/kaldb/elasticsearchApi/BulkIngestResponse.java rename to kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestResponse.java index 8d40b9d867..c3be3bec47 100644 --- a/kaldb/src/main/java/com/slack/kaldb/elasticsearchApi/BulkIngestResponse.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestResponse.java @@ -1,3 +1,3 @@ -package com.slack.kaldb.elasticsearchApi; +package com.slack.kaldb.bulkIngestApi; public record BulkIngestResponse(int totalDocs, long failedDocs, String errorMsg) {} diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java new file mode 100644 index 0000000000..53cfc53fc3 --- /dev/null +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java @@ -0,0 +1,70 @@ +package com.slack.kaldb.bulkIngestApi; + +import com.google.common.util.concurrent.AbstractIdleService; +import com.slack.kaldb.metadata.core.KaldbMetadataStoreChangeListener; +import com.slack.kaldb.metadata.dataset.DatasetMetadata; +import com.slack.kaldb.metadata.dataset.DatasetMetadataStore; +import com.slack.kaldb.preprocessor.PreprocessorRateLimiter; +import com.slack.kaldb.proto.config.KaldbConfigs; +import com.slack.service.murron.trace.Trace; +import io.micrometer.core.instrument.Timer; +import io.micrometer.prometheus.PrometheusMeterRegistry; +import java.util.List; +import java.util.function.BiPredicate; + +public class DatasetRateLimitingService extends AbstractIdleService { + private final DatasetMetadataStore datasetMetadataStore; + private final KaldbMetadataStoreChangeListener datasetListener = + (datasetMetadata) -> updateRateLimiter(); + + private final PreprocessorRateLimiter rateLimiter; + private BiPredicate> rateLimiterPredicate; + + private final PrometheusMeterRegistry meterRegistry; + public static final String RATE_LIMIT_RELOAD_TIMER = + "preprocessor_dataset_rate_limit_reload_timer"; + private final Timer rateLimitReloadtimer; + + public DatasetRateLimitingService( + DatasetMetadataStore datasetMetadataStore, + KaldbConfigs.PreprocessorConfig preprocessorConfig, + PrometheusMeterRegistry meterRegistry) { + this.datasetMetadataStore = datasetMetadataStore; + this.meterRegistry = meterRegistry; + + this.rateLimiter = + new PreprocessorRateLimiter( + meterRegistry, + preprocessorConfig.getPreprocessorInstanceCount(), + preprocessorConfig.getRateLimiterMaxBurstSeconds(), + true); + + this.rateLimitReloadtimer = meterRegistry.timer(RATE_LIMIT_RELOAD_TIMER); + } + + private void updateRateLimiter() { + Timer.Sample sample = Timer.start(meterRegistry); + try { + List datasetMetadataList = datasetMetadataStore.listSync(); + this.rateLimiterPredicate = rateLimiter.createBulkIngestRateLimiter(datasetMetadataList); + } finally { + // TODO: re-work this so that we can add success/failure tags and capture them + sample.stop(rateLimitReloadtimer); + } + } + + @Override + protected void startUp() throws Exception { + updateRateLimiter(); + datasetMetadataStore.addListener(datasetListener); + } + + @Override + protected void shutDown() throws Exception { + datasetMetadataStore.removeListener(datasetListener); + } + + public boolean test(String index, List value) { + return rateLimiterPredicate.test(index, value); + } +} diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducer.java new file mode 100644 index 0000000000..c65093c7f5 --- /dev/null +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducer.java @@ -0,0 +1,247 @@ +package com.slack.kaldb.bulkIngestApi; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.slack.kaldb.metadata.dataset.DatasetMetadata.MATCH_ALL_SERVICE; +import static com.slack.kaldb.metadata.dataset.DatasetMetadata.MATCH_STAR_SERVICE; + +import com.google.common.util.concurrent.AbstractExecutionThreadService; +import com.slack.kaldb.metadata.core.KaldbMetadataStoreChangeListener; +import com.slack.kaldb.metadata.dataset.DatasetMetadata; +import com.slack.kaldb.metadata.dataset.DatasetMetadataStore; +import com.slack.kaldb.preprocessor.PreprocessorService; +import com.slack.kaldb.proto.config.KaldbConfigs; +import com.slack.kaldb.util.RuntimeHalterImpl; +import com.slack.service.murron.trace.Trace; +import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics; +import io.micrometer.prometheus.PrometheusMeterRegistry; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.AuthorizationException; +import org.apache.kafka.common.errors.OutOfOrderSequenceException; +import org.apache.kafka.common.errors.ProducerFencedException; +import org.apache.kafka.common.errors.TimeoutException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TransactionBatchingKafkaProducer extends AbstractExecutionThreadService { + private static final Logger LOG = LoggerFactory.getLogger(TransactionBatchingKafkaProducer.class); + + private final KafkaProducer kafkaProducer; + private final KafkaClientMetrics kafkaMetrics; + + private final KaldbConfigs.PreprocessorConfig preprocessorConfig; + + private final DatasetMetadataStore datasetMetadataStore; + private final KaldbMetadataStoreChangeListener datasetListener = + (datasetMetadata) -> cacheSortedDataset(); + + protected List throughputSortedDatasets; + + private final BlockingQueue pendingRequests = new ArrayBlockingQueue<>(500); + + public TransactionBatchingKafkaProducer( + final DatasetMetadataStore datasetMetadataStore, + final KaldbConfigs.PreprocessorConfig preprocessorConfig, + final PrometheusMeterRegistry meterRegistry) { + checkArgument( + !preprocessorConfig.getBootstrapServers().isEmpty(), + "Kafka bootstrapServers must be provided"); + + checkArgument( + !preprocessorConfig.getDownstreamTopic().isEmpty(), + "Kafka downstreamTopic must be provided"); + + this.preprocessorConfig = preprocessorConfig; + this.datasetMetadataStore = datasetMetadataStore; + + // since we use a new transaction ID every time we start a preprocessor there can be some zombie + // transactions? + // I think they will remain in kafka till they expire. They should never be readable if the + // consumer sets isolation.level as "read_committed" + // see "zombie fencing" https://www.confluent.io/blog/transactions-apache-kafka/ + this.kafkaProducer = createKafkaTransactionProducer(UUID.randomUUID().toString()); + + this.kafkaMetrics = new KafkaClientMetrics(kafkaProducer); + this.kafkaMetrics.bindTo(meterRegistry); + + this.kafkaProducer.initTransactions(); + } + + private void cacheSortedDataset() { + // we sort the datasets to rank from which dataset do we start matching candidate service names + // in the future we can change the ordering from sort to something else + this.throughputSortedDatasets = + datasetMetadataStore.listSync().stream() + .sorted(Comparator.comparingLong(DatasetMetadata::getThroughputBytes).reversed()) + .toList(); + } + + @Override + protected void startUp() throws Exception { + cacheSortedDataset(); + datasetMetadataStore.addListener(datasetListener); + } + + @Override + protected void run() throws Exception { + while (isRunning()) { + List requests = new ArrayList<>(); + pendingRequests.drainTo(requests); + if (requests.isEmpty()) { + try { + Thread.sleep(2000); + } catch (InterruptedException e) { + return; + } + } else { + transactionCommit(requests); + } + } + } + + @Override + protected void shutDown() throws Exception { + datasetMetadataStore.removeListener(datasetListener); + + kafkaProducer.close(); + if (kafkaMetrics != null) { + kafkaMetrics.close(); + } + } + + public BatchRequest createRequest(Map> inputDocs) { + BatchRequest request = new BatchRequest(inputDocs); + // todo - add can throw exceptions + pendingRequests.add(request); + return request; + } + + protected Map transactionCommit(List requests) { + Map responseMap = new HashMap<>(); + try { + kafkaProducer.beginTransaction(); + for (BatchRequest request : requests) { + responseMap.put(request, produceDocuments(request.getInputDocs())); + } + kafkaProducer.commitTransaction(); + } catch (Exception e) { + LOG.warn("failed transaction with error", e); + try { + kafkaProducer.abortTransaction(); + } catch (ProducerFencedException err) { + LOG.error("Could not abort transaction", err); + } + + for (BatchRequest request : requests) { + responseMap.put( + request, + new BulkIngestResponse( + 0, + request.getInputDocs().values().stream().mapToInt(List::size).sum(), + e.getMessage())); + } + } + + for (Map.Entry entry : responseMap.entrySet()) { + BatchRequest key = entry.getKey(); + BulkIngestResponse value = entry.getValue(); + key.setResponse(value); + } + return responseMap; + } + + @SuppressWarnings("FutureReturnValueIgnored") + private BulkIngestResponse produceDocuments(Map> indexDocs) { + int totalDocs = indexDocs.values().stream().mapToInt(List::size).sum(); + + // we cannot create a generic pool of producers because the kafka API expects the transaction ID + // to be a property while creating the producer object. + for (Map.Entry> indexDoc : indexDocs.entrySet()) { + String index = indexDoc.getKey(); + + // call once per batch and use the same partition for better batching + // todo - this probably shouldn't be tied to the transaction batching logic? + int partition = getPartition(index); + + // since there isn't a dataset provisioned for this service/index we will not index this set + // of docs + if (partition < 0) { + LOG.warn("index=" + index + " does not have a provisioned dataset associated with it"); + continue; + } + + // KafkaProducer does not allow creating multiple transactions from a single object - + // rightfully so. + // Till we fix the producer design to allow for multiple /_bulk requests to be able to + // write to the same txn + // we will limit producing documents 1 thread at a time + try { + for (Trace.Span doc : indexDoc.getValue()) { + ProducerRecord producerRecord = + new ProducerRecord<>( + preprocessorConfig.getDownstreamTopic(), partition, index, doc.toByteArray()); + + // we intentionally supress FutureReturnValueIgnored here in errorprone - this is because + // we wrap this in a transaction, which is responsible for flushing all of the pending + // messages + kafkaProducer.send(producerRecord); + } + } catch (TimeoutException te) { + LOG.error("Commit transaction timeout", te); + // the commitTransaction waits till "max.block.ms" after which it will time out + // in that case we cannot call abort exception because that throws the following error + // "Cannot attempt operation `abortTransaction` because the previous + // call to `commitTransaction` timed out and must be retried" + // so for now we just restart the preprocessor + new RuntimeHalterImpl() + .handleFatal( + new Throwable( + "KafkaProducer needs to shutdown as we don't have retry yet and we cannot call abortTxn on timeout", + te)); + } catch (ProducerFencedException | OutOfOrderSequenceException | AuthorizationException e) { + // We can't recover from these exceptions, so our only option is to close the producer and + // exit. + new RuntimeHalterImpl().handleFatal(new Throwable("KafkaProducer needs to shutdown ", e)); + } + } + + return new BulkIngestResponse(totalDocs, 0, ""); + } + + private KafkaProducer createKafkaTransactionProducer(String transactionId) { + Properties props = new Properties(); + props.put("bootstrap.servers", preprocessorConfig.getBootstrapServers()); + props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); + props.put("transactional.id", transactionId); + props.put("linger.ms", 250); + props.put("max.block.ms", "10000"); + props.put("compression.type", "snappy"); + return new KafkaProducer<>(props); + } + + private int getPartition(String index) { + for (DatasetMetadata datasetMetadata : throughputSortedDatasets) { + String serviceNamePattern = datasetMetadata.getServiceNamePattern(); + + if (serviceNamePattern.equals(MATCH_ALL_SERVICE) + || serviceNamePattern.equals(MATCH_STAR_SERVICE) + || index.equals(serviceNamePattern)) { + List partitions = PreprocessorService.getActivePartitionList(datasetMetadata); + return partitions.get(ThreadLocalRandom.current().nextInt(partitions.size())); + } + } + // We don't have a provisioned service for this index + return -1; + } +} diff --git a/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java b/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java index 4e2a94ab15..291fc7543f 100644 --- a/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java +++ b/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java @@ -5,6 +5,9 @@ import com.google.common.util.concurrent.ServiceManager; import com.slack.kaldb.blobfs.BlobFs; import com.slack.kaldb.blobfs.s3.S3CrtBlobFs; +import com.slack.kaldb.bulkIngestApi.BulkIngestApi; +import com.slack.kaldb.bulkIngestApi.DatasetRateLimitingService; +import com.slack.kaldb.bulkIngestApi.TransactionBatchingKafkaProducer; import com.slack.kaldb.chunkManager.CachingChunkManager; import com.slack.kaldb.chunkManager.IndexingChunkManager; import com.slack.kaldb.clusterManager.ClusterHpaMetricService; @@ -388,12 +391,17 @@ private static Set getServices( KaldbConfigs.NodeRole.PREPROCESSOR, List.of(datasetMetadataStore))); if (preprocessorConfig.getUseBulkApi()) { - // TODO: using an armeria service that is also a guava service does not look elegant - // explore ways where we can control the lifecycle without the need for a guava service here - OpenSearchBulkIngestApi openSearchBulkApiService = - new OpenSearchBulkIngestApi(datasetMetadataStore, preprocessorConfig, meterRegistry); + TransactionBatchingKafkaProducer transactionBatchingKafkaProducer = + new TransactionBatchingKafkaProducer( + datasetMetadataStore, preprocessorConfig, meterRegistry); + services.add(transactionBatchingKafkaProducer); + DatasetRateLimitingService datasetRateLimitingService = + new DatasetRateLimitingService(datasetMetadataStore, preprocessorConfig, meterRegistry); + services.add(datasetRateLimitingService); + + BulkIngestApi openSearchBulkApiService = + new BulkIngestApi(transactionBatchingKafkaProducer, datasetRateLimitingService); armeriaServiceBuilder.withAnnotatedService(openSearchBulkApiService); - services.add(openSearchBulkApiService); } else { PreprocessorService preprocessorService = new PreprocessorService(datasetMetadataStore, preprocessorConfig, meterRegistry); diff --git a/kaldb/src/main/java/com/slack/kaldb/server/OpenSearchBulkIngestApi.java b/kaldb/src/main/java/com/slack/kaldb/server/OpenSearchBulkIngestApi.java deleted file mode 100644 index d8079e1c81..0000000000 --- a/kaldb/src/main/java/com/slack/kaldb/server/OpenSearchBulkIngestApi.java +++ /dev/null @@ -1,365 +0,0 @@ -package com.slack.kaldb.server; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkState; -import static com.linecorp.armeria.common.HttpStatus.INTERNAL_SERVER_ERROR; -import static com.linecorp.armeria.common.HttpStatus.TOO_MANY_REQUESTS; -import static com.slack.kaldb.metadata.dataset.DatasetMetadata.MATCH_ALL_SERVICE; -import static com.slack.kaldb.metadata.dataset.DatasetMetadata.MATCH_STAR_SERVICE; -import static com.slack.kaldb.preprocessor.PreprocessorService.CONFIG_RELOAD_TIMER; -import static com.slack.kaldb.preprocessor.PreprocessorService.INITIALIZE_RATE_LIMIT_WARM; -import static com.slack.kaldb.preprocessor.PreprocessorService.filterValidDatasetMetadata; -import static com.slack.kaldb.preprocessor.PreprocessorService.sortDatasetsOnThroughput; - -import com.google.common.util.concurrent.AbstractService; -import com.linecorp.armeria.common.HttpResponse; -import com.linecorp.armeria.server.annotation.Blocking; -import com.linecorp.armeria.server.annotation.Post; -import com.slack.kaldb.elasticsearchApi.BulkIngestResponse; -import com.slack.kaldb.metadata.core.KaldbMetadataStoreChangeListener; -import com.slack.kaldb.metadata.dataset.DatasetMetadata; -import com.slack.kaldb.metadata.dataset.DatasetMetadataStore; -import com.slack.kaldb.preprocessor.PreprocessorRateLimiter; -import com.slack.kaldb.preprocessor.PreprocessorService; -import com.slack.kaldb.preprocessor.ingest.OpenSearchBulkApiRequestParser; -import com.slack.kaldb.proto.config.KaldbConfigs; -import com.slack.kaldb.util.RuntimeHalterImpl; -import com.slack.service.murron.trace.Trace; -import io.micrometer.core.instrument.Timer; -import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics; -import io.micrometer.prometheus.PrometheusMeterRegistry; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.UUID; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadLocalRandom; -import java.util.function.BiPredicate; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.errors.AuthorizationException; -import org.apache.kafka.common.errors.OutOfOrderSequenceException; -import org.apache.kafka.common.errors.ProducerFencedException; -import org.apache.kafka.common.errors.TimeoutException; -import org.opensearch.action.index.IndexRequest; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * batching is important - if we send one doc a time we will create a transaction per request which - * is expensive - */ -public class OpenSearchBulkIngestApi extends AbstractService { - - private static final Logger LOG = LoggerFactory.getLogger(OpenSearchBulkIngestApi.class); - private final PrometheusMeterRegistry meterRegistry; - - private final KaldbConfigs.PreprocessorConfig preprocessorConfig; - private final DatasetMetadataStore datasetMetadataStore; - - private final KaldbMetadataStoreChangeListener datasetListener = - (datasetMetadata) -> load(); - - private final PreprocessorRateLimiter rateLimiter; - private BiPredicate> rateLimiterPredicate; - protected List throughputSortedDatasets; - - private final Timer configReloadTimer; - - private final KafkaProducer kafkaProducer; - private final KafkaClientMetrics kafkaMetrics; - - private Thread producerThread; - - @Override - protected void doStart() { - try { - LOG.info("Starting OpenSearchBulkIngestApi service"); - load(); - datasetMetadataStore.addListener(datasetListener); - LOG.info("OpenSearchBulkIngestAPI service started"); - producerThread = Thread.ofVirtual().start(this::run); - notifyStarted(); - } catch (Throwable t) { - notifyFailed(t); - } - } - - @Override - protected void doStop() { - try { - LOG.info("Stopping OpenSearchBulkIngestApi service"); - datasetMetadataStore.removeListener(datasetListener); - if (producerThread != null) { - producerThread.interrupt(); - } - kafkaProducer.close(); - if (kafkaMetrics != null) { - kafkaMetrics.close(); - } - LOG.info("OpenSearchBulkIngestApi service closed"); - notifyStopped(); - } catch (Throwable t) { - notifyFailed(t); - } - } - - public void load() { - Timer.Sample sample = Timer.start(meterRegistry); - try { - List datasetMetadataList = datasetMetadataStore.listSync(); - // only attempt to register stream processing on valid dataset configurations - List datasetMetadataToProcesses = - filterValidDatasetMetadata(datasetMetadataList); - - checkState(!datasetMetadataToProcesses.isEmpty(), "dataset metadata list must not be empty"); - - this.throughputSortedDatasets = sortDatasetsOnThroughput(datasetMetadataToProcesses); - this.rateLimiterPredicate = - rateLimiter.createBulkIngestRateLimiter(datasetMetadataToProcesses); - } catch (Exception e) { - notifyFailed(e); - } finally { - // TODO: re-work this so that we can add success/failure tags and capture them - sample.stop(configReloadTimer); - } - } - - public OpenSearchBulkIngestApi( - DatasetMetadataStore datasetMetadataStore, - KaldbConfigs.PreprocessorConfig preprocessorConfig, - PrometheusMeterRegistry meterRegistry) { - this(datasetMetadataStore, preprocessorConfig, meterRegistry, INITIALIZE_RATE_LIMIT_WARM); - } - - public OpenSearchBulkIngestApi( - DatasetMetadataStore datasetMetadataStore, - KaldbConfigs.PreprocessorConfig preprocessorConfig, - PrometheusMeterRegistry meterRegistry, - boolean initializeRateLimitWarm) { - - checkArgument( - !preprocessorConfig.getBootstrapServers().isEmpty(), - "Kafka bootstrapServers must be provided"); - - checkArgument( - !preprocessorConfig.getDownstreamTopic().isEmpty(), - "Kafka downstreamTopic must be provided"); - - this.datasetMetadataStore = datasetMetadataStore; - this.preprocessorConfig = preprocessorConfig; - this.meterRegistry = meterRegistry; - this.rateLimiter = - new PreprocessorRateLimiter( - meterRegistry, - preprocessorConfig.getPreprocessorInstanceCount(), - preprocessorConfig.getRateLimiterMaxBurstSeconds(), - initializeRateLimitWarm); - - this.configReloadTimer = meterRegistry.timer(CONFIG_RELOAD_TIMER); - - // since we use a new transaction ID every time we start a preprocessor there can be some zombie - // transactions? - // I think they will remain in kafka till they expire. They should never be readable if the - // consumer sets isolation.level as "read_committed" - // see "zombie fencing" https://www.confluent.io/blog/transactions-apache-kafka/ - this.kafkaProducer = createKafkaTransactionProducer(UUID.randomUUID().toString()); - kafkaMetrics = new KafkaClientMetrics(kafkaProducer); - kafkaMetrics.bindTo(meterRegistry); - this.kafkaProducer.initTransactions(); - } - - /** - * 1. Kaldb does not support the concept of "updates". It's always an add 2. The "index" is used - * as the span name - */ - @Blocking - @Post("/_bulk") - public HttpResponse addDocument(String bulkRequest) { - try { - List indexRequests = - OpenSearchBulkApiRequestParser.parseBulkRequest(bulkRequest); - Map> docs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); - // our rate limiter doesn't have a way to acquire permits across multiple datasets - // so today as a limitation we reject any request that has documents against multiple indexes - // We think most indexing requests will be against 1 index - if (docs.keySet().size() > 1) { - BulkIngestResponse response = - new BulkIngestResponse(0, 0, "request must contain only 1 unique index"); - return HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response); - } - - for (Map.Entry> indexDocs : docs.entrySet()) { - final String index = indexDocs.getKey(); - if (!rateLimiterPredicate.test(index, indexDocs.getValue())) { - BulkIngestResponse response = new BulkIngestResponse(0, 0, "rate limit exceeded"); - return HttpResponse.ofJson(TOO_MANY_REQUESTS, response); - } - } - BulkIngestResponse response = createRequest(docs).getResponse(); - return HttpResponse.ofJson(response); - } catch (Exception e) { - LOG.error("Request failed ", e); - BulkIngestResponse response = new BulkIngestResponse(0, 0, e.getMessage()); - return HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response); - } - } - - static class BatchRequest { - private final Map> inputDocs; - private final SynchronousQueue internalResponse = new SynchronousQueue<>(); - - public BatchRequest(Map> inputDocs) { - this.inputDocs = inputDocs; - } - - public Map> getInputDocs() { - return inputDocs; - } - - public void setResponse(BulkIngestResponse response) { - internalResponse.add(response); - } - - public BulkIngestResponse getResponse() throws InterruptedException { - return internalResponse.take(); - } - } - - BlockingQueue pendingRequests = new ArrayBlockingQueue<>(500); - - public BatchRequest createRequest(Map> inputDocs) { - BatchRequest request = new BatchRequest(inputDocs); - // todo - add can throw exceptions - pendingRequests.add(request); - return request; - } - - public void run() { - while (true) { - List requests = new ArrayList<>(); - pendingRequests.drainTo(requests); - - if (requests.isEmpty()) { - try { - Thread.sleep(2000); - } catch (InterruptedException e) { - return; - } - } else { - try { - Map responseMap = new HashMap<>(); - kafkaProducer.beginTransaction(); - for (BatchRequest request : requests) { - responseMap.put(request, produceDocuments(request.getInputDocs())); - } - kafkaProducer.commitTransaction(); - responseMap.forEach(BatchRequest::setResponse); - } catch (Exception e) { - LOG.warn("failed transaction with error", e); - try { - kafkaProducer.abortTransaction(); - } catch (ProducerFencedException err) { - LOG.error("Could not abort transaction", err); - } - - for (BatchRequest request : requests) { - request.setResponse( - new BulkIngestResponse( - 0, - request.inputDocs.values().stream().mapToInt(List::size).sum(), - e.getMessage())); - } - } - } - } - } - - @SuppressWarnings("FutureReturnValueIgnored") - public BulkIngestResponse produceDocuments(Map> indexDocs) { - int totalDocs = indexDocs.values().stream().mapToInt(List::size).sum(); - - // we cannot create a generic pool of producers because the kafka API expects the transaction ID - // to be a property while creating the producer object. - for (Map.Entry> indexDoc : indexDocs.entrySet()) { - String index = indexDoc.getKey(); - // call once per batch and use the same partition for better batching - int partition = getPartition(index); - - // since there isn't a dataset provisioned for this service/index we will not index this set - // of docs - if (partition < 0) { - LOG.warn("index=" + index + " does not have a provisioned dataset associated with it"); - continue; - } - - // KafkaProducer does not allow creating multiple transactions from a single object - - // rightfully so. - // Till we fix the producer design to allow for multiple /_bulk requests to be able to - // write to the same txn - // we will limit producing documents 1 thread at a time - try { - for (Trace.Span doc : indexDoc.getValue()) { - ProducerRecord producerRecord = - new ProducerRecord<>( - preprocessorConfig.getDownstreamTopic(), partition, index, doc.toByteArray()); - - // we intentionally supress FutureReturnValueIgnored here in errorprone - this is because - // we wrap this in a transaction, which is responsible for flushing all of the pending - // messages - kafkaProducer.send(producerRecord); - } - } catch (TimeoutException te) { - LOG.error("Commit transaction timeout", te); - // the commitTransaction waits till "max.block.ms" after which it will time out - // in that case we cannot call abort exception because that throws the following error - // "Cannot attempt operation `abortTransaction` because the previous - // call to `commitTransaction` timed out and must be retried" - // so for now we just restart the preprocessor - new RuntimeHalterImpl() - .handleFatal( - new Throwable( - "KafkaProducer needs to shutdown as we don't have retry yet and we cannot call abortTxn on timeout", - te)); - } catch (ProducerFencedException | OutOfOrderSequenceException | AuthorizationException e) { - // We can't recover from these exceptions, so our only option is to close the producer and - // exit. - new RuntimeHalterImpl().handleFatal(new Throwable("KafkaProducer needs to shutdown ", e)); - } - } - - return new BulkIngestResponse(totalDocs, 0, ""); - } - - private int getPartition(String index) { - for (DatasetMetadata datasetMetadata : throughputSortedDatasets) { - String serviceNamePattern = datasetMetadata.getServiceNamePattern(); - - if (serviceNamePattern.equals(MATCH_ALL_SERVICE) - || serviceNamePattern.equals(MATCH_STAR_SERVICE) - || index.equals(serviceNamePattern)) { - List partitions = PreprocessorService.getActivePartitionList(datasetMetadata); - return partitions.get(ThreadLocalRandom.current().nextInt(partitions.size())); - } - } - // We don't have a provisioned service for this index - return -1; - } - - private KafkaProducer createKafkaTransactionProducer(String transactionId) { - Properties props = new Properties(); - props.put("bootstrap.servers", preprocessorConfig.getBootstrapServers()); - props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); - props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); - props.put("transactional.id", transactionId); - props.put("linger.ms", 250); - props.put("max.block.ms", "10000"); - props.put("compression.type", "snappy"); - return new KafkaProducer<>(props); - } -} diff --git a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducerTest.java b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducerTest.java new file mode 100644 index 0000000000..6385c17c13 --- /dev/null +++ b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducerTest.java @@ -0,0 +1,262 @@ +package com.slack.kaldb.bulkIngestApi; + +import static com.slack.kaldb.server.KaldbConfig.DEFAULT_START_STOP_DURATION; +import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import brave.Tracing; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; +import com.slack.kaldb.metadata.core.CuratorBuilder; +import com.slack.kaldb.metadata.dataset.DatasetMetadata; +import com.slack.kaldb.metadata.dataset.DatasetMetadataStore; +import com.slack.kaldb.metadata.dataset.DatasetPartitionMetadata; +import com.slack.kaldb.proto.config.KaldbConfigs; +import com.slack.kaldb.testlib.TestKafkaServer; +import com.slack.service.murron.trace.Trace; +import io.micrometer.prometheus.PrometheusConfig; +import io.micrometer.prometheus.PrometheusMeterRegistry; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import org.apache.curator.test.TestingServer; +import org.apache.curator.x.async.AsyncCuratorFramework; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class TransactionBatchingKafkaProducerTest { + private static final Logger LOG = + LoggerFactory.getLogger(TransactionBatchingKafkaProducerTest.class); + private static PrometheusMeterRegistry meterRegistry; + private static AsyncCuratorFramework curatorFramework; + private static KaldbConfigs.PreprocessorConfig preprocessorConfig; + private static DatasetMetadataStore datasetMetadataStore; + private static TestingServer zkServer; + private static TestKafkaServer kafkaServer; + + private TransactionBatchingKafkaProducer transactionBatchingKafkaProducer; + + static String INDEX_NAME = "testtransactionindex"; + + private static String DOWNSTREAM_TOPIC = "test-transaction-topic-out"; + + @BeforeEach + public void bootstrapCluster() throws Exception { + Tracing.newBuilder().build(); + meterRegistry = new PrometheusMeterRegistry(PrometheusConfig.DEFAULT); + + zkServer = new TestingServer(); + KaldbConfigs.ZookeeperConfig zkConfig = + KaldbConfigs.ZookeeperConfig.newBuilder() + .setZkConnectString(zkServer.getConnectString()) + .setZkPathPrefix("testZK") + .setZkSessionTimeoutMs(1000) + .setZkConnectionTimeoutMs(1000) + .setSleepBetweenRetriesMs(1000) + .build(); + curatorFramework = CuratorBuilder.build(meterRegistry, zkConfig); + + kafkaServer = new TestKafkaServer(); + kafkaServer.createTopicWithPartitions(DOWNSTREAM_TOPIC, 5); + + KaldbConfigs.ServerConfig serverConfig = + KaldbConfigs.ServerConfig.newBuilder() + .setServerPort(8080) + .setServerAddress("localhost") + .build(); + preprocessorConfig = + KaldbConfigs.PreprocessorConfig.newBuilder() + .setBootstrapServers(kafkaServer.getBroker().getBrokerList().get()) + .setUseBulkApi(true) + .setServerConfig(serverConfig) + .setPreprocessorInstanceCount(1) + .setRateLimiterMaxBurstSeconds(1) + .setDownstreamTopic(DOWNSTREAM_TOPIC) + .build(); + + datasetMetadataStore = new DatasetMetadataStore(curatorFramework, true); + DatasetMetadata datasetMetadata = + new DatasetMetadata( + INDEX_NAME, + "owner", + 1, + List.of(new DatasetPartitionMetadata(1, Long.MAX_VALUE, List.of("0"))), + INDEX_NAME); + // Create an entry while init. Update the entry on every test run + datasetMetadataStore.createSync(datasetMetadata); + + transactionBatchingKafkaProducer = + new TransactionBatchingKafkaProducer( + datasetMetadataStore, preprocessorConfig, meterRegistry); + transactionBatchingKafkaProducer.startAsync(); + transactionBatchingKafkaProducer.awaitRunning(DEFAULT_START_STOP_DURATION); + } + + @Test + public void testDocumentInKafkaTransactionError() throws Exception { + KafkaConsumer kafkaConsumer = getTestKafkaConsumer(); + + // we want to inject a failure in the second doc and test if the abort transaction works and we + // don't index the first document + Trace.Span doc1 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("error1")).build(); + Trace.Span doc2 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("error2")).build(); + Trace.Span doc3 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("error3")).build(); + Trace.Span doc4 = spy(Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("error4")).build()); + when(doc4.toByteArray()).thenThrow(new RuntimeException("exception")); + Trace.Span doc5 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("error5")).build(); + + Map> indexDocs = + Map.of(INDEX_NAME, List.of(doc1, doc2, doc3, doc4, doc5)); + + BatchRequest request1 = new BatchRequest(indexDocs); + Thread.ofVirtual() + .start( + () -> { + try { + // because of the synchronous queue, we need someone consuming the response before + // we attempt to set it + request1.getResponse(); + } catch (InterruptedException ignored) { + } + }); + BulkIngestResponse responseObj = + (BulkIngestResponse) + transactionBatchingKafkaProducer + .transactionCommit(List.of(request1)) + .values() + .toArray()[0]; + assertThat(responseObj.totalDocs()).isEqualTo(0); + assertThat(responseObj.failedDocs()).isEqualTo(5); + assertThat(responseObj.errorMsg()).isNotNull(); + + await() + .until( + () -> { + @SuppressWarnings("OptionalGetWithoutIsPresent") + long partitionOffset = + (Long) + kafkaConsumer + .endOffsets(List.of(new TopicPartition(DOWNSTREAM_TOPIC, 0))) + .values() + .stream() + .findFirst() + .get(); + LOG.debug( + "Current partitionOffset - {}. expecting offset to be less than 5", + partitionOffset); + return partitionOffset > 0 && partitionOffset < 5; + }); + + ConsumerRecords records = + kafkaConsumer.poll(Duration.of(10, ChronoUnit.SECONDS)); + + assertThat(records.count()).isEqualTo(0); + + long currentPartitionOffset = + (Long) + kafkaConsumer + .endOffsets(List.of(new TopicPartition(DOWNSTREAM_TOPIC, 0))) + .values() + .stream() + .findFirst() + .get(); + + Trace.Span doc6 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("no_error6")).build(); + Trace.Span doc7 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("no_error7")).build(); + Trace.Span doc8 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("no_error8")).build(); + Trace.Span doc9 = + spy(Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("no_error9")).build()); + Trace.Span doc10 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("no_error10")).build(); + + indexDocs = Map.of(INDEX_NAME, List.of(doc6, doc7, doc8, doc9, doc10)); + + BatchRequest request2 = new BatchRequest(indexDocs); + Thread.ofVirtual() + .start( + () -> { + try { + // because of the synchronous queue, we need someone consuming the response before + // we attempt to set it + request2.getResponse(); + } catch (InterruptedException ignored) { + } + }); + responseObj = + (BulkIngestResponse) + transactionBatchingKafkaProducer + .transactionCommit(List.of(request2)) + .values() + .toArray()[0]; + assertThat(responseObj.totalDocs()).isEqualTo(5); + assertThat(responseObj.failedDocs()).isEqualTo(0); + assertThat(responseObj.errorMsg()).isNotNull(); + + // 5 docs. 1 control batch. initial offset was 1 after the first failed batch + validateOffset(kafkaConsumer, currentPartitionOffset + 5 + 1); + records = kafkaConsumer.poll(Duration.of(10, ChronoUnit.SECONDS)); + + assertThat(records.count()).isEqualTo(5); + records.forEach( + record -> + LOG.info( + "Trace= + " + TraceSpanParserSilenceError(record.value()).getId().toStringUtf8())); + + // close the kafka consumer used in the test + kafkaConsumer.close(); + } + + public KafkaConsumer getTestKafkaConsumer() { + // used to verify the message exist on the downstream topic + Properties properties = kafkaServer.getBroker().consumerConfig(); + properties.put( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringDeserializer"); + properties.put( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + properties.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 30000); + properties.put("isolation.level", "read_committed"); + KafkaConsumer kafkaConsumer = new KafkaConsumer(properties); + kafkaConsumer.subscribe(List.of(DOWNSTREAM_TOPIC)); + return kafkaConsumer; + } + + public void validateOffset(KafkaConsumer kafkaConsumer, long expectedOffset) { + await() + .until( + () -> { + @SuppressWarnings("OptionalGetWithoutIsPresent") + long partitionOffset = + (Long) + kafkaConsumer + .endOffsets(List.of(new TopicPartition(DOWNSTREAM_TOPIC, 0))) + .values() + .stream() + .findFirst() + .get(); + LOG.debug( + "Current partitionOffset - {}. expecting offset to be - {}", + partitionOffset, + expectedOffset); + return partitionOffset == expectedOffset; + }); + } + + private static Trace.Span TraceSpanParserSilenceError(byte[] data) { + try { + return Trace.Span.parseFrom(data); + } catch (InvalidProtocolBufferException e) { + return Trace.Span.newBuilder().build(); + } + } +} diff --git a/kaldb/src/test/java/com/slack/kaldb/server/OpenSearchBulkIngestApiTest.java b/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java similarity index 64% rename from kaldb/src/test/java/com/slack/kaldb/server/OpenSearchBulkIngestApiTest.java rename to kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java index 3581237c96..144521c546 100644 --- a/kaldb/src/test/java/com/slack/kaldb/server/OpenSearchBulkIngestApiTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java @@ -5,15 +5,16 @@ import static com.linecorp.armeria.common.HttpStatus.TOO_MANY_REQUESTS; import static com.slack.kaldb.server.KaldbConfig.DEFAULT_START_STOP_DURATION; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; import static org.awaitility.Awaitility.await; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; import brave.Tracing; -import com.google.protobuf.ByteString; import com.google.protobuf.InvalidProtocolBufferException; import com.linecorp.armeria.common.AggregatedHttpResponse; -import com.slack.kaldb.elasticsearchApi.BulkIngestResponse; +import com.slack.kaldb.bulkIngestApi.BulkIngestApi; +import com.slack.kaldb.bulkIngestApi.BulkIngestResponse; +import com.slack.kaldb.bulkIngestApi.DatasetRateLimitingService; +import com.slack.kaldb.bulkIngestApi.TransactionBatchingKafkaProducer; import com.slack.kaldb.metadata.core.CuratorBuilder; import com.slack.kaldb.metadata.dataset.DatasetMetadata; import com.slack.kaldb.metadata.dataset.DatasetMetadataStore; @@ -21,16 +22,19 @@ import com.slack.kaldb.preprocessor.PreprocessorRateLimiter; import com.slack.kaldb.preprocessor.ingest.OpenSearchBulkApiRequestParser; import com.slack.kaldb.proto.config.KaldbConfigs; +import com.slack.kaldb.testlib.MetricsUtil; import com.slack.kaldb.testlib.TestKafkaServer; import com.slack.kaldb.util.JsonUtil; import com.slack.service.murron.trace.Trace; import io.micrometer.prometheus.PrometheusConfig; import io.micrometer.prometheus.PrometheusMeterRegistry; +import java.io.IOException; import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.CompletableFuture; import org.apache.curator.test.TestingServer; import org.apache.curator.x.async.AsyncCuratorFramework; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -44,17 +48,18 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class OpenSearchBulkIngestApiTest { - - private static final Logger LOG = LoggerFactory.getLogger(OpenSearchBulkIngestApi.class); - +public class BulkIngestApiTest { + private static final Logger LOG = LoggerFactory.getLogger(BulkIngestApi.class); private static PrometheusMeterRegistry meterRegistry; private static AsyncCuratorFramework curatorFramework; private static KaldbConfigs.PreprocessorConfig preprocessorConfig; private static DatasetMetadataStore datasetMetadataStore; private static TestingServer zkServer; private static TestKafkaServer kafkaServer; - private OpenSearchBulkIngestApi openSearchBulkAPI; + private BulkIngestApi bulkApi; + + private TransactionBatchingKafkaProducer transactionBatchingKafkaProducer; + private DatasetRateLimitingService datasetRateLimitingService; static String INDEX_NAME = "testindex"; @@ -105,18 +110,30 @@ public void bootstrapCluster() throws Exception { // Create an entry while init. Update the entry on every test run datasetMetadataStore.createSync(datasetMetadata); - openSearchBulkAPI = - new OpenSearchBulkIngestApi(datasetMetadataStore, preprocessorConfig, meterRegistry, false); + datasetRateLimitingService = + new DatasetRateLimitingService(datasetMetadataStore, preprocessorConfig, meterRegistry); + transactionBatchingKafkaProducer = + new TransactionBatchingKafkaProducer( + datasetMetadataStore, preprocessorConfig, meterRegistry); + + datasetRateLimitingService.startAsync(); + transactionBatchingKafkaProducer.startAsync(); - openSearchBulkAPI.startAsync(); - openSearchBulkAPI.awaitRunning(DEFAULT_START_STOP_DURATION); + datasetRateLimitingService.awaitRunning(DEFAULT_START_STOP_DURATION); + transactionBatchingKafkaProducer.awaitRunning(DEFAULT_START_STOP_DURATION); + + bulkApi = new BulkIngestApi(transactionBatchingKafkaProducer, datasetRateLimitingService); } // I looked at making this a @BeforeEach. it's possible if you annotate a test with a @Tag and // pass throughputBytes. // However, decided not to go with that because it involved hardcoding the throughput bytes // when defining the test. We need it to be dynamic based on the size of the docs - public void updateDatasetThroughput(int throughputBytes) throws Exception { + public void updateDatasetThroughput(int throughputBytes) { + double timerCount = + MetricsUtil.getTimerCount( + DatasetRateLimitingService.RATE_LIMIT_RELOAD_TIMER, meterRegistry); + // dataset metadata already exists. Update with the throughput value DatasetMetadata datasetMetadata = new DatasetMetadata( @@ -127,17 +144,13 @@ public void updateDatasetThroughput(int throughputBytes) throws Exception { INDEX_NAME); datasetMetadataStore.updateSync(datasetMetadata); - // Need to wait until we've verified the rate limit has been correctly loaded + // Need to wait until the rate limit has been loaded await() .until( () -> - openSearchBulkAPI.throughputSortedDatasets.stream() - .filter(datasetMetadata1 -> datasetMetadata1.name.equals(INDEX_NAME)) - .findFirst(), - (storedDatasetMetadata) -> { - //noinspection OptionalGetWithoutIsPresent - return storedDatasetMetadata.get().getThroughputBytes() == throughputBytes; - }); + MetricsUtil.getTimerCount( + DatasetRateLimitingService.RATE_LIMIT_RELOAD_TIMER, meterRegistry) + > timerCount); } @AfterEach @@ -145,9 +158,13 @@ public void updateDatasetThroughput(int throughputBytes) throws Exception { // Instead of calling stop from every test and ensuring it's part of a finally block we just call // the shutdown code with the @AfterEach annotation public void shutdownOpenSearchAPI() throws Exception { - if (openSearchBulkAPI != null) { - openSearchBulkAPI.stopAsync(); - openSearchBulkAPI.awaitTerminated(DEFAULT_START_STOP_DURATION); + if (datasetRateLimitingService != null) { + datasetRateLimitingService.stopAsync(); + datasetRateLimitingService.awaitTerminated(DEFAULT_START_STOP_DURATION); + } + if (transactionBatchingKafkaProducer != null) { + transactionBatchingKafkaProducer.stopAsync(); + transactionBatchingKafkaProducer.awaitTerminated(DEFAULT_START_STOP_DURATION); } kafkaServer.close(); curatorFramework.unwrap().close(); @@ -190,7 +207,7 @@ public void testBulkApiBasic() throws Exception { updateDatasetThroughput(throughputBytes); // test with empty causes a parse exception - AggregatedHttpResponse response = openSearchBulkAPI.addDocument("{}\n").aggregate().join(); + AggregatedHttpResponse response = bulkApi.addDocument("{}\n").aggregate().join(); assertThat(response.status().isSuccess()).isEqualTo(false); assertThat(response.status().code()).isEqualTo(INTERNAL_SERVER_ERROR.code()); BulkIngestResponse responseObj = @@ -200,20 +217,49 @@ public void testBulkApiBasic() throws Exception { // test with request1 twice. first one should succeed, second one will fail because of rate // limiter - response = openSearchBulkAPI.addDocument(request1).aggregate().join(); - assertThat(response.status().isSuccess()).isEqualTo(true); - assertThat(response.status().code()).isEqualTo(OK.code()); - responseObj = JsonUtil.read(response.contentUtf8(), BulkIngestResponse.class); - assertThat(responseObj.totalDocs()).isEqualTo(1); - assertThat(responseObj.failedDocs()).isEqualTo(0); - - response = openSearchBulkAPI.addDocument(request1).aggregate().join(); - assertThat(response.status().isSuccess()).isEqualTo(false); - assertThat(response.status().code()).isEqualTo(TOO_MANY_REQUESTS.code()); - responseObj = JsonUtil.read(response.contentUtf8(), BulkIngestResponse.class); - assertThat(responseObj.totalDocs()).isEqualTo(0); - assertThat(responseObj.failedDocs()).isEqualTo(0); - assertThat(responseObj.errorMsg()).isEqualTo("rate limit exceeded"); + CompletableFuture response1 = + bulkApi + .addDocument(request1) + .aggregate() + .thenApply( + httpResponse -> { + assertThat(httpResponse.status().isSuccess()).isEqualTo(true); + assertThat(httpResponse.status().code()).isEqualTo(OK.code()); + BulkIngestResponse httpResponseObj = null; + try { + httpResponseObj = + JsonUtil.read(httpResponse.contentUtf8(), BulkIngestResponse.class); + } catch (IOException e) { + fail("", e); + } + assertThat(httpResponseObj.totalDocs()).isEqualTo(1); + assertThat(httpResponseObj.failedDocs()).isEqualTo(0); + return httpResponse; + }); + + CompletableFuture response2 = + bulkApi + .addDocument(request1) + .aggregate() + .thenApply( + httpResponse -> { + assertThat(httpResponse.status().isSuccess()).isEqualTo(false); + assertThat(httpResponse.status().code()).isEqualTo(TOO_MANY_REQUESTS.code()); + BulkIngestResponse httpResponseObj = null; + try { + httpResponseObj = + JsonUtil.read(httpResponse.contentUtf8(), BulkIngestResponse.class); + } catch (IOException e) { + fail("", e); + } + assertThat(httpResponseObj.totalDocs()).isEqualTo(0); + assertThat(httpResponseObj.failedDocs()).isEqualTo(0); + assertThat(httpResponseObj.errorMsg()).isEqualTo("rate limit exceeded"); + return httpResponse; + }); + + await().until(response1::isDone); + await().until(response2::isDone); // test with multiple indexes String request2 = @@ -223,7 +269,7 @@ public void testBulkApiBasic() throws Exception { { "index": {"_index": "testindex2", "_id": "1"} } { "field1" : "value1" } """; - response = openSearchBulkAPI.addDocument(request2).aggregate().join(); + response = bulkApi.addDocument(request2).aggregate().join(); assertThat(response.status().isSuccess()).isEqualTo(false); assertThat(response.status().code()).isEqualTo(INTERNAL_SERVER_ERROR.code()); responseObj = JsonUtil.read(response.contentUtf8(), BulkIngestResponse.class); @@ -251,7 +297,7 @@ public void testDocumentInKafkaSimple() throws Exception { KafkaConsumer kafkaConsumer = getTestKafkaConsumer(); - AggregatedHttpResponse response = openSearchBulkAPI.addDocument(request1).aggregate().join(); + AggregatedHttpResponse response = bulkApi.addDocument(request1).aggregate().join(); assertThat(response.status().isSuccess()).isEqualTo(true); assertThat(response.status().code()).isEqualTo(OK.code()); BulkIngestResponse responseObj = @@ -279,89 +325,6 @@ record -> kafkaConsumer.close(); } - @Test - public void testDocumentInKafkaTransactionError() throws Exception { - updateDatasetThroughput(100_1000); - - KafkaConsumer kafkaConsumer = getTestKafkaConsumer(); - - // we want to inject a failure in the second doc and test if the abort transaction works and we - // don't index the first document - Trace.Span doc1 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("error1")).build(); - Trace.Span doc2 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("error2")).build(); - Trace.Span doc3 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("error3")).build(); - Trace.Span doc4 = spy(Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("error4")).build()); - when(doc4.toByteArray()).thenThrow(new RuntimeException("exception")); - Trace.Span doc5 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("error5")).build(); - - Map> indexDocs = - Map.of("testindex", List.of(doc1, doc2, doc3, doc4, doc5)); - - BulkIngestResponse responseObj = openSearchBulkAPI.produceDocuments(indexDocs); - assertThat(responseObj.totalDocs()).isEqualTo(0); - assertThat(responseObj.failedDocs()).isEqualTo(5); - assertThat(responseObj.errorMsg()).isNotNull(); - - await() - .until( - () -> { - @SuppressWarnings("OptionalGetWithoutIsPresent") - long partitionOffset = - (Long) - kafkaConsumer - .endOffsets(List.of(new TopicPartition(DOWNSTREAM_TOPIC, 0))) - .values() - .stream() - .findFirst() - .get(); - LOG.debug( - "Current partitionOffset - {}. expecting offset to be less than 5", - partitionOffset); - return partitionOffset > 0 && partitionOffset < 5; - }); - - ConsumerRecords records = - kafkaConsumer.poll(Duration.of(10, ChronoUnit.SECONDS)); - - assertThat(records.count()).isEqualTo(0); - - long currentPartitionOffset = - (Long) - kafkaConsumer - .endOffsets(List.of(new TopicPartition(DOWNSTREAM_TOPIC, 0))) - .values() - .stream() - .findFirst() - .get(); - - Trace.Span doc6 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("no_error6")).build(); - Trace.Span doc7 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("no_error7")).build(); - Trace.Span doc8 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("no_error8")).build(); - Trace.Span doc9 = - spy(Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("no_error9")).build()); - Trace.Span doc10 = Trace.Span.newBuilder().setId(ByteString.copyFromUtf8("no_error10")).build(); - - indexDocs = Map.of("testindex", List.of(doc6, doc7, doc8, doc9, doc10)); - - responseObj = openSearchBulkAPI.produceDocuments(indexDocs); - assertThat(responseObj.totalDocs()).isEqualTo(5); - assertThat(responseObj.failedDocs()).isEqualTo(0); - assertThat(responseObj.errorMsg()).isNotNull(); - - // 5 docs. 1 control batch. initial offset was 1 after the first failed batch - validateOffset(kafkaConsumer, currentPartitionOffset + 5 + 1); - records = kafkaConsumer.poll(Duration.of(10, ChronoUnit.SECONDS)); - - assertThat(records.count()).isEqualTo(5); - records.forEach( - record -> - LOG.info( - "Trace= + " + TraceSpanParserSilenceError(record.value()).getId().toStringUtf8())); - - // close the kafka consumer used in the test - kafkaConsumer.close(); - } - public void validateOffset(KafkaConsumer kafkaConsumer, long expectedOffset) { await() .until( From abdd9de8d9eccc6ae8e24018d777043da2122a1d Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Mon, 18 Dec 2023 16:55:17 -0700 Subject: [PATCH 03/18] Cleanup & rename refactor --- .../kaldb/bulkIngestApi/BulkIngestApi.java | 16 ++++--- ...ucer.java => BulkIngestKafkaProducer.java} | 30 ++++++------- ...tchRequest.java => BulkIngestRequest.java} | 4 +- .../DatasetRateLimitingService.java | 2 +- .../OpensearchBulkApiRequestParser.java} | 10 ++--- .../java/com/slack/kaldb/server/Kaldb.java | 11 +++-- ....java => BulkIngestKafkaProducerTest.java} | 30 +++++-------- .../OpensearchBulkApiRequestParserTest.java} | 42 +++++++++---------- .../slack/kaldb/server/BulkIngestApiTest.java | 31 +++++++------- 9 files changed, 83 insertions(+), 93 deletions(-) rename kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/{TransactionBatchingKafkaProducer.java => BulkIngestKafkaProducer.java} (89%) rename kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/{BatchRequest.java => BulkIngestRequest.java} (85%) rename kaldb/src/main/java/com/slack/kaldb/{preprocessor/ingest/OpenSearchBulkApiRequestParser.java => bulkIngestApi/OpensearchBulkApiRequestParser.java} (94%) rename kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/{TransactionBatchingKafkaProducerTest.java => BulkIngestKafkaProducerTest.java} (91%) rename kaldb/src/test/java/com/slack/kaldb/{preprocessor/ingest/OpenSearchBulkRequestTest.java => bulkIngestApi/OpensearchBulkApiRequestParserTest.java} (83%) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java index bdc41d3f67..5aff97b48d 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java @@ -6,7 +6,6 @@ import com.linecorp.armeria.common.HttpResponse; import com.linecorp.armeria.server.annotation.Blocking; import com.linecorp.armeria.server.annotation.Post; -import com.slack.kaldb.preprocessor.ingest.OpenSearchBulkApiRequestParser; import com.slack.service.murron.trace.Trace; import java.util.List; import java.util.Map; @@ -16,14 +15,14 @@ public class BulkIngestApi { private static final Logger LOG = LoggerFactory.getLogger(BulkIngestApi.class); - private final TransactionBatchingKafkaProducer transactionBatchingKafkaProducer; + private final BulkIngestKafkaProducer bulkIngestKafkaProducer; private final DatasetRateLimitingService datasetRateLimitingService; public BulkIngestApi( - TransactionBatchingKafkaProducer transactionBatchingKafkaProducer, + BulkIngestKafkaProducer bulkIngestKafkaProducer, DatasetRateLimitingService datasetRateLimitingService) { - this.transactionBatchingKafkaProducer = transactionBatchingKafkaProducer; + this.bulkIngestKafkaProducer = bulkIngestKafkaProducer; this.datasetRateLimitingService = datasetRateLimitingService; } @@ -36,9 +35,9 @@ public BulkIngestApi( public HttpResponse addDocument(String bulkRequest) { try { List indexRequests = - OpenSearchBulkApiRequestParser.parseBulkRequest(bulkRequest); + OpensearchBulkApiRequestParser.parseBulkRequest(bulkRequest); Map> docs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); // todo - our rate limiter doesn't have a way to acquire permits across multiple datasets // so today as a limitation we reject any request that has documents against multiple indexes @@ -51,13 +50,12 @@ public HttpResponse addDocument(String bulkRequest) { for (Map.Entry> indexDocs : docs.entrySet()) { final String index = indexDocs.getKey(); - if (!datasetRateLimitingService.test(index, indexDocs.getValue())) { + if (!datasetRateLimitingService.tryAcquire(index, indexDocs.getValue())) { BulkIngestResponse response = new BulkIngestResponse(0, 0, "rate limit exceeded"); return HttpResponse.ofJson(TOO_MANY_REQUESTS, response); } } - BulkIngestResponse response = - transactionBatchingKafkaProducer.createRequest(docs).getResponse(); + BulkIngestResponse response = bulkIngestKafkaProducer.createRequest(docs).getResponse(); return HttpResponse.ofJson(response); } catch (Exception e) { LOG.error("Request failed ", e); diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java similarity index 89% rename from kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducer.java rename to kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index c65093c7f5..d7111ea767 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -33,10 +33,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class TransactionBatchingKafkaProducer extends AbstractExecutionThreadService { - private static final Logger LOG = LoggerFactory.getLogger(TransactionBatchingKafkaProducer.class); +public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { + private static final Logger LOG = LoggerFactory.getLogger(BulkIngestKafkaProducer.class); - private final KafkaProducer kafkaProducer; + private final KafkaProducer kafkaProducer; private final KafkaClientMetrics kafkaMetrics; private final KaldbConfigs.PreprocessorConfig preprocessorConfig; @@ -47,9 +47,10 @@ public class TransactionBatchingKafkaProducer extends AbstractExecutionThreadSer protected List throughputSortedDatasets; - private final BlockingQueue pendingRequests = new ArrayBlockingQueue<>(500); + // todo - parameterize the capacity option? + private final BlockingQueue pendingRequests = new ArrayBlockingQueue<>(500); - public TransactionBatchingKafkaProducer( + public BulkIngestKafkaProducer( final DatasetMetadataStore datasetMetadataStore, final KaldbConfigs.PreprocessorConfig preprocessorConfig, final PrometheusMeterRegistry meterRegistry) { @@ -95,7 +96,7 @@ protected void startUp() throws Exception { @Override protected void run() throws Exception { while (isRunning()) { - List requests = new ArrayList<>(); + List requests = new ArrayList<>(); pendingRequests.drainTo(requests); if (requests.isEmpty()) { try { @@ -119,18 +120,19 @@ protected void shutDown() throws Exception { } } - public BatchRequest createRequest(Map> inputDocs) { - BatchRequest request = new BatchRequest(inputDocs); + public BulkIngestRequest createRequest(Map> inputDocs) { + BulkIngestRequest request = new BulkIngestRequest(inputDocs); // todo - add can throw exceptions pendingRequests.add(request); return request; } - protected Map transactionCommit(List requests) { - Map responseMap = new HashMap<>(); + protected Map transactionCommit( + List requests) { + Map responseMap = new HashMap<>(); try { kafkaProducer.beginTransaction(); - for (BatchRequest request : requests) { + for (BulkIngestRequest request : requests) { responseMap.put(request, produceDocuments(request.getInputDocs())); } kafkaProducer.commitTransaction(); @@ -142,7 +144,7 @@ protected Map transactionCommit(List transactionCommit(List entry : responseMap.entrySet()) { - BatchRequest key = entry.getKey(); + for (Map.Entry entry : responseMap.entrySet()) { + BulkIngestRequest key = entry.getKey(); BulkIngestResponse value = entry.getValue(); key.setResponse(value); } diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BatchRequest.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestRequest.java similarity index 85% rename from kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BatchRequest.java rename to kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestRequest.java index d1ff7b24ce..0b6427782c 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BatchRequest.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestRequest.java @@ -5,11 +5,11 @@ import java.util.Map; import java.util.concurrent.SynchronousQueue; -public class BatchRequest { +public class BulkIngestRequest { private final Map> inputDocs; private final SynchronousQueue internalResponse = new SynchronousQueue<>(); - protected BatchRequest(Map> inputDocs) { + protected BulkIngestRequest(Map> inputDocs) { this.inputDocs = inputDocs; } diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java index 53cfc53fc3..fed2532333 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java @@ -64,7 +64,7 @@ protected void shutDown() throws Exception { datasetMetadataStore.removeListener(datasetListener); } - public boolean test(String index, List value) { + public boolean tryAcquire(String index, List value) { return rateLimiterPredicate.test(index, value); } } diff --git a/kaldb/src/main/java/com/slack/kaldb/preprocessor/ingest/OpenSearchBulkApiRequestParser.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParser.java similarity index 94% rename from kaldb/src/main/java/com/slack/kaldb/preprocessor/ingest/OpenSearchBulkApiRequestParser.java rename to kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParser.java index 891e056ab1..c7457ecad6 100644 --- a/kaldb/src/main/java/com/slack/kaldb/preprocessor/ingest/OpenSearchBulkApiRequestParser.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParser.java @@ -1,4 +1,4 @@ -package com.slack.kaldb.preprocessor.ingest; +package com.slack.kaldb.bulkIngestApi; import com.google.protobuf.ByteString; import com.slack.kaldb.writer.SpanFormatter; @@ -22,11 +22,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class OpenSearchBulkApiRequestParser { +public class OpensearchBulkApiRequestParser { - public static final Logger LOG = LoggerFactory.getLogger(OpenSearchBulkApiRequestParser.class); + public static final Logger LOG = LoggerFactory.getLogger(OpensearchBulkApiRequestParser.class); - private static String SERVICE_NAME_KEY = "service_name"; + private static final String SERVICE_NAME_KEY = "service_name"; public static Trace.Span fromIngestDocument(IngestDocument ingestDocument) { ZonedDateTime timestamp = @@ -81,7 +81,7 @@ public static Map> convertIndexRequestToTraceFormat( } IngestDocument ingestDocument = convertRequestToDocument(indexRequest); List docs = indexDocs.computeIfAbsent(index, key -> new ArrayList<>()); - docs.add(OpenSearchBulkApiRequestParser.fromIngestDocument(ingestDocument)); + docs.add(OpensearchBulkApiRequestParser.fromIngestDocument(ingestDocument)); } return indexDocs; } diff --git a/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java b/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java index 291fc7543f..3760b4e5b4 100644 --- a/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java +++ b/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java @@ -6,8 +6,8 @@ import com.slack.kaldb.blobfs.BlobFs; import com.slack.kaldb.blobfs.s3.S3CrtBlobFs; import com.slack.kaldb.bulkIngestApi.BulkIngestApi; +import com.slack.kaldb.bulkIngestApi.BulkIngestKafkaProducer; import com.slack.kaldb.bulkIngestApi.DatasetRateLimitingService; -import com.slack.kaldb.bulkIngestApi.TransactionBatchingKafkaProducer; import com.slack.kaldb.chunkManager.CachingChunkManager; import com.slack.kaldb.chunkManager.IndexingChunkManager; import com.slack.kaldb.clusterManager.ClusterHpaMetricService; @@ -391,16 +391,15 @@ private static Set getServices( KaldbConfigs.NodeRole.PREPROCESSOR, List.of(datasetMetadataStore))); if (preprocessorConfig.getUseBulkApi()) { - TransactionBatchingKafkaProducer transactionBatchingKafkaProducer = - new TransactionBatchingKafkaProducer( - datasetMetadataStore, preprocessorConfig, meterRegistry); - services.add(transactionBatchingKafkaProducer); + BulkIngestKafkaProducer bulkIngestKafkaProducer = + new BulkIngestKafkaProducer(datasetMetadataStore, preprocessorConfig, meterRegistry); + services.add(bulkIngestKafkaProducer); DatasetRateLimitingService datasetRateLimitingService = new DatasetRateLimitingService(datasetMetadataStore, preprocessorConfig, meterRegistry); services.add(datasetRateLimitingService); BulkIngestApi openSearchBulkApiService = - new BulkIngestApi(transactionBatchingKafkaProducer, datasetRateLimitingService); + new BulkIngestApi(bulkIngestKafkaProducer, datasetRateLimitingService); armeriaServiceBuilder.withAnnotatedService(openSearchBulkApiService); } else { PreprocessorService preprocessorService = diff --git a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducerTest.java b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java similarity index 91% rename from kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducerTest.java rename to kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java index 6385c17c13..bf9238af76 100644 --- a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/TransactionBatchingKafkaProducerTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java @@ -34,9 +34,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class TransactionBatchingKafkaProducerTest { - private static final Logger LOG = - LoggerFactory.getLogger(TransactionBatchingKafkaProducerTest.class); +class BulkIngestKafkaProducerTest { + private static final Logger LOG = LoggerFactory.getLogger(BulkIngestKafkaProducerTest.class); private static PrometheusMeterRegistry meterRegistry; private static AsyncCuratorFramework curatorFramework; private static KaldbConfigs.PreprocessorConfig preprocessorConfig; @@ -44,7 +43,7 @@ class TransactionBatchingKafkaProducerTest { private static TestingServer zkServer; private static TestKafkaServer kafkaServer; - private TransactionBatchingKafkaProducer transactionBatchingKafkaProducer; + private BulkIngestKafkaProducer bulkIngestKafkaProducer; static String INDEX_NAME = "testtransactionindex"; @@ -95,11 +94,10 @@ public void bootstrapCluster() throws Exception { // Create an entry while init. Update the entry on every test run datasetMetadataStore.createSync(datasetMetadata); - transactionBatchingKafkaProducer = - new TransactionBatchingKafkaProducer( - datasetMetadataStore, preprocessorConfig, meterRegistry); - transactionBatchingKafkaProducer.startAsync(); - transactionBatchingKafkaProducer.awaitRunning(DEFAULT_START_STOP_DURATION); + bulkIngestKafkaProducer = + new BulkIngestKafkaProducer(datasetMetadataStore, preprocessorConfig, meterRegistry); + bulkIngestKafkaProducer.startAsync(); + bulkIngestKafkaProducer.awaitRunning(DEFAULT_START_STOP_DURATION); } @Test @@ -118,7 +116,7 @@ public void testDocumentInKafkaTransactionError() throws Exception { Map> indexDocs = Map.of(INDEX_NAME, List.of(doc1, doc2, doc3, doc4, doc5)); - BatchRequest request1 = new BatchRequest(indexDocs); + BulkIngestRequest request1 = new BulkIngestRequest(indexDocs); Thread.ofVirtual() .start( () -> { @@ -131,10 +129,7 @@ public void testDocumentInKafkaTransactionError() throws Exception { }); BulkIngestResponse responseObj = (BulkIngestResponse) - transactionBatchingKafkaProducer - .transactionCommit(List.of(request1)) - .values() - .toArray()[0]; + bulkIngestKafkaProducer.transactionCommit(List.of(request1)).values().toArray()[0]; assertThat(responseObj.totalDocs()).isEqualTo(0); assertThat(responseObj.failedDocs()).isEqualTo(5); assertThat(responseObj.errorMsg()).isNotNull(); @@ -180,7 +175,7 @@ public void testDocumentInKafkaTransactionError() throws Exception { indexDocs = Map.of(INDEX_NAME, List.of(doc6, doc7, doc8, doc9, doc10)); - BatchRequest request2 = new BatchRequest(indexDocs); + BulkIngestRequest request2 = new BulkIngestRequest(indexDocs); Thread.ofVirtual() .start( () -> { @@ -193,10 +188,7 @@ public void testDocumentInKafkaTransactionError() throws Exception { }); responseObj = (BulkIngestResponse) - transactionBatchingKafkaProducer - .transactionCommit(List.of(request2)) - .values() - .toArray()[0]; + bulkIngestKafkaProducer.transactionCommit(List.of(request2)).values().toArray()[0]; assertThat(responseObj.totalDocs()).isEqualTo(5); assertThat(responseObj.failedDocs()).isEqualTo(0); assertThat(responseObj.errorMsg()).isNotNull(); diff --git a/kaldb/src/test/java/com/slack/kaldb/preprocessor/ingest/OpenSearchBulkRequestTest.java b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParserTest.java similarity index 83% rename from kaldb/src/test/java/com/slack/kaldb/preprocessor/ingest/OpenSearchBulkRequestTest.java rename to kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParserTest.java index 62b3141f38..cdbabaad84 100644 --- a/kaldb/src/test/java/com/slack/kaldb/preprocessor/ingest/OpenSearchBulkRequestTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParserTest.java @@ -1,5 +1,6 @@ -package com.slack.kaldb.preprocessor.ingest; +package com.slack.kaldb.bulkIngestApi; +import static com.slack.kaldb.bulkIngestApi.OpensearchBulkApiRequestParser.convertRequestToDocument; import static org.assertj.core.api.Assertions.assertThat; import com.google.common.io.Resources; @@ -15,7 +16,7 @@ import org.opensearch.action.index.IndexRequest; import org.opensearch.ingest.IngestDocument; -public class OpenSearchBulkRequestTest { +public class OpensearchBulkApiRequestParserTest { private String getRawQueryString(String filename) throws IOException { return Resources.toString( @@ -27,14 +28,14 @@ private String getRawQueryString(String filename) throws IOException { public void testSimpleIndexRequest() throws Exception { String rawRequest = getRawQueryString("index_simple"); - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); assertThat(indexRequests.get(0).index()).isEqualTo("test"); assertThat(indexRequests.get(0).id()).isEqualTo("1"); assertThat(indexRequests.get(0).sourceAsMap().size()).isEqualTo(2); Map> indexDocs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("test").size()).isEqualTo(1); @@ -54,10 +55,10 @@ public void testSimpleIndexRequest() throws Exception { public void testIndexNoFields() throws Exception { String rawRequest = getRawQueryString("index_no_fields"); - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); Map> indexDocs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("test").size()).isEqualTo(1); @@ -77,10 +78,10 @@ public void testIndexNoFields() throws Exception { public void testIndexNoFieldsNoId() throws Exception { String rawRequest = getRawQueryString("index_no_fields_no_id"); - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); Map> indexDocs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("test").size()).isEqualTo(1); @@ -100,27 +101,27 @@ public void testIndexNoFieldsNoId() throws Exception { public void testIndexEmptyRequest() throws Exception { String rawRequest = getRawQueryString("index_empty_request"); - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); Map> indexDocs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(0); } @Test public void testOtherBulkRequests() throws Exception { String rawRequest = getRawQueryString("non_index"); - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(0); } @Test public void testIndexRequestWithSpecialChars() throws Exception { String rawRequest = getRawQueryString("index_request_with_special_chars"); - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); Map> indexDocs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("index_name").size()).isEqualTo(1); @@ -139,11 +140,11 @@ public void testIndexRequestWithSpecialChars() throws Exception { @Test public void testBulkRequests() throws Exception { String rawRequest = getRawQueryString("bulk_requests"); - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); Map> indexDocs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("test").size()).isEqualTo(1); @@ -162,11 +163,11 @@ public void testBulkRequests() throws Exception { @Test public void testUpdatesAgainstTwoIndexes() throws Exception { String rawRequest = getRawQueryString("two_indexes"); - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(2); Map> indexDocs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(2); assertThat(indexDocs.get("test1").size()).isEqualTo(1); assertThat(indexDocs.get("test2").size()).isEqualTo(1); @@ -179,12 +180,11 @@ public void testUpdatesAgainstTwoIndexes() throws Exception { public void testTraceSpanGeneratedTimestamp() throws IOException { String rawRequest = getRawQueryString("index_simple"); - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); - IngestDocument ingestDocument = - OpenSearchBulkApiRequestParser.convertRequestToDocument(indexRequests.get(0)); - Trace.Span span = OpenSearchBulkApiRequestParser.fromIngestDocument(ingestDocument); + IngestDocument ingestDocument = convertRequestToDocument(indexRequests.get(0)); + Trace.Span span = OpensearchBulkApiRequestParser.fromIngestDocument(ingestDocument); // timestamp is in microseconds based on the trace.proto definition Instant ingestDocumentTime = diff --git a/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java b/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java index 144521c546..a1bbc14616 100644 --- a/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java @@ -12,15 +12,15 @@ import com.google.protobuf.InvalidProtocolBufferException; import com.linecorp.armeria.common.AggregatedHttpResponse; import com.slack.kaldb.bulkIngestApi.BulkIngestApi; +import com.slack.kaldb.bulkIngestApi.BulkIngestKafkaProducer; import com.slack.kaldb.bulkIngestApi.BulkIngestResponse; import com.slack.kaldb.bulkIngestApi.DatasetRateLimitingService; -import com.slack.kaldb.bulkIngestApi.TransactionBatchingKafkaProducer; +import com.slack.kaldb.bulkIngestApi.OpensearchBulkApiRequestParser; import com.slack.kaldb.metadata.core.CuratorBuilder; import com.slack.kaldb.metadata.dataset.DatasetMetadata; import com.slack.kaldb.metadata.dataset.DatasetMetadataStore; import com.slack.kaldb.metadata.dataset.DatasetPartitionMetadata; import com.slack.kaldb.preprocessor.PreprocessorRateLimiter; -import com.slack.kaldb.preprocessor.ingest.OpenSearchBulkApiRequestParser; import com.slack.kaldb.proto.config.KaldbConfigs; import com.slack.kaldb.testlib.MetricsUtil; import com.slack.kaldb.testlib.TestKafkaServer; @@ -58,7 +58,7 @@ public class BulkIngestApiTest { private static TestKafkaServer kafkaServer; private BulkIngestApi bulkApi; - private TransactionBatchingKafkaProducer transactionBatchingKafkaProducer; + private BulkIngestKafkaProducer bulkIngestKafkaProducer; private DatasetRateLimitingService datasetRateLimitingService; static String INDEX_NAME = "testindex"; @@ -112,17 +112,16 @@ public void bootstrapCluster() throws Exception { datasetRateLimitingService = new DatasetRateLimitingService(datasetMetadataStore, preprocessorConfig, meterRegistry); - transactionBatchingKafkaProducer = - new TransactionBatchingKafkaProducer( - datasetMetadataStore, preprocessorConfig, meterRegistry); + bulkIngestKafkaProducer = + new BulkIngestKafkaProducer(datasetMetadataStore, preprocessorConfig, meterRegistry); datasetRateLimitingService.startAsync(); - transactionBatchingKafkaProducer.startAsync(); + bulkIngestKafkaProducer.startAsync(); datasetRateLimitingService.awaitRunning(DEFAULT_START_STOP_DURATION); - transactionBatchingKafkaProducer.awaitRunning(DEFAULT_START_STOP_DURATION); + bulkIngestKafkaProducer.awaitRunning(DEFAULT_START_STOP_DURATION); - bulkApi = new BulkIngestApi(transactionBatchingKafkaProducer, datasetRateLimitingService); + bulkApi = new BulkIngestApi(bulkIngestKafkaProducer, datasetRateLimitingService); } // I looked at making this a @BeforeEach. it's possible if you annotate a test with a @Tag and @@ -162,9 +161,9 @@ public void shutdownOpenSearchAPI() throws Exception { datasetRateLimitingService.stopAsync(); datasetRateLimitingService.awaitTerminated(DEFAULT_START_STOP_DURATION); } - if (transactionBatchingKafkaProducer != null) { - transactionBatchingKafkaProducer.stopAsync(); - transactionBatchingKafkaProducer.awaitTerminated(DEFAULT_START_STOP_DURATION); + if (bulkIngestKafkaProducer != null) { + bulkIngestKafkaProducer.stopAsync(); + bulkIngestKafkaProducer.awaitTerminated(DEFAULT_START_STOP_DURATION); } kafkaServer.close(); curatorFramework.unwrap().close(); @@ -197,9 +196,9 @@ public void testBulkApiBasic() throws Exception { """; // get num bytes that can be used to create the dataset. When we make 2 successive calls the // second one should fail - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(request1); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(request1); Map> indexDocs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("testindex").size()).isEqualTo(1); assertThat(indexDocs.get("testindex").get(0).getId().toStringUtf8()).isEqualTo("1"); @@ -287,9 +286,9 @@ public void testDocumentInKafkaSimple() throws Exception { { "index": {"_index": "testindex", "_id": "2"} } { "field1" : "value2" } """; - List indexRequests = OpenSearchBulkApiRequestParser.parseBulkRequest(request1); + List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(request1); Map> indexDocs = - OpenSearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("testindex").size()).isEqualTo(2); int throughputBytes = PreprocessorRateLimiter.getSpanBytes(indexDocs.get("testindex")); From 5fa8cdffde75a99f9d004241a4d368653607a202 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Tue, 19 Dec 2023 11:14:42 -0700 Subject: [PATCH 04/18] Additional cleanup, documentation, error handling, and metrics added --- kaldb/pom.xml | 8 +++ .../kaldb/bulkIngestApi/BulkIngestApi.java | 23 +++++---- .../BulkIngestKafkaProducer.java | 50 ++++++++++++++++--- .../bulkIngestApi/BulkIngestRequest.java | 9 +++- .../bulkIngestApi/BulkIngestResponse.java | 1 + .../DatasetRateLimitingService.java | 6 ++- .../BulkApiRequestParser.java} | 25 +++++++--- .../BulkApiRequestParserTest.java} | 40 +++++++-------- .../slack/kaldb/server/BulkIngestApiTest.java | 24 ++------- 9 files changed, 116 insertions(+), 70 deletions(-) rename kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/{OpensearchBulkApiRequestParser.java => opensearch/BulkApiRequestParser.java} (82%) rename kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/{OpensearchBulkApiRequestParserTest.java => opensearch/BulkApiRequestParserTest.java} (78%) diff --git a/kaldb/pom.xml b/kaldb/pom.xml index 0421035554..2a666aff8d 100644 --- a/kaldb/pom.xml +++ b/kaldb/pom.xml @@ -687,6 +687,14 @@ false + + + + com.google.googlejavaformat + google-java-format + 1.19.1 + + diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java index 5aff97b48d..d2494da076 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java @@ -6,13 +6,18 @@ import com.linecorp.armeria.common.HttpResponse; import com.linecorp.armeria.server.annotation.Blocking; import com.linecorp.armeria.server.annotation.Post; +import com.slack.kaldb.bulkIngestApi.opensearch.BulkApiRequestParser; import com.slack.service.murron.trace.Trace; import java.util.List; import java.util.Map; -import org.opensearch.action.index.IndexRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * This class is responsible for defining the http endpoint behavior for the bulk ingest. It is + * expected to handle appropriate rate limiting, error handling, and submit the parsed messages to + * Kafka for ingestion. + */ public class BulkIngestApi { private static final Logger LOG = LoggerFactory.getLogger(BulkIngestApi.class); private final BulkIngestKafkaProducer bulkIngestKafkaProducer; @@ -26,18 +31,13 @@ public BulkIngestApi( this.datasetRateLimitingService = datasetRateLimitingService; } - /** - * 1. Kaldb does not support the concept of "updates". It's always an add 2. The "index" is used - * as the span name - */ @Blocking @Post("/_bulk") public HttpResponse addDocument(String bulkRequest) { + // 1. Kaldb does not support the concept of "updates". It's always an add. + // 2. The "index" is used as the span name try { - List indexRequests = - OpensearchBulkApiRequestParser.parseBulkRequest(bulkRequest); - Map> docs = - OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + Map> docs = BulkApiRequestParser.parseRequest(bulkRequest); // todo - our rate limiter doesn't have a way to acquire permits across multiple datasets // so today as a limitation we reject any request that has documents against multiple indexes @@ -55,7 +55,10 @@ public HttpResponse addDocument(String bulkRequest) { return HttpResponse.ofJson(TOO_MANY_REQUESTS, response); } } - BulkIngestResponse response = bulkIngestKafkaProducer.createRequest(docs).getResponse(); + + // getResponse will cause this thread to wait until the batch producer submits or it hits + // Armeria timeout + BulkIngestResponse response = bulkIngestKafkaProducer.submitRequest(docs).getResponse(); return HttpResponse.ofJson(response); } catch (Exception e) { LOG.error("Request failed ", e); diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index d7111ea767..414dd9af82 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -12,6 +12,7 @@ import com.slack.kaldb.proto.config.KaldbConfigs; import com.slack.kaldb.util.RuntimeHalterImpl; import com.slack.service.murron.trace.Trace; +import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics; import io.micrometer.prometheus.PrometheusMeterRegistry; import java.util.ArrayList; @@ -24,6 +25,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.errors.AuthorizationException; @@ -43,12 +45,24 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { private final DatasetMetadataStore datasetMetadataStore; private final KaldbMetadataStoreChangeListener datasetListener = - (datasetMetadata) -> cacheSortedDataset(); + (_) -> cacheSortedDataset(); protected List throughputSortedDatasets; - // todo - parameterize the capacity option? - private final BlockingQueue pendingRequests = new ArrayBlockingQueue<>(500); + private final BlockingQueue pendingRequests; + + private final Integer producerSleep; + + public static final String FAILED_SET_RESPONSE_COUNTER = + "bulk_ingest_producer_failed_set_response"; + private final Counter failedSetResponseCounter; + + public static final String OVER_LIMIT_PENDING_REQUESTS = + "bulk_ingest_producer_over_limit_pending"; + private final Counter overLimitPendingRequests; + + public static final String BATCH_SIZE_GAUGE = "bulk_ingest_producer_batch_size"; + private final AtomicInteger batchSizeGauge; public BulkIngestKafkaProducer( final DatasetMetadataStore datasetMetadataStore, @@ -65,6 +79,14 @@ public BulkIngestKafkaProducer( this.preprocessorConfig = preprocessorConfig; this.datasetMetadataStore = datasetMetadataStore; + // todo - consider making these a configurable value, or determine a way to derive a reasonable + // value automatically + this.pendingRequests = + new ArrayBlockingQueue<>( + Integer.parseInt(System.getProperty("kalDb.bulkIngest.pendingLimit", "500"))); + this.producerSleep = + Integer.parseInt(System.getProperty("kalDb.bulkIngest.producerSleep", "2000")); + // since we use a new transaction ID every time we start a preprocessor there can be some zombie // transactions? // I think they will remain in kafka till they expire. They should never be readable if the @@ -75,6 +97,10 @@ public BulkIngestKafkaProducer( this.kafkaMetrics = new KafkaClientMetrics(kafkaProducer); this.kafkaMetrics.bindTo(meterRegistry); + this.failedSetResponseCounter = meterRegistry.counter(FAILED_SET_RESPONSE_COUNTER); + this.overLimitPendingRequests = meterRegistry.counter(OVER_LIMIT_PENDING_REQUESTS); + this.batchSizeGauge = meterRegistry.gauge(BATCH_SIZE_GAUGE, new AtomicInteger(0)); + this.kafkaProducer.initTransactions(); } @@ -98,9 +124,10 @@ protected void run() throws Exception { while (isRunning()) { List requests = new ArrayList<>(); pendingRequests.drainTo(requests); + batchSizeGauge.set(requests.size()); if (requests.isEmpty()) { try { - Thread.sleep(2000); + Thread.sleep(producerSleep); } catch (InterruptedException e) { return; } @@ -120,10 +147,14 @@ protected void shutDown() throws Exception { } } - public BulkIngestRequest createRequest(Map> inputDocs) { + public BulkIngestRequest submitRequest(Map> inputDocs) { BulkIngestRequest request = new BulkIngestRequest(inputDocs); - // todo - add can throw exceptions - pendingRequests.add(request); + try { + pendingRequests.add(request); + } catch (IllegalStateException e) { + overLimitPendingRequests.increment(); + throw e; + } return request; } @@ -157,7 +188,10 @@ protected Map transactionCommit( for (Map.Entry entry : responseMap.entrySet()) { BulkIngestRequest key = entry.getKey(); BulkIngestResponse value = entry.getValue(); - key.setResponse(value); + if (!key.setResponse(value)) { + LOG.warn("Failed to add result to the bulk ingest request, consumer thread went away?"); + failedSetResponseCounter.increment(); + } } return responseMap; } diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestRequest.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestRequest.java index 0b6427782c..8b47a0310b 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestRequest.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestRequest.java @@ -5,6 +5,11 @@ import java.util.Map; import java.util.concurrent.SynchronousQueue; +/** + * Wrapper object to enable building a bulk request and awaiting on an asynchronous response to be + * populated. As this uses a synchronous queue internally, it expects a thread to already be waiting + * on getResponse when setResponse is invoked with the result data. + */ public class BulkIngestRequest { private final Map> inputDocs; private final SynchronousQueue internalResponse = new SynchronousQueue<>(); @@ -17,8 +22,8 @@ Map> getInputDocs() { return inputDocs; } - void setResponse(BulkIngestResponse response) { - internalResponse.add(response); + boolean setResponse(BulkIngestResponse response) { + return internalResponse.offer(response); } public BulkIngestResponse getResponse() throws InterruptedException { diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestResponse.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestResponse.java index c3be3bec47..fa1947ae56 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestResponse.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestResponse.java @@ -1,3 +1,4 @@ package com.slack.kaldb.bulkIngestApi; +/** Metadata object for the result of a bulk ingest request */ public record BulkIngestResponse(int totalDocs, long failedDocs, String errorMsg) {} diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java index fed2532333..c8cd900aba 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/DatasetRateLimitingService.java @@ -12,10 +12,14 @@ import java.util.List; import java.util.function.BiPredicate; +/** + * Guava service that maintains an rate limiting object consistent with the value stored in the + * dataset metadata store. + */ public class DatasetRateLimitingService extends AbstractIdleService { private final DatasetMetadataStore datasetMetadataStore; private final KaldbMetadataStoreChangeListener datasetListener = - (datasetMetadata) -> updateRateLimiter(); + (_) -> updateRateLimiter(); private final PreprocessorRateLimiter rateLimiter; private BiPredicate> rateLimiterPredicate; diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParser.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParser.java similarity index 82% rename from kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParser.java rename to kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParser.java index c7457ecad6..f6c323baa7 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParser.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParser.java @@ -1,4 +1,4 @@ -package com.slack.kaldb.bulkIngestApi; +package com.slack.kaldb.bulkIngestApi.opensearch; import com.google.protobuf.ByteString; import com.slack.kaldb.writer.SpanFormatter; @@ -22,13 +22,22 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class OpensearchBulkApiRequestParser { +/** + * This class uses the Opensearch libraries to parse the bulk ingest request into documents which + * can be inserted into Kafka. The goal of this is to leverage Opensearch where possible, while + * preventing opensearch abstractions from leaking further into KalDB. + */ +public class BulkApiRequestParser { - public static final Logger LOG = LoggerFactory.getLogger(OpensearchBulkApiRequestParser.class); + private static final Logger LOG = LoggerFactory.getLogger(BulkApiRequestParser.class); private static final String SERVICE_NAME_KEY = "service_name"; - public static Trace.Span fromIngestDocument(IngestDocument ingestDocument) { + public static Map> parseRequest(String postBody) throws IOException { + return convertIndexRequestToTraceFormat(parseBulkRequest(postBody)); + } + + protected static Trace.Span fromIngestDocument(IngestDocument ingestDocument) { ZonedDateTime timestamp = (ZonedDateTime) ingestDocument @@ -69,9 +78,9 @@ public static Trace.Span fromIngestDocument(IngestDocument ingestDocument) { return spanBuilder.build(); } - // key - index. value - list of docs to be indexed - public static Map> convertIndexRequestToTraceFormat( + protected static Map> convertIndexRequestToTraceFormat( List indexRequests) { + // key - index. value - list of docs to be indexed Map> indexDocs = new HashMap<>(); for (IndexRequest indexRequest : indexRequests) { @@ -81,7 +90,7 @@ public static Map> convertIndexRequestToTraceFormat( } IngestDocument ingestDocument = convertRequestToDocument(indexRequest); List docs = indexDocs.computeIfAbsent(index, key -> new ArrayList<>()); - docs.add(OpensearchBulkApiRequestParser.fromIngestDocument(ingestDocument)); + docs.add(BulkApiRequestParser.fromIngestDocument(ingestDocument)); } return indexDocs; } @@ -101,7 +110,7 @@ protected static IngestDocument convertRequestToDocument(IndexRequest indexReque // and transform it } - public static List parseBulkRequest(String postBody) throws IOException { + protected static List parseBulkRequest(String postBody) throws IOException { List indexRequests = new ArrayList<>(); BulkRequest bulkRequest = new BulkRequest(); // calls parse under the hood diff --git a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParserTest.java b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParserTest.java similarity index 78% rename from kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParserTest.java rename to kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParserTest.java index cdbabaad84..c7b735da31 100644 --- a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/OpensearchBulkApiRequestParserTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParserTest.java @@ -1,6 +1,6 @@ -package com.slack.kaldb.bulkIngestApi; +package com.slack.kaldb.bulkIngestApi.opensearch; -import static com.slack.kaldb.bulkIngestApi.OpensearchBulkApiRequestParser.convertRequestToDocument; +import static com.slack.kaldb.bulkIngestApi.opensearch.BulkApiRequestParser.convertRequestToDocument; import static org.assertj.core.api.Assertions.assertThat; import com.google.common.io.Resources; @@ -16,7 +16,7 @@ import org.opensearch.action.index.IndexRequest; import org.opensearch.ingest.IngestDocument; -public class OpensearchBulkApiRequestParserTest { +public class BulkApiRequestParserTest { private String getRawQueryString(String filename) throws IOException { return Resources.toString( @@ -28,14 +28,14 @@ private String getRawQueryString(String filename) throws IOException { public void testSimpleIndexRequest() throws Exception { String rawRequest = getRawQueryString("index_simple"); - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); assertThat(indexRequests.get(0).index()).isEqualTo("test"); assertThat(indexRequests.get(0).id()).isEqualTo("1"); assertThat(indexRequests.get(0).sourceAsMap().size()).isEqualTo(2); Map> indexDocs = - OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + BulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("test").size()).isEqualTo(1); @@ -55,10 +55,10 @@ public void testSimpleIndexRequest() throws Exception { public void testIndexNoFields() throws Exception { String rawRequest = getRawQueryString("index_no_fields"); - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); Map> indexDocs = - OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + BulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("test").size()).isEqualTo(1); @@ -78,10 +78,10 @@ public void testIndexNoFields() throws Exception { public void testIndexNoFieldsNoId() throws Exception { String rawRequest = getRawQueryString("index_no_fields_no_id"); - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); Map> indexDocs = - OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + BulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("test").size()).isEqualTo(1); @@ -101,27 +101,27 @@ public void testIndexNoFieldsNoId() throws Exception { public void testIndexEmptyRequest() throws Exception { String rawRequest = getRawQueryString("index_empty_request"); - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); Map> indexDocs = - OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + BulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(0); } @Test public void testOtherBulkRequests() throws Exception { String rawRequest = getRawQueryString("non_index"); - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(0); } @Test public void testIndexRequestWithSpecialChars() throws Exception { String rawRequest = getRawQueryString("index_request_with_special_chars"); - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); Map> indexDocs = - OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + BulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("index_name").size()).isEqualTo(1); @@ -140,11 +140,11 @@ public void testIndexRequestWithSpecialChars() throws Exception { @Test public void testBulkRequests() throws Exception { String rawRequest = getRawQueryString("bulk_requests"); - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); Map> indexDocs = - OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + BulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(1); assertThat(indexDocs.get("test").size()).isEqualTo(1); @@ -163,11 +163,11 @@ public void testBulkRequests() throws Exception { @Test public void testUpdatesAgainstTwoIndexes() throws Exception { String rawRequest = getRawQueryString("two_indexes"); - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(2); Map> indexDocs = - OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); + BulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); assertThat(indexDocs.keySet().size()).isEqualTo(2); assertThat(indexDocs.get("test1").size()).isEqualTo(1); assertThat(indexDocs.get("test2").size()).isEqualTo(1); @@ -180,11 +180,11 @@ public void testUpdatesAgainstTwoIndexes() throws Exception { public void testTraceSpanGeneratedTimestamp() throws IOException { String rawRequest = getRawQueryString("index_simple"); - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(rawRequest); + List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); IngestDocument ingestDocument = convertRequestToDocument(indexRequests.get(0)); - Trace.Span span = OpensearchBulkApiRequestParser.fromIngestDocument(ingestDocument); + Trace.Span span = BulkApiRequestParser.fromIngestDocument(ingestDocument); // timestamp is in microseconds based on the trace.proto definition Instant ingestDocumentTime = diff --git a/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java b/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java index a1bbc14616..3c32218568 100644 --- a/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java @@ -15,12 +15,10 @@ import com.slack.kaldb.bulkIngestApi.BulkIngestKafkaProducer; import com.slack.kaldb.bulkIngestApi.BulkIngestResponse; import com.slack.kaldb.bulkIngestApi.DatasetRateLimitingService; -import com.slack.kaldb.bulkIngestApi.OpensearchBulkApiRequestParser; import com.slack.kaldb.metadata.core.CuratorBuilder; import com.slack.kaldb.metadata.dataset.DatasetMetadata; import com.slack.kaldb.metadata.dataset.DatasetMetadataStore; import com.slack.kaldb.metadata.dataset.DatasetPartitionMetadata; -import com.slack.kaldb.preprocessor.PreprocessorRateLimiter; import com.slack.kaldb.proto.config.KaldbConfigs; import com.slack.kaldb.testlib.MetricsUtil; import com.slack.kaldb.testlib.TestKafkaServer; @@ -29,10 +27,10 @@ import io.micrometer.prometheus.PrometheusConfig; import io.micrometer.prometheus.PrometheusMeterRegistry; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.List; -import java.util.Map; import java.util.Properties; import java.util.concurrent.CompletableFuture; import org.apache.curator.test.TestingServer; @@ -44,7 +42,6 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.opensearch.action.index.IndexRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -194,16 +191,7 @@ public void testBulkApiBasic() throws Exception { { "index": {"_index": "testindex", "_id": "1"} } { "field1" : "value1" } """; - // get num bytes that can be used to create the dataset. When we make 2 successive calls the - // second one should fail - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(request1); - Map> indexDocs = - OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); - assertThat(indexDocs.keySet().size()).isEqualTo(1); - assertThat(indexDocs.get("testindex").size()).isEqualTo(1); - assertThat(indexDocs.get("testindex").get(0).getId().toStringUtf8()).isEqualTo("1"); - int throughputBytes = PreprocessorRateLimiter.getSpanBytes(indexDocs.get("testindex")); - updateDatasetThroughput(throughputBytes); + updateDatasetThroughput(request1.getBytes(StandardCharsets.UTF_8).length); // test with empty causes a parse exception AggregatedHttpResponse response = bulkApi.addDocument("{}\n").aggregate().join(); @@ -286,13 +274,7 @@ public void testDocumentInKafkaSimple() throws Exception { { "index": {"_index": "testindex", "_id": "2"} } { "field1" : "value2" } """; - List indexRequests = OpensearchBulkApiRequestParser.parseBulkRequest(request1); - Map> indexDocs = - OpensearchBulkApiRequestParser.convertIndexRequestToTraceFormat(indexRequests); - assertThat(indexDocs.keySet().size()).isEqualTo(1); - assertThat(indexDocs.get("testindex").size()).isEqualTo(2); - int throughputBytes = PreprocessorRateLimiter.getSpanBytes(indexDocs.get("testindex")); - updateDatasetThroughput(throughputBytes); + updateDatasetThroughput(request1.getBytes(StandardCharsets.UTF_8).length); KafkaConsumer kafkaConsumer = getTestKafkaConsumer(); From 200af949fd7f80109ea33242452680d7f60f28e2 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Tue, 19 Dec 2023 14:41:22 -0700 Subject: [PATCH 05/18] Rework exception handling to better catch errors --- .../BulkIngestKafkaProducer.java | 52 +++++++++---------- 1 file changed, 25 insertions(+), 27 deletions(-) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index 414dd9af82..b91d9a9ccc 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -167,6 +167,22 @@ protected Map transactionCommit( responseMap.put(request, produceDocuments(request.getInputDocs())); } kafkaProducer.commitTransaction(); + } catch (TimeoutException te) { + LOG.error("Commit transaction timeout", te); + // the commitTransaction waits till "max.block.ms" after which it will time out + // in that case we cannot call abort exception because that throws the following error + // "Cannot attempt operation `abortTransaction` because the previous + // call to `commitTransaction` timed out and must be retried" + // so for now we just restart the preprocessor + new RuntimeHalterImpl() + .handleFatal( + new Throwable( + "KafkaProducer needs to shutdown as we don't have retry yet and we cannot call abortTxn on timeout", + te)); + } catch (ProducerFencedException | OutOfOrderSequenceException | AuthorizationException e) { + // We can't recover from these exceptions, so our only option is to close the producer and + // exit. + new RuntimeHalterImpl().handleFatal(new Throwable("KafkaProducer needs to shutdown ", e)); } catch (Exception e) { LOG.warn("failed transaction with error", e); try { @@ -221,33 +237,15 @@ private BulkIngestResponse produceDocuments(Map> indexD // Till we fix the producer design to allow for multiple /_bulk requests to be able to // write to the same txn // we will limit producing documents 1 thread at a time - try { - for (Trace.Span doc : indexDoc.getValue()) { - ProducerRecord producerRecord = - new ProducerRecord<>( - preprocessorConfig.getDownstreamTopic(), partition, index, doc.toByteArray()); - - // we intentionally supress FutureReturnValueIgnored here in errorprone - this is because - // we wrap this in a transaction, which is responsible for flushing all of the pending - // messages - kafkaProducer.send(producerRecord); - } - } catch (TimeoutException te) { - LOG.error("Commit transaction timeout", te); - // the commitTransaction waits till "max.block.ms" after which it will time out - // in that case we cannot call abort exception because that throws the following error - // "Cannot attempt operation `abortTransaction` because the previous - // call to `commitTransaction` timed out and must be retried" - // so for now we just restart the preprocessor - new RuntimeHalterImpl() - .handleFatal( - new Throwable( - "KafkaProducer needs to shutdown as we don't have retry yet and we cannot call abortTxn on timeout", - te)); - } catch (ProducerFencedException | OutOfOrderSequenceException | AuthorizationException e) { - // We can't recover from these exceptions, so our only option is to close the producer and - // exit. - new RuntimeHalterImpl().handleFatal(new Throwable("KafkaProducer needs to shutdown ", e)); + for (Trace.Span doc : indexDoc.getValue()) { + ProducerRecord producerRecord = + new ProducerRecord<>( + preprocessorConfig.getDownstreamTopic(), partition, index, doc.toByteArray()); + + // we intentionally supress FutureReturnValueIgnored here in errorprone - this is because + // we wrap this in a transaction, which is responsible for flushing all of the pending + // messages + kafkaProducer.send(producerRecord); } } From ad0f22052823bb743ea79391738afad98076cc20 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Tue, 19 Dec 2023 15:12:51 -0700 Subject: [PATCH 06/18] Adjust producer linger/batch configs --- .../slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index b91d9a9ccc..b6d1c48c59 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -258,8 +258,9 @@ private KafkaProducer createKafkaTransactionProducer(String tran props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); props.put("transactional.id", transactionId); - props.put("linger.ms", 250); - props.put("max.block.ms", "10000"); + props.put("linger.ms", 500); + props.put("batch.size", 128000); + props.put("max.block.ms", 10000); props.put("compression.type", "snappy"); return new KafkaProducer<>(props); } From 1d92605b6c29165f02206b24639f0b66ad9f52c6 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Thu, 21 Dec 2023 10:49:38 -0700 Subject: [PATCH 07/18] Add bulk ingest edge instrumentation --- .../kaldb/bulkIngestApi/BulkIngestApi.java | 13 ++++++++-- .../opensearch/BulkApiRequestParser.java | 8 +++--- .../java/com/slack/kaldb/server/Kaldb.java | 2 +- .../opensearch/BulkApiRequestParserTest.java | 26 ++++++++++--------- .../slack/kaldb/server/BulkIngestApiTest.java | 2 +- 5 files changed, 30 insertions(+), 21 deletions(-) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java index d2494da076..943e2ab4d8 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java @@ -8,6 +8,9 @@ import com.linecorp.armeria.server.annotation.Post; import com.slack.kaldb.bulkIngestApi.opensearch.BulkApiRequestParser; import com.slack.service.murron.trace.Trace; +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.MeterRegistry; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import org.slf4j.Logger; @@ -22,13 +25,17 @@ public class BulkIngestApi { private static final Logger LOG = LoggerFactory.getLogger(BulkIngestApi.class); private final BulkIngestKafkaProducer bulkIngestKafkaProducer; private final DatasetRateLimitingService datasetRateLimitingService; + private final Counter incomingByteTotal; + private final String BULK_INGEST_INCOMING_BYTE_TOTAL = "kaldb_preprocessor_incoming_byte"; public BulkIngestApi( BulkIngestKafkaProducer bulkIngestKafkaProducer, - DatasetRateLimitingService datasetRateLimitingService) { + DatasetRateLimitingService datasetRateLimitingService, + MeterRegistry meterRegistry) { this.bulkIngestKafkaProducer = bulkIngestKafkaProducer; this.datasetRateLimitingService = datasetRateLimitingService; + this.incomingByteTotal = meterRegistry.counter(BULK_INGEST_INCOMING_BYTE_TOTAL); } @Blocking @@ -37,7 +44,9 @@ public HttpResponse addDocument(String bulkRequest) { // 1. Kaldb does not support the concept of "updates". It's always an add. // 2. The "index" is used as the span name try { - Map> docs = BulkApiRequestParser.parseRequest(bulkRequest); + byte[] bulkRequestBytes = bulkRequest.getBytes(StandardCharsets.UTF_8); + incomingByteTotal.increment(bulkRequestBytes.length); + Map> docs = BulkApiRequestParser.parseRequest(bulkRequestBytes); // todo - our rate limiter doesn't have a way to acquire permits across multiple datasets // so today as a limitation we reject any request that has documents against multiple indexes diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParser.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParser.java index f6c323baa7..527ecb77c5 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParser.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParser.java @@ -4,7 +4,6 @@ import com.slack.kaldb.writer.SpanFormatter; import com.slack.service.murron.trace.Trace; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.util.ArrayList; @@ -33,7 +32,7 @@ public class BulkApiRequestParser { private static final String SERVICE_NAME_KEY = "service_name"; - public static Map> parseRequest(String postBody) throws IOException { + public static Map> parseRequest(byte[] postBody) throws IOException { return convertIndexRequestToTraceFormat(parseBulkRequest(postBody)); } @@ -110,12 +109,11 @@ protected static IngestDocument convertRequestToDocument(IndexRequest indexReque // and transform it } - protected static List parseBulkRequest(String postBody) throws IOException { + protected static List parseBulkRequest(byte[] postBody) throws IOException { List indexRequests = new ArrayList<>(); BulkRequest bulkRequest = new BulkRequest(); // calls parse under the hood - byte[] bytes = postBody.getBytes(StandardCharsets.UTF_8); - bulkRequest.add(bytes, 0, bytes.length, null, MediaTypeRegistry.JSON); + bulkRequest.add(postBody, 0, postBody.length, null, MediaTypeRegistry.JSON); List> requests = bulkRequest.requests(); for (DocWriteRequest request : requests) { if (request.opType() == DocWriteRequest.OpType.INDEX) { diff --git a/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java b/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java index 3760b4e5b4..8c93374235 100644 --- a/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java +++ b/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java @@ -399,7 +399,7 @@ private static Set getServices( services.add(datasetRateLimitingService); BulkIngestApi openSearchBulkApiService = - new BulkIngestApi(bulkIngestKafkaProducer, datasetRateLimitingService); + new BulkIngestApi(bulkIngestKafkaProducer, datasetRateLimitingService, meterRegistry); armeriaServiceBuilder.withAnnotatedService(openSearchBulkApiService); } else { PreprocessorService preprocessorService = diff --git a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParserTest.java b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParserTest.java index c7b735da31..c25d76a548 100644 --- a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParserTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/opensearch/BulkApiRequestParserTest.java @@ -7,6 +7,7 @@ import com.slack.service.murron.trace.Trace; import java.io.IOException; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.time.Instant; import java.time.temporal.ChronoUnit; import java.util.List; @@ -18,15 +19,16 @@ public class BulkApiRequestParserTest { - private String getRawQueryString(String filename) throws IOException { + private byte[] getRawQueryBytes(String filename) throws IOException { return Resources.toString( - Resources.getResource(String.format("opensearchRequest/bulk/%s.ndjson", filename)), - Charset.defaultCharset()); + Resources.getResource(String.format("opensearchRequest/bulk/%s.ndjson", filename)), + Charset.defaultCharset()) + .getBytes(StandardCharsets.UTF_8); } @Test public void testSimpleIndexRequest() throws Exception { - String rawRequest = getRawQueryString("index_simple"); + byte[] rawRequest = getRawQueryBytes("index_simple"); List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); @@ -53,7 +55,7 @@ public void testSimpleIndexRequest() throws Exception { @Test public void testIndexNoFields() throws Exception { - String rawRequest = getRawQueryString("index_no_fields"); + byte[] rawRequest = getRawQueryBytes("index_no_fields"); List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); @@ -76,7 +78,7 @@ public void testIndexNoFields() throws Exception { @Test public void testIndexNoFieldsNoId() throws Exception { - String rawRequest = getRawQueryString("index_no_fields_no_id"); + byte[] rawRequest = getRawQueryBytes("index_no_fields_no_id"); List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); @@ -99,7 +101,7 @@ public void testIndexNoFieldsNoId() throws Exception { @Test public void testIndexEmptyRequest() throws Exception { - String rawRequest = getRawQueryString("index_empty_request"); + byte[] rawRequest = getRawQueryBytes("index_empty_request"); List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); @@ -110,14 +112,14 @@ public void testIndexEmptyRequest() throws Exception { @Test public void testOtherBulkRequests() throws Exception { - String rawRequest = getRawQueryString("non_index"); + byte[] rawRequest = getRawQueryBytes("non_index"); List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(0); } @Test public void testIndexRequestWithSpecialChars() throws Exception { - String rawRequest = getRawQueryString("index_request_with_special_chars"); + byte[] rawRequest = getRawQueryBytes("index_request_with_special_chars"); List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); Map> indexDocs = @@ -139,7 +141,7 @@ public void testIndexRequestWithSpecialChars() throws Exception { @Test public void testBulkRequests() throws Exception { - String rawRequest = getRawQueryString("bulk_requests"); + byte[] rawRequest = getRawQueryBytes("bulk_requests"); List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); @@ -162,7 +164,7 @@ public void testBulkRequests() throws Exception { @Test public void testUpdatesAgainstTwoIndexes() throws Exception { - String rawRequest = getRawQueryString("two_indexes"); + byte[] rawRequest = getRawQueryBytes("two_indexes"); List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(2); @@ -178,7 +180,7 @@ public void testUpdatesAgainstTwoIndexes() throws Exception { @Test public void testTraceSpanGeneratedTimestamp() throws IOException { - String rawRequest = getRawQueryString("index_simple"); + byte[] rawRequest = getRawQueryBytes("index_simple"); List indexRequests = BulkApiRequestParser.parseBulkRequest(rawRequest); assertThat(indexRequests.size()).isEqualTo(1); diff --git a/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java b/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java index 3c32218568..2126569aa7 100644 --- a/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java @@ -118,7 +118,7 @@ public void bootstrapCluster() throws Exception { datasetRateLimitingService.awaitRunning(DEFAULT_START_STOP_DURATION); bulkIngestKafkaProducer.awaitRunning(DEFAULT_START_STOP_DURATION); - bulkApi = new BulkIngestApi(bulkIngestKafkaProducer, datasetRateLimitingService); + bulkApi = new BulkIngestApi(bulkIngestKafkaProducer, datasetRateLimitingService, meterRegistry); } // I looked at making this a @BeforeEach. it's possible if you annotate a test with a @Tag and From 51379e91cb050f58a49fc6d7750013a1b1590978 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Thu, 21 Dec 2023 11:28:54 -0700 Subject: [PATCH 08/18] Add support for additional kafka props on bulk ingest --- config/config.yaml | 5 +- .../BulkIngestKafkaProducer.java | 58 ++++++++++++++----- .../preprocessor/PreprocessorService.java | 6 +- .../com/slack/kaldb/writer/KafkaUtils.java | 35 +++++++++++ .../writer/kafka/KaldbKafkaConsumer.java | 37 +++--------- kaldb/src/main/proto/kaldb_configs.proto | 4 +- .../BulkIngestKafkaProducerTest.java | 8 ++- .../slack/kaldb/server/BulkIngestApiTest.java | 8 ++- .../slack/kaldb/server/KaldbConfigTest.java | 12 +++- .../slack/kaldb/writer/KafkaUtilsTest.java | 48 +++++++++++++++ .../writer/kafka/KaldbKafkaConsumerTest.java | 34 ----------- kaldb/src/test/resources/test_config.json | 4 ++ kaldb/src/test/resources/test_config.yaml | 4 ++ 13 files changed, 172 insertions(+), 91 deletions(-) create mode 100644 kaldb/src/main/java/com/slack/kaldb/writer/KafkaUtils.java create mode 100644 kaldb/src/test/java/com/slack/kaldb/writer/KafkaUtilsTest.java diff --git a/config/config.yaml b/config/config.yaml index 9835654318..6c95431fd1 100644 --- a/config/config.yaml +++ b/config/config.yaml @@ -125,6 +125,10 @@ preprocessorConfig: numStreamThreads: ${KAFKA_STREAM_THREADS:-2} processingGuarantee: ${KAFKA_STREAM_PROCESSING_GUARANTEE:-at_least_once} additionalProps: ${KAFKA_ADDITIONAL_PROPS:-} + kafkaConfig: + kafkaTopic: ${KAFKA_TOPIC:-test-topic} + kafkaBootStrapServers: ${KAFKA_BOOTSTRAP_SERVERS:-localhost:9092} + additionalProps: ${KAFKA_ADDITIONAL_PROPS:-} serverConfig: serverPort: ${KALDB_PREPROCESSOR_SERVER_PORT:-8086} @@ -136,5 +140,4 @@ preprocessorConfig: dataTransformer: ${PREPROCESSOR_TRANSFORMER:-json} rateLimiterMaxBurstSeconds: ${PREPROCESSOR_RATE_LIMITER_MAX_BURST_SECONDS:-1} kafkaPartitionStickyTimeoutMs: ${KAFKA_PARTITION_STICKY_TIMEOUT_MS:-0} - bootstrapServers: ${KAFKA_BOOTSTRAP_SERVERS:-localhost:9092} useBulkApi: ${KALDB_PREPROCESSOR_USE_BULK_API:-false} diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index b6d1c48c59..ed74e474fc 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -11,6 +11,7 @@ import com.slack.kaldb.preprocessor.PreprocessorService; import com.slack.kaldb.proto.config.KaldbConfigs; import com.slack.kaldb.util.RuntimeHalterImpl; +import com.slack.kaldb.writer.KafkaUtils; import com.slack.service.murron.trace.Trace; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics; @@ -21,12 +22,14 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.UUID; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.errors.AuthorizationException; import org.apache.kafka.common.errors.OutOfOrderSequenceException; @@ -41,7 +44,7 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { private final KafkaProducer kafkaProducer; private final KafkaClientMetrics kafkaMetrics; - private final KaldbConfigs.PreprocessorConfig preprocessorConfig; + private final KaldbConfigs.KafkaConfig kafkaConfig; private final DatasetMetadataStore datasetMetadataStore; private final KaldbMetadataStoreChangeListener datasetListener = @@ -64,19 +67,28 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { public static final String BATCH_SIZE_GAUGE = "bulk_ingest_producer_batch_size"; private final AtomicInteger batchSizeGauge; + private static final Set OVERRIDABLE_CONFIGS = + Set.of( + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + ProducerConfig.LINGER_MS_CONFIG, + ProducerConfig.BATCH_SIZE_CONFIG, + ProducerConfig.MAX_BLOCK_MS_CONFIG, + ProducerConfig.COMPRESSION_TYPE_CONFIG); + public BulkIngestKafkaProducer( final DatasetMetadataStore datasetMetadataStore, final KaldbConfigs.PreprocessorConfig preprocessorConfig, final PrometheusMeterRegistry meterRegistry) { + + this.kafkaConfig = preprocessorConfig.getKafkaConfig(); + checkArgument( - !preprocessorConfig.getBootstrapServers().isEmpty(), + !kafkaConfig.getKafkaBootStrapServers().isEmpty(), "Kafka bootstrapServers must be provided"); - checkArgument( - !preprocessorConfig.getDownstreamTopic().isEmpty(), - "Kafka downstreamTopic must be provided"); + checkArgument(!kafkaConfig.getKafkaTopic().isEmpty(), "Kafka topic must be provided"); - this.preprocessorConfig = preprocessorConfig; this.datasetMetadataStore = datasetMetadataStore; // todo - consider making these a configurable value, or determine a way to derive a reasonable @@ -239,8 +251,7 @@ private BulkIngestResponse produceDocuments(Map> indexD // we will limit producing documents 1 thread at a time for (Trace.Span doc : indexDoc.getValue()) { ProducerRecord producerRecord = - new ProducerRecord<>( - preprocessorConfig.getDownstreamTopic(), partition, index, doc.toByteArray()); + new ProducerRecord<>(kafkaConfig.getKafkaTopic(), partition, index, doc.toByteArray()); // we intentionally supress FutureReturnValueIgnored here in errorprone - this is because // we wrap this in a transaction, which is responsible for flushing all of the pending @@ -254,14 +265,29 @@ private BulkIngestResponse produceDocuments(Map> indexD private KafkaProducer createKafkaTransactionProducer(String transactionId) { Properties props = new Properties(); - props.put("bootstrap.servers", preprocessorConfig.getBootstrapServers()); - props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); - props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); - props.put("transactional.id", transactionId); - props.put("linger.ms", 500); - props.put("batch.size", 128000); - props.put("max.block.ms", 10000); - props.put("compression.type", "snappy"); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaConfig.getKafkaBootStrapServers()); + props.put( + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringSerializer"); + props.put( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArraySerializer"); + props.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionId); + props.put(ProducerConfig.LINGER_MS_CONFIG, 500); + props.put(ProducerConfig.BATCH_SIZE_CONFIG, 128000); + props.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000); + props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, "snappy"); + + // don't override the properties that we have already set explicitly using named properties + for (Map.Entry additionalProp : + kafkaConfig.getAdditionalPropsMap().entrySet()) { + props = + KafkaUtils.maybeOverrideProps( + props, + additionalProp.getKey(), + additionalProp.getValue(), + OVERRIDABLE_CONFIGS.contains(additionalProp.getKey())); + } return new KafkaProducer<>(props); } diff --git a/kaldb/src/main/java/com/slack/kaldb/preprocessor/PreprocessorService.java b/kaldb/src/main/java/com/slack/kaldb/preprocessor/PreprocessorService.java index 1e87100fbd..5351ec9b01 100644 --- a/kaldb/src/main/java/com/slack/kaldb/preprocessor/PreprocessorService.java +++ b/kaldb/src/main/java/com/slack/kaldb/preprocessor/PreprocessorService.java @@ -2,7 +2,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.slack.kaldb.server.KaldbConfig.DEFAULT_START_STOP_DURATION; -import static com.slack.kaldb.writer.kafka.KaldbKafkaConsumer.maybeOverride; import com.google.common.util.concurrent.AbstractService; import com.slack.kaldb.metadata.core.KaldbMetadataStoreChangeListener; @@ -10,6 +9,7 @@ import com.slack.kaldb.metadata.dataset.DatasetMetadataStore; import com.slack.kaldb.metadata.dataset.DatasetPartitionMetadata; import com.slack.kaldb.proto.config.KaldbConfigs; +import com.slack.kaldb.writer.KafkaUtils; import com.slack.service.murron.trace.Trace; import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.Timer; @@ -313,7 +313,9 @@ protected static Properties makeKafkaStreamsProps( // don't override any property we already set for (Map.Entry additionalProp : kafkaStreamConfig.getAdditionalPropsMap().entrySet()) { - maybeOverride(props, additionalProp.getKey(), additionalProp.getValue(), false); + props = + KafkaUtils.maybeOverrideProps( + props, additionalProp.getKey(), additionalProp.getValue(), false); } return props; diff --git a/kaldb/src/main/java/com/slack/kaldb/writer/KafkaUtils.java b/kaldb/src/main/java/com/slack/kaldb/writer/KafkaUtils.java new file mode 100644 index 0000000000..7c351a6f00 --- /dev/null +++ b/kaldb/src/main/java/com/slack/kaldb/writer/KafkaUtils.java @@ -0,0 +1,35 @@ +package com.slack.kaldb.writer; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Properties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Shared kafka functions for producers, consumers, and stream applications */ +public class KafkaUtils { + private static final Logger LOG = LoggerFactory.getLogger(KafkaUtils.class); + + @VisibleForTesting + public static Properties maybeOverrideProps( + Properties inputProps, String key, String value, boolean override) { + Properties changedProps = (Properties) inputProps.clone(); + String userValue = changedProps.getProperty(key); + if (userValue != null) { + if (override) { + LOG.warn( + String.format( + "Property %s is provided but will be overridden from %s to %s", + key, userValue, value)); + changedProps.setProperty(key, value); + } else { + LOG.warn( + String.format( + "Property %s is provided but won't be overridden from %s to %s", + key, userValue, value)); + } + } else { + changedProps.setProperty(key, value); + } + return changedProps; + } +} diff --git a/kaldb/src/main/java/com/slack/kaldb/writer/kafka/KaldbKafkaConsumer.java b/kaldb/src/main/java/com/slack/kaldb/writer/kafka/KaldbKafkaConsumer.java index 368cd158c7..7e151df9d5 100644 --- a/kaldb/src/main/java/com/slack/kaldb/writer/kafka/KaldbKafkaConsumer.java +++ b/kaldb/src/main/java/com/slack/kaldb/writer/kafka/KaldbKafkaConsumer.java @@ -9,6 +9,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.slack.kaldb.proto.config.KaldbConfigs; import com.slack.kaldb.server.KaldbConfig; +import com.slack.kaldb.writer.KafkaUtils; import com.slack.kaldb.writer.LogMessageWriterImpl; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.MeterRegistry; @@ -82,40 +83,16 @@ public static Properties makeKafkaConsumerProps(KaldbConfigs.KafkaConfig kafkaCo // don't override the properties that we have already set explicitly using named properties for (Map.Entry additionalProp : kafkaConfig.getAdditionalPropsMap().entrySet()) { - maybeOverride( - props, - additionalProp.getKey(), - additionalProp.getValue(), - OVERRIDABLE_CONFIGS.contains(additionalProp.getKey())); + props = + KafkaUtils.maybeOverrideProps( + props, + additionalProp.getKey(), + additionalProp.getValue(), + OVERRIDABLE_CONFIGS.contains(additionalProp.getKey())); } return props; } - @VisibleForTesting - public static boolean maybeOverride( - Properties props, String key, String value, boolean override) { - boolean overridden = false; - String userValue = props.getProperty(key); - if (userValue != null) { - if (override) { - LOG.warn( - String.format( - "Property %s is provided but will be overridden from %s to %s", - key, userValue, value)); - props.setProperty(key, value); - overridden = true; - } else { - LOG.warn( - String.format( - "Property %s is provided but won't be overridden from %s to %s", - key, userValue, value)); - } - } else { - props.setProperty(key, value); - } - return overridden; - } - private KafkaConsumer kafkaConsumer; private final TopicPartition topicPartition; diff --git a/kaldb/src/main/proto/kaldb_configs.proto b/kaldb/src/main/proto/kaldb_configs.proto index 17b1656939..21ac233489 100644 --- a/kaldb/src/main/proto/kaldb_configs.proto +++ b/kaldb/src/main/proto/kaldb_configs.proto @@ -266,8 +266,8 @@ message PreprocessorConfig { // more docs on PreprocessorPartitioner#getDatasetPartitionSuppliers int32 kafka_partition_sticky_timeout_ms = 8; - // this value needs to be set if the bulk API is used to bootstrap the producer kafka + // Kafka config needs to be set if the bulk API is used to bootstrap the producer kafka // we plan on moving everything to the bulk API and removing KafkaStreamConfig in the future - string bootstrap_servers = 9; + KafkaConfig kafka_config = 9; bool use_bulk_api = 10; } diff --git a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java index bf9238af76..a1ba185a47 100644 --- a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java @@ -73,14 +73,18 @@ public void bootstrapCluster() throws Exception { .setServerPort(8080) .setServerAddress("localhost") .build(); + KaldbConfigs.KafkaConfig kafkaConfig = + KaldbConfigs.KafkaConfig.newBuilder() + .setKafkaBootStrapServers(kafkaServer.getBroker().getBrokerList().get()) + .setKafkaTopic(DOWNSTREAM_TOPIC) + .build(); preprocessorConfig = KaldbConfigs.PreprocessorConfig.newBuilder() - .setBootstrapServers(kafkaServer.getBroker().getBrokerList().get()) + .setKafkaConfig(kafkaConfig) .setUseBulkApi(true) .setServerConfig(serverConfig) .setPreprocessorInstanceCount(1) .setRateLimiterMaxBurstSeconds(1) - .setDownstreamTopic(DOWNSTREAM_TOPIC) .build(); datasetMetadataStore = new DatasetMetadataStore(curatorFramework, true); diff --git a/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java b/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java index 2126569aa7..c112d35bc0 100644 --- a/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/server/BulkIngestApiTest.java @@ -86,14 +86,18 @@ public void bootstrapCluster() throws Exception { .setServerPort(8080) .setServerAddress("localhost") .build(); + KaldbConfigs.KafkaConfig kafkaConfig = + KaldbConfigs.KafkaConfig.newBuilder() + .setKafkaBootStrapServers(kafkaServer.getBroker().getBrokerList().get()) + .setKafkaTopic(DOWNSTREAM_TOPIC) + .build(); preprocessorConfig = KaldbConfigs.PreprocessorConfig.newBuilder() - .setBootstrapServers(kafkaServer.getBroker().getBrokerList().get()) + .setKafkaConfig(kafkaConfig) .setUseBulkApi(true) .setServerConfig(serverConfig) .setPreprocessorInstanceCount(1) .setRateLimiterMaxBurstSeconds(1) - .setDownstreamTopic(DOWNSTREAM_TOPIC) .build(); datasetMetadataStore = new DatasetMetadataStore(curatorFramework, true); diff --git a/kaldb/src/test/java/com/slack/kaldb/server/KaldbConfigTest.java b/kaldb/src/test/java/com/slack/kaldb/server/KaldbConfigTest.java index 9f72902287..adb80f0fea 100644 --- a/kaldb/src/test/java/com/slack/kaldb/server/KaldbConfigTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/server/KaldbConfigTest.java @@ -295,7 +295,11 @@ public void testParseKaldbJsonConfigFile() throws IOException { assertThat(preprocessorConfig.getDataTransformer()).isEqualTo("api_log"); assertThat(preprocessorConfig.getRateLimiterMaxBurstSeconds()).isEqualTo(2); assertThat(preprocessorConfig.getUseBulkApi()).isEqualTo(false); - assertThat(preprocessorConfig.getBootstrapServers()).isEqualTo("localhost:9092"); + + final KaldbConfigs.KafkaConfig preprocessorKafkaConfig = + config.getPreprocessorConfig().getKafkaConfig(); + assertThat(preprocessorKafkaConfig.getKafkaBootStrapServers()).isEqualTo("localhost:9092"); + assertThat(preprocessorKafkaConfig.getKafkaTopic()).isEqualTo("test-topic"); final KaldbConfigs.ServerConfig preprocessorServerConfig = preprocessorConfig.getServerConfig(); assertThat(preprocessorServerConfig.getServerPort()).isEqualTo(8085); @@ -466,8 +470,12 @@ public void testParseKaldbYamlConfigFile() throws IOException { assertThat(preprocessorConfig.getDataTransformer()).isEqualTo("api_log"); assertThat(preprocessorConfig.getRateLimiterMaxBurstSeconds()).isEqualTo(2); + final KaldbConfigs.KafkaConfig preprocessorKafkaConfig = + config.getPreprocessorConfig().getKafkaConfig(); + assertThat(preprocessorKafkaConfig.getKafkaBootStrapServers()).isEqualTo("localhost:9092"); + assertThat(preprocessorKafkaConfig.getKafkaTopic()).isEqualTo("test-topic"); + assertThat(preprocessorConfig.getUseBulkApi()).isEqualTo(true); - assertThat(preprocessorConfig.getBootstrapServers()).isEqualTo("localhost:9092"); final KaldbConfigs.ServerConfig preprocessorServerConfig = preprocessorConfig.getServerConfig(); assertThat(preprocessorServerConfig.getServerPort()).isEqualTo(8085); diff --git a/kaldb/src/test/java/com/slack/kaldb/writer/KafkaUtilsTest.java b/kaldb/src/test/java/com/slack/kaldb/writer/KafkaUtilsTest.java new file mode 100644 index 0000000000..e3e425044b --- /dev/null +++ b/kaldb/src/test/java/com/slack/kaldb/writer/KafkaUtilsTest.java @@ -0,0 +1,48 @@ +package com.slack.kaldb.writer; + +import static org.assertj.core.api.Assertions.assertThat; + +import com.slack.kaldb.proto.config.KaldbConfigs; +import com.slack.kaldb.testlib.TestKafkaServer; +import com.slack.kaldb.writer.kafka.KaldbKafkaConsumer; +import java.util.Properties; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.junit.jupiter.api.Test; + +class KafkaUtilsTest { + public static final String TEST_KAFKA_CLIENT_GROUP = "test_kaldb_consumer"; + + @Test + public void testOverridingProperties() { + KaldbConfigs.KafkaConfig kafkaConfig = + KaldbConfigs.KafkaConfig.newBuilder() + .setKafkaTopic(TestKafkaServer.TEST_KAFKA_TOPIC) + .setKafkaTopicPartition("0") + .setKafkaBootStrapServers("bootstrap_server") + .setKafkaClientGroup(TEST_KAFKA_CLIENT_GROUP) + .setEnableKafkaAutoCommit("true") + .setKafkaAutoCommitInterval("5000") + .setKafkaSessionTimeout("5000") + .build(); + + Properties properties = KaldbKafkaConsumer.makeKafkaConsumerProps(kafkaConfig); + assertThat(properties.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) + .isEqualTo("org.apache.kafka.common.serialization.StringDeserializer"); + + kafkaConfig = + KaldbConfigs.KafkaConfig.newBuilder() + .setKafkaTopic(TestKafkaServer.TEST_KAFKA_TOPIC) + .setKafkaTopicPartition("0") + .setKafkaBootStrapServers("bootstrap_server") + .setKafkaClientGroup(TEST_KAFKA_CLIENT_GROUP) + .setEnableKafkaAutoCommit("true") + .setKafkaAutoCommitInterval("5000") + .setKafkaSessionTimeout("5000") + .putAdditionalProps(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "test_serializer") + .build(); + + properties = KaldbKafkaConsumer.makeKafkaConsumerProps(kafkaConfig); + assertThat(properties.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) + .isEqualTo("test_serializer"); + } +} diff --git a/kaldb/src/test/java/com/slack/kaldb/writer/kafka/KaldbKafkaConsumerTest.java b/kaldb/src/test/java/com/slack/kaldb/writer/kafka/KaldbKafkaConsumerTest.java index fae04c0700..2185beb1bf 100644 --- a/kaldb/src/test/java/com/slack/kaldb/writer/kafka/KaldbKafkaConsumerTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/writer/kafka/KaldbKafkaConsumerTest.java @@ -117,40 +117,6 @@ public void tearDown() throws Exception { metricsRegistry.close(); } - @Test - public void testOverridingProperties() { - KaldbConfigs.KafkaConfig kafkaConfig = - KaldbConfigs.KafkaConfig.newBuilder() - .setKafkaTopic(TestKafkaServer.TEST_KAFKA_TOPIC) - .setKafkaTopicPartition("0") - .setKafkaBootStrapServers("bootstrap_server") - .setKafkaClientGroup(TEST_KAFKA_CLIENT_GROUP) - .setEnableKafkaAutoCommit("true") - .setKafkaAutoCommitInterval("5000") - .setKafkaSessionTimeout("5000") - .build(); - - Properties properties = KaldbKafkaConsumer.makeKafkaConsumerProps(kafkaConfig); - assertThat(properties.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) - .isEqualTo("org.apache.kafka.common.serialization.StringDeserializer"); - - kafkaConfig = - KaldbConfigs.KafkaConfig.newBuilder() - .setKafkaTopic(TestKafkaServer.TEST_KAFKA_TOPIC) - .setKafkaTopicPartition("0") - .setKafkaBootStrapServers("bootstrap_server") - .setKafkaClientGroup(TEST_KAFKA_CLIENT_GROUP) - .setEnableKafkaAutoCommit("true") - .setKafkaAutoCommitInterval("5000") - .setKafkaSessionTimeout("5000") - .putAdditionalProps(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "test_serializer") - .build(); - - properties = KaldbKafkaConsumer.makeKafkaConsumerProps(kafkaConfig); - assertThat(properties.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) - .isEqualTo("test_serializer"); - } - @Test public void testGetEndOffsetForPartition() throws Exception { EphemeralKafkaBroker broker = kafkaServer.getBroker(); diff --git a/kaldb/src/test/resources/test_config.json b/kaldb/src/test/resources/test_config.json index fa675fa2a8..1d1f4e6833 100644 --- a/kaldb/src/test/resources/test_config.json +++ b/kaldb/src/test/resources/test_config.json @@ -142,6 +142,10 @@ "numStreamThreads": 2, "processingGuarantee": "at_least_once" }, + "kafkaConfig": { + "kafkaTopic": "test-topic", + "kafkaBootStrapServers": "localhost:9092" + }, "serverConfig": { "serverPort": 8085, "serverAddress": "localhost", diff --git a/kaldb/src/test/resources/test_config.yaml b/kaldb/src/test/resources/test_config.yaml index 00c96a30da..e5c62cd740 100644 --- a/kaldb/src/test/resources/test_config.yaml +++ b/kaldb/src/test/resources/test_config.yaml @@ -114,6 +114,10 @@ preprocessorConfig: applicationId: kaldb_preprocessor numStreamThreads: 2 processingGuarantee: at_least_once + kafkaConfig: + kafkaTopic: test-topic + kafkaBootStrapServers: "localhost:9092" + serverConfig: serverPort: 8085 serverAddress: localhost From 12db93c7080acdcadfab9d1424be4393d9bec7c0 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Tue, 2 Jan 2024 12:13:31 -0700 Subject: [PATCH 09/18] Remove default configs in favor of env settings --- .../kaldb/bulkIngestApi/BulkIngestKafkaProducer.java | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index ed74e474fc..2780d58e3b 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -69,12 +69,7 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { private static final Set OVERRIDABLE_CONFIGS = Set.of( - ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - ProducerConfig.LINGER_MS_CONFIG, - ProducerConfig.BATCH_SIZE_CONFIG, - ProducerConfig.MAX_BLOCK_MS_CONFIG, - ProducerConfig.COMPRESSION_TYPE_CONFIG); + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); public BulkIngestKafkaProducer( final DatasetMetadataStore datasetMetadataStore, @@ -273,10 +268,6 @@ private KafkaProducer createKafkaTransactionProducer(String tran ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); props.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionId); - props.put(ProducerConfig.LINGER_MS_CONFIG, 500); - props.put(ProducerConfig.BATCH_SIZE_CONFIG, 128000); - props.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000); - props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, "snappy"); // don't override the properties that we have already set explicitly using named properties for (Map.Entry additionalProp : From faf5c7f03ce6758a4d4c2633c1f5fa0fe9f6a33f Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Tue, 2 Jan 2024 13:27:39 -0700 Subject: [PATCH 10/18] Add ability to override blocking task executor limits for bulk ingest --- .../main/java/com/slack/kaldb/server/ArmeriaService.java | 5 +++++ kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java | 9 +++++++++ 2 files changed, 14 insertions(+) diff --git a/kaldb/src/main/java/com/slack/kaldb/server/ArmeriaService.java b/kaldb/src/main/java/com/slack/kaldb/server/ArmeriaService.java index 331c5f1ed0..697969553b 100644 --- a/kaldb/src/main/java/com/slack/kaldb/server/ArmeriaService.java +++ b/kaldb/src/main/java/com/slack/kaldb/server/ArmeriaService.java @@ -96,6 +96,11 @@ public Builder withRequestTimeout(Duration requestTimeout) { return this; } + public Builder withBlockingTaskExecutorThreads(int numThreads) { + serverBuilder.blockingTaskExecutor(numThreads); + return this; + } + public Builder withTracing(KaldbConfigs.TracingConfig tracingConfig) { // span handlers is an ordered list, so we need to be careful with ordering if (tracingConfig.getCommonTagsCount() > 0) { diff --git a/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java b/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java index 8c93374235..df8f699f5a 100644 --- a/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java +++ b/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java @@ -401,6 +401,15 @@ private static Set getServices( BulkIngestApi openSearchBulkApiService = new BulkIngestApi(bulkIngestKafkaProducer, datasetRateLimitingService, meterRegistry); armeriaServiceBuilder.withAnnotatedService(openSearchBulkApiService); + + // todo - consider making this a configurable value or determine an automatic value + // reasonably + int blockingTaskExecutorThreads = + Integer.parseInt(System.getProperty("kalDb.bulkIngest.blockingExecutorThreads", "0")); + if (blockingTaskExecutorThreads > 0) { + LOG.info("kalDb.bulkIngest.blockingExecutorThreads={}", blockingTaskExecutorThreads); + armeriaServiceBuilder.withBlockingTaskExecutorThreads(blockingTaskExecutorThreads); + } } else { PreprocessorService preprocessorService = new PreprocessorService(datasetMetadataStore, preprocessorConfig, meterRegistry); From 8b2f6d3e7ef4ee0533541f283a1901db6fbd470c Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Wed, 3 Jan 2024 09:39:46 -0700 Subject: [PATCH 11/18] Add timer for bulk ingest --- .../com/slack/kaldb/bulkIngestApi/BulkIngestApi.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java index 943e2ab4d8..1e27f0d636 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java @@ -10,6 +10,7 @@ import com.slack.service.murron.trace.Trace; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.MeterRegistry; +import io.micrometer.core.instrument.Timer; import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -25,8 +26,11 @@ public class BulkIngestApi { private static final Logger LOG = LoggerFactory.getLogger(BulkIngestApi.class); private final BulkIngestKafkaProducer bulkIngestKafkaProducer; private final DatasetRateLimitingService datasetRateLimitingService; + private final MeterRegistry meterRegistry; private final Counter incomingByteTotal; + private final Timer bulkIngestTimer; private final String BULK_INGEST_INCOMING_BYTE_TOTAL = "kaldb_preprocessor_incoming_byte"; + private final String BULK_INGEST_TIMER = "kaldb_preprocessor_bulk_ingest"; public BulkIngestApi( BulkIngestKafkaProducer bulkIngestKafkaProducer, @@ -35,7 +39,9 @@ public BulkIngestApi( this.bulkIngestKafkaProducer = bulkIngestKafkaProducer; this.datasetRateLimitingService = datasetRateLimitingService; + this.meterRegistry = meterRegistry; this.incomingByteTotal = meterRegistry.counter(BULK_INGEST_INCOMING_BYTE_TOTAL); + this.bulkIngestTimer = meterRegistry.timer(BULK_INGEST_TIMER); } @Blocking @@ -43,6 +49,7 @@ public BulkIngestApi( public HttpResponse addDocument(String bulkRequest) { // 1. Kaldb does not support the concept of "updates". It's always an add. // 2. The "index" is used as the span name + Timer.Sample sample = Timer.start(meterRegistry); try { byte[] bulkRequestBytes = bulkRequest.getBytes(StandardCharsets.UTF_8); incomingByteTotal.increment(bulkRequestBytes.length); @@ -73,6 +80,8 @@ public HttpResponse addDocument(String bulkRequest) { LOG.error("Request failed ", e); BulkIngestResponse response = new BulkIngestResponse(0, 0, e.getMessage()); return HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response); + } finally { + sample.stop(bulkIngestTimer); } } } From 5129539a1d27525f48ab0e5c5fdeac6de7576a20 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Wed, 3 Jan 2024 11:25:51 -0700 Subject: [PATCH 12/18] Add a stall counter for bulk ingest --- .../slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index 2780d58e3b..51a7771b87 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -64,6 +64,9 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { "bulk_ingest_producer_over_limit_pending"; private final Counter overLimitPendingRequests; + public static final String STALL_COUNTER = "bulk_ingest_producer_stall_counter"; + private final Counter stallCounter; + public static final String BATCH_SIZE_GAUGE = "bulk_ingest_producer_batch_size"; private final AtomicInteger batchSizeGauge; @@ -106,6 +109,7 @@ public BulkIngestKafkaProducer( this.failedSetResponseCounter = meterRegistry.counter(FAILED_SET_RESPONSE_COUNTER); this.overLimitPendingRequests = meterRegistry.counter(OVER_LIMIT_PENDING_REQUESTS); + this.stallCounter = meterRegistry.counter(STALL_COUNTER); this.batchSizeGauge = meterRegistry.gauge(BATCH_SIZE_GAUGE, new AtomicInteger(0)); this.kafkaProducer.initTransactions(); @@ -134,6 +138,7 @@ protected void run() throws Exception { batchSizeGauge.set(requests.size()); if (requests.isEmpty()) { try { + stallCounter.increment(); Thread.sleep(producerSleep); } catch (InterruptedException e) { return; From 1d4aeefbe6ffae271459e9825283d2999a44a740 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Wed, 3 Jan 2024 13:37:11 -0700 Subject: [PATCH 13/18] Test pooling kafka producers --- kaldb/pom.xml | 5 + .../BulkIngestKafkaProducer.java | 104 ++++++++++++++---- 2 files changed, 88 insertions(+), 21 deletions(-) diff --git a/kaldb/pom.xml b/kaldb/pom.xml index 2a666aff8d..d383082fae 100644 --- a/kaldb/pom.xml +++ b/kaldb/pom.xml @@ -209,6 +209,11 @@ kafka-streams ${kafka.version} + + org.apache.commons + commons-pool2 + 2.12.0 + diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index 51a7771b87..df6d666da1 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -4,6 +4,7 @@ import static com.slack.kaldb.metadata.dataset.DatasetMetadata.MATCH_ALL_SERVICE; import static com.slack.kaldb.metadata.dataset.DatasetMetadata.MATCH_STAR_SERVICE; +import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.AbstractExecutionThreadService; import com.slack.kaldb.metadata.core.KaldbMetadataStoreChangeListener; import com.slack.kaldb.metadata.dataset.DatasetMetadata; @@ -14,6 +15,7 @@ import com.slack.kaldb.writer.KafkaUtils; import com.slack.service.murron.trace.Trace; import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Tag; import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics; import io.micrometer.prometheus.PrometheusMeterRegistry; import java.util.ArrayList; @@ -26,8 +28,15 @@ import java.util.UUID; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.pool2.BasePooledObjectFactory; +import org.apache.commons.pool2.ObjectPool; +import org.apache.commons.pool2.PooledObject; +import org.apache.commons.pool2.impl.DefaultPooledObject; +import org.apache.commons.pool2.impl.GenericObjectPool; +import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; @@ -41,8 +50,11 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { private static final Logger LOG = LoggerFactory.getLogger(BulkIngestKafkaProducer.class); - private final KafkaProducer kafkaProducer; - private final KafkaClientMetrics kafkaMetrics; + private final ObjectPool> kafkaProducerPool; + + private final AtomicInteger poolCounter = new AtomicInteger(0); + + private final Map metricsMap = new ConcurrentHashMap<>(); private final KaldbConfigs.KafkaConfig kafkaConfig; @@ -102,17 +114,37 @@ public BulkIngestKafkaProducer( // I think they will remain in kafka till they expire. They should never be readable if the // consumer sets isolation.level as "read_committed" // see "zombie fencing" https://www.confluent.io/blog/transactions-apache-kafka/ - this.kafkaProducer = createKafkaTransactionProducer(UUID.randomUUID().toString()); - - this.kafkaMetrics = new KafkaClientMetrics(kafkaProducer); - this.kafkaMetrics.bindTo(meterRegistry); + GenericObjectPoolConfig> config = new GenericObjectPoolConfig<>(); + this.kafkaProducerPool = + new GenericObjectPool<>( + new BasePooledObjectFactory<>() { + @Override + public KafkaProducer create() { + KafkaProducer producer = + createKafkaTransactionProducer(UUID.randomUUID().toString()); + int metricsCounter = poolCounter.getAndIncrement(); + KafkaClientMetrics clientMetrics = + new KafkaClientMetrics( + producer, + List.of(Tag.of("bulk_instance_id", String.valueOf(metricsCounter)))); + clientMetrics.bindTo(meterRegistry); + metricsMap.put(metricsCounter, clientMetrics); + producer.initTransactions(); + return producer; + } + + @Override + public PooledObject> wrap( + KafkaProducer stringKafkaProducer) { + return new DefaultPooledObject<>(stringKafkaProducer); + } + }, + config); this.failedSetResponseCounter = meterRegistry.counter(FAILED_SET_RESPONSE_COUNTER); this.overLimitPendingRequests = meterRegistry.counter(OVER_LIMIT_PENDING_REQUESTS); this.stallCounter = meterRegistry.counter(STALL_COUNTER); this.batchSizeGauge = meterRegistry.gauge(BATCH_SIZE_GAUGE, new AtomicInteger(0)); - - this.kafkaProducer.initTransactions(); } private void cacheSortedDataset() { @@ -144,7 +176,21 @@ protected void run() throws Exception { return; } } else { - transactionCommit(requests); + try { + KafkaProducer kafkaProducer = kafkaProducerPool.borrowObject(); + Thread.ofVirtual() + .start( + () -> { + transactionCommit(requests, kafkaProducer); + try { + kafkaProducerPool.returnObject(kafkaProducer); + } catch (Exception e) { + LOG.error("Error attempting to return kafka producer to pool", e); + } + }); + } catch (Exception e) { + LOG.error("Error attempting to borrow kafka producer from pool", e); + } } } } @@ -152,11 +198,8 @@ protected void run() throws Exception { @Override protected void shutDown() throws Exception { datasetMetadataStore.removeListener(datasetListener); - - kafkaProducer.close(); - if (kafkaMetrics != null) { - kafkaMetrics.close(); - } + kafkaProducerPool.close(); + metricsMap.forEach((_, kafkaMetrics) -> kafkaMetrics.close()); } public BulkIngestRequest submitRequest(Map> inputDocs) { @@ -170,15 +213,30 @@ public BulkIngestRequest submitRequest(Map> inputDocs) return request; } + @Deprecated + @VisibleForTesting + protected Map transactionCommit( + List requests) throws Exception { + KafkaProducer kafkaProducer = kafkaProducerPool.borrowObject(); + Map response = + transactionCommit(requests, kafkaProducer); + kafkaProducerPool.returnObject(kafkaProducer); + return response; + } + protected Map transactionCommit( - List requests) { + List requests, KafkaProducer kafkaProducer) { Map responseMap = new HashMap<>(); + + // KafkaProducer kafkaProducer = null; try { + // kafkaProducer = kafkaProducerPool.borrowObject(); kafkaProducer.beginTransaction(); for (BulkIngestRequest request : requests) { - responseMap.put(request, produceDocuments(request.getInputDocs())); + responseMap.put(request, produceDocuments(request.getInputDocs(), kafkaProducer)); } kafkaProducer.commitTransaction(); + // kafkaProducerPool.returnObject(kafkaProducer); } catch (TimeoutException te) { LOG.error("Commit transaction timeout", te); // the commitTransaction waits till "max.block.ms" after which it will time out @@ -197,10 +255,13 @@ protected Map transactionCommit( new RuntimeHalterImpl().handleFatal(new Throwable("KafkaProducer needs to shutdown ", e)); } catch (Exception e) { LOG.warn("failed transaction with error", e); - try { - kafkaProducer.abortTransaction(); - } catch (ProducerFencedException err) { - LOG.error("Could not abort transaction", err); + if (kafkaProducer != null) { + try { + kafkaProducer.abortTransaction(); + // kafkaProducerPool.returnObject(kafkaProducer); + } catch (ProducerFencedException err) { + LOG.error("Could not abort transaction", err); + } } for (BulkIngestRequest request : requests) { @@ -225,7 +286,8 @@ protected Map transactionCommit( } @SuppressWarnings("FutureReturnValueIgnored") - private BulkIngestResponse produceDocuments(Map> indexDocs) { + private BulkIngestResponse produceDocuments( + Map> indexDocs, KafkaProducer kafkaProducer) { int totalDocs = indexDocs.values().stream().mapToInt(List::size).sum(); // we cannot create a generic pool of producers because the kafka API expects the transaction ID From 412a86f00c2e567d65119776bf0655055e12857f Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Wed, 3 Jan 2024 17:43:21 -0700 Subject: [PATCH 14/18] Increase default kafka producer pool, make configurable --- .../com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index df6d666da1..cfd5a91211 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -115,6 +115,8 @@ public BulkIngestKafkaProducer( // consumer sets isolation.level as "read_committed" // see "zombie fencing" https://www.confluent.io/blog/transactions-apache-kafka/ GenericObjectPoolConfig> config = new GenericObjectPoolConfig<>(); + config.setMaxTotal( + Integer.parseInt(System.getProperty("kalDb.bulkIngest.pooledKafkaProducers", "16"))); this.kafkaProducerPool = new GenericObjectPool<>( new BasePooledObjectFactory<>() { From e2c2727795fdc62b64db62302fadd368423d56ee Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Thu, 4 Jan 2024 10:32:07 -0700 Subject: [PATCH 15/18] Move bulk ingest to non-blocking virtual threads --- .../kaldb/bulkIngestApi/BulkIngestApi.java | 41 +++++++++++++------ .../BulkIngestKafkaProducer.java | 28 +++++++++---- 2 files changed, 49 insertions(+), 20 deletions(-) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java index 1e27f0d636..e9fbeda8f1 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java @@ -4,7 +4,6 @@ import static com.linecorp.armeria.common.HttpStatus.TOO_MANY_REQUESTS; import com.linecorp.armeria.common.HttpResponse; -import com.linecorp.armeria.server.annotation.Blocking; import com.linecorp.armeria.server.annotation.Post; import com.slack.kaldb.bulkIngestApi.opensearch.BulkApiRequestParser; import com.slack.service.murron.trace.Trace; @@ -14,6 +13,7 @@ import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,44 +44,59 @@ public BulkIngestApi( this.bulkIngestTimer = meterRegistry.timer(BULK_INGEST_TIMER); } - @Blocking @Post("/_bulk") public HttpResponse addDocument(String bulkRequest) { // 1. Kaldb does not support the concept of "updates". It's always an add. // 2. The "index" is used as the span name + CompletableFuture future = new CompletableFuture<>(); Timer.Sample sample = Timer.start(meterRegistry); try { byte[] bulkRequestBytes = bulkRequest.getBytes(StandardCharsets.UTF_8); incomingByteTotal.increment(bulkRequestBytes.length); Map> docs = BulkApiRequestParser.parseRequest(bulkRequestBytes); - // todo - our rate limiter doesn't have a way to acquire permits across multiple datasets - // so today as a limitation we reject any request that has documents against multiple indexes + // todo - our rate limiter doesn't have a way to acquire permits across multiple + // datasets + // so today as a limitation we reject any request that has documents against + // multiple indexes // We think most indexing requests will be against 1 index if (docs.keySet().size() > 1) { BulkIngestResponse response = new BulkIngestResponse(0, 0, "request must contain only 1 unique index"); - return HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response); + future.complete(HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response)); } for (Map.Entry> indexDocs : docs.entrySet()) { final String index = indexDocs.getKey(); if (!datasetRateLimitingService.tryAcquire(index, indexDocs.getValue())) { BulkIngestResponse response = new BulkIngestResponse(0, 0, "rate limit exceeded"); - return HttpResponse.ofJson(TOO_MANY_REQUESTS, response); + future.complete(HttpResponse.ofJson(TOO_MANY_REQUESTS, response)); } } - // getResponse will cause this thread to wait until the batch producer submits or it hits - // Armeria timeout - BulkIngestResponse response = bulkIngestKafkaProducer.submitRequest(docs).getResponse(); - return HttpResponse.ofJson(response); + // getResponse will cause this thread to wait until the batch producer submits or it + // hits Armeria timeout + Thread.ofVirtual() + .start( + () -> { + try { + BulkIngestResponse response = + bulkIngestKafkaProducer.submitRequest(docs).getResponse(); + future.complete(HttpResponse.ofJson(response)); + } catch (InterruptedException e) { + LOG.error("Request failed ", e); + future.complete( + HttpResponse.ofJson( + INTERNAL_SERVER_ERROR, new BulkIngestResponse(0, 0, e.getMessage()))); + } + }); } catch (Exception e) { LOG.error("Request failed ", e); BulkIngestResponse response = new BulkIngestResponse(0, 0, e.getMessage()); - return HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response); - } finally { - sample.stop(bulkIngestTimer); + future.complete(HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response)); } + future.thenRun(() -> sample.stop(bulkIngestTimer)); + + return HttpResponse.of(future); } } diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index cfd5a91211..06248ef6c6 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -26,9 +26,9 @@ import java.util.Properties; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.pool2.BasePooledObjectFactory; @@ -76,6 +76,9 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { "bulk_ingest_producer_over_limit_pending"; private final Counter overLimitPendingRequests; + public static final String PENDING_REQUEST_QUEUE_SIZE = "bulk_ingest_pending_request_queue_size"; + private AtomicInteger pendingRequestQueueSize; + public static final String STALL_COUNTER = "bulk_ingest_producer_stall_counter"; private final Counter stallCounter; @@ -86,6 +89,8 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { Set.of( ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); + public final int maxBatchSize; + public BulkIngestKafkaProducer( final DatasetMetadataStore datasetMetadataStore, final KaldbConfigs.PreprocessorConfig preprocessorConfig, @@ -103,11 +108,13 @@ public BulkIngestKafkaProducer( // todo - consider making these a configurable value, or determine a way to derive a reasonable // value automatically - this.pendingRequests = - new ArrayBlockingQueue<>( - Integer.parseInt(System.getProperty("kalDb.bulkIngest.pendingLimit", "500"))); + this.pendingRequests = new LinkedBlockingQueue<>(); + // new ArrayBlockingQueue<>( + // Integer.parseInt(System.getProperty("kalDb.bulkIngest.pendingLimit", "500"))); this.producerSleep = - Integer.parseInt(System.getProperty("kalDb.bulkIngest.producerSleep", "2000")); + Integer.parseInt(System.getProperty("kalDb.bulkIngest.producerSleep", "200")); + this.maxBatchSize = + Integer.parseInt(System.getProperty("kalDb.bulkIngest.maxBatchSize", "1000")); // since we use a new transaction ID every time we start a preprocessor there can be some zombie // transactions? @@ -116,7 +123,10 @@ public BulkIngestKafkaProducer( // see "zombie fencing" https://www.confluent.io/blog/transactions-apache-kafka/ GenericObjectPoolConfig> config = new GenericObjectPoolConfig<>(); config.setMaxTotal( - Integer.parseInt(System.getProperty("kalDb.bulkIngest.pooledKafkaProducers", "16"))); + Integer.parseInt( + System.getProperty( + "kalDb.bulkIngest.pooledKafkaProducers", + String.valueOf(Runtime.getRuntime().availableProcessors())))); this.kafkaProducerPool = new GenericObjectPool<>( new BasePooledObjectFactory<>() { @@ -147,6 +157,8 @@ public PooledObject> wrap( this.overLimitPendingRequests = meterRegistry.counter(OVER_LIMIT_PENDING_REQUESTS); this.stallCounter = meterRegistry.counter(STALL_COUNTER); this.batchSizeGauge = meterRegistry.gauge(BATCH_SIZE_GAUGE, new AtomicInteger(0)); + this.pendingRequestQueueSize = + meterRegistry.gauge(PENDING_REQUEST_QUEUE_SIZE, new AtomicInteger(0)); } private void cacheSortedDataset() { @@ -167,8 +179,10 @@ protected void startUp() throws Exception { @Override protected void run() throws Exception { while (isRunning()) { + pendingRequestQueueSize.set(pendingRequests.size()); + List requests = new ArrayList<>(); - pendingRequests.drainTo(requests); + pendingRequests.drainTo(requests, maxBatchSize); batchSizeGauge.set(requests.size()); if (requests.isEmpty()) { try { From b17b82091818aef6a16541eec314763f3f2fb842 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Fri, 5 Jan 2024 11:28:08 -0700 Subject: [PATCH 16/18] Remove pooled producer logic, unused metrics, early exit --- kaldb/pom.xml | 5 - .../kaldb/bulkIngestApi/BulkIngestApi.java | 7 +- .../BulkIngestKafkaProducer.java | 122 +++--------------- 3 files changed, 23 insertions(+), 111 deletions(-) diff --git a/kaldb/pom.xml b/kaldb/pom.xml index d383082fae..2a666aff8d 100644 --- a/kaldb/pom.xml +++ b/kaldb/pom.xml @@ -209,11 +209,6 @@ kafka-streams ${kafka.version} - - org.apache.commons - commons-pool2 - 2.12.0 - diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java index e9fbeda8f1..78f4db41a7 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java @@ -50,6 +50,8 @@ public HttpResponse addDocument(String bulkRequest) { // 2. The "index" is used as the span name CompletableFuture future = new CompletableFuture<>(); Timer.Sample sample = Timer.start(meterRegistry); + future.thenRun(() -> sample.stop(bulkIngestTimer)); + try { byte[] bulkRequestBytes = bulkRequest.getBytes(StandardCharsets.UTF_8); incomingByteTotal.increment(bulkRequestBytes.length); @@ -64,6 +66,7 @@ public HttpResponse addDocument(String bulkRequest) { BulkIngestResponse response = new BulkIngestResponse(0, 0, "request must contain only 1 unique index"); future.complete(HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response)); + return HttpResponse.of(future); } for (Map.Entry> indexDocs : docs.entrySet()) { @@ -71,11 +74,10 @@ public HttpResponse addDocument(String bulkRequest) { if (!datasetRateLimitingService.tryAcquire(index, indexDocs.getValue())) { BulkIngestResponse response = new BulkIngestResponse(0, 0, "rate limit exceeded"); future.complete(HttpResponse.ofJson(TOO_MANY_REQUESTS, response)); + return HttpResponse.of(future); } } - // getResponse will cause this thread to wait until the batch producer submits or it - // hits Armeria timeout Thread.ofVirtual() .start( () -> { @@ -95,7 +97,6 @@ public HttpResponse addDocument(String bulkRequest) { BulkIngestResponse response = new BulkIngestResponse(0, 0, e.getMessage()); future.complete(HttpResponse.ofJson(INTERNAL_SERVER_ERROR, response)); } - future.thenRun(() -> sample.stop(bulkIngestTimer)); return HttpResponse.of(future); } diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index 06248ef6c6..cac2f97901 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -4,7 +4,6 @@ import static com.slack.kaldb.metadata.dataset.DatasetMetadata.MATCH_ALL_SERVICE; import static com.slack.kaldb.metadata.dataset.DatasetMetadata.MATCH_STAR_SERVICE; -import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.AbstractExecutionThreadService; import com.slack.kaldb.metadata.core.KaldbMetadataStoreChangeListener; import com.slack.kaldb.metadata.dataset.DatasetMetadata; @@ -15,7 +14,6 @@ import com.slack.kaldb.writer.KafkaUtils; import com.slack.service.murron.trace.Trace; import io.micrometer.core.instrument.Counter; -import io.micrometer.core.instrument.Tag; import io.micrometer.core.instrument.binder.kafka.KafkaClientMetrics; import io.micrometer.prometheus.PrometheusMeterRegistry; import java.util.ArrayList; @@ -27,16 +25,9 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.pool2.BasePooledObjectFactory; -import org.apache.commons.pool2.ObjectPool; -import org.apache.commons.pool2.PooledObject; -import org.apache.commons.pool2.impl.DefaultPooledObject; -import org.apache.commons.pool2.impl.GenericObjectPool; -import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; @@ -50,11 +41,9 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { private static final Logger LOG = LoggerFactory.getLogger(BulkIngestKafkaProducer.class); - private final ObjectPool> kafkaProducerPool; + private final KafkaProducer kafkaProducer; - private final AtomicInteger poolCounter = new AtomicInteger(0); - - private final Map metricsMap = new ConcurrentHashMap<>(); + private final KafkaClientMetrics kafkaMetrics; private final KaldbConfigs.KafkaConfig kafkaConfig; @@ -71,14 +60,6 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { public static final String FAILED_SET_RESPONSE_COUNTER = "bulk_ingest_producer_failed_set_response"; private final Counter failedSetResponseCounter; - - public static final String OVER_LIMIT_PENDING_REQUESTS = - "bulk_ingest_producer_over_limit_pending"; - private final Counter overLimitPendingRequests; - - public static final String PENDING_REQUEST_QUEUE_SIZE = "bulk_ingest_pending_request_queue_size"; - private AtomicInteger pendingRequestQueueSize; - public static final String STALL_COUNTER = "bulk_ingest_producer_stall_counter"; private final Counter stallCounter; @@ -89,8 +70,6 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { Set.of( ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); - public final int maxBatchSize; - public BulkIngestKafkaProducer( final DatasetMetadataStore datasetMetadataStore, final KaldbConfigs.PreprocessorConfig preprocessorConfig, @@ -105,60 +84,26 @@ public BulkIngestKafkaProducer( checkArgument(!kafkaConfig.getKafkaTopic().isEmpty(), "Kafka topic must be provided"); this.datasetMetadataStore = datasetMetadataStore; - - // todo - consider making these a configurable value, or determine a way to derive a reasonable - // value automatically this.pendingRequests = new LinkedBlockingQueue<>(); - // new ArrayBlockingQueue<>( - // Integer.parseInt(System.getProperty("kalDb.bulkIngest.pendingLimit", "500"))); + + // todo - consider making this a configurable value or removing the config this.producerSleep = - Integer.parseInt(System.getProperty("kalDb.bulkIngest.producerSleep", "200")); - this.maxBatchSize = - Integer.parseInt(System.getProperty("kalDb.bulkIngest.maxBatchSize", "1000")); + Integer.parseInt(System.getProperty("kalDb.bulkIngest.producerSleep", "50")); // since we use a new transaction ID every time we start a preprocessor there can be some zombie // transactions? // I think they will remain in kafka till they expire. They should never be readable if the // consumer sets isolation.level as "read_committed" // see "zombie fencing" https://www.confluent.io/blog/transactions-apache-kafka/ - GenericObjectPoolConfig> config = new GenericObjectPoolConfig<>(); - config.setMaxTotal( - Integer.parseInt( - System.getProperty( - "kalDb.bulkIngest.pooledKafkaProducers", - String.valueOf(Runtime.getRuntime().availableProcessors())))); - this.kafkaProducerPool = - new GenericObjectPool<>( - new BasePooledObjectFactory<>() { - @Override - public KafkaProducer create() { - KafkaProducer producer = - createKafkaTransactionProducer(UUID.randomUUID().toString()); - int metricsCounter = poolCounter.getAndIncrement(); - KafkaClientMetrics clientMetrics = - new KafkaClientMetrics( - producer, - List.of(Tag.of("bulk_instance_id", String.valueOf(metricsCounter)))); - clientMetrics.bindTo(meterRegistry); - metricsMap.put(metricsCounter, clientMetrics); - producer.initTransactions(); - return producer; - } - - @Override - public PooledObject> wrap( - KafkaProducer stringKafkaProducer) { - return new DefaultPooledObject<>(stringKafkaProducer); - } - }, - config); + this.kafkaProducer = createKafkaTransactionProducer(UUID.randomUUID().toString()); + this.kafkaMetrics = new KafkaClientMetrics(kafkaProducer); + this.kafkaMetrics.bindTo(meterRegistry); this.failedSetResponseCounter = meterRegistry.counter(FAILED_SET_RESPONSE_COUNTER); - this.overLimitPendingRequests = meterRegistry.counter(OVER_LIMIT_PENDING_REQUESTS); this.stallCounter = meterRegistry.counter(STALL_COUNTER); this.batchSizeGauge = meterRegistry.gauge(BATCH_SIZE_GAUGE, new AtomicInteger(0)); - this.pendingRequestQueueSize = - meterRegistry.gauge(PENDING_REQUEST_QUEUE_SIZE, new AtomicInteger(0)); + + this.kafkaProducer.initTransactions(); } private void cacheSortedDataset() { @@ -179,10 +124,8 @@ protected void startUp() throws Exception { @Override protected void run() throws Exception { while (isRunning()) { - pendingRequestQueueSize.set(pendingRequests.size()); - List requests = new ArrayList<>(); - pendingRequests.drainTo(requests, maxBatchSize); + pendingRequests.drainTo(requests); batchSizeGauge.set(requests.size()); if (requests.isEmpty()) { try { @@ -192,21 +135,7 @@ protected void run() throws Exception { return; } } else { - try { - KafkaProducer kafkaProducer = kafkaProducerPool.borrowObject(); - Thread.ofVirtual() - .start( - () -> { - transactionCommit(requests, kafkaProducer); - try { - kafkaProducerPool.returnObject(kafkaProducer); - } catch (Exception e) { - LOG.error("Error attempting to return kafka producer to pool", e); - } - }); - } catch (Exception e) { - LOG.error("Error attempting to borrow kafka producer from pool", e); - } + transactionCommit(requests); } } } @@ -214,34 +143,21 @@ protected void run() throws Exception { @Override protected void shutDown() throws Exception { datasetMetadataStore.removeListener(datasetListener); - kafkaProducerPool.close(); - metricsMap.forEach((_, kafkaMetrics) -> kafkaMetrics.close()); + + kafkaProducer.close(); + if (kafkaMetrics != null) { + kafkaMetrics.close(); + } } public BulkIngestRequest submitRequest(Map> inputDocs) { BulkIngestRequest request = new BulkIngestRequest(inputDocs); - try { - pendingRequests.add(request); - } catch (IllegalStateException e) { - overLimitPendingRequests.increment(); - throw e; - } + pendingRequests.add(request); return request; } - @Deprecated - @VisibleForTesting - protected Map transactionCommit( - List requests) throws Exception { - KafkaProducer kafkaProducer = kafkaProducerPool.borrowObject(); - Map response = - transactionCommit(requests, kafkaProducer); - kafkaProducerPool.returnObject(kafkaProducer); - return response; - } - protected Map transactionCommit( - List requests, KafkaProducer kafkaProducer) { + List requests) { Map responseMap = new HashMap<>(); // KafkaProducer kafkaProducer = null; From e242efd53110ce548cfa151f7ea606019c3eb624 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Fri, 5 Jan 2024 11:52:19 -0700 Subject: [PATCH 17/18] Remove unused blocking config --- .../main/java/com/slack/kaldb/server/ArmeriaService.java | 5 ----- kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java | 9 --------- 2 files changed, 14 deletions(-) diff --git a/kaldb/src/main/java/com/slack/kaldb/server/ArmeriaService.java b/kaldb/src/main/java/com/slack/kaldb/server/ArmeriaService.java index 697969553b..331c5f1ed0 100644 --- a/kaldb/src/main/java/com/slack/kaldb/server/ArmeriaService.java +++ b/kaldb/src/main/java/com/slack/kaldb/server/ArmeriaService.java @@ -96,11 +96,6 @@ public Builder withRequestTimeout(Duration requestTimeout) { return this; } - public Builder withBlockingTaskExecutorThreads(int numThreads) { - serverBuilder.blockingTaskExecutor(numThreads); - return this; - } - public Builder withTracing(KaldbConfigs.TracingConfig tracingConfig) { // span handlers is an ordered list, so we need to be careful with ordering if (tracingConfig.getCommonTagsCount() > 0) { diff --git a/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java b/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java index df8f699f5a..8c93374235 100644 --- a/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java +++ b/kaldb/src/main/java/com/slack/kaldb/server/Kaldb.java @@ -401,15 +401,6 @@ private static Set getServices( BulkIngestApi openSearchBulkApiService = new BulkIngestApi(bulkIngestKafkaProducer, datasetRateLimitingService, meterRegistry); armeriaServiceBuilder.withAnnotatedService(openSearchBulkApiService); - - // todo - consider making this a configurable value or determine an automatic value - // reasonably - int blockingTaskExecutorThreads = - Integer.parseInt(System.getProperty("kalDb.bulkIngest.blockingExecutorThreads", "0")); - if (blockingTaskExecutorThreads > 0) { - LOG.info("kalDb.bulkIngest.blockingExecutorThreads={}", blockingTaskExecutorThreads); - armeriaServiceBuilder.withBlockingTaskExecutorThreads(blockingTaskExecutorThreads); - } } else { PreprocessorService preprocessorService = new PreprocessorService(datasetMetadataStore, preprocessorConfig, meterRegistry); From 16683f9a2520e12dd6366e728cfcc0fe34360e8b Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Mon, 8 Jan 2024 12:41:48 -0700 Subject: [PATCH 18/18] PR feedback --- .../slack/kaldb/bulkIngestApi/BulkIngestApi.java | 2 ++ .../bulkIngestApi/BulkIngestKafkaProducer.java | 14 +++++++------- .../bulkIngestApi/BulkIngestKafkaProducerTest.java | 10 ++++++++-- 3 files changed, 17 insertions(+), 9 deletions(-) diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java index 78f4db41a7..baf3686fbd 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestApi.java @@ -78,6 +78,8 @@ public HttpResponse addDocument(String bulkRequest) { } } + // todo - explore the possibility of using the blocking task executor backed by virtual + // threads to fulfill this Thread.ofVirtual() .start( () -> { diff --git a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java index cac2f97901..651c49b6c4 100644 --- a/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java +++ b/kaldb/src/main/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducer.java @@ -55,7 +55,7 @@ public class BulkIngestKafkaProducer extends AbstractExecutionThreadService { private final BlockingQueue pendingRequests; - private final Integer producerSleep; + private final Integer producerSleepMs; public static final String FAILED_SET_RESPONSE_COUNTER = "bulk_ingest_producer_failed_set_response"; @@ -87,8 +87,8 @@ public BulkIngestKafkaProducer( this.pendingRequests = new LinkedBlockingQueue<>(); // todo - consider making this a configurable value or removing the config - this.producerSleep = - Integer.parseInt(System.getProperty("kalDb.bulkIngest.producerSleep", "50")); + this.producerSleepMs = + Integer.parseInt(System.getProperty("kalDb.bulkIngest.producerSleepMs", "50")); // since we use a new transaction ID every time we start a preprocessor there can be some zombie // transactions? @@ -130,12 +130,12 @@ protected void run() throws Exception { if (requests.isEmpty()) { try { stallCounter.increment(); - Thread.sleep(producerSleep); + Thread.sleep(producerSleepMs); } catch (InterruptedException e) { return; } } else { - transactionCommit(requests); + produceDocumentsAndCommit(requests); } } } @@ -156,7 +156,7 @@ public BulkIngestRequest submitRequest(Map> inputDocs) return request; } - protected Map transactionCommit( + protected Map produceDocumentsAndCommit( List requests) { Map responseMap = new HashMap<>(); @@ -247,7 +247,7 @@ private BulkIngestResponse produceDocuments( ProducerRecord producerRecord = new ProducerRecord<>(kafkaConfig.getKafkaTopic(), partition, index, doc.toByteArray()); - // we intentionally supress FutureReturnValueIgnored here in errorprone - this is because + // we intentionally suppress FutureReturnValueIgnored here in errorprone - this is because // we wrap this in a transaction, which is responsible for flushing all of the pending // messages kafkaProducer.send(producerRecord); diff --git a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java index a1ba185a47..3035a09180 100644 --- a/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java +++ b/kaldb/src/test/java/com/slack/kaldb/bulkIngestApi/BulkIngestKafkaProducerTest.java @@ -133,7 +133,10 @@ public void testDocumentInKafkaTransactionError() throws Exception { }); BulkIngestResponse responseObj = (BulkIngestResponse) - bulkIngestKafkaProducer.transactionCommit(List.of(request1)).values().toArray()[0]; + bulkIngestKafkaProducer + .produceDocumentsAndCommit(List.of(request1)) + .values() + .toArray()[0]; assertThat(responseObj.totalDocs()).isEqualTo(0); assertThat(responseObj.failedDocs()).isEqualTo(5); assertThat(responseObj.errorMsg()).isNotNull(); @@ -192,7 +195,10 @@ public void testDocumentInKafkaTransactionError() throws Exception { }); responseObj = (BulkIngestResponse) - bulkIngestKafkaProducer.transactionCommit(List.of(request2)).values().toArray()[0]; + bulkIngestKafkaProducer + .produceDocumentsAndCommit(List.of(request2)) + .values() + .toArray()[0]; assertThat(responseObj.totalDocs()).isEqualTo(5); assertThat(responseObj.failedDocs()).isEqualTo(0); assertThat(responseObj.errorMsg()).isNotNull();