From d53fe23fdcc02a0be5be8fef4576a1c42db17574 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Fri, 18 Oct 2024 15:21:54 -0700 Subject: [PATCH 01/32] =?UTF-8?q?Refactored=20`produceToStoreBufferService?= =?UTF-8?q?OrKafka()`=20from=20`StoreIngestionTask`=20into=20`StorePartiti?= =?UTF-8?q?onDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 4 +- .../consumer/StorageUtilizationManager.java | 4 +- .../kafka/consumer/StoreIngestionTask.java | 194 +++++++++--------- .../consumer/StorePartitionDataReceiver.java | 90 +++++++- 4 files changed, 195 insertions(+), 97 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 97423460f92..85471674f93 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -2281,7 +2281,7 @@ protected void recordHeartbeatReceived( } @Override - protected Iterable> validateAndFilterOutDuplicateMessagesFromLeaderTopic( + public Iterable> validateAndFilterOutDuplicateMessagesFromLeaderTopic( Iterable> records, String kafkaUrl, PubSubTopicPartition topicPartition) { @@ -2365,7 +2365,7 @@ protected Iterable> validate * {@link DelegateConsumerRecordResult#QUEUED_TO_DRAINER}. * * This function assumes {@link #shouldProcessRecord(PubSubMessage)} has been called which happens in - * {@link StoreIngestionTask#produceToStoreBufferServiceOrKafka(Iterable, PubSubTopicPartition, String, int)} + * {@link StorePartitionDataReceiver#produceToStoreBufferServiceOrKafka(Iterable, PubSubTopicPartition, String, int)} * before calling this and the it was decided that this record needs to be processed. It does not perform any * validation check on the PartitionConsumptionState object to keep the goal of the function simple and not overload. * diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorageUtilizationManager.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorageUtilizationManager.java index 87150f5e98c..cc83fd1128a 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorageUtilizationManager.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorageUtilizationManager.java @@ -216,8 +216,8 @@ public void removePartition(int partition) { /** * Enforce partition level quota for the map. * This function could be invoked by multiple threads when shared consumer is being used. - * Check {@link StoreIngestionTask#produceToStoreBufferServiceOrKafka} and {@link StoreIngestionTask#checkIngestionProgress} - * to find more details. + * Check {@link StorePartitionDataReceiver#produceToStoreBufferServiceOrKafka} and + * {@link StoreIngestionTask#checkIngestionProgress} to find more details. */ public void checkAllPartitionsQuota() { try (AutoCloseableLock ignored = AutoCloseableLock.of(hybridStoreDiskQuotaLock)) { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index e1620eb5ccf..68617d9fd25 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -1134,12 +1134,12 @@ protected void produceToStoreBufferService( } } - protected abstract Iterable> validateAndFilterOutDuplicateMessagesFromLeaderTopic( + public abstract Iterable> validateAndFilterOutDuplicateMessagesFromLeaderTopic( Iterable> records, String kafkaUrl, PubSubTopicPartition topicPartition); - private int handleSingleMessage( + public int handleSingleMessage( PubSubMessageProcessedResultWrapper consumerRecordWrapper, PubSubTopicPartition topicPartition, PartitionConsumptionState partitionConsumptionState, @@ -1211,94 +1211,100 @@ private int handleSingleMessage( return record.getPayloadSize(); } - /** - * This function is in charge of producing the consumer records to the writer buffers maintained by {@link StoreBufferService}. - * - * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after this call. - * - * @param records : received consumer records - * @param topicPartition - * @throws InterruptedException - */ - protected void produceToStoreBufferServiceOrKafka( - Iterable> records, - PubSubTopicPartition topicPartition, - String kafkaUrl, - int kafkaClusterId) throws InterruptedException { - PartitionConsumptionState partitionConsumptionState = - partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); - if (partitionConsumptionState == null) { - throw new VeniceException( - "PartitionConsumptionState should present for store version: " + kafkaVersionTopic + ", partition: " - + topicPartition.getPartitionNumber()); - } - /** - * Validate and filter out duplicate messages from the real-time topic as early as possible, so that - * the following batch processing logic won't spend useless efforts on duplicate messages. - */ - records = validateAndFilterOutDuplicateMessagesFromLeaderTopic(records, kafkaUrl, topicPartition); - - if ((isActiveActiveReplicationEnabled || isWriteComputationEnabled) + // /** + // * This function is in charge of producing the consumer records to the writer buffers maintained by {@link + // StoreBufferService}. + // * + // * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after + // this call. + // * + // * @param records : received consumer records + // * @param topicPartition + // * @throws InterruptedException + // */ + // protected void produceToStoreBufferServiceOrKafka( + // Iterable> records, + // PubSubTopicPartition topicPartition, + // String kafkaUrl, + // int kafkaClusterId) throws InterruptedException { + // PartitionConsumptionState partitionConsumptionState = + // partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); + // if (partitionConsumptionState == null) { + // throw new VeniceException( + // "PartitionConsumptionState should present for store version: " + kafkaVersionTopic + ", partition: " + // + topicPartition.getPartitionNumber()); + // } + // /** + // * Validate and filter out duplicate messages from the real-time topic as early as possible, so that + // * the following batch processing logic won't spend useless efforts on duplicate messages. + // */ + // records = validateAndFilterOutDuplicateMessagesFromLeaderTopic(records, kafkaUrl, topicPartition); + // + // if (shouldProduceInBatch(records)) { + // produceToStoreBufferServiceOrKafkaInBatch( + // records, + // topicPartition, + // partitionConsumptionState, + // kafkaUrl, + // kafkaClusterId); + // return; + // } + // + // long totalBytesRead = 0; + // ValueHolder elapsedTimeForPuttingIntoQueue = new ValueHolder<>(0d); + // boolean metricsEnabled = emitMetrics.get(); + // long beforeProcessingBatchRecordsTimestampMs = System.currentTimeMillis(); + // + // partitionConsumptionState = partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); + // for (PubSubMessage record: records) { + // long beforeProcessingPerRecordTimestampNs = System.nanoTime(); + // partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); + // if (!shouldProcessRecord(record)) { + // partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); + // continue; + // } + // + // // Check schema id availability before putting consumer record to drainer queue + // waitReadyToProcessRecord(record); + // + // totalBytesRead += handleSingleMessage( + // new PubSubMessageProcessedResultWrapper<>(record), + // topicPartition, + // partitionConsumptionState, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs, + // beforeProcessingBatchRecordsTimestampMs, + // metricsEnabled, + // elapsedTimeForPuttingIntoQueue); + // } + // + // /** + // * Even if the records list is empty, we still need to check quota to potentially resume partition + // */ + // storageUtilizationManager.enforcePartitionQuota(topicPartition.getPartitionNumber(), totalBytesRead); + // + // if (metricsEnabled) { + // if (totalBytesRead > 0) { + // hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); + // } + // if (elapsedTimeForPuttingIntoQueue.getValue() > 0) { + // hostLevelIngestionStats.recordConsumerRecordsQueuePutLatency( + // elapsedTimeForPuttingIntoQueue.getValue(), + // beforeProcessingBatchRecordsTimestampMs); + // } + // + // hostLevelIngestionStats.recordStorageQuotaUsed(storageUtilizationManager.getDiskQuotaUsage()); + // } + // } + + public boolean shouldProduceInBatch(Iterable> records) { + return (isActiveActiveReplicationEnabled || isWriteComputationEnabled) && serverConfig.isAAWCWorkloadParallelProcessingEnabled() - && IngestionBatchProcessor.isAllMessagesFromRTTopic(records)) { - produceToStoreBufferServiceOrKafkaInBatch( - records, - topicPartition, - partitionConsumptionState, - kafkaUrl, - kafkaClusterId); - return; - } - - long totalBytesRead = 0; - ValueHolder elapsedTimeForPuttingIntoQueue = new ValueHolder<>(0d); - boolean metricsEnabled = emitMetrics.get(); - long beforeProcessingBatchRecordsTimestampMs = System.currentTimeMillis(); - - partitionConsumptionState = partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); - for (PubSubMessage record: records) { - long beforeProcessingPerRecordTimestampNs = System.nanoTime(); - partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); - if (!shouldProcessRecord(record)) { - partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); - continue; - } - - // Check schema id availability before putting consumer record to drainer queue - waitReadyToProcessRecord(record); - - totalBytesRead += handleSingleMessage( - new PubSubMessageProcessedResultWrapper<>(record), - topicPartition, - partitionConsumptionState, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs, - metricsEnabled, - elapsedTimeForPuttingIntoQueue); - } - - /** - * Even if the records list is empty, we still need to check quota to potentially resume partition - */ - storageUtilizationManager.enforcePartitionQuota(topicPartition.getPartitionNumber(), totalBytesRead); - - if (metricsEnabled) { - if (totalBytesRead > 0) { - hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); - } - if (elapsedTimeForPuttingIntoQueue.getValue() > 0) { - hostLevelIngestionStats.recordConsumerRecordsQueuePutLatency( - elapsedTimeForPuttingIntoQueue.getValue(), - beforeProcessingBatchRecordsTimestampMs); - } - - hostLevelIngestionStats.recordStorageQuotaUsed(storageUtilizationManager.getDiskQuotaUsage()); - } + && IngestionBatchProcessor.isAllMessagesFromRTTopic(records); } - protected void produceToStoreBufferServiceOrKafkaInBatch( + public void produceToStoreBufferServiceOrKafkaInBatch( Iterable> records, PubSubTopicPartition topicPartition, PartitionConsumptionState partitionConsumptionState, @@ -1547,9 +1553,9 @@ protected void checkIngestionProgress(Store store) throws InterruptedException { /** * While using the shared consumer, we still need to check hybrid quota here since the actual disk usage could change * because of compaction or the disk quota could be adjusted even there is no record write. - * Since {@link #produceToStoreBufferServiceOrKafka} is only being invoked by {@link KafkaConsumerService} when there - * are available records, this function needs to check whether we need to resume the consumption when there are - * paused consumption because of hybrid quota violation. + * Since {@link StorePartitionDataReceiver#produceToStoreBufferServiceOrKafka} is only being invoked by + * {@link KafkaConsumerService} when there are available records, this function needs to check whether we need to + * resume the consumption when there are paused consumption because of hybrid quota violation. */ if (storageUtilizationManager.hasPausedPartitionIngestion()) { storageUtilizationManager.checkAllPartitionsQuota(); @@ -3164,7 +3170,7 @@ private boolean processControlMessage( case START_OF_PUSH: /** * N.B.: The processing for SOP happens at the very beginning of the pipeline, in: - * {@link #produceToStoreBufferServiceOrKafka(Iterable, boolean, PubSubTopicPartition, String, int)} + * {@link StorePartitionDataReceiver#produceToStoreBufferServiceOrKafka} */ break; case END_OF_PUSH: @@ -3911,7 +3917,7 @@ private int processKafkaDataMessage( * needs to #checkValueSchemaAvail * @param record */ - private void waitReadyToProcessRecord(PubSubMessage record) + public void waitReadyToProcessRecord(PubSubMessage record) throws InterruptedException { KafkaMessageEnvelope kafkaValue = record.getValue(); if (record.getKey().isControlMessage() || kafkaValue == null) { @@ -4675,6 +4681,10 @@ Lazy getGracefulShutdownLatch() { return gracefulShutdownLatch; } + public StorageUtilizationManager getStorageUtilizationManager() { + return storageUtilizationManager; + } + // For unit test purpose. void setVersionRole(PartitionReplicaIngestionContext.VersionRole versionRole) { this.versionRole = versionRole; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index e5a59294b20..1b0492359f0 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -2,6 +2,7 @@ import com.linkedin.avroutil1.compatibility.shaded.org.apache.commons.lang3.Validate; import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; +import com.linkedin.davinci.stats.HostLevelIngestionStats; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.message.KafkaKey; @@ -10,6 +11,7 @@ import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.utils.ExceptionUtils; import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.ValueHolder; import java.util.List; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -75,12 +77,98 @@ public void write(List> cons * all the buffered messages for the paused partitions, but just slightly more complicate. * */ - storeIngestionTask.produceToStoreBufferServiceOrKafka(consumedData, topicPartition, kafkaUrl, kafkaClusterId); + produceToStoreBufferServiceOrKafka(consumedData, topicPartition, kafkaUrl, kafkaClusterId); } catch (Exception e) { handleDataReceiverException(e); } } + /** + * This function is in charge of producing the consumer records to the writer buffers maintained by {@link StoreBufferService}. + * + * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after this call. + * + * @param records : received consumer records + */ + protected void produceToStoreBufferServiceOrKafka( + Iterable> records, + PubSubTopicPartition topicPartition, + String kafkaUrl, + int kafkaClusterId) throws InterruptedException { + final int partition = topicPartition.getPartitionNumber(); + PartitionConsumptionState partitionConsumptionState = storeIngestionTask.getPartitionConsumptionState(partition); + if (partitionConsumptionState == null) { + throw new VeniceException( + "PartitionConsumptionState should present for store version: " + storeIngestionTask.getKafkaVersionTopic() + + ", partition: " + partition); + } + + /** + * Validate and filter out duplicate messages from the real-time topic as early as possible, so that + * the following batch processing logic won't spend useless efforts on duplicate messages. + */ + records = + storeIngestionTask.validateAndFilterOutDuplicateMessagesFromLeaderTopic(records, kafkaUrl, topicPartition); + + if (storeIngestionTask.shouldProduceInBatch(records)) { + storeIngestionTask.produceToStoreBufferServiceOrKafkaInBatch( + records, + topicPartition, + partitionConsumptionState, + kafkaUrl, + kafkaClusterId); + return; + } + + long totalBytesRead = 0; + ValueHolder elapsedTimeForPuttingIntoQueue = new ValueHolder<>(0d); + boolean metricsEnabled = storeIngestionTask.isMetricsEmissionEnabled(); + long beforeProcessingBatchRecordsTimestampMs = System.currentTimeMillis(); + + for (PubSubMessage record: records) { + long beforeProcessingPerRecordTimestampNs = System.nanoTime(); + partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); + if (!storeIngestionTask.shouldProcessRecord(record)) { + partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); + continue; + } + + // Check schema id availability before putting consumer record to drainer queue + storeIngestionTask.waitReadyToProcessRecord(record); + + totalBytesRead += storeIngestionTask.handleSingleMessage( + new PubSubMessageProcessedResultWrapper<>(record), + topicPartition, + partitionConsumptionState, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs, + metricsEnabled, + elapsedTimeForPuttingIntoQueue); + } + + /** + * Even if the records list is empty, we still need to check quota to potentially resume partition + */ + final StorageUtilizationManager storageUtilizationManager = storeIngestionTask.getStorageUtilizationManager(); + storageUtilizationManager.enforcePartitionQuota(partition, totalBytesRead); + + if (metricsEnabled) { + HostLevelIngestionStats hostLevelIngestionStats = storeIngestionTask.getHostLevelIngestionStats(); + if (totalBytesRead > 0) { + hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); + } + if (elapsedTimeForPuttingIntoQueue.getValue() > 0) { + hostLevelIngestionStats.recordConsumerRecordsQueuePutLatency( + elapsedTimeForPuttingIntoQueue.getValue(), + beforeProcessingBatchRecordsTimestampMs); + } + + hostLevelIngestionStats.recordStorageQuotaUsed(storageUtilizationManager.getDiskQuotaUsage()); + } + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); From 4bb2c5a8485e942e6ec83108076dcf299bbb0d64 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Fri, 18 Oct 2024 15:41:31 -0700 Subject: [PATCH 02/32] =?UTF-8?q?Refactored=20`produceToStoreBufferService?= =?UTF-8?q?OrKafkaInBatch()`=20from=20`StoreIngestionTask`=20into=20`Store?= =?UTF-8?q?PartitionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../kafka/consumer/StoreIngestionTask.java | 200 +++++++++--------- .../consumer/StorePartitionDataReceiver.java | 109 +++++++++- 2 files changed, 205 insertions(+), 104 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 68617d9fd25..a94cf8ef240 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -42,7 +42,6 @@ import com.linkedin.davinci.store.StoragePartitionConfig; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; import com.linkedin.davinci.store.record.ValueRecord; -import com.linkedin.davinci.utils.ByteArrayKey; import com.linkedin.davinci.utils.ChunkAssembler; import com.linkedin.davinci.validation.KafkaDataIntegrityValidator; import com.linkedin.davinci.validation.PartitionTracker; @@ -129,10 +128,8 @@ import java.util.Collection; import java.util.Collections; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.NavigableMap; import java.util.Objects; import java.util.Optional; import java.util.Properties; @@ -150,7 +147,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.ReentrantLock; import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; @@ -1304,104 +1300,104 @@ public boolean shouldProduceInBatch(Iterable> records, - PubSubTopicPartition topicPartition, - PartitionConsumptionState partitionConsumptionState, - String kafkaUrl, - int kafkaClusterId) throws InterruptedException { - long totalBytesRead = 0; - ValueHolder elapsedTimeForPuttingIntoQueue = new ValueHolder<>(0d); - boolean metricsEnabled = emitMetrics.get(); - long beforeProcessingBatchRecordsTimestampMs = System.currentTimeMillis(); - /** - * Split the records into mini batches. - */ - int batchSize = serverConfig.getAAWCWorkloadParallelProcessingThreadPoolSize(); - List>> batches = new ArrayList<>(); - List> ongoingBatch = new ArrayList<>(batchSize); - Iterator> iter = records.iterator(); - while (iter.hasNext()) { - PubSubMessage record = iter.next(); - if (partitionConsumptionState != null) { - partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); - } - if (!shouldProcessRecord(record)) { - if (partitionConsumptionState != null) { - partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); - } - continue; - } - waitReadyToProcessRecord(record); - ongoingBatch.add(record); - if (ongoingBatch.size() == batchSize) { - batches.add(ongoingBatch); - ongoingBatch = new ArrayList<>(batchSize); - } - } - if (!ongoingBatch.isEmpty()) { - batches.add(ongoingBatch); - } - if (batches.isEmpty()) { - return; - } - IngestionBatchProcessor ingestionBatchProcessor = getIngestionBatchProcessor(); - if (ingestionBatchProcessor == null) { - throw new VeniceException( - "IngestionBatchProcessor object should present for store version: " + kafkaVersionTopic); - } - /** - * Process records batch by batch. - */ - for (List> batch: batches) { - NavigableMap keyLockMap = ingestionBatchProcessor.lockKeys(batch); - try { - long beforeProcessingPerRecordTimestampNs = System.nanoTime(); - List> processedResults = - ingestionBatchProcessor.process( - batch, - partitionConsumptionState, - topicPartition.getPartitionNumber(), - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs); - - for (PubSubMessageProcessedResultWrapper processedRecord: processedResults) { - totalBytesRead += handleSingleMessage( - processedRecord, - topicPartition, - partitionConsumptionState, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs, - metricsEnabled, - elapsedTimeForPuttingIntoQueue); - } - } finally { - ingestionBatchProcessor.unlockKeys(keyLockMap); - } - } - - /** - * Even if the records list is empty, we still need to check quota to potentially resume partition - */ - storageUtilizationManager.enforcePartitionQuota(topicPartition.getPartitionNumber(), totalBytesRead); - - if (metricsEnabled) { - if (totalBytesRead > 0) { - hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); - } - if (elapsedTimeForPuttingIntoQueue.getValue() > 0) { - hostLevelIngestionStats.recordConsumerRecordsQueuePutLatency( - elapsedTimeForPuttingIntoQueue.getValue(), - beforeProcessingBatchRecordsTimestampMs); - } - - hostLevelIngestionStats.recordStorageQuotaUsed(storageUtilizationManager.getDiskQuotaUsage()); - } - } + // public void produceToStoreBufferServiceOrKafkaInBatch( + // Iterable> records, + // PubSubTopicPartition topicPartition, + // PartitionConsumptionState partitionConsumptionState, + // String kafkaUrl, + // int kafkaClusterId) throws InterruptedException { + // long totalBytesRead = 0; + // ValueHolder elapsedTimeForPuttingIntoQueue = new ValueHolder<>(0d); + // boolean metricsEnabled = emitMetrics.get(); + // long beforeProcessingBatchRecordsTimestampMs = System.currentTimeMillis(); + // /** + // * Split the records into mini batches. + // */ + // int batchSize = serverConfig.getAAWCWorkloadParallelProcessingThreadPoolSize(); + // List>> batches = new ArrayList<>(); + // List> ongoingBatch = new ArrayList<>(batchSize); + // Iterator> iter = records.iterator(); + // while (iter.hasNext()) { + // PubSubMessage record = iter.next(); + // if (partitionConsumptionState != null) { + // partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); + // } + // if (!shouldProcessRecord(record)) { + // if (partitionConsumptionState != null) { + // partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); + // } + // continue; + // } + // waitReadyToProcessRecord(record); + // ongoingBatch.add(record); + // if (ongoingBatch.size() == batchSize) { + // batches.add(ongoingBatch); + // ongoingBatch = new ArrayList<>(batchSize); + // } + // } + // if (!ongoingBatch.isEmpty()) { + // batches.add(ongoingBatch); + // } + // if (batches.isEmpty()) { + // return; + // } + // IngestionBatchProcessor ingestionBatchProcessor = getIngestionBatchProcessor(); + // if (ingestionBatchProcessor == null) { + // throw new VeniceException( + // "IngestionBatchProcessor object should present for store version: " + kafkaVersionTopic); + // } + // /** + // * Process records batch by batch. + // */ + // for (List> batch: batches) { + // NavigableMap keyLockMap = ingestionBatchProcessor.lockKeys(batch); + // try { + // long beforeProcessingPerRecordTimestampNs = System.nanoTime(); + // List> processedResults = + // ingestionBatchProcessor.process( + // batch, + // partitionConsumptionState, + // topicPartition.getPartitionNumber(), + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs, + // beforeProcessingBatchRecordsTimestampMs); + // + // for (PubSubMessageProcessedResultWrapper processedRecord: processedResults) { + // totalBytesRead += handleSingleMessage( + // processedRecord, + // topicPartition, + // partitionConsumptionState, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs, + // beforeProcessingBatchRecordsTimestampMs, + // metricsEnabled, + // elapsedTimeForPuttingIntoQueue); + // } + // } finally { + // ingestionBatchProcessor.unlockKeys(keyLockMap); + // } + // } + // + // /** + // * Even if the records list is empty, we still need to check quota to potentially resume partition + // */ + // storageUtilizationManager.enforcePartitionQuota(topicPartition.getPartitionNumber(), totalBytesRead); + // + // if (metricsEnabled) { + // if (totalBytesRead > 0) { + // hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); + // } + // if (elapsedTimeForPuttingIntoQueue.getValue() > 0) { + // hostLevelIngestionStats.recordConsumerRecordsQueuePutLatency( + // elapsedTimeForPuttingIntoQueue.getValue(), + // beforeProcessingBatchRecordsTimestampMs); + // } + // + // hostLevelIngestionStats.recordStorageQuotaUsed(storageUtilizationManager.getDiskQuotaUsage()); + // } + // } // For testing purpose List getPartitionIngestionExceptionList() { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 1b0492359f0..6ba7c686e89 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -3,6 +3,7 @@ import com.linkedin.avroutil1.compatibility.shaded.org.apache.commons.lang3.Validate; import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; import com.linkedin.davinci.stats.HostLevelIngestionStats; +import com.linkedin.davinci.utils.ByteArrayKey; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.message.KafkaKey; @@ -12,7 +13,11 @@ import com.linkedin.venice.utils.ExceptionUtils; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.ValueHolder; +import java.util.ArrayList; +import java.util.Iterator; import java.util.List; +import java.util.NavigableMap; +import java.util.concurrent.locks.ReentrantLock; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -111,7 +116,7 @@ protected void produceToStoreBufferServiceOrKafka( storeIngestionTask.validateAndFilterOutDuplicateMessagesFromLeaderTopic(records, kafkaUrl, topicPartition); if (storeIngestionTask.shouldProduceInBatch(records)) { - storeIngestionTask.produceToStoreBufferServiceOrKafkaInBatch( + produceToStoreBufferServiceOrKafkaInBatch( records, topicPartition, partitionConsumptionState, @@ -148,13 +153,113 @@ protected void produceToStoreBufferServiceOrKafka( elapsedTimeForPuttingIntoQueue); } + updateMetricsAndEnforceQuota( + totalBytesRead, + partition, + elapsedTimeForPuttingIntoQueue, + beforeProcessingBatchRecordsTimestampMs); + } + + public void produceToStoreBufferServiceOrKafkaInBatch( + Iterable> records, + PubSubTopicPartition topicPartition, + PartitionConsumptionState partitionConsumptionState, + String kafkaUrl, + int kafkaClusterId) throws InterruptedException { + long totalBytesRead = 0; + ValueHolder elapsedTimeForPuttingIntoQueue = new ValueHolder<>(0d); + boolean metricsEnabled = storeIngestionTask.isMetricsEmissionEnabled(); + long beforeProcessingBatchRecordsTimestampMs = System.currentTimeMillis(); + /** + * Split the records into mini batches. + */ + int batchSize = storeIngestionTask.getServerConfig().getAAWCWorkloadParallelProcessingThreadPoolSize(); + List>> batches = new ArrayList<>(); + List> ongoingBatch = new ArrayList<>(batchSize); + Iterator> iter = records.iterator(); + while (iter.hasNext()) { + PubSubMessage record = iter.next(); + if (partitionConsumptionState != null) { + partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); + } + if (!storeIngestionTask.shouldProcessRecord(record)) { + if (partitionConsumptionState != null) { + partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); + } + continue; + } + storeIngestionTask.waitReadyToProcessRecord(record); + ongoingBatch.add(record); + if (ongoingBatch.size() == batchSize) { + batches.add(ongoingBatch); + ongoingBatch = new ArrayList<>(batchSize); + } + } + if (!ongoingBatch.isEmpty()) { + batches.add(ongoingBatch); + } + if (batches.isEmpty()) { + return; + } + IngestionBatchProcessor ingestionBatchProcessor = storeIngestionTask.getIngestionBatchProcessor(); + if (ingestionBatchProcessor == null) { + throw new VeniceException( + "IngestionBatchProcessor object should present for store version: " + + storeIngestionTask.getKafkaVersionTopic()); + } + /** + * Process records batch by batch. + */ + for (List> batch: batches) { + NavigableMap keyLockMap = ingestionBatchProcessor.lockKeys(batch); + try { + long beforeProcessingPerRecordTimestampNs = System.nanoTime(); + List> processedResults = + ingestionBatchProcessor.process( + batch, + partitionConsumptionState, + topicPartition.getPartitionNumber(), + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); + + for (PubSubMessageProcessedResultWrapper processedRecord: processedResults) { + totalBytesRead += storeIngestionTask.handleSingleMessage( + processedRecord, + topicPartition, + partitionConsumptionState, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs, + metricsEnabled, + elapsedTimeForPuttingIntoQueue); + } + } finally { + ingestionBatchProcessor.unlockKeys(keyLockMap); + } + } + + updateMetricsAndEnforceQuota( + totalBytesRead, + topicPartition.getPartitionNumber(), + elapsedTimeForPuttingIntoQueue, + beforeProcessingBatchRecordsTimestampMs); + } + + private void updateMetricsAndEnforceQuota( + long totalBytesRead, + int partition, + ValueHolder elapsedTimeForPuttingIntoQueue, + long beforeProcessingBatchRecordsTimestampMs) { /** * Even if the records list is empty, we still need to check quota to potentially resume partition */ final StorageUtilizationManager storageUtilizationManager = storeIngestionTask.getStorageUtilizationManager(); storageUtilizationManager.enforcePartitionQuota(partition, totalBytesRead); - if (metricsEnabled) { + if (storeIngestionTask.isMetricsEmissionEnabled()) { HostLevelIngestionStats hostLevelIngestionStats = storeIngestionTask.getHostLevelIngestionStats(); if (totalBytesRead > 0) { hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); From 776d65f84f1436bc69a3b6bebab09a493343a944 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Fri, 18 Oct 2024 16:54:52 -0700 Subject: [PATCH 03/32] =?UTF-8?q?Refactored=20`waitReadyToProcessRecord()`?= =?UTF-8?q?=20and=20the=20call=20chain=20down=20to=20right=20before=20`wai?= =?UTF-8?q?tUntilValueSchemaAvailable()`=20from=20`StoreIngestionTask`=20i?= =?UTF-8?q?nto=20`StorePartitionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../kafka/consumer/StoreIngestionTask.java | 215 +++++++++--------- .../consumer/StorePartitionDataReceiver.java | 101 +++++++- 2 files changed, 211 insertions(+), 105 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index a94cf8ef240..13f3b2a6e75 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -71,7 +71,6 @@ import com.linkedin.venice.kafka.protocol.StartOfIncrementalPush; import com.linkedin.venice.kafka.protocol.StartOfPush; import com.linkedin.venice.kafka.protocol.TopicSwitch; -import com.linkedin.venice.kafka.protocol.Update; import com.linkedin.venice.kafka.protocol.enums.ControlMessageType; import com.linkedin.venice.kafka.protocol.enums.MessageType; import com.linkedin.venice.kafka.protocol.state.PartitionState; @@ -170,7 +169,7 @@ public abstract class StoreIngestionTask implements Runnable, Closeable { public static long SCHEMA_POLLING_DELAY_MS = SECONDS.toMillis(5); public static long STORE_VERSION_POLLING_DELAY_MS = MINUTES.toMillis(1); - private static final long SCHEMA_POLLING_TIMEOUT_MS = MINUTES.toMillis(5); + public static final long SCHEMA_POLLING_TIMEOUT_MS = MINUTES.toMillis(5); private static final long SOP_POLLING_TIMEOUT_MS = HOURS.toMillis(1); protected static final long WAITING_TIME_FOR_LAST_RECORD_TO_BE_PROCESSED = MINUTES.toMillis(1); // 1 min @@ -3908,97 +3907,100 @@ private int processKafkaDataMessage( return keyLen + valueLen; } - /** - * This method checks whether the given record needs to be checked schema availability. Only PUT and UPDATE message - * needs to #checkValueSchemaAvail - * @param record - */ - public void waitReadyToProcessRecord(PubSubMessage record) - throws InterruptedException { - KafkaMessageEnvelope kafkaValue = record.getValue(); - if (record.getKey().isControlMessage() || kafkaValue == null) { - return; - } - - switch (MessageType.valueOf(kafkaValue)) { - case PUT: - Put put = (Put) kafkaValue.payloadUnion; - waitReadyToProcessDataRecord(put.schemaId); - try { - deserializeValue(put.schemaId, put.putValue, record); - } catch (Exception e) { - PartitionConsumptionState pcs = - partitionConsumptionStateMap.get(record.getTopicPartition().getPartitionNumber()); - LeaderFollowerStateType state = pcs == null ? null : pcs.getLeaderFollowerState(); - throw new VeniceException( - "Failed to deserialize PUT for: " + record.getTopicPartition() + ", offset: " + record.getOffset() - + ", schema id: " + put.schemaId + ", LF state: " + state, - e); - } - break; - case UPDATE: - Update update = (Update) kafkaValue.payloadUnion; - waitReadyToProcessDataRecord(update.schemaId); - break; - case DELETE: - /* we don't need to check schema availability for DELETE */ - break; - default: - throw new VeniceMessageException( - ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); - } - } - - /** - * Check whether the given schema id is available for current store. - * The function will bypass the check if schema id is -1 (VPJ job is still using it before we finishes the integration with schema registry). - * Right now, this function is maintaining a local cache for schema id of current store considering that the value schema is immutable; - * If the schema id is not available, this function will polling until the schema appears or timeout: {@link #SCHEMA_POLLING_TIMEOUT_MS}; - * - * @param schemaId - */ - private void waitReadyToProcessDataRecord(int schemaId) throws InterruptedException { - if (schemaId == -1) { - // TODO: Once Venice Client (VeniceShellClient) finish the integration with schema registry, - // we need to remove this check here. - return; - } - - if (schemaId == AvroProtocolDefinition.CHUNK.getCurrentProtocolVersion() - || schemaId == AvroProtocolDefinition.CHUNKED_VALUE_MANIFEST.getCurrentProtocolVersion()) { - StoreVersionState storeVersionState = waitVersionStateAvailable(kafkaVersionTopic); - if (!storeVersionState.chunked) { - throw new VeniceException( - "Detected chunking in a store-version where chunking is NOT enabled. Will abort ingestion."); - } - return; - } - - waitUntilValueSchemaAvailable(schemaId); - } - - protected StoreVersionState waitVersionStateAvailable(String kafkaTopic) throws InterruptedException { - long startTime = System.currentTimeMillis(); - long elapsedTime; - StoreVersionState state; - for (;;) { - state = storageEngine.getStoreVersionState(); - elapsedTime = System.currentTimeMillis() - startTime; - - if (state != null) { - return state; - } - - if (elapsedTime > SCHEMA_POLLING_TIMEOUT_MS || !isRunning()) { - LOGGER.warn("Version state is not available for {} after {}", kafkaTopic, elapsedTime); - throw new VeniceException("Store version state is not available for " + kafkaTopic); - } - - Thread.sleep(SCHEMA_POLLING_DELAY_MS); - } - } + // /** + // * This method checks whether the given record needs to be checked schema availability. Only PUT and UPDATE message + // * needs to #checkValueSchemaAvail + // * @param record + // */ + // public void waitReadyToProcessRecord(PubSubMessage record) + // throws InterruptedException { + // KafkaMessageEnvelope kafkaValue = record.getValue(); + // if (record.getKey().isControlMessage() || kafkaValue == null) { + // return; + // } + // + // switch (MessageType.valueOf(kafkaValue)) { + // case PUT: + // Put put = (Put) kafkaValue.payloadUnion; + // waitReadyToProcessDataRecord(put.schemaId); + // try { + // deserializeValue(put.schemaId, put.putValue, record); + // } catch (Exception e) { + // PartitionConsumptionState pcs = + // partitionConsumptionStateMap.get(record.getTopicPartition().getPartitionNumber()); + // LeaderFollowerStateType state = pcs == null ? null : pcs.getLeaderFollowerState(); + // throw new VeniceException( + // "Failed to deserialize PUT for: " + record.getTopicPartition() + ", offset: " + record.getOffset() + // + ", schema id: " + put.schemaId + ", LF state: " + state, + // e); + // } + // break; + // case UPDATE: + // Update update = (Update) kafkaValue.payloadUnion; + // waitReadyToProcessDataRecord(update.schemaId); + // break; + // case DELETE: + // /* we don't need to check schema availability for DELETE */ + // break; + // default: + // throw new VeniceMessageException( + // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); + // } + // } + // + // /** + // * Check whether the given schema id is available for current store. + // * The function will bypass the check if schema id is -1 (VPJ job is still using it before we finishes the + // integration with schema registry). + // * Right now, this function is maintaining a local cache for schema id of current store considering that the value + // schema is immutable; + // * If the schema id is not available, this function will polling until the schema appears or timeout: {@link + // #SCHEMA_POLLING_TIMEOUT_MS}; + // * + // * @param schemaId + // */ + // public void waitReadyToProcessDataRecord(int schemaId) throws InterruptedException { + // if (schemaId == -1) { + // // TODO: Once Venice Client (VeniceShellClient) finish the integration with schema registry, + // // we need to remove this check here. + // return; + // } + // + // if (schemaId == AvroProtocolDefinition.CHUNK.getCurrentProtocolVersion() + // || schemaId == AvroProtocolDefinition.CHUNKED_VALUE_MANIFEST.getCurrentProtocolVersion()) { + // StoreVersionState storeVersionState = waitVersionStateAvailable(kafkaVersionTopic); + // if (!storeVersionState.chunked) { + // throw new VeniceException( + // "Detected chunking in a store-version where chunking is NOT enabled. Will abort ingestion."); + // } + // return; + // } + // + // waitUntilValueSchemaAvailable(schemaId); + // } + // + // protected StoreVersionState waitVersionStateAvailable(String kafkaTopic) throws InterruptedException { + // long startTime = System.currentTimeMillis(); + // long elapsedTime; + // StoreVersionState state; + // for (;;) { + // state = storageEngine.getStoreVersionState(); + // elapsedTime = System.currentTimeMillis() - startTime; + // + // if (state != null) { + // return state; + // } + // + // if (elapsedTime > SCHEMA_POLLING_TIMEOUT_MS || !isRunning()) { + // LOGGER.warn("Version state is not available for {} after {}", kafkaTopic, elapsedTime); + // throw new VeniceException("Store version state is not available for " + kafkaTopic); + // } + // + // Thread.sleep(SCHEMA_POLLING_DELAY_MS); + // } + // } - private void waitUntilValueSchemaAvailable(int schemaId) throws InterruptedException { + void waitUntilValueSchemaAvailable(int schemaId) throws InterruptedException { // Considering value schema is immutable for an existing store, we can cache it locally if (availableSchemaIds.get(schemaId) != null) { return; @@ -4043,17 +4045,16 @@ private void waitUntilValueSchemaAvailable(int schemaId) throws InterruptedExcep } /** - * Deserialize a value using the schema that serializes it. Exception will be thrown and ingestion will fail if the - * value cannot be deserialized. Currently, the deserialization dry-run won't happen in the following cases: - * - * 1. Value is chunked. A single piece of value cannot be deserialized. In this case, the schema id is not added in - * availableSchemaIds by {@link StoreIngestionTask#waitUntilValueSchemaAvailable}. - * 2. Ingestion isolation is enabled, in which case ingestion happens on forked process instead of this main process. - */ - private void deserializeValue( - int schemaId, - ByteBuffer value, - PubSubMessage record) throws IOException { + * Deserialize a value using the schema that serializes it. Exception will be thrown and ingestion will fail if the + * value cannot be deserialized. Currently, the deserialization dry-run won't happen in the following cases: + * + * 1. Value is chunked. A single piece of value cannot be deserialized. In this case, the schema id is not added in + * availableSchemaIds by {@link StoreIngestionTask#waitUntilValueSchemaAvailable}. + * 2. Ingestion isolation is enabled, in which case ingestion happens on forked process instead of this main + process. + */ + void deserializeValue(int schemaId, ByteBuffer value, PubSubMessage record) + throws IOException { if (schemaId < 0 || deserializedSchemaIds.get(schemaId) != null || availableSchemaIds.get(schemaId) == null) { return; } @@ -4681,6 +4682,14 @@ public StorageUtilizationManager getStorageUtilizationManager() { return storageUtilizationManager; } + public SparseConcurrentList getAvailableSchemaIds() { + return availableSchemaIds; + } + + public SparseConcurrentList getDeserializedSchemaIds() { + return deserializedSchemaIds; + } + // For unit test purpose. void setVersionRole(PartitionReplicaIngestionContext.VersionRole versionRole) { this.versionRole = versionRole; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 6ba7c686e89..9dd3e51737d 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -5,11 +5,17 @@ import com.linkedin.davinci.stats.HostLevelIngestionStats; import com.linkedin.davinci.utils.ByteArrayKey; import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.exceptions.VeniceMessageException; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; +import com.linkedin.venice.kafka.protocol.Put; +import com.linkedin.venice.kafka.protocol.Update; +import com.linkedin.venice.kafka.protocol.enums.MessageType; +import com.linkedin.venice.kafka.protocol.state.StoreVersionState; import com.linkedin.venice.message.KafkaKey; import com.linkedin.venice.pubsub.api.PubSubMessage; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.utils.ExceptionUtils; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.ValueHolder; @@ -139,7 +145,7 @@ protected void produceToStoreBufferServiceOrKafka( } // Check schema id availability before putting consumer record to drainer queue - storeIngestionTask.waitReadyToProcessRecord(record); + waitReadyToProcessRecord(record); totalBytesRead += storeIngestionTask.handleSingleMessage( new PubSubMessageProcessedResultWrapper<>(record), @@ -188,7 +194,7 @@ public void produceToStoreBufferServiceOrKafkaInBatch( } continue; } - storeIngestionTask.waitReadyToProcessRecord(record); + waitReadyToProcessRecord(record); ongoingBatch.add(record); if (ongoingBatch.size() == batchSize) { batches.add(ongoingBatch); @@ -274,6 +280,97 @@ private void updateMetricsAndEnforceQuota( } } + /** + * This method checks whether the given record needs to be checked schema availability. Only PUT and UPDATE message + * needs to #checkValueSchemaAvail + * @param record + */ + private void waitReadyToProcessRecord(PubSubMessage record) + throws InterruptedException { + KafkaMessageEnvelope kafkaValue = record.getValue(); + if (record.getKey().isControlMessage() || kafkaValue == null) { + return; + } + + switch (MessageType.valueOf(kafkaValue)) { + case PUT: + Put put = (Put) kafkaValue.payloadUnion; + waitReadyToProcessDataRecord(put.schemaId); + try { + storeIngestionTask.deserializeValue(put.schemaId, put.putValue, record); + } catch (Exception e) { + PartitionConsumptionState pcs = + storeIngestionTask.getPartitionConsumptionState(record.getTopicPartition().getPartitionNumber()); + LeaderFollowerStateType state = pcs == null ? null : pcs.getLeaderFollowerState(); + throw new VeniceException( + "Failed to deserialize PUT for: " + record.getTopicPartition() + ", offset: " + record.getOffset() + + ", schema id: " + put.schemaId + ", LF state: " + state, + e); + } + break; + case UPDATE: + Update update = (Update) kafkaValue.payloadUnion; + waitReadyToProcessDataRecord(update.schemaId); + break; + case DELETE: + /* we don't need to check schema availability for DELETE */ + break; + default: + throw new VeniceMessageException( + storeIngestionTask.getIngestionTaskName() + " : Invalid/Unrecognized operation type submitted: " + + kafkaValue.messageType); + } + } + + /** + * Check whether the given schema id is available for current store. + * The function will bypass the check if schema id is -1 (VPJ job is still using it before we finishes the integration with schema registry). + * Right now, this function is maintaining a local cache for schema id of current store considering that the value schema is immutable; + * If the schema id is not available, this function will polling until the schema appears or timeout: {@link StoreIngestionTask#SCHEMA_POLLING_TIMEOUT_MS}; + * + * @param schemaId + */ + private void waitReadyToProcessDataRecord(int schemaId) throws InterruptedException { + if (schemaId == -1) { + // TODO: Once Venice Client (VeniceShellClient) finish the integration with schema registry, + // we need to remove this check here. + return; + } + + if (schemaId == AvroProtocolDefinition.CHUNK.getCurrentProtocolVersion() + || schemaId == AvroProtocolDefinition.CHUNKED_VALUE_MANIFEST.getCurrentProtocolVersion()) { + StoreVersionState storeVersionState = waitVersionStateAvailable(storeIngestionTask.getKafkaVersionTopic()); + if (!storeVersionState.chunked) { + throw new VeniceException( + "Detected chunking in a store-version where chunking is NOT enabled. Will abort ingestion."); + } + return; + } + + storeIngestionTask.waitUntilValueSchemaAvailable(schemaId); + } + + private StoreVersionState waitVersionStateAvailable(String kafkaTopic) throws InterruptedException { + long startTime = System.currentTimeMillis(); + long elapsedTime; + StoreVersionState state; + for (;;) { + state = storeIngestionTask.getStorageEngine().getStoreVersionState(); + elapsedTime = System.currentTimeMillis() - startTime; + + if (state != null) { + return state; + } + + if (elapsedTime > StoreIngestionTask.SCHEMA_POLLING_TIMEOUT_MS || !storeIngestionTask.isRunning()) { + LOGGER.warn("Version state is not available for {} after {}", kafkaTopic, elapsedTime); + throw new VeniceException("Store version state is not available for " + kafkaTopic); + } + + Thread.sleep(StoreIngestionTask.SCHEMA_POLLING_DELAY_MS); + } + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); From f1be5cf7b10668570f16e9e0950c0de244be9e17 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Fri, 18 Oct 2024 17:09:00 -0700 Subject: [PATCH 04/32] =?UTF-8?q?Refactored=20`handleSingleMessage()`=20fr?= =?UTF-8?q?om=20`StoreIngestionTask`=20into=20`StorePartitionDataReceiver`?= =?UTF-8?q?.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../kafka/consumer/StoreIngestionTask.java | 150 +++++++++--------- .../consumer/StorePartitionDataReceiver.java | 74 ++++++++- 2 files changed, 149 insertions(+), 75 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 13f3b2a6e75..ec60734e829 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -112,7 +112,6 @@ import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Timer; import com.linkedin.venice.utils.Utils; -import com.linkedin.venice.utils.ValueHolder; import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import com.linkedin.venice.utils.lazy.Lazy; @@ -1134,77 +1133,77 @@ public abstract Iterable> va String kafkaUrl, PubSubTopicPartition topicPartition); - public int handleSingleMessage( - PubSubMessageProcessedResultWrapper consumerRecordWrapper, - PubSubTopicPartition topicPartition, - PartitionConsumptionState partitionConsumptionState, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingPerRecordTimestampNs, - long beforeProcessingBatchRecordsTimestampMs, - boolean metricsEnabled, - ValueHolder elapsedTimeForPuttingIntoQueue) throws InterruptedException { - PubSubMessage record = consumerRecordWrapper.getMessage(); - if (record.getKey().isControlMessage()) { - ControlMessage controlMessage = (ControlMessage) record.getValue().payloadUnion; - if (ControlMessageType.valueOf(controlMessage.controlMessageType) == ControlMessageType.START_OF_PUSH) { - /** - * N.B.: The rest of the {@link ControlMessage} types are handled by: - * {@link #processControlMessage(KafkaMessageEnvelope, ControlMessage, int, long, PartitionConsumptionState)} - * - * But for the SOP in particular, we want to process it here, at the start of the pipeline, to ensure that the - * {@link StoreVersionState} is properly primed, as other functions below this point, but prior to being - * enqueued into the {@link StoreBufferService} rely on this state to be there. - */ - processStartOfPush( - record.getValue(), - controlMessage, - record.getTopicPartition().getPartitionNumber(), - partitionConsumptionStateMap.get(topicPartition.getPartitionNumber())); - } - } - - // This function may modify the original record in KME and it is unsafe to use the payload from KME directly after - // this call. - DelegateConsumerRecordResult delegateConsumerRecordResult = delegateConsumerRecord( - consumerRecordWrapper, - topicPartition.getPartitionNumber(), - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs); - - switch (delegateConsumerRecordResult) { - case QUEUED_TO_DRAINER: - long queuePutStartTimeInNS = metricsEnabled ? System.nanoTime() : 0; - - // blocking call - storeBufferService.putConsumerRecord( - record, - this, - null, - topicPartition.getPartitionNumber(), - kafkaUrl, - beforeProcessingPerRecordTimestampNs); - - if (metricsEnabled) { - elapsedTimeForPuttingIntoQueue.setValue( - elapsedTimeForPuttingIntoQueue.getValue() + LatencyUtils.getElapsedTimeFromNSToMS(queuePutStartTimeInNS)); - } - break; - case PRODUCED_TO_KAFKA: - case SKIPPED_MESSAGE: - break; - default: - throw new VeniceException( - ingestionTaskName + " received unknown DelegateConsumerRecordResult enum for " - + record.getTopicPartition()); - } - // Update the latest message consumed time - partitionConsumptionState.setLatestMessageConsumedTimestampInMs(beforeProcessingBatchRecordsTimestampMs); - - return record.getPayloadSize(); - } + // public int handleSingleMessage( + // PubSubMessageProcessedResultWrapper consumerRecordWrapper, + // PubSubTopicPartition topicPartition, + // PartitionConsumptionState partitionConsumptionState, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingPerRecordTimestampNs, + // long beforeProcessingBatchRecordsTimestampMs, + // boolean metricsEnabled, + // ValueHolder elapsedTimeForPuttingIntoQueue) throws InterruptedException { + // PubSubMessage record = consumerRecordWrapper.getMessage(); + // if (record.getKey().isControlMessage()) { + // ControlMessage controlMessage = (ControlMessage) record.getValue().payloadUnion; + // if (ControlMessageType.valueOf(controlMessage.controlMessageType) == ControlMessageType.START_OF_PUSH) { + // /** + // * N.B.: The rest of the {@link ControlMessage} types are handled by: + // * {@link #processControlMessage(KafkaMessageEnvelope, ControlMessage, int, long, PartitionConsumptionState)} + // * + // * But for the SOP in particular, we want to process it here, at the start of the pipeline, to ensure that the + // * {@link StoreVersionState} is properly primed, as other functions below this point, but prior to being + // * enqueued into the {@link StoreBufferService} rely on this state to be there. + // */ + // processStartOfPush( + // record.getValue(), + // controlMessage, + // record.getTopicPartition().getPartitionNumber(), + // partitionConsumptionStateMap.get(topicPartition.getPartitionNumber())); + // } + // } + // + // // This function may modify the original record in KME and it is unsafe to use the payload from KME directly after + // // this call. + // DelegateConsumerRecordResult delegateConsumerRecordResult = delegateConsumerRecord( + // consumerRecordWrapper, + // topicPartition.getPartitionNumber(), + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs, + // beforeProcessingBatchRecordsTimestampMs); + // + // switch (delegateConsumerRecordResult) { + // case QUEUED_TO_DRAINER: + // long queuePutStartTimeInNS = metricsEnabled ? System.nanoTime() : 0; + // + // // blocking call + // storeBufferService.putConsumerRecord( + // record, + // this, + // null, + // topicPartition.getPartitionNumber(), + // kafkaUrl, + // beforeProcessingPerRecordTimestampNs); + // + // if (metricsEnabled) { + // elapsedTimeForPuttingIntoQueue.setValue( + // elapsedTimeForPuttingIntoQueue.getValue() + LatencyUtils.getElapsedTimeFromNSToMS(queuePutStartTimeInNS)); + // } + // break; + // case PRODUCED_TO_KAFKA: + // case SKIPPED_MESSAGE: + // break; + // default: + // throw new VeniceException( + // ingestionTaskName + " received unknown DelegateConsumerRecordResult enum for " + // + record.getTopicPartition()); + // } + // // Update the latest message consumed time + // partitionConsumptionState.setLatestMessageConsumedTimestampInMs(beforeProcessingBatchRecordsTimestampMs); + // + // return record.getPayloadSize(); + // } // /** // * This function is in charge of producing the consumer records to the writer buffers maintained by {@link @@ -2914,7 +2913,7 @@ private void syncEndOfPushTimestampToMetadataService(long endOfPushTimestamp) { }); } - private void processStartOfPush( + void processStartOfPush( KafkaMessageEnvelope startOfPushKME, ControlMessage controlMessage, int partition, @@ -4682,6 +4681,11 @@ public StorageUtilizationManager getStorageUtilizationManager() { return storageUtilizationManager; } + void putConsumerRecord(PubSubMessage record, int partition, String url, long t) + throws InterruptedException { + storeBufferService.putConsumerRecord(record, this, null, partition, url, t); + } + public SparseConcurrentList getAvailableSchemaIds() { return availableSchemaIds; } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 9dd3e51737d..26fc82fe79c 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -6,9 +6,11 @@ import com.linkedin.davinci.utils.ByteArrayKey; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceMessageException; +import com.linkedin.venice.kafka.protocol.ControlMessage; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.Put; import com.linkedin.venice.kafka.protocol.Update; +import com.linkedin.venice.kafka.protocol.enums.ControlMessageType; import com.linkedin.venice.kafka.protocol.enums.MessageType; import com.linkedin.venice.kafka.protocol.state.StoreVersionState; import com.linkedin.venice.message.KafkaKey; @@ -17,6 +19,7 @@ import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.utils.ExceptionUtils; +import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.ValueHolder; import java.util.ArrayList; @@ -147,7 +150,7 @@ protected void produceToStoreBufferServiceOrKafka( // Check schema id availability before putting consumer record to drainer queue waitReadyToProcessRecord(record); - totalBytesRead += storeIngestionTask.handleSingleMessage( + totalBytesRead += handleSingleMessage( new PubSubMessageProcessedResultWrapper<>(record), topicPartition, partitionConsumptionState, @@ -231,7 +234,7 @@ public void produceToStoreBufferServiceOrKafkaInBatch( beforeProcessingBatchRecordsTimestampMs); for (PubSubMessageProcessedResultWrapper processedRecord: processedResults) { - totalBytesRead += storeIngestionTask.handleSingleMessage( + totalBytesRead += handleSingleMessage( processedRecord, topicPartition, partitionConsumptionState, @@ -371,6 +374,73 @@ private StoreVersionState waitVersionStateAvailable(String kafkaTopic) throws In } } + private int handleSingleMessage( + PubSubMessageProcessedResultWrapper consumerRecordWrapper, + PubSubTopicPartition topicPartition, + PartitionConsumptionState partitionConsumptionState, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingPerRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs, + boolean metricsEnabled, + ValueHolder elapsedTimeForPuttingIntoQueue) throws InterruptedException { + final int partition = topicPartition.getPartitionNumber(); + PubSubMessage record = consumerRecordWrapper.getMessage(); + if (record.getKey().isControlMessage()) { + ControlMessage controlMessage = (ControlMessage) record.getValue().payloadUnion; + if (ControlMessageType.valueOf(controlMessage.controlMessageType) == ControlMessageType.START_OF_PUSH) { + /** + * N.B.: The rest of the {@link ControlMessage} types are handled by: {@link StoreIngestionTask#processControlMessage} + * + * But for the SOP in particular, we want to process it here, at the start of the pipeline, to ensure that the + * {@link StoreVersionState} is properly primed, as other functions below this point, but prior to being + * enqueued into the {@link StoreBufferService} rely on this state to be there. + */ + storeIngestionTask.processStartOfPush( + record.getValue(), + controlMessage, + record.getTopicPartition().getPartitionNumber(), + storeIngestionTask.getPartitionConsumptionState(partition)); + } + } + + // This function may modify the original record in KME and it is unsafe to use the payload from KME directly after + // this call. + StoreIngestionTask.DelegateConsumerRecordResult delegateConsumerRecordResult = + storeIngestionTask.delegateConsumerRecord( + consumerRecordWrapper, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); + + switch (delegateConsumerRecordResult) { + case QUEUED_TO_DRAINER: + long queuePutStartTimeInNS = metricsEnabled ? System.nanoTime() : 0; + + // blocking call + storeIngestionTask.putConsumerRecord(record, partition, kafkaUrl, beforeProcessingPerRecordTimestampNs); + + if (metricsEnabled) { + elapsedTimeForPuttingIntoQueue.setValue( + elapsedTimeForPuttingIntoQueue.getValue() + LatencyUtils.getElapsedTimeFromNSToMS(queuePutStartTimeInNS)); + } + break; + case PRODUCED_TO_KAFKA: + case SKIPPED_MESSAGE: + break; + default: + throw new VeniceException( + storeIngestionTask.getIngestionTaskName() + " received unknown DelegateConsumerRecordResult enum for " + + record.getTopicPartition()); + } + // Update the latest message consumed time + partitionConsumptionState.setLatestMessageConsumedTimestampInMs(beforeProcessingBatchRecordsTimestampMs); + + return record.getPayloadSize(); + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); From 8eda4a0559b6d6dc7ec3d43aaebb82371519aca6 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Fri, 18 Oct 2024 17:31:55 -0700 Subject: [PATCH 05/32] =?UTF-8?q?Refactored=20`validateAndFilterOutDuplica?= =?UTF-8?q?teMessagesFromLeaderTopic()`=20from=20`StoreIngestionTask`=20/?= =?UTF-8?q?=20`LeaderFollowerStoreIngestionTask`=20into=20`StorePartitionD?= =?UTF-8?q?ataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 155 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 23 ++- .../consumer/StorePartitionDataReceiver.java | 76 ++++++++- 3 files changed, 171 insertions(+), 83 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 85471674f93..6aac1e6e31f 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -33,14 +33,11 @@ import com.linkedin.davinci.store.view.ChangeCaptureViewWriter; import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.validation.KafkaDataIntegrityValidator; -import com.linkedin.davinci.validation.PartitionTracker; import com.linkedin.venice.common.VeniceSystemStoreUtils; import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceMessageException; import com.linkedin.venice.exceptions.VeniceTimeoutException; -import com.linkedin.venice.exceptions.validation.DuplicateDataException; -import com.linkedin.venice.exceptions.validation.FatalDataValidationException; import com.linkedin.venice.guid.GuidUtils; import com.linkedin.venice.kafka.protocol.ControlMessage; import com.linkedin.venice.kafka.protocol.Delete; @@ -92,7 +89,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -1261,6 +1257,7 @@ private void checkAndUpdateDataRecoveryStatusOfHybridStore(PartitionConsumptionS * * If buffer replay is disable, all replicas will stick to version topic, no one is going to produce any message. */ + @Override protected boolean shouldProduceToVersionTopic(PartitionConsumptionState partitionConsumptionState) { if (!isLeader(partitionConsumptionState)) { return false; // Not leader @@ -2280,78 +2277,79 @@ protected void recordHeartbeatReceived( } } - @Override - public Iterable> validateAndFilterOutDuplicateMessagesFromLeaderTopic( - Iterable> records, - String kafkaUrl, - PubSubTopicPartition topicPartition) { - PartitionConsumptionState partitionConsumptionState = - partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); - if (partitionConsumptionState == null) { - // The partition is likely unsubscribed, will skip these messages. - LOGGER.warn( - "No partition consumption state for store version: {}, partition:{}, will filter out all the messages", - kafkaVersionTopic, - topicPartition.getPartitionNumber()); - return Collections.emptyList(); - } - boolean isEndOfPushReceived = partitionConsumptionState.isEndOfPushReceived(); - if (!shouldProduceToVersionTopic(partitionConsumptionState)) { - return records; - } - /** - * Just to note this code is getting executed in Leader only. Leader DIV check progress is always ahead of the - * actual data persisted on disk. Leader DIV check results will not be persisted on disk. - */ - Iterator> iter = records.iterator(); - while (iter.hasNext()) { - PubSubMessage record = iter.next(); - boolean isRealTimeMsg = record.getTopicPartition().getPubSubTopic().isRealTime(); - try { - /** - * TODO: An improvement can be made to fail all future versions for fatal DIV exceptions after EOP. - */ - if (!isGlobalRtDivEnabled) { - validateMessage( - PartitionTracker.VERSION_TOPIC, - this.kafkaDataIntegrityValidatorForLeaders, - record, - isEndOfPushReceived, - partitionConsumptionState); - } else { - validateMessage( - PartitionTracker.TopicType.of( - isRealTimeMsg - ? PartitionTracker.TopicType.REALTIME_TOPIC_TYPE - : PartitionTracker.TopicType.VERSION_TOPIC_TYPE, - kafkaUrl), - this.kafkaDataIntegrityValidatorForLeaders, - record, - isEndOfPushReceived, - partitionConsumptionState); - } - versionedDIVStats.recordSuccessMsg(storeName, versionNumber); - } catch (FatalDataValidationException e) { - if (!isEndOfPushReceived) { - throw e; - } - } catch (DuplicateDataException e) { - /** - * Skip duplicated messages; leader must not produce duplicated messages from RT to VT, because leader will - * override the DIV info for messages from RT; as a result, both leaders and followers will persisted duplicated - * messages to disk, and potentially rewind a k/v pair to an old value. - */ - divErrorMetricCallback.accept(e); - LOGGER.debug( - "Skipping a duplicate record from: {} offset: {} for replica: {}", - record.getTopicPartition(), - record.getOffset(), - partitionConsumptionState.getReplicaId()); - iter.remove(); - } - } - return records; - } + // @Override + // public Iterable> + // validateAndFilterOutDuplicateMessagesFromLeaderTopic( + // Iterable> records, + // String kafkaUrl, + // PubSubTopicPartition topicPartition) { + // PartitionConsumptionState partitionConsumptionState = + // partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); + // if (partitionConsumptionState == null) { + // // The partition is likely unsubscribed, will skip these messages. + // LOGGER.warn( + // "No partition consumption state for store version: {}, partition:{}, will filter out all the messages", + // kafkaVersionTopic, + // topicPartition.getPartitionNumber()); + // return Collections.emptyList(); + // } + // boolean isEndOfPushReceived = partitionConsumptionState.isEndOfPushReceived(); + // if (!shouldProduceToVersionTopic(partitionConsumptionState)) { + // return records; + // } + // /** + // * Just to note this code is getting executed in Leader only. Leader DIV check progress is always ahead of the + // * actual data persisted on disk. Leader DIV check results will not be persisted on disk. + // */ + // Iterator> iter = records.iterator(); + // while (iter.hasNext()) { + // PubSubMessage record = iter.next(); + // boolean isRealTimeMsg = record.getTopicPartition().getPubSubTopic().isRealTime(); + // try { + // /** + // * TODO: An improvement can be made to fail all future versions for fatal DIV exceptions after EOP. + // */ + // if (!isGlobalRtDivEnabled) { + // validateMessage( + // PartitionTracker.VERSION_TOPIC, + // this.kafkaDataIntegrityValidatorForLeaders, + // record, + // isEndOfPushReceived, + // partitionConsumptionState); + // } else { + // validateMessage( + // PartitionTracker.TopicType.of( + // isRealTimeMsg + // ? PartitionTracker.TopicType.REALTIME_TOPIC_TYPE + // : PartitionTracker.TopicType.VERSION_TOPIC_TYPE, + // kafkaUrl), + // this.kafkaDataIntegrityValidatorForLeaders, + // record, + // isEndOfPushReceived, + // partitionConsumptionState); + // } + // versionedDIVStats.recordSuccessMsg(storeName, versionNumber); + // } catch (FatalDataValidationException e) { + // if (!isEndOfPushReceived) { + // throw e; + // } + // } catch (DuplicateDataException e) { + // /** + // * Skip duplicated messages; leader must not produce duplicated messages from RT to VT, because leader will + // * override the DIV info for messages from RT; as a result, both leaders and followers will persisted duplicated + // * messages to disk, and potentially rewind a k/v pair to an old value. + // */ + // divErrorMetricCallback.accept(e); + // LOGGER.debug( + // "Skipping a duplicate record from: {} offset: {} for replica: {}", + // record.getTopicPartition(), + // record.getOffset(), + // partitionConsumptionState.getReplicaId()); + // iter.remove(); + // } + // } + // return records; + // } /** * The goal of this function is to possibly produce the incoming kafka message consumed from local VT, remote VT, RT or SR topic to @@ -4026,4 +4024,9 @@ private void maybeQueueCMWritesToVersionTopic( produceCall.run(); } } + + @Override + public KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders() { + return kafkaDataIntegrityValidatorForLeaders; + } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index ec60734e829..60f0b164df7 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -1128,11 +1128,12 @@ protected void produceToStoreBufferService( } } - public abstract Iterable> validateAndFilterOutDuplicateMessagesFromLeaderTopic( - Iterable> records, - String kafkaUrl, - PubSubTopicPartition topicPartition); - + // public abstract Iterable> + // validateAndFilterOutDuplicateMessagesFromLeaderTopic( + // Iterable> records, + // String kafkaUrl, + // PubSubTopicPartition topicPartition); + // // public int handleSingleMessage( // PubSubMessageProcessedResultWrapper consumerRecordWrapper, // PubSubTopicPartition topicPartition, @@ -3123,6 +3124,8 @@ protected void processControlMessageForViews( // NoOp } + protected abstract boolean shouldProduceToVersionTopic(PartitionConsumptionState partitionConsumptionState); + protected boolean processTopicSwitch( ControlMessage controlMessage, int partition, @@ -4694,6 +4697,16 @@ public SparseConcurrentList getDeserializedSchemaIds() { return deserializedSchemaIds; } + public boolean isGlobalRtDivEnabled() { + return isGlobalRtDivEnabled; + } + + public Consumer getDivErrorMetricCallback() { + return divErrorMetricCallback; + } + + public abstract KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders(); + // For unit test purpose. void setVersionRole(PartitionReplicaIngestionContext.VersionRole versionRole) { this.versionRole = versionRole; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 26fc82fe79c..07073f7be5c 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -4,8 +4,12 @@ import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; import com.linkedin.davinci.stats.HostLevelIngestionStats; import com.linkedin.davinci.utils.ByteArrayKey; +import com.linkedin.davinci.validation.PartitionTracker; +import com.linkedin.davinci.validation.PartitionTracker.TopicType; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceMessageException; +import com.linkedin.venice.exceptions.validation.DuplicateDataException; +import com.linkedin.venice.exceptions.validation.FatalDataValidationException; import com.linkedin.venice.kafka.protocol.ControlMessage; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.Put; @@ -23,6 +27,7 @@ import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.ValueHolder; import java.util.ArrayList; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.NavigableMap; @@ -121,8 +126,7 @@ protected void produceToStoreBufferServiceOrKafka( * Validate and filter out duplicate messages from the real-time topic as early as possible, so that * the following batch processing logic won't spend useless efforts on duplicate messages. */ - records = - storeIngestionTask.validateAndFilterOutDuplicateMessagesFromLeaderTopic(records, kafkaUrl, topicPartition); + records = validateAndFilterOutDuplicateMessagesFromLeaderTopic(records, kafkaUrl, topicPartition); if (storeIngestionTask.shouldProduceInBatch(records)) { produceToStoreBufferServiceOrKafkaInBatch( @@ -283,6 +287,74 @@ private void updateMetricsAndEnforceQuota( } } + public Iterable> validateAndFilterOutDuplicateMessagesFromLeaderTopic( + Iterable> records, + String kafkaUrl, + PubSubTopicPartition topicPartition) { + PartitionConsumptionState partitionConsumptionState = + storeIngestionTask.getPartitionConsumptionState(topicPartition.getPartitionNumber()); + if (partitionConsumptionState == null) { + // The partition is likely unsubscribed, will skip these messages. + LOGGER.warn( + "No partition consumption state for store version: {}, partition:{}, will filter out all the messages", + storeIngestionTask.getKafkaVersionTopic(), + topicPartition.getPartitionNumber()); + return Collections.emptyList(); + } + boolean isEndOfPushReceived = partitionConsumptionState.isEndOfPushReceived(); + if (!storeIngestionTask.shouldProduceToVersionTopic(partitionConsumptionState)) { + return records; + } + /** + * Just to note this code is getting executed in Leader only. Leader DIV check progress is always ahead of the + * actual data persisted on disk. Leader DIV check results will not be persisted on disk. + */ + Iterator> iter = records.iterator(); + while (iter.hasNext()) { + PubSubMessage record = iter.next(); + boolean isRealTimeMsg = record.getTopicPartition().getPubSubTopic().isRealTime(); + try { + /** + * TODO: An improvement can be made to fail all future versions for fatal DIV exceptions after EOP. + */ + TopicType topicType; + if (storeIngestionTask.isGlobalRtDivEnabled()) { + final int topicTypeEnumCode = isRealTimeMsg ? TopicType.REALTIME_TOPIC_TYPE : TopicType.VERSION_TOPIC_TYPE; + topicType = TopicType.of(topicTypeEnumCode, kafkaUrl); + } else { + topicType = PartitionTracker.VERSION_TOPIC; + } + + storeIngestionTask.validateMessage( + topicType, + storeIngestionTask.getKafkaDataIntegrityValidatorForLeaders(), + record, + isEndOfPushReceived, + partitionConsumptionState); + storeIngestionTask.getVersionedDIVStats() + .recordSuccessMsg(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); + } catch (FatalDataValidationException e) { + if (!isEndOfPushReceived) { + throw e; + } + } catch (DuplicateDataException e) { + /** + * Skip duplicated messages; leader must not produce duplicated messages from RT to VT, because leader will + * override the DIV info for messages from RT; as a result, both leaders and followers will persisted duplicated + * messages to disk, and potentially rewind a k/v pair to an old value. + */ + storeIngestionTask.getDivErrorMetricCallback().accept(e); + LOGGER.debug( + "Skipping a duplicate record from: {} offset: {} for replica: {}", + record.getTopicPartition(), + record.getOffset(), + partitionConsumptionState.getReplicaId()); + iter.remove(); + } + } + return records; + } + /** * This method checks whether the given record needs to be checked schema availability. Only PUT and UPDATE message * needs to #checkValueSchemaAvail From 6eea45c66ea9e61841d8d04eb4128940f2b29fdb Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Wed, 5 Feb 2025 13:36:16 -0800 Subject: [PATCH 06/32] =?UTF-8?q?Refactored=20delegateConsumerRecord()=20(?= =?UTF-8?q?which=20was=20an=20absolute=20behemoth)=20from=20LeaderFollower?= =?UTF-8?q?StoreIngestionTask=20/=20ActiveActiveStoreIngestionTask=20into?= =?UTF-8?q?=20StorePartitionDataReceiver.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 106 +-- .../LeaderFollowerStoreIngestionTask.java | 792 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 126 ++- .../consumer/StorePartitionDataReceiver.java | 482 ++++++++++- .../LeaderFollowerStoreIngestionTaskTest.java | 8 +- 5 files changed, 1030 insertions(+), 484 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index adb5dd90324..7c85d5ee657 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -22,7 +22,6 @@ import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; import com.linkedin.davinci.store.record.ByteBufferValueRecord; import com.linkedin.davinci.store.record.ValueRecord; -import com.linkedin.davinci.utils.ByteArrayKey; import com.linkedin.venice.exceptions.PersistenceFailureException; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceMessageException; @@ -67,7 +66,6 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; -import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; import org.apache.avro.generic.GenericRecord; @@ -193,54 +191,57 @@ public static int getKeyLevelLockMaxPoolSizeBasedOnServerConfig(VeniceServerConf * serverConfig.getKafkaClusterIdToUrlMap().size() * multiplier + 1; } - @Override - protected DelegateConsumerRecordResult delegateConsumerRecord( - PubSubMessageProcessedResultWrapper consumerRecordWrapper, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingPerRecordTimestampNs, - long beforeProcessingBatchRecordsTimestampMs) { - if (!consumerRecordWrapper.getMessage().getTopicPartition().getPubSubTopic().isRealTime()) { - /** - * We don't need to lock the partition here because during VT consumption there is only one consumption source. - */ - return super.delegateConsumerRecord( - consumerRecordWrapper, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs); - } else { - /** - * The below flow must be executed in a critical session for the same key: - * Read existing value/RMD from transient record cache/disk -> perform DCR and decide incoming value wins - * -> update transient record cache -> produce to VT (just call send, no need to wait for the produce future in the critical session) - * - * Otherwise, there could be race conditions: - * [fabric A thread]Read from transient record cache -> [fabric A thread]perform DCR and decide incoming value wins - * -> [fabric B thread]read from transient record cache -> [fabric B thread]perform DCR and decide incoming value wins - * -> [fabric B thread]update transient record cache -> [fabric B thread]produce to VT -> [fabric A thread]update transient record cache - * -> [fabric A thread]produce to VT - */ - final ByteArrayKey byteArrayKey = ByteArrayKey.wrap(consumerRecordWrapper.getMessage().getKey().getKey()); - ReentrantLock keyLevelLock = this.keyLevelLocksManager.get().acquireLockByKey(byteArrayKey); - keyLevelLock.lock(); - try { - return super.delegateConsumerRecord( - consumerRecordWrapper, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs); - } finally { - keyLevelLock.unlock(); - this.keyLevelLocksManager.get().releaseLock(byteArrayKey); - } - } - } + // @Override + // protected DelegateConsumerRecordResult delegateConsumerRecord( + // PubSubMessageProcessedResultWrapper consumerRecordWrapper, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingPerRecordTimestampNs, + // long beforeProcessingBatchRecordsTimestampMs) { + // if (!consumerRecordWrapper.getMessage().getTopicPartition().getPubSubTopic().isRealTime()) { + // /** + // * We don't need to lock the partition here because during VT consumption there is only one consumption source. + // */ + // return super.delegateConsumerRecord( + // consumerRecordWrapper, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs, + // beforeProcessingBatchRecordsTimestampMs); + // } else { + // /** + // * The below flow must be executed in a critical session for the same key: + // * Read existing value/RMD from transient record cache/disk -> perform DCR and decide incoming value wins + // * -> update transient record cache -> produce to VT (just call send, no need to wait for the produce future in the + // critical session) + // * + // * Otherwise, there could be race conditions: + // * [fabric A thread]Read from transient record cache -> [fabric A thread]perform DCR and decide incoming value wins + // * -> [fabric B thread]read from transient record cache -> [fabric B thread]perform DCR and decide incoming value + // wins + // * -> [fabric B thread]update transient record cache -> [fabric B thread]produce to VT -> [fabric A thread]update + // transient record cache + // * -> [fabric A thread]produce to VT + // */ + // final ByteArrayKey byteArrayKey = ByteArrayKey.wrap(consumerRecordWrapper.getMessage().getKey().getKey()); + // ReentrantLock keyLevelLock = this.keyLevelLocksManager.get().acquireLockByKey(byteArrayKey); + // keyLevelLock.lock(); + // try { + // return super.delegateConsumerRecord( + // consumerRecordWrapper, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs, + // beforeProcessingBatchRecordsTimestampMs); + // } finally { + // keyLevelLock.unlock(); + // this.keyLevelLocksManager.get().releaseLock(byteArrayKey); + // } + // } + // } @Override protected void putInStorageEngine(int partition, byte[] keyBytes, Put put) { @@ -1443,6 +1444,11 @@ int getRmdProtocolVersionId() { return rmdProtocolVersionId; } + @Override + public final Lazy getKeyLevelLocksManager() { + return keyLevelLocksManager; + } + protected BiConsumer getProduceToTopicFunction( PartitionConsumptionState partitionConsumptionState, byte[] key, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 6aac1e6e31f..a6b1d78b437 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -8,11 +8,9 @@ import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.PAUSE_TRANSITION_FROM_STANDBY_TO_LEADER; import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.STANDBY; import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_PUSH; -import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.START_OF_SEGMENT; import static com.linkedin.venice.kafka.protocol.enums.MessageType.UPDATE; import static com.linkedin.venice.pubsub.api.PubSubMessageHeaders.VENICE_LEADER_COMPLETION_STATE_HEADER; import static com.linkedin.venice.writer.VeniceWriter.APP_DEFAULT_LOGICAL_TS; -import static com.linkedin.venice.writer.VeniceWriter.DEFAULT_LEADER_METADATA_WRAPPER; import static java.lang.Long.max; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -87,7 +85,6 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -97,7 +94,6 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -2067,7 +2063,8 @@ protected final void recordAssembledRecordSizeRatio(double ratio, long currentTi } } - private void recordRegionHybridConsumptionStats( + @Override + protected void recordRegionHybridConsumptionStats( int kafkaClusterId, int producedRecordSize, long upstreamOffset, @@ -2155,6 +2152,7 @@ protected boolean checkAndLogIfLagIsAcceptableForHybridStore( * HeartBeat SOS messages carry the leader completion state in the header. This function extracts the leader completion * state from that header and updates the {@param partitionConsumptionState} accordingly. */ + @Override protected void getAndUpdateLeaderCompletedState( KafkaKey kafkaKey, KafkaMessageEnvelope kafkaValue, @@ -2201,7 +2199,8 @@ protected void getAndUpdateLeaderCompletedState( * colo or remote colo, as the header inherited from an incorrect version or remote colos might * provide incorrect information about the leader state. */ - private void propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( + @Override + protected final void propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( PartitionConsumptionState partitionConsumptionState, PubSubMessage consumerRecord, LeaderProducedRecordContext leaderProducedRecordContext, @@ -2350,367 +2349,386 @@ protected void recordHeartbeatReceived( // } // return records; // } - - /** - * The goal of this function is to possibly produce the incoming kafka message consumed from local VT, remote VT, RT or SR topic to - * local VT if needed. It's decided based on the function output of {@link #shouldProduceToVersionTopic} and message type. - * It also perform any necessary additional computation operation such as for write-compute/update message. - * It returns a boolean indicating if it was produced to kafka or not. - * - * This function should be called as one of the first steps in processing pipeline for all messages consumed from any kafka topic. - * - * The caller of this function should only process this {@param consumerRecord} further if the return is - * {@link DelegateConsumerRecordResult#QUEUED_TO_DRAINER}. - * - * This function assumes {@link #shouldProcessRecord(PubSubMessage)} has been called which happens in - * {@link StorePartitionDataReceiver#produceToStoreBufferServiceOrKafka(Iterable, PubSubTopicPartition, String, int)} - * before calling this and the it was decided that this record needs to be processed. It does not perform any - * validation check on the PartitionConsumptionState object to keep the goal of the function simple and not overload. - * - * Also DIV validation is done here if the message is received from RT topic. For more info please see - * please see {@literal StoreIngestionTask#internalProcessConsumerRecord(PubSubMessage, PartitionConsumptionState, LeaderProducedRecordContext, int, String, long)} - * - * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after this function. - * - * @return a {@link DelegateConsumerRecordResult} indicating what to do with the record - */ - @Override - protected DelegateConsumerRecordResult delegateConsumerRecord( - PubSubMessageProcessedResultWrapper consumerRecordWrapper, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingPerRecordTimestampNs, - long beforeProcessingBatchRecordsTimestampMs) { - PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); - try { - KafkaKey kafkaKey = consumerRecord.getKey(); - KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); - /** - * partitionConsumptionState must be in a valid state and no error reported. This is made sure by calling - * {@link shouldProcessRecord} before processing any record. - * - * ^ This is no longer true because with shared consumer the partitionConsumptionState could have been removed - * from unsubscribe action in the StoreIngestionTask thread. Today, when unsubscribing - * {@link StoreIngestionTask.waitForAllMessageToBeProcessedFromTopicPartition} only ensure the buffer queue is - * drained before unsubscribe. Records being processed by shared consumer may see invalid partitionConsumptionState. - */ - PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(partition); - if (partitionConsumptionState == null) { - // The partition is likely unsubscribed, will skip these messages. - return DelegateConsumerRecordResult.SKIPPED_MESSAGE; - } - boolean produceToLocalKafka = shouldProduceToVersionTopic(partitionConsumptionState); - // UPDATE message is only expected in LEADER which must be produced to kafka. - MessageType msgType = MessageType.valueOf(kafkaValue); - if (msgType == UPDATE && !produceToLocalKafka) { - throw new VeniceMessageException( - ingestionTaskName + " hasProducedToKafka: Received UPDATE message in non-leader for: " - + consumerRecord.getTopicPartition() + " Offset " + consumerRecord.getOffset()); - } else if (msgType == MessageType.CONTROL_MESSAGE) { - ControlMessage controlMessage = (ControlMessage) kafkaValue.payloadUnion; - getAndUpdateLeaderCompletedState( - kafkaKey, - kafkaValue, - controlMessage, - consumerRecord.getPubSubMessageHeaders(), - partitionConsumptionState); - } - - /** - * return early if it needs not be produced to local VT such as cases like - * (i) it's a follower or (ii) leader is consuming from VT - */ - if (!produceToLocalKafka) { - /** - * For the local consumption, the batch data won't be produce to the local VT again, so we will switch - * to real-time writer upon EOP here and for the remote consumption of VT, the switch will be handled - * in the following section as it needs to flush the messages and then switch. - */ - if (isLeader(partitionConsumptionState) && msgType == MessageType.CONTROL_MESSAGE - && ControlMessageType.valueOf((ControlMessage) kafkaValue.payloadUnion).equals(END_OF_PUSH)) { - LOGGER.info( - "Switching to the VeniceWriter for real-time workload for topic: {}, partition: {}", - getVersionTopic().getName(), - partition); - // Just to be extra safe - partitionConsumptionState.getVeniceWriterLazyRef().ifPresent(vw -> vw.flush()); - partitionConsumptionState.setVeniceWriterLazyRef(veniceWriterForRealTime); - } - /** - * Materialized view need to produce to the corresponding view topic for the batch portion of the data. This is - * achieved in the following ways: - * 1. Remote fabric(s) will leverage NR where the leader will replicate VT from NR source fabric and produce - * to local view topic(s). - * 2. NR source fabric's view topic will be produced by VPJ. This is because there is no checkpointing and - * easy way to add checkpointing for leaders consuming the local VT. Making it difficult and error prone if - * we let the leader produce to view topic(s) in NR source fabric. - */ - return DelegateConsumerRecordResult.QUEUED_TO_DRAINER; - } - - // If we are here the message must be produced to local kafka or silently consumed. - LeaderProducedRecordContext leaderProducedRecordContext; - // No need to resolve cluster id and kafka url because sep topics are real time topic and it's not VT - validateRecordBeforeProducingToLocalKafka(consumerRecord, partitionConsumptionState, kafkaUrl, kafkaClusterId); - - if (consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { - recordRegionHybridConsumptionStats( - kafkaClusterId, - consumerRecord.getPayloadSize(), - consumerRecord.getOffset(), - beforeProcessingBatchRecordsTimestampMs); - updateLatestInMemoryLeaderConsumedRTOffset(partitionConsumptionState, kafkaUrl, consumerRecord.getOffset()); - } - - // heavy leader processing starts here - versionedIngestionStats.recordLeaderPreprocessingLatency( - storeName, - versionNumber, - LatencyUtils.getElapsedTimeFromNSToMS(beforeProcessingPerRecordTimestampNs), - beforeProcessingBatchRecordsTimestampMs); - - if (kafkaKey.isControlMessage()) { - boolean producedFinally = true; - ControlMessage controlMessage = (ControlMessage) kafkaValue.getPayloadUnion(); - ControlMessageType controlMessageType = ControlMessageType.valueOf(controlMessage); - leaderProducedRecordContext = LeaderProducedRecordContext - .newControlMessageRecord(kafkaClusterId, consumerRecord.getOffset(), kafkaKey.getKey(), controlMessage); - switch (controlMessageType) { - case START_OF_PUSH: - /** - * N.B.: This is expected to be the first time we call {@link veniceWriter#get()}. During this time - * since startOfPush has not been processed yet, {@link StoreVersionState} is not created yet (unless - * this is a server restart scenario). So the {@link com.linkedin.venice.writer.VeniceWriter#isChunkingEnabled} field - * will not be set correctly at this point. This is fine as chunking is mostly not applicable for control messages. - * This chunking flag for the veniceWriter will happen be set correctly in - * {@link StoreIngestionTask#processStartOfPush(ControlMessage, int, long, PartitionConsumptionState)}, - * which will be called when this message gets processed in drainer thread after successfully producing - * to kafka. - * - * Note update: the first time we call {@link veniceWriter#get()} is different in various use cases: - * 1. For hybrid store with L/F enabled, the first time a VeniceWriter is created is after leader switches to RT and - * consumes the first message; potential message type: SOS, EOS, data message. - * 2. For store version generated by stream reprocessing push type, the first time is after leader switches to - * SR topic and consumes the first message; potential message type: SOS, EOS, data message (consider server restart). - * 3. For store with native replication enabled, the first time is after leader switches to remote topic and start - * consumes the first message; potential message type: SOS, EOS, SOP, EOP, data message (consider server restart). - */ - case END_OF_PUSH: - // CMs that are produced with DIV pass-through mode can break DIV without synchronization with view writers. - // This is because for data (PUT) records we queue their produceToLocalKafka behind the completion of view - // writers. The main SIT will move on to subsequent messages and for CMs that don't need to be propagated - // to view topics we are producing them directly. If we don't check the previous write before producing the - // CMs then in the VT we might get out of order messages and with pass-through DIV that's going to be an - // issue. e.g. a PUT record belonging to seg:0 can come after the EOS of seg:0 due to view writer delays. - // Since SOP and EOP are rare we can simply wait for the last VT produce future. - checkAndWaitForLastVTProduceFuture(partitionConsumptionState); - /** - * Simply produce this EOP to local VT. It will be processed in order in the drainer queue later - * after successfully producing to kafka. - */ - produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() - .get() - .put( - consumerRecord.getKey(), - consumerRecord.getValue(), - callback, - consumerRecord.getTopicPartition().getPartitionNumber(), - leaderMetadataWrapper), - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs); - partitionConsumptionState.getVeniceWriterLazyRef().get().flush(); - // Switch the writer for real-time workload - LOGGER.info( - "Switching to the VeniceWriter for real-time workload for topic: {}, partition: {}", - getVersionTopic().getName(), - partition); - partitionConsumptionState.setVeniceWriterLazyRef(veniceWriterForRealTime); - break; - case START_OF_SEGMENT: - case END_OF_SEGMENT: - /** - * SOS and EOS will be produced to the local version topic with DIV pass-through mode by leader in the following cases: - * 1. SOS and EOS are from stream-reprocessing topics (use cases: stream-reprocessing) - * 2. SOS and EOS are from version topics in a remote fabric (use cases: native replication for remote fabrics) - * - * SOS and EOS will not be produced to local version topic in the following cases: - * 1. SOS and EOS are from real-time topics (use cases: hybrid ingestion, incremental push to RT) - * 2. SOS and EOS are from version topics in local fabric, which has 2 different scenarios: - * i. native replication is enabled, but the current fabric is the source fabric (use cases: native repl for source fabric) - * ii. native replication is not enabled; it doesn't matter whether current replica is leader or follower, - * messages from local VT doesn't need to be reproduced into local VT again (use case: local batch consumption) - * - * There is one exception that overrules the above conditions. i.e. if the SOS is a heartbeat from the RT topic. - * In such case the heartbeat is produced to VT with updated {@link LeaderMetadataWrapper}. - * - * We want to ensure correct ordering for any SOS and EOS that we do decide to write to VT. This is done by - * coordinating with the corresponding {@link PartitionConsumptionState#getLastVTProduceCallFuture}. - * However, this coordination is only needed if there are view writers. i.e. the VT writes and CM writes - * need to be in the same mode. Either both coordinate with lastVTProduceCallFuture or neither. - */ - if (!consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { - final LeaderProducedRecordContext segmentCMLeaderProduceRecordContext = leaderProducedRecordContext; - maybeQueueCMWritesToVersionTopic( - partitionConsumptionState, - () -> produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - segmentCMLeaderProduceRecordContext, - (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() - .get() - .put( - consumerRecord.getKey(), - consumerRecord.getValue(), - callback, - consumerRecord.getTopicPartition().getPartitionNumber(), - leaderMetadataWrapper), - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs)); - } else { - if (controlMessageType == START_OF_SEGMENT - && Arrays.equals(consumerRecord.getKey().getKey(), KafkaKey.HEART_BEAT.getKey())) { - final LeaderProducedRecordContext heartbeatLeaderProducedRecordContext = leaderProducedRecordContext; - maybeQueueCMWritesToVersionTopic( - partitionConsumptionState, - () -> propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( - partitionConsumptionState, - consumerRecord, - heartbeatLeaderProducedRecordContext, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs)); - } else { - /** - * Based on current design handling this case (specially EOS) is tricky as we don't produce the SOS/EOS - * received from RT to local VT. But ideally EOS must be queued in-order (after all previous data message - * PUT/UPDATE/DELETE) to drainer. But since the queueing of data message to drainer - * happens in Kafka producer callback there is no way to queue this EOS to drainer in-order. - * - * Usually following processing in Leader for other control message. - * 1. DIV: - * 2. updateOffset: - * 3. stats maintenance as in {@link StoreIngestionTask#processKafkaDataMessage(PubSubMessage, PartitionConsumptionState, LeaderProducedRecordContext)} - * - * For #1 Since we have moved the DIV validation in this function, We are good with DIV part which is the most critical one. - * For #2 Leader will not update the offset for SOS/EOS. From Server restart point of view this is tolerable. This was the case in previous design also. So there is no change in behaviour. - * For #3 stat counter update will not happen for SOS/EOS message. This should not be a big issue. If needed we can copy some of the stats maintenance - * work here. - * - * So in summary NO further processing is needed SOS/EOS received from RT topics. Just silently drop the message here. - * We should not return false here. - */ - producedFinally = false; - } - } - break; - case START_OF_INCREMENTAL_PUSH: - case END_OF_INCREMENTAL_PUSH: - // For inc push to RT policy, the control msg is written in RT topic, we will need to calculate the - // destination partition in VT correctly. - int versionTopicPartitionToBeProduced = consumerRecord.getTopicPartition().getPartitionNumber(); - /** - * We are using {@link VeniceWriter#asyncSendControlMessage()} api instead of {@link VeniceWriter#put()} since we have - * to calculate DIV for this message but keeping the ControlMessage content unchanged. {@link VeniceWriter#put()} does not - * allow that. - */ - produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() - .get() - .asyncSendControlMessage( - controlMessage, - versionTopicPartitionToBeProduced, - new HashMap<>(), - callback, - leaderMetadataWrapper), - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs); - break; - case TOPIC_SWITCH: - /** - * For TOPIC_SWITCH message we should use -1 as consumedOffset. This will ensure that it does not update the - * setLeaderUpstreamOffset in: - * {@link #updateOffsetsAsRemoteConsumeLeader(PartitionConsumptionState, LeaderProducedRecordContext, String, PubSubMessage, UpdateVersionTopicOffset, UpdateUpstreamTopicOffset)} - * The leaderUpstreamOffset is set from the TS message config itself. We should not override it. - */ - if (isDataRecovery && !partitionConsumptionState.isBatchOnly()) { - // Ignore remote VT's TS message since we might need to consume more RT or incremental push data from VT - // that's no longer in the local/remote RT due to retention. - return DelegateConsumerRecordResult.SKIPPED_MESSAGE; - } - leaderProducedRecordContext = - LeaderProducedRecordContext.newControlMessageRecord(kafkaKey.getKey(), controlMessage); - produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() - .get() - .asyncSendControlMessage( - controlMessage, - consumerRecord.getTopicPartition().getPartitionNumber(), - new HashMap<>(), - callback, - DEFAULT_LEADER_METADATA_WRAPPER), - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs); - break; - case VERSION_SWAP: - return DelegateConsumerRecordResult.QUEUED_TO_DRAINER; - default: - // do nothing - break; - } - if (!isSegmentControlMsg(controlMessageType)) { - LOGGER.info( - "Replica: {} hasProducedToKafka: {}; ControlMessage: {}; Incoming record topic-partition: {}; offset: {}", - partitionConsumptionState.getReplicaId(), - producedFinally, - controlMessageType.name(), - consumerRecord.getTopicPartition(), - consumerRecord.getOffset()); - } - } else if (kafkaValue == null) { - throw new VeniceMessageException( - partitionConsumptionState.getReplicaId() - + " hasProducedToKafka: Given null Venice Message. TopicPartition: " - + consumerRecord.getTopicPartition() + " Offset " + consumerRecord.getOffset()); - } else { - // This function may modify the original record in KME and it is unsafe to use the payload from KME directly - // after this call. - processMessageAndMaybeProduceToKafka( - consumerRecordWrapper, - partitionConsumptionState, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs); - } - return DelegateConsumerRecordResult.PRODUCED_TO_KAFKA; - } catch (Exception e) { - throw new VeniceException( - ingestionTaskName + " hasProducedToKafka: exception for message received from: " - + consumerRecord.getTopicPartition() + ", Offset: " + consumerRecord.getOffset() + ". Bubbling up.", - e); - } - } + // + // /** + // * The goal of this function is to possibly produce the incoming kafka message consumed from local VT, remote VT, RT + // or SR topic to + // * local VT if needed. It's decided based on the function output of {@link #shouldProduceToVersionTopic} and message + // type. + // * It also perform any necessary additional computation operation such as for write-compute/update message. + // * It returns a boolean indicating if it was produced to kafka or not. + // * + // * This function should be called as one of the first steps in processing pipeline for all messages consumed from + // any kafka topic. + // * + // * The caller of this function should only process this {@param consumerRecord} further if the return is + // * {@link DelegateConsumerRecordResult#QUEUED_TO_DRAINER}. + // * + // * This function assumes {@link #shouldProcessRecord(PubSubMessage)} has been called which happens in + // * {@link StorePartitionDataReceiver#produceToStoreBufferServiceOrKafka(Iterable, PubSubTopicPartition, String, + // int)} + // * before calling this and the it was decided that this record needs to be processed. It does not perform any + // * validation check on the PartitionConsumptionState object to keep the goal of the function simple and not + // overload. + // * + // * Also DIV validation is done here if the message is received from RT topic. For more info please see + // * please see {@literal StoreIngestionTask#internalProcessConsumerRecord(PubSubMessage, PartitionConsumptionState, + // LeaderProducedRecordContext, int, String, long)} + // * + // * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after + // this function. + // * + // * @return a {@link DelegateConsumerRecordResult} indicating what to do with the record + // */ + // @Override + // protected DelegateConsumerRecordResult delegateConsumerRecord( + // PubSubMessageProcessedResultWrapper consumerRecordWrapper, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingPerRecordTimestampNs, + // long beforeProcessingBatchRecordsTimestampMs) { + // PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); + // try { + // KafkaKey kafkaKey = consumerRecord.getKey(); + // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + // /** + // * partitionConsumptionState must be in a valid state and no error reported. This is made sure by calling + // * {@link shouldProcessRecord} before processing any record. + // * + // * ^ This is no longer true because with shared consumer the partitionConsumptionState could have been removed + // * from unsubscribe action in the StoreIngestionTask thread. Today, when unsubscribing + // * {@link StoreIngestionTask.waitForAllMessageToBeProcessedFromTopicPartition} only ensure the buffer queue is + // * drained before unsubscribe. Records being processed by shared consumer may see invalid partitionConsumptionState. + // */ + // PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(partition); + // if (partitionConsumptionState == null) { + // // The partition is likely unsubscribed, will skip these messages. + // return DelegateConsumerRecordResult.SKIPPED_MESSAGE; + // } + // boolean produceToLocalKafka = shouldProduceToVersionTopic(partitionConsumptionState); + // // UPDATE message is only expected in LEADER which must be produced to kafka. + // MessageType msgType = MessageType.valueOf(kafkaValue); + // if (msgType == UPDATE && !produceToLocalKafka) { + // throw new VeniceMessageException( + // ingestionTaskName + " hasProducedToKafka: Received UPDATE message in non-leader for: " + // + consumerRecord.getTopicPartition() + " Offset " + consumerRecord.getOffset()); + // } else if (msgType == MessageType.CONTROL_MESSAGE) { + // ControlMessage controlMessage = (ControlMessage) kafkaValue.payloadUnion; + // getAndUpdateLeaderCompletedState( + // kafkaKey, + // kafkaValue, + // controlMessage, + // consumerRecord.getPubSubMessageHeaders(), + // partitionConsumptionState); + // } + // + // /** + // * return early if it needs not be produced to local VT such as cases like + // * (i) it's a follower or (ii) leader is consuming from VT + // */ + // if (!produceToLocalKafka) { + // /** + // * For the local consumption, the batch data won't be produce to the local VT again, so we will switch + // * to real-time writer upon EOP here and for the remote consumption of VT, the switch will be handled + // * in the following section as it needs to flush the messages and then switch. + // */ + // if (isLeader(partitionConsumptionState) && msgType == MessageType.CONTROL_MESSAGE + // && ControlMessageType.valueOf((ControlMessage) kafkaValue.payloadUnion).equals(END_OF_PUSH)) { + // LOGGER.info( + // "Switching to the VeniceWriter for real-time workload for topic: {}, partition: {}", + // getVersionTopic().getName(), + // partition); + // // Just to be extra safe + // partitionConsumptionState.getVeniceWriterLazyRef().ifPresent(vw -> vw.flush()); + // partitionConsumptionState.setVeniceWriterLazyRef(veniceWriterForRealTime); + // } + // /** + // * Materialized view need to produce to the corresponding view topic for the batch portion of the data. This is + // * achieved in the following ways: + // * 1. Remote fabric(s) will leverage NR where the leader will replicate VT from NR source fabric and produce + // * to local view topic(s). + // * 2. NR source fabric's view topic will be produced by VPJ. This is because there is no checkpointing and + // * easy way to add checkpointing for leaders consuming the local VT. Making it difficult and error prone if + // * we let the leader produce to view topic(s) in NR source fabric. + // */ + // return DelegateConsumerRecordResult.QUEUED_TO_DRAINER; + // } + // + // // If we are here the message must be produced to local kafka or silently consumed. + // LeaderProducedRecordContext leaderProducedRecordContext; + // // No need to resolve cluster id and kafka url because sep topics are real time topic and it's not VT + // validateRecordBeforeProducingToLocalKafka(consumerRecord, partitionConsumptionState, kafkaUrl, kafkaClusterId); + // + // if (consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { + // recordRegionHybridConsumptionStats( + // kafkaClusterId, + // consumerRecord.getPayloadSize(), + // consumerRecord.getOffset(), + // beforeProcessingBatchRecordsTimestampMs); + // updateLatestInMemoryLeaderConsumedRTOffset(partitionConsumptionState, kafkaUrl, consumerRecord.getOffset()); + // } + // + // // heavy leader processing starts here + // versionedIngestionStats.recordLeaderPreprocessingLatency( + // storeName, + // versionNumber, + // LatencyUtils.getElapsedTimeFromNSToMS(beforeProcessingPerRecordTimestampNs), + // beforeProcessingBatchRecordsTimestampMs); + // + // if (kafkaKey.isControlMessage()) { + // boolean producedFinally = true; + // ControlMessage controlMessage = (ControlMessage) kafkaValue.getPayloadUnion(); + // ControlMessageType controlMessageType = ControlMessageType.valueOf(controlMessage); + // leaderProducedRecordContext = LeaderProducedRecordContext + // .newControlMessageRecord(kafkaClusterId, consumerRecord.getOffset(), kafkaKey.getKey(), controlMessage); + // switch (controlMessageType) { + // case START_OF_PUSH: + // /** + // * N.B.: This is expected to be the first time we call {@link veniceWriter#get()}. During this time + // * since startOfPush has not been processed yet, {@link StoreVersionState} is not created yet (unless + // * this is a server restart scenario). So the {@link com.linkedin.venice.writer.VeniceWriter#isChunkingEnabled} + // field + // * will not be set correctly at this point. This is fine as chunking is mostly not applicable for control messages. + // * This chunking flag for the veniceWriter will happen be set correctly in + // * {@link StoreIngestionTask#processStartOfPush(ControlMessage, int, long, PartitionConsumptionState)}, + // * which will be called when this message gets processed in drainer thread after successfully producing + // * to kafka. + // * + // * Note update: the first time we call {@link veniceWriter#get()} is different in various use cases: + // * 1. For hybrid store with L/F enabled, the first time a VeniceWriter is created is after leader switches to RT and + // * consumes the first message; potential message type: SOS, EOS, data message. + // * 2. For store version generated by stream reprocessing push type, the first time is after leader switches to + // * SR topic and consumes the first message; potential message type: SOS, EOS, data message (consider server + // restart). + // * 3. For store with native replication enabled, the first time is after leader switches to remote topic and start + // * consumes the first message; potential message type: SOS, EOS, SOP, EOP, data message (consider server restart). + // */ + // case END_OF_PUSH: + // // CMs that are produced with DIV pass-through mode can break DIV without synchronization with view writers. + // // This is because for data (PUT) records we queue their produceToLocalKafka behind the completion of view + // // writers. The main SIT will move on to subsequent messages and for CMs that don't need to be propagated + // // to view topics we are producing them directly. If we don't check the previous write before producing the + // // CMs then in the VT we might get out of order messages and with pass-through DIV that's going to be an + // // issue. e.g. a PUT record belonging to seg:0 can come after the EOS of seg:0 due to view writer delays. + // // Since SOP and EOP are rare we can simply wait for the last VT produce future. + // checkAndWaitForLastVTProduceFuture(partitionConsumptionState); + // /** + // * Simply produce this EOP to local VT. It will be processed in order in the drainer queue later + // * after successfully producing to kafka. + // */ + // produceToLocalKafka( + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() + // .get() + // .put( + // consumerRecord.getKey(), + // consumerRecord.getValue(), + // callback, + // consumerRecord.getTopicPartition().getPartitionNumber(), + // leaderMetadataWrapper), + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs); + // partitionConsumptionState.getVeniceWriterLazyRef().get().flush(); + // // Switch the writer for real-time workload + // LOGGER.info( + // "Switching to the VeniceWriter for real-time workload for topic: {}, partition: {}", + // getVersionTopic().getName(), + // partition); + // partitionConsumptionState.setVeniceWriterLazyRef(veniceWriterForRealTime); + // break; + // case START_OF_SEGMENT: + // case END_OF_SEGMENT: + // /** + // * SOS and EOS will be produced to the local version topic with DIV pass-through mode by leader in the following + // cases: + // * 1. SOS and EOS are from stream-reprocessing topics (use cases: stream-reprocessing) + // * 2. SOS and EOS are from version topics in a remote fabric (use cases: native replication for remote fabrics) + // * + // * SOS and EOS will not be produced to local version topic in the following cases: + // * 1. SOS and EOS are from real-time topics (use cases: hybrid ingestion, incremental push to RT) + // * 2. SOS and EOS are from version topics in local fabric, which has 2 different scenarios: + // * i. native replication is enabled, but the current fabric is the source fabric (use cases: native repl for source + // fabric) + // * ii. native replication is not enabled; it doesn't matter whether current replica is leader or follower, + // * messages from local VT doesn't need to be reproduced into local VT again (use case: local batch consumption) + // * + // * There is one exception that overrules the above conditions. i.e. if the SOS is a heartbeat from the RT topic. + // * In such case the heartbeat is produced to VT with updated {@link LeaderMetadataWrapper}. + // * + // * We want to ensure correct ordering for any SOS and EOS that we do decide to write to VT. This is done by + // * coordinating with the corresponding {@link PartitionConsumptionState#getLastVTProduceCallFuture}. + // * However, this coordination is only needed if there are view writers. i.e. the VT writes and CM writes + // * need to be in the same mode. Either both coordinate with lastVTProduceCallFuture or neither. + // */ + // if (!consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { + // final LeaderProducedRecordContext segmentCMLeaderProduceRecordContext = leaderProducedRecordContext; + // maybeQueueCMWritesToVersionTopic( + // partitionConsumptionState, + // () -> produceToLocalKafka( + // consumerRecord, + // partitionConsumptionState, + // segmentCMLeaderProduceRecordContext, + // (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() + // .get() + // .put( + // consumerRecord.getKey(), + // consumerRecord.getValue(), + // callback, + // consumerRecord.getTopicPartition().getPartitionNumber(), + // leaderMetadataWrapper), + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs)); + // } else { + // if (controlMessageType == START_OF_SEGMENT + // && Arrays.equals(consumerRecord.getKey().getKey(), KafkaKey.HEART_BEAT.getKey())) { + // final LeaderProducedRecordContext heartbeatLeaderProducedRecordContext = leaderProducedRecordContext; + // maybeQueueCMWritesToVersionTopic( + // partitionConsumptionState, + // () -> propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( + // partitionConsumptionState, + // consumerRecord, + // heartbeatLeaderProducedRecordContext, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs)); + // } else { + // /** + // * Based on current design handling this case (specially EOS) is tricky as we don't produce the SOS/EOS + // * received from RT to local VT. But ideally EOS must be queued in-order (after all previous data message + // * PUT/UPDATE/DELETE) to drainer. But since the queueing of data message to drainer + // * happens in Kafka producer callback there is no way to queue this EOS to drainer in-order. + // * + // * Usually following processing in Leader for other control message. + // * 1. DIV: + // * 2. updateOffset: + // * 3. stats maintenance as in {@link StoreIngestionTask#processKafkaDataMessage(PubSubMessage, + // PartitionConsumptionState, LeaderProducedRecordContext)} + // * + // * For #1 Since we have moved the DIV validation in this function, We are good with DIV part which is the most + // critical one. + // * For #2 Leader will not update the offset for SOS/EOS. From Server restart point of view this is tolerable. This + // was the case in previous design also. So there is no change in behaviour. + // * For #3 stat counter update will not happen for SOS/EOS message. This should not be a big issue. If needed we can + // copy some of the stats maintenance + // * work here. + // * + // * So in summary NO further processing is needed SOS/EOS received from RT topics. Just silently drop the message + // here. + // * We should not return false here. + // */ + // producedFinally = false; + // } + // } + // break; + // case START_OF_INCREMENTAL_PUSH: + // case END_OF_INCREMENTAL_PUSH: + // // For inc push to RT policy, the control msg is written in RT topic, we will need to calculate the + // // destination partition in VT correctly. + // int versionTopicPartitionToBeProduced = consumerRecord.getTopicPartition().getPartitionNumber(); + // /** + // * We are using {@link VeniceWriter#asyncSendControlMessage()} api instead of {@link VeniceWriter#put()} since we + // have + // * to calculate DIV for this message but keeping the ControlMessage content unchanged. {@link VeniceWriter#put()} + // does not + // * allow that. + // */ + // produceToLocalKafka( + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() + // .get() + // .asyncSendControlMessage( + // controlMessage, + // versionTopicPartitionToBeProduced, + // new HashMap<>(), + // callback, + // leaderMetadataWrapper), + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs); + // break; + // case TOPIC_SWITCH: + // /** + // * For TOPIC_SWITCH message we should use -1 as consumedOffset. This will ensure that it does not update the + // * setLeaderUpstreamOffset in: + // * {@link #updateOffsetsAsRemoteConsumeLeader(PartitionConsumptionState, LeaderProducedRecordContext, String, + // PubSubMessage, UpdateVersionTopicOffset, UpdateUpstreamTopicOffset)} + // * The leaderUpstreamOffset is set from the TS message config itself. We should not override it. + // */ + // if (isDataRecovery && !partitionConsumptionState.isBatchOnly()) { + // // Ignore remote VT's TS message since we might need to consume more RT or incremental push data from VT + // // that's no longer in the local/remote RT due to retention. + // return DelegateConsumerRecordResult.SKIPPED_MESSAGE; + // } + // leaderProducedRecordContext = + // LeaderProducedRecordContext.newControlMessageRecord(kafkaKey.getKey(), controlMessage); + // produceToLocalKafka( + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() + // .get() + // .asyncSendControlMessage( + // controlMessage, + // consumerRecord.getTopicPartition().getPartitionNumber(), + // new HashMap<>(), + // callback, + // DEFAULT_LEADER_METADATA_WRAPPER), + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs); + // break; + // case VERSION_SWAP: + // return DelegateConsumerRecordResult.QUEUED_TO_DRAINER; + // default: + // // do nothing + // break; + // } + // if (!isSegmentControlMsg(controlMessageType)) { + // LOGGER.info( + // "Replica: {} hasProducedToKafka: {}; ControlMessage: {}; Incoming record topic-partition: {}; offset: {}", + // partitionConsumptionState.getReplicaId(), + // producedFinally, + // controlMessageType.name(), + // consumerRecord.getTopicPartition(), + // consumerRecord.getOffset()); + // } + // } else if (kafkaValue == null) { + // throw new VeniceMessageException( + // partitionConsumptionState.getReplicaId() + // + " hasProducedToKafka: Given null Venice Message. TopicPartition: " + // + consumerRecord.getTopicPartition() + " Offset " + consumerRecord.getOffset()); + // } else { + // // This function may modify the original record in KME and it is unsafe to use the payload from KME directly + // // after this call. + // processMessageAndMaybeProduceToKafka( + // consumerRecordWrapper, + // partitionConsumptionState, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingPerRecordTimestampNs, + // beforeProcessingBatchRecordsTimestampMs); + // } + // return DelegateConsumerRecordResult.PRODUCED_TO_KAFKA; + // } catch (Exception e) { + // throw new VeniceException( + // ingestionTaskName + " hasProducedToKafka: exception for message received from: " + // + consumerRecord.getTopicPartition() + ", Offset: " + consumerRecord.getOffset() + ". Bubbling up.", + // e); + // } + // } /** * Besides draining messages in the drainer queue, wait for the last producer future. @@ -2801,7 +2819,8 @@ protected void waitForAllMessageToBeProcessedFromTopicPartition( * * Extend this function when there is new check needed. */ - private void validateRecordBeforeProducingToLocalKafka( + @Override + protected void validateRecordBeforeProducingToLocalKafka( PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, String kafkaUrl, @@ -3056,6 +3075,7 @@ protected long getLatestConsumedUpstreamOffsetForHybridOffsetLagMeasurement( return pcs.getLeaderConsumedUpstreamRTOffset(OffsetRecord.NON_AA_REPLICATION_UPSTREAM_OFFSET_MAP_KEY); } + @Override protected void updateLatestInMemoryLeaderConsumedRTOffset( PartitionConsumptionState pcs, String ignoredKafkaUrl, @@ -3333,6 +3353,7 @@ private PubSubMessageProcessedResult processMessage( } } + @Override protected void processMessageAndMaybeProduceToKafka( PubSubMessageProcessedResultWrapper consumerRecordWrapper, PartitionConsumptionState partitionConsumptionState, @@ -3705,6 +3726,10 @@ protected Lazy> getVeniceWriter( return partitionConsumptionState.getVeniceWriterLazyRef(); } + protected void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState) { + partitionConsumptionState.setVeniceWriterLazyRef(veniceWriterForRealTime); + } + // test method protected void addPartitionConsumptionState(Integer partition, PartitionConsumptionState pcs) { partitionConsumptionStateMap.put(partition, pcs); @@ -3995,38 +4020,17 @@ Set getKafkaUrlSetFromTopicSwitch(TopicSwitchWrapper topicSwitchWrapper) return topicSwitchWrapper.getSourceServers(); } - private void checkAndWaitForLastVTProduceFuture(PartitionConsumptionState partitionConsumptionState) - throws ExecutionException, InterruptedException { - partitionConsumptionState.getLastVTProduceCallFuture().get(); - } - protected boolean hasViewWriters() { return viewWriters != null && !viewWriters.isEmpty(); } - private void maybeQueueCMWritesToVersionTopic( - PartitionConsumptionState partitionConsumptionState, - Runnable produceCall) { - if (hasViewWriters()) { - CompletableFuture propagateSegmentCMWrite = new CompletableFuture<>(); - partitionConsumptionState.getLastVTProduceCallFuture().whenCompleteAsync((value, exception) -> { - if (exception == null) { - produceCall.run(); - propagateSegmentCMWrite.complete(null); - } else { - VeniceException veniceException = new VeniceException(exception); - this.setIngestionException(partitionConsumptionState.getPartition(), veniceException); - propagateSegmentCMWrite.completeExceptionally(veniceException); - } - }); - partitionConsumptionState.setLastVTProduceCallFuture(propagateSegmentCMWrite); - } else { - produceCall.run(); - } - } - @Override public KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders() { return kafkaDataIntegrityValidatorForLeaders; } + + @Override + public Lazy getKeyLevelLocksManager() { + throw new VeniceException("getKeyLevelLocksManager() should only be called in active-active replication mode"); + } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 60f0b164df7..7619fe65b1d 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -86,6 +86,7 @@ import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubMessage; +import com.linkedin.venice.pubsub.api.PubSubMessageHeaders; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.pubsub.api.exceptions.PubSubUnsubscribedTopicPartitionException; @@ -115,6 +116,8 @@ import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import com.linkedin.venice.utils.lazy.Lazy; +import com.linkedin.venice.writer.ChunkAwareCallback; +import com.linkedin.venice.writer.LeaderMetadataWrapper; import it.unimi.dsi.fastutil.objects.Object2IntMap; import java.io.Closeable; import java.io.IOException; @@ -145,6 +148,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; @@ -3177,7 +3181,7 @@ private boolean processControlMessage( case END_OF_SEGMENT: case VERSION_SWAP: /** - * Nothing to do here as all the processing is being done in {@link StoreIngestionTask#delegateConsumerRecord(ConsumerRecord, int, String)}. + * Nothing to do here as all the processing is being done in {@link StorePartitionDataReceiver#delegateConsumerRecord}. */ break; case START_OF_INCREMENTAL_PUSH: @@ -4423,34 +4427,36 @@ protected void waitForAllMessageToBeProcessedFromTopicPartition( storeBufferService.drainBufferedRecordsFromTopicPartition(topicPartition); } - protected abstract DelegateConsumerRecordResult delegateConsumerRecord( - PubSubMessageProcessedResultWrapper consumerRecordWrapper, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingPerRecordTimestampNs, - long beforeProcessingBatchRecordsTimestampMs); - - /** - * This enum represents all potential results after calling {@link #delegateConsumerRecord(PubSubMessageProcessedResultWrapper, int, String, int, long, long)}. - */ - protected enum DelegateConsumerRecordResult { - /** - * The consumer record has been produced to local version topic by leader. - */ - PRODUCED_TO_KAFKA, - /** - * The consumer record has been put into drainer queue; the following cases will result in putting to drainer directly: - * 1. Online/Offline ingestion task - * 2. Follower replicas - * 3. Leader is consuming from local version topics - */ - QUEUED_TO_DRAINER, - /** - * The consumer record is skipped. e.g. remote VT's TS message during data recovery. - */ - SKIPPED_MESSAGE - } + // protected abstract DelegateConsumerRecordResult delegateConsumerRecord( + // PubSubMessageProcessedResultWrapper consumerRecordWrapper, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingPerRecordTimestampNs, + // long beforeProcessingBatchRecordsTimestampMs); + // + // /** + // * This enum represents all potential results after calling {@link + // #delegateConsumerRecord(PubSubMessageProcessedResultWrapper, int, String, int, long, long)}. + // */ + // protected enum DelegateConsumerRecordResult { + // /** + // * The consumer record has been produced to local version topic by leader. + // */ + // PRODUCED_TO_KAFKA, + // /** + // * The consumer record has been put into drainer queue; the following cases will result in putting to drainer + // directly: + // * 1. Online/Offline ingestion task + // * 2. Follower replicas + // * 3. Leader is consuming from local version topics + // */ + // QUEUED_TO_DRAINER, + // /** + // * The consumer record is skipped. e.g. remote VT's TS message during data recovery. + // */ + // SKIPPED_MESSAGE + // } /** * The method measures the time between receiving the message from the local VT and when the message is committed in @@ -4705,8 +4711,70 @@ public Consumer getDivErrorMetricCallback() { return divErrorMetricCallback; } + public abstract Lazy getKeyLevelLocksManager(); + public abstract KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders(); + protected abstract void getAndUpdateLeaderCompletedState( + KafkaKey kafkaKey, + KafkaMessageEnvelope kafkaValue, + ControlMessage controlMessage, + PubSubMessageHeaders pubSubMessageHeaders, + PartitionConsumptionState partitionConsumptionState); + + protected abstract void validateRecordBeforeProducingToLocalKafka( + PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, + String kafkaUrl, + int kafkaClusterId); + + protected abstract void recordRegionHybridConsumptionStats( + int kafkaClusterId, + int producedRecordSize, + long upstreamOffset, + long currentTimeMs); + + protected abstract void updateLatestInMemoryLeaderConsumedRTOffset( + PartitionConsumptionState pcs, + String ignoredKafkaUrl, + long offset); + + protected abstract void produceToLocalKafka( + PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, + LeaderProducedRecordContext leaderProducedRecordContext, + BiConsumer produceFunction, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs); + + protected abstract void propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( + PartitionConsumptionState partitionConsumptionState, + PubSubMessage consumerRecord, + LeaderProducedRecordContext leaderProducedRecordContext, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs); + + protected abstract void processMessageAndMaybeProduceToKafka( + PubSubMessageProcessedResultWrapper consumerRecordWrapper, + PartitionConsumptionState partitionConsumptionState, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs); + + protected abstract void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState); + + protected abstract boolean hasViewWriters(); + + public boolean isDataRecovery() { + return isDataRecovery; + } + // For unit test purpose. void setVersionRole(PartitionReplicaIngestionContext.VersionRole versionRole) { this.versionRole = versionRole; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 07073f7be5c..8dd5cc9a450 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -1,5 +1,7 @@ package com.linkedin.davinci.kafka.consumer; +import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_PUSH; + import com.linkedin.avroutil1.compatibility.shaded.org.apache.commons.lang3.Validate; import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; import com.linkedin.davinci.stats.HostLevelIngestionStats; @@ -26,11 +28,16 @@ import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.ValueHolder; +import com.linkedin.venice.writer.VeniceWriter; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.NavigableMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.locks.ReentrantLock; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -478,14 +485,13 @@ private int handleSingleMessage( // This function may modify the original record in KME and it is unsafe to use the payload from KME directly after // this call. - StoreIngestionTask.DelegateConsumerRecordResult delegateConsumerRecordResult = - storeIngestionTask.delegateConsumerRecord( - consumerRecordWrapper, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingPerRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs); + DelegateConsumerRecordResult delegateConsumerRecordResult = delegateConsumerRecordMaybeWithLock( + consumerRecordWrapper, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); switch (delegateConsumerRecordResult) { case QUEUED_TO_DRAINER: @@ -513,6 +519,466 @@ private int handleSingleMessage( return record.getPayloadSize(); } + /** + * This enum represents all potential results after calling {@link #delegateConsumerRecord(PubSubMessageProcessedResultWrapper, int, String, int, long, long)}. + */ + protected enum DelegateConsumerRecordResult { + /** + * The consumer record has been produced to local version topic by leader. + */ + PRODUCED_TO_KAFKA, + /** + * The consumer record has been put into drainer queue; the following cases will result in putting to drainer directly: + * 1. Online/Offline ingestion task + * 2. Follower replicas + * 3. Leader is consuming from local version topics + */ + QUEUED_TO_DRAINER, + /** + * The consumer record is skipped. e.g. remote VT's TS message during data recovery. + */ + SKIPPED_MESSAGE + } + + private DelegateConsumerRecordResult delegateConsumerRecordMaybeWithLock( + PubSubMessageProcessedResultWrapper consumerRecordWrapper, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingPerRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs) { + if (!storeIngestionTask.isActiveActiveReplicationEnabled() + || !consumerRecordWrapper.getMessage().getTopicPartition().getPubSubTopic().isRealTime()) { + /** + * We don't need to lock the partition here because during VT consumption there is only one consumption source. + */ + return delegateConsumerRecord( + consumerRecordWrapper, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); + } else { + /** + * The below flow must be executed in a critical session for the same key: + * Read existing value/RMD from transient record cache/disk -> perform DCR and decide incoming value wins + * -> update transient record cache -> produce to VT (just call send, no need to wait for the produce future in the critical session) + * + * Otherwise, there could be race conditions: + * [fabric A thread]Read from transient record cache -> [fabric A thread]perform DCR and decide incoming value wins + * -> [fabric B thread]read from transient record cache -> [fabric B thread]perform DCR and decide incoming value wins + * -> [fabric B thread]update transient record cache -> [fabric B thread]produce to VT -> [fabric A thread]update transient record cache + * -> [fabric A thread]produce to VT + */ + final ByteArrayKey byteArrayKey = ByteArrayKey.wrap(consumerRecordWrapper.getMessage().getKey().getKey()); + ReentrantLock keyLevelLock = storeIngestionTask.getKeyLevelLocksManager().get().acquireLockByKey(byteArrayKey); + keyLevelLock.lock(); + try { + return delegateConsumerRecord( + consumerRecordWrapper, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); + } finally { + keyLevelLock.unlock(); + storeIngestionTask.getKeyLevelLocksManager().get().releaseLock(byteArrayKey); + } + } + } + + /** + * The goal of this function is to possibly produce the incoming kafka message consumed from local VT, remote VT, RT or SR topic to + * local VT if needed. It's decided based on the function output of {@link StoreIngestionTask#shouldProduceToVersionTopic} and message type. + * It also perform any necessary additional computation operation such as for write-compute/update message. + * It returns a boolean indicating if it was produced to kafka or not. + * + * This function should be called as one of the first steps in processing pipeline for all messages consumed from any kafka topic. + * + * The caller of this function should only process this {@param consumerRecord} further if the return is + * {@link DelegateConsumerRecordResult#QUEUED_TO_DRAINER}. + * + * This function assumes {@link LeaderFollowerStoreIngestionTask#shouldProcessRecord(PubSubMessage)} has been called which happens in + * {@link StorePartitionDataReceiver#produceToStoreBufferServiceOrKafka(Iterable, PubSubTopicPartition, String, int)} + * before calling this and the it was decided that this record needs to be processed. It does not perform any + * validation check on the PartitionConsumptionState object to keep the goal of the function simple and not overload. + * + * Also DIV validation is done here if the message is received from RT topic. For more info please see + * please see {@link StoreIngestionTask#internalProcessConsumerRecord} + * + * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after this function. + * + * @return a {@link DelegateConsumerRecordResult} indicating what to do with the record + */ + DelegateConsumerRecordResult delegateConsumerRecord( + PubSubMessageProcessedResultWrapper consumerRecordWrapper, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingPerRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs) { + PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); + try { + KafkaKey kafkaKey = consumerRecord.getKey(); + KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + /** + * partitionConsumptionState must be in a valid state and no error reported. This is made sure by calling + * {@link shouldProcessRecord} before processing any record. + * + * ^ This is no longer true because with shared consumer the partitionConsumptionState could have been removed + * from unsubscribe action in the StoreIngestionTask thread. Today, when unsubscribing + * {@link StoreIngestionTask.waitForAllMessageToBeProcessedFromTopicPartition} only ensure the buffer queue is + * drained before unsubscribe. Records being processed by shared consumer may see invalid partitionConsumptionState. + */ + PartitionConsumptionState partitionConsumptionState = storeIngestionTask.getPartitionConsumptionState(partition); + if (partitionConsumptionState == null) { + // The partition is likely unsubscribed, will skip these messages. + return DelegateConsumerRecordResult.SKIPPED_MESSAGE; + } + boolean shouldProduceToLocalKafka = storeIngestionTask.shouldProduceToVersionTopic(partitionConsumptionState); + // UPDATE message is only expected in LEADER which must be produced to kafka. + MessageType msgType = MessageType.valueOf(kafkaValue); + if (msgType == MessageType.UPDATE && !shouldProduceToLocalKafka) { + throw new VeniceMessageException( + storeIngestionTask.getIngestionTaskName() + + " hasProducedToKafka: Received UPDATE message in non-leader for: " + + consumerRecord.getTopicPartition() + " Offset " + consumerRecord.getOffset()); + } else if (msgType == MessageType.CONTROL_MESSAGE) { + ControlMessage controlMessage = (ControlMessage) kafkaValue.payloadUnion; + storeIngestionTask.getAndUpdateLeaderCompletedState( + kafkaKey, + kafkaValue, + controlMessage, + consumerRecord.getPubSubMessageHeaders(), + partitionConsumptionState); + } + + /** + * return early if it needs not be produced to local VT such as cases like + * (i) it's a follower or (ii) leader is consuming from VT + */ + if (!shouldProduceToLocalKafka) { + /** + * For the local consumption, the batch data won't be produce to the local VT again, so we will switch + * to real-time writer upon EOP here and for the remote consumption of VT, the switch will be handled + * in the following section as it needs to flush the messages and then switch. + */ + if (LeaderFollowerStoreIngestionTask.isLeader(partitionConsumptionState) + && msgType == MessageType.CONTROL_MESSAGE + && ControlMessageType.valueOf((ControlMessage) kafkaValue.payloadUnion).equals(END_OF_PUSH)) { + LOGGER.info( + "Switching to the VeniceWriter for real-time workload for topic: {}, partition: {}", + storeIngestionTask.getVersionTopic().getName(), + partition); + // Just to be extra safe + partitionConsumptionState.getVeniceWriterLazyRef().ifPresent(vw -> vw.flush()); + storeIngestionTask.setRealTimeVeniceWriterRef(partitionConsumptionState); + } + return DelegateConsumerRecordResult.QUEUED_TO_DRAINER; + } + + // If we are here the message must be produced to local kafka or silently consumed. + LeaderProducedRecordContext leaderProducedRecordContext; + // No need to resolve cluster id and kafka url because sep topics are real time topic and it's not VT + storeIngestionTask.validateRecordBeforeProducingToLocalKafka( + consumerRecord, + partitionConsumptionState, + kafkaUrl, + kafkaClusterId); + + if (consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { + storeIngestionTask.recordRegionHybridConsumptionStats( + // convert the cluster id back to the original cluster id for monitoring purpose + storeIngestionTask.getServerConfig() + .getEquivalentKafkaClusterIdForSepTopic( + storeIngestionTask.getServerConfig().getEquivalentKafkaClusterIdForSepTopic(kafkaClusterId)), + consumerRecord.getPayloadSize(), + consumerRecord.getOffset(), + beforeProcessingBatchRecordsTimestampMs); + storeIngestionTask.updateLatestInMemoryLeaderConsumedRTOffset( + partitionConsumptionState, + kafkaUrl, + consumerRecord.getOffset()); + } + + // heavy leader processing starts here + storeIngestionTask.getVersionIngestionStats() + .recordLeaderPreprocessingLatency( + storeIngestionTask.getStoreName(), + storeIngestionTask.getVersionNumber(), + LatencyUtils.getElapsedTimeFromNSToMS(beforeProcessingPerRecordTimestampNs), + beforeProcessingBatchRecordsTimestampMs); + + if (kafkaKey.isControlMessage()) { + boolean producedFinally = true; + ControlMessage controlMessage = (ControlMessage) kafkaValue.getPayloadUnion(); + ControlMessageType controlMessageType = ControlMessageType.valueOf(controlMessage); + leaderProducedRecordContext = LeaderProducedRecordContext + .newControlMessageRecord(kafkaClusterId, consumerRecord.getOffset(), kafkaKey.getKey(), controlMessage); + switch (controlMessageType) { + case START_OF_PUSH: + /** + * N.B.: This is expected to be the first time we call {@link veniceWriter#get()}. During this time + * since startOfPush has not been processed yet, {@link StoreVersionState} is not created yet (unless + * this is a server restart scenario). So the {@link com.linkedin.venice.writer.VeniceWriter#isChunkingEnabled} field + * will not be set correctly at this point. This is fine as chunking is mostly not applicable for control messages. + * This chunking flag for the veniceWriter will happen be set correctly in + * {@link StoreIngestionTask#processStartOfPush(ControlMessage, int, long, PartitionConsumptionState)}, + * which will be called when this message gets processed in drainer thread after successfully producing + * to kafka. + * + * Note update: the first time we call {@link veniceWriter#get()} is different in various use cases: + * 1. For hybrid store with L/F enabled, the first time a VeniceWriter is created is after leader switches to RT and + * consumes the first message; potential message type: SOS, EOS, data message. + * 2. For store version generated by stream reprocessing push type, the first time is after leader switches to + * SR topic and consumes the first message; potential message type: SOS, EOS, data message (consider server restart). + * 3. For store with native replication enabled, the first time is after leader switches to remote topic and start + * consumes the first message; potential message type: SOS, EOS, SOP, EOP, data message (consider server restart). + */ + case END_OF_PUSH: + // CMs that are produced with DIV pass-through mode can break DIV without synchronization with view writers. + // This is because for data (PUT) records we queue their produceToLocalKafka behind the completion of view + // writers. The main SIT will move on to subsequent messages and for CMs that don't need to be propagated + // to view topics we are producing them directly. If we don't check the previous write before producing the + // CMs then in the VT we might get out of order messages and with pass-through DIV that's going to be an + // issue. e.g. a PUT record belonging to seg:0 can come after the EOS of seg:0 due to view writer delays. + // Since SOP and EOP are rare we can simply wait for the last VT produce future. + checkAndWaitForLastVTProduceFuture(partitionConsumptionState); + /** + * Simply produce this EOP to local VT. It will be processed in order in the drainer queue later + * after successfully producing to kafka. + */ + storeIngestionTask.produceToLocalKafka( + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() + .get() + .put( + consumerRecord.getKey(), + consumerRecord.getValue(), + callback, + consumerRecord.getTopicPartition().getPartitionNumber(), + leaderMetadataWrapper), + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs); + partitionConsumptionState.getVeniceWriterLazyRef().get().flush(); + // Switch the writer for real-time workload + LOGGER.info( + "Switching to the VeniceWriter for real-time workload for topic: {}, partition: {}", + storeIngestionTask.getVersionTopic().getName(), + partition); + storeIngestionTask.setRealTimeVeniceWriterRef(partitionConsumptionState); + break; + case START_OF_SEGMENT: + case END_OF_SEGMENT: + /** + * SOS and EOS will be produced to the local version topic with DIV pass-through mode by leader in the following cases: + * 1. SOS and EOS are from stream-reprocessing topics (use cases: stream-reprocessing) + * 2. SOS and EOS are from version topics in a remote fabric (use cases: native replication for remote fabrics) + * + * SOS and EOS will not be produced to local version topic in the following cases: + * 1. SOS and EOS are from real-time topics (use cases: hybrid ingestion, incremental push to RT) + * 2. SOS and EOS are from version topics in local fabric, which has 2 different scenarios: + * i. native replication is enabled, but the current fabric is the source fabric (use cases: native repl for source fabric) + * ii. native replication is not enabled; it doesn't matter whether current replica is leader or follower, + * messages from local VT doesn't need to be reproduced into local VT again (use case: local batch consumption) + * + * There is one exception that overrules the above conditions. i.e. if the SOS is a heartbeat from the RT topic. + * In such case the heartbeat is produced to VT with updated {@link LeaderMetadataWrapper}. + * + * We want to ensure correct ordering for any SOS and EOS that we do decide to write to VT. This is done by + * coordinating with the corresponding {@link PartitionConsumptionState#getLastVTProduceCallFuture}. + * However, this coordination is only needed if there are view writers. i.e. the VT writes and CM writes + * need to be in the same mode. Either both coordinate with lastVTProduceCallFuture or neither. + */ + if (!consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { + final LeaderProducedRecordContext segmentCMLeaderProduceRecordContext = leaderProducedRecordContext; + maybeQueueCMWritesToVersionTopic( + partitionConsumptionState, + () -> storeIngestionTask.produceToLocalKafka( + consumerRecord, + partitionConsumptionState, + segmentCMLeaderProduceRecordContext, + (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() + .get() + .put( + consumerRecord.getKey(), + consumerRecord.getValue(), + callback, + consumerRecord.getTopicPartition().getPartitionNumber(), + leaderMetadataWrapper), + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs)); + } else { + if (controlMessageType == ControlMessageType.START_OF_SEGMENT + && Arrays.equals(consumerRecord.getKey().getKey(), KafkaKey.HEART_BEAT.getKey())) { + final LeaderProducedRecordContext heartbeatLeaderProducedRecordContext = leaderProducedRecordContext; + maybeQueueCMWritesToVersionTopic( + partitionConsumptionState, + () -> storeIngestionTask.propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( + partitionConsumptionState, + consumerRecord, + heartbeatLeaderProducedRecordContext, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs)); + } else { + /** + * Based on current design handling this case (specially EOS) is tricky as we don't produce the SOS/EOS + * received from RT to local VT. But ideally EOS must be queued in-order (after all previous data message + * PUT/UPDATE/DELETE) to drainer. But since the queueing of data message to drainer + * happens in Kafka producer callback there is no way to queue this EOS to drainer in-order. + * + * Usually following processing in Leader for other control message. + * 1. DIV: + * 2. updateOffset: + * 3. stats maintenance as in {@link StoreIngestionTask#processKafkaDataMessage} + * + * For #1 Since we have moved the DIV validation in this function, We are good with DIV part which is the most critical one. + * For #2 Leader will not update the offset for SOS/EOS. From Server restart point of view this is tolerable. This was the case in previous design also. So there is no change in behaviour. + * For #3 stat counter update will not happen for SOS/EOS message. This should not be a big issue. If needed we can copy some of the stats maintenance + * work here. + * + * So in summary NO further processing is needed SOS/EOS received from RT topics. Just silently drop the message here. + * We should not return false here. + */ + producedFinally = false; + } + } + break; + case START_OF_INCREMENTAL_PUSH: + case END_OF_INCREMENTAL_PUSH: + // For inc push to RT policy, the control msg is written in RT topic, we will need to calculate the + // destination partition in VT correctly. + int versionTopicPartitionToBeProduced = consumerRecord.getTopicPartition().getPartitionNumber(); + /** + * We are using {@link VeniceWriter#asyncSendControlMessage()} api instead of {@link VeniceWriter#put()} since we have + * to calculate DIV for this message but keeping the ControlMessage content unchanged. {@link VeniceWriter#put()} does not + * allow that. + */ + storeIngestionTask.produceToLocalKafka( + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() + .get() + .asyncSendControlMessage( + controlMessage, + versionTopicPartitionToBeProduced, + new HashMap<>(), + callback, + leaderMetadataWrapper), + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs); + break; + case TOPIC_SWITCH: + /** + * For TOPIC_SWITCH message we should use -1 as consumedOffset. This will ensure that it does not update the + * setLeaderUpstreamOffset in: + * {@link #updateOffsetsAsRemoteConsumeLeader(PartitionConsumptionState, LeaderProducedRecordContext, String, PubSubMessage, LeaderFollowerStoreIngestionTask.UpdateVersionTopicOffset, LeaderFollowerStoreIngestionTask.UpdateUpstreamTopicOffset)} + * The leaderUpstreamOffset is set from the TS message config itself. We should not override it. + */ + if (storeIngestionTask.isDataRecovery() && !partitionConsumptionState.isBatchOnly()) { + // Ignore remote VT's TS message since we might need to consume more RT or incremental push data from VT + // that's no longer in the local/remote RT due to retention. + return DelegateConsumerRecordResult.SKIPPED_MESSAGE; + } + leaderProducedRecordContext = + LeaderProducedRecordContext.newControlMessageRecord(kafkaKey.getKey(), controlMessage); + storeIngestionTask.produceToLocalKafka( + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() + .get() + .asyncSendControlMessage( + controlMessage, + consumerRecord.getTopicPartition().getPartitionNumber(), + new HashMap<>(), + callback, + VeniceWriter.DEFAULT_LEADER_METADATA_WRAPPER), + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs); + break; + case VERSION_SWAP: + return DelegateConsumerRecordResult.QUEUED_TO_DRAINER; + default: + // do nothing + break; + } + if (!storeIngestionTask.isSegmentControlMsg(controlMessageType)) { + LOGGER.info( + "Replica: {} hasProducedToKafka: {}; ControlMessage: {}; Incoming record topic-partition: {}; offset: {}", + partitionConsumptionState.getReplicaId(), + producedFinally, + controlMessageType.name(), + consumerRecord.getTopicPartition(), + consumerRecord.getOffset()); + } + } else if (kafkaValue == null) { + throw new VeniceMessageException( + partitionConsumptionState.getReplicaId() + + " hasProducedToKafka: Given null Venice Message. TopicPartition: " + + consumerRecord.getTopicPartition() + " Offset " + consumerRecord.getOffset()); + } else { + // This function may modify the original record in KME and it is unsafe to use the payload from KME directly + // after this call. + storeIngestionTask.processMessageAndMaybeProduceToKafka( + consumerRecordWrapper, + partitionConsumptionState, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); + } + return DelegateConsumerRecordResult.PRODUCED_TO_KAFKA; + } catch (Exception e) { + throw new VeniceException( + storeIngestionTask.getIngestionTaskName() + " hasProducedToKafka: exception for message received from: " + + consumerRecord.getTopicPartition() + ", Offset: " + consumerRecord.getOffset() + ". Bubbling up.", + e); + } + } + + private void checkAndWaitForLastVTProduceFuture(PartitionConsumptionState partitionConsumptionState) + throws ExecutionException, InterruptedException { + partitionConsumptionState.getLastVTProduceCallFuture().get(); + } + + private void maybeQueueCMWritesToVersionTopic( + PartitionConsumptionState partitionConsumptionState, + Runnable produceCall) { + if (storeIngestionTask.hasViewWriters()) { + CompletableFuture propagateSegmentCMWrite = new CompletableFuture<>(); + partitionConsumptionState.getLastVTProduceCallFuture().whenCompleteAsync((value, exception) -> { + if (exception == null) { + produceCall.run(); + propagateSegmentCMWrite.complete(null); + } else { + VeniceException veniceException = new VeniceException(exception); + storeIngestionTask.setIngestionException(partitionConsumptionState.getPartition(), veniceException); + propagateSegmentCMWrite.completeExceptionally(veniceException); + } + }); + partitionConsumptionState.setLastVTProduceCallFuture(propagateSegmentCMWrite); + } else { + produceCall.run(); + } + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTaskTest.java index c3b4e17d430..bb1f7a318b4 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTaskTest.java @@ -206,6 +206,7 @@ public void setUp() throws InterruptedException { mockPartitionConsumptionState = mock(PartitionConsumptionState.class); mockConsumerAction = mock(ConsumerAction.class); + mockTopicPartition = mock(PubSubTopicPartition.class); mockProperties = new Properties(); mockBooleanSupplier = mock(BooleanSupplier.class); @@ -247,7 +248,6 @@ public void testVeniceWriterInProcessConsumerAction() throws InterruptedExceptio mock(LeaderFollowerPartitionStateModel.LeaderSessionIdChecker.class); when(mockConsumerAction.getLeaderSessionIdChecker()).thenReturn(mockLeaderSessionIdChecker); when(mockLeaderSessionIdChecker.isSessionIdValid()).thenReturn(true); - mockTopicPartition = mock(PubSubTopicPartition.class); OffsetRecord mockOffsetRecord = mock(OffsetRecord.class); when(mockConsumerAction.getTopicPartition()).thenReturn(mockTopicPartition); when(mockPartitionConsumptionState.getOffsetRecord()).thenReturn(mockOffsetRecord); @@ -332,8 +332,10 @@ public void testControlMessagesAreInOrderWithPassthroughDIV() throws Interrupted .thenReturn(nextVTWriteFuture); VeniceWriter veniceWriter = mock(VeniceWriter.class); doReturn(Lazy.of(() -> veniceWriter)).when(mockPartitionConsumptionState).getVeniceWriterLazyRef(); - leaderFollowerStoreIngestionTask.delegateConsumerRecord(firstCM, 0, "testURL", 0, 0, 0); - leaderFollowerStoreIngestionTask.delegateConsumerRecord(secondCM, 0, "testURL", 0, 0, 0); + StorePartitionDataReceiver storePartitionDataReceiver = + spy(new StorePartitionDataReceiver(leaderFollowerStoreIngestionTask, mockTopicPartition, "testURL", 0)); + storePartitionDataReceiver.delegateConsumerRecord(firstCM, 0, "testURL", 0, 0, 0); + storePartitionDataReceiver.delegateConsumerRecord(secondCM, 0, "testURL", 0, 0, 0); // The CM write should be queued but not executed yet since the previous VT write future is still incomplete verify(veniceWriter, never()).put(any(), any(), any(), anyInt(), any()); lastVTWriteFuture.complete(null); From d764c8198e85fb6afa04bdb66b3180b1279a8984 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Fri, 18 Oct 2024 20:34:00 -0700 Subject: [PATCH 07/32] =?UTF-8?q?Refactored=20`validateRecordBeforeProduci?= =?UTF-8?q?ngToLocalKafka()`=20from=20`LeaderFollowerStoreIngestionTask`?= =?UTF-8?q?=20into=20`StorePartitionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 70 +++++++++---------- .../kafka/consumer/StoreIngestionTask.java | 14 ++-- .../consumer/StorePartitionDataReceiver.java | 41 +++++++++-- 3 files changed, 79 insertions(+), 46 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index a6b1d78b437..3d380f76186 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -2814,41 +2814,41 @@ protected void waitForAllMessageToBeProcessedFromTopicPartition( } } - /** - * Checks before producing local version topic. - * - * Extend this function when there is new check needed. - */ - @Override - protected void validateRecordBeforeProducingToLocalKafka( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - String kafkaUrl, - int kafkaClusterId) { - // Check whether the message is from local version topic; leader shouldn't consume from local VT and then produce - // back to VT again - // localKafkaClusterId will always be the regular one without "_sep" suffix so kafkaClusterId should be converted - // for comparison. Like-wise for the kafkaUrl. - if (kafkaClusterId == localKafkaClusterId - && consumerRecord.getTopicPartition().getPubSubTopic().equals(this.versionTopic) - && kafkaUrl.equals(this.localKafkaServer)) { - // N.B.: Ideally, the first two conditions should be sufficient, but for some reasons, in certain tests, the - // third condition also ends up being necessary. In any case, doing the cluster ID check should be a - // fast short-circuit in normal cases. - try { - int partitionId = partitionConsumptionState.getPartition(); - setIngestionException( - partitionId, - new VeniceException( - "Store version " + this.kafkaVersionTopic + " partition " + partitionId - + " is consuming from local version topic and producing back to local version topic" - + ", kafkaClusterId = " + kafkaClusterId + ", kafkaUrl = " + kafkaUrl + ", this.localKafkaServer = " - + this.localKafkaServer)); - } catch (VeniceException offerToQueueException) { - setLastStoreIngestionException(offerToQueueException); - } - } - } + // /** + // * Checks before producing local version topic. + // * + // * Extend this function when there is new check needed. + // */ + // @Override + // protected void validateRecordBeforeProducingToLocalKafka( + // PubSubMessage consumerRecord, + // PartitionConsumptionState partitionConsumptionState, + // String kafkaUrl, + // int kafkaClusterId) { + // // Check whether the message is from local version topic; leader shouldn't consume from local VT and then produce + // // back to VT again + // // localKafkaClusterId will always be the regular one without "_sep" suffix so kafkaClusterId should be converted + // // for comparison. Like-wise for the kafkaUrl. + // if (kafkaClusterId == localKafkaClusterId + // && consumerRecord.getTopicPartition().getPubSubTopic().equals(this.versionTopic) + // && kafkaUrl.equals(this.localKafkaServer)) { + // // N.B.: Ideally, the first two conditions should be sufficient, but for some reasons, in certain tests, the + // // third condition also ends up being necessary. In any case, doing the cluster ID check should be a + // // fast short-circuit in normal cases. + // try { + // int partitionId = partitionConsumptionState.getPartition(); + // setIngestionException( + // partitionId, + // new VeniceException( + // "Store version " + this.kafkaVersionTopic + " partition " + partitionId + // + " is consuming from local version topic and producing back to local version topic" + // + ", kafkaClusterId = " + kafkaClusterId + ", kafkaUrl = " + kafkaUrl + ", this.localKafkaServer = " + // + this.localKafkaServer)); + // } catch (VeniceException offerToQueueException) { + // setLastStoreIngestionException(offerToQueueException); + // } + // } + // } // calculate the replication once per partition, checking Leader instance will make sure we calculate it just once // per partition. diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 7619fe65b1d..6943292e2b6 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -4722,12 +4722,6 @@ protected abstract void getAndUpdateLeaderCompletedState( PubSubMessageHeaders pubSubMessageHeaders, PartitionConsumptionState partitionConsumptionState); - protected abstract void validateRecordBeforeProducingToLocalKafka( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - String kafkaUrl, - int kafkaClusterId); - protected abstract void recordRegionHybridConsumptionStats( int kafkaClusterId, int producedRecordSize, @@ -4775,6 +4769,14 @@ public boolean isDataRecovery() { return isDataRecovery; } + public String getLocalKafkaServer() { + return localKafkaServer; + } + + public int getLocalKafkaClusterId() { + return localKafkaClusterId; + } + // For unit test purpose. void setVersionRole(PartitionReplicaIngestionContext.VersionRole versionRole) { this.versionRole = versionRole; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 8dd5cc9a450..f0977935331 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -682,11 +682,7 @@ DelegateConsumerRecordResult delegateConsumerRecord( // If we are here the message must be produced to local kafka or silently consumed. LeaderProducedRecordContext leaderProducedRecordContext; // No need to resolve cluster id and kafka url because sep topics are real time topic and it's not VT - storeIngestionTask.validateRecordBeforeProducingToLocalKafka( - consumerRecord, - partitionConsumptionState, - kafkaUrl, - kafkaClusterId); + validateRecordBeforeProducingToLocalKafka(consumerRecord, partitionConsumptionState, kafkaUrl, kafkaClusterId); if (consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { storeIngestionTask.recordRegionHybridConsumptionStats( @@ -979,6 +975,41 @@ private void maybeQueueCMWritesToVersionTopic( } } + /** + * Checks before producing local version topic. + * + * Extend this function when there is new check needed. + */ + private void validateRecordBeforeProducingToLocalKafka( + PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, + String kafkaUrl, + int kafkaClusterId) { + // Check whether the message is from local version topic; leader shouldn't consume from local VT and then produce + // back to VT again + // localKafkaClusterId will always be the regular one without "_sep" suffix so kafkaClusterId should be converted + // for comparison. Like-wise for the kafkaUrl. + if (kafkaClusterId == storeIngestionTask.getLocalKafkaClusterId() + && consumerRecord.getTopicPartition().getPubSubTopic().equals(storeIngestionTask.getVersionTopic()) + && kafkaUrl.equals(storeIngestionTask.getLocalKafkaServer())) { + // N.B.: Ideally, the first two conditions should be sufficient, but for some reasons, in certain tests, the + // third condition also ends up being necessary. In any case, doing the cluster ID check should be a + // fast short-circuit in normal cases. + try { + int partitionId = partitionConsumptionState.getPartition(); + storeIngestionTask.setIngestionException( + partitionId, + new VeniceException( + "Store version " + storeIngestionTask.getVersionTopic() + " partition " + partitionId + + " is consuming from local version topic and producing back to local version topic" + + ", kafkaClusterId = " + kafkaClusterId + ", kafkaUrl = " + kafkaUrl + ", this.localKafkaServer = " + + storeIngestionTask.getLocalKafkaServer())); + } catch (VeniceException offerToQueueException) { + storeIngestionTask.setLastStoreIngestionException(offerToQueueException); + } + } + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); From 865bda300b65fd9ffa154390e74a01a51f0b0dad Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Fri, 18 Oct 2024 20:37:53 -0700 Subject: [PATCH 08/32] =?UTF-8?q?Refactored=20`recordRegionHybridConsumpti?= =?UTF-8?q?onStats()`=20from=20`LeaderFollowerStoreIngestionTask`=20into?= =?UTF-8?q?=20`StorePartitionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 34 +++++++++---------- .../kafka/consumer/StoreIngestionTask.java | 6 ---- .../consumer/StorePartitionDataReceiver.java | 21 +++++++++++- 3 files changed, 37 insertions(+), 24 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 3d380f76186..b95df89f598 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -2063,23 +2063,23 @@ protected final void recordAssembledRecordSizeRatio(double ratio, long currentTi } } - @Override - protected void recordRegionHybridConsumptionStats( - int kafkaClusterId, - int producedRecordSize, - long upstreamOffset, - long currentTimeMs) { - if (kafkaClusterId >= 0) { - versionedIngestionStats.recordRegionHybridConsumption( - storeName, - versionNumber, - kafkaClusterId, - producedRecordSize, - upstreamOffset, - currentTimeMs); - hostLevelIngestionStats.recordTotalRegionHybridBytesConsumed(kafkaClusterId, producedRecordSize, currentTimeMs); - } - } + // @Override + // protected void recordRegionHybridConsumptionStats( + // int kafkaClusterId, + // int producedRecordSize, + // long upstreamOffset, + // long currentTimeMs) { + // if (kafkaClusterId >= 0) { + // versionedIngestionStats.recordRegionHybridConsumption( + // storeName, + // versionNumber, + // kafkaClusterId, + // producedRecordSize, + // upstreamOffset, + // currentTimeMs); + // hostLevelIngestionStats.recordTotalRegionHybridBytesConsumed(kafkaClusterId, producedRecordSize, currentTimeMs); + // } + // } @Override protected boolean isHybridFollower(PartitionConsumptionState partitionConsumptionState) { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 6943292e2b6..2cdea5d2f48 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -4722,12 +4722,6 @@ protected abstract void getAndUpdateLeaderCompletedState( PubSubMessageHeaders pubSubMessageHeaders, PartitionConsumptionState partitionConsumptionState); - protected abstract void recordRegionHybridConsumptionStats( - int kafkaClusterId, - int producedRecordSize, - long upstreamOffset, - long currentTimeMs); - protected abstract void updateLatestInMemoryLeaderConsumedRTOffset( PartitionConsumptionState pcs, String ignoredKafkaUrl, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index f0977935331..9e232d31c26 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -685,7 +685,7 @@ DelegateConsumerRecordResult delegateConsumerRecord( validateRecordBeforeProducingToLocalKafka(consumerRecord, partitionConsumptionState, kafkaUrl, kafkaClusterId); if (consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { - storeIngestionTask.recordRegionHybridConsumptionStats( + recordRegionHybridConsumptionStats( // convert the cluster id back to the original cluster id for monitoring purpose storeIngestionTask.getServerConfig() .getEquivalentKafkaClusterIdForSepTopic( @@ -1010,6 +1010,25 @@ private void validateRecordBeforeProducingToLocalKafka( } } + private void recordRegionHybridConsumptionStats( + int kafkaClusterId, + int producedRecordSize, + long upstreamOffset, + long currentTimeMs) { + if (kafkaClusterId >= 0) { + storeIngestionTask.getVersionIngestionStats() + .recordRegionHybridConsumption( + storeIngestionTask.getStoreName(), + storeIngestionTask.getVersionNumber(), + kafkaClusterId, + producedRecordSize, + upstreamOffset, + currentTimeMs); + storeIngestionTask.getHostLevelIngestionStats() + .recordTotalRegionHybridBytesConsumed(kafkaClusterId, producedRecordSize, currentTimeMs); + } + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); From 0e5d0e1f1f14fba26854404051e872f02f1747f2 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Wed, 5 Feb 2025 13:40:02 -0800 Subject: [PATCH 09/32] =?UTF-8?q?Refactored=20propagateHeartbeatFromUpstre?= =?UTF-8?q?amTopicToLocalVersionTopic()=20from=20LeaderFollowerStoreIngest?= =?UTF-8?q?ionTask=20into=20StorePartitionDataReceiver.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 142 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 16 +- .../consumer/StorePartitionDataReceiver.java | 72 ++++++++- 3 files changed, 160 insertions(+), 70 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index b95df89f598..8ee3deb4f3d 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -11,7 +11,6 @@ import static com.linkedin.venice.kafka.protocol.enums.MessageType.UPDATE; import static com.linkedin.venice.pubsub.api.PubSubMessageHeaders.VENICE_LEADER_COMPLETION_STATE_HEADER; import static com.linkedin.venice.writer.VeniceWriter.APP_DEFAULT_LOGICAL_TS; -import static java.lang.Long.max; import static java.util.concurrent.TimeUnit.MILLISECONDS; import com.linkedin.davinci.client.DaVinciRecordTransformerConfig; @@ -2191,54 +2190,61 @@ protected void getAndUpdateLeaderCompletedState( } } - /** - * Leaders propagate HB SOS message from RT to local VT (to all subpartitions in case if amplification - * Factor is configured to be more than 1) with updated LeaderCompleteState header: - * Adding the headers during this phase instead of adding it to RT directly simplifies the logic - * of how to identify the HB SOS from the correct version or whether the HB SOS is from the local - * colo or remote colo, as the header inherited from an incorrect version or remote colos might - * provide incorrect information about the leader state. - */ - @Override - protected final void propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( - PartitionConsumptionState partitionConsumptionState, - PubSubMessage consumerRecord, - LeaderProducedRecordContext leaderProducedRecordContext, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs) { - LeaderProducerCallback callback = createProducerCallback( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - partition, - kafkaUrl, - beforeProcessingRecordTimestampNs); - LeaderMetadataWrapper leaderMetadataWrapper = new LeaderMetadataWrapper(consumerRecord.getOffset(), kafkaClusterId); - LeaderCompleteState leaderCompleteState = - LeaderCompleteState.getLeaderCompleteState(partitionConsumptionState.isCompletionReported()); - /** - * The maximum value between the original producer timestamp and the timestamp when the message is added to the RT topic is used: - * This approach addresses scenarios wrt clock drift where the producer's timestamp is consistently delayed by several minutes, - * causing it not to align with the {@link com.linkedin.davinci.config.VeniceServerConfig#leaderCompleteStateCheckValidIntervalMs} - * interval. The likelihood of simultaneous significant time discrepancies between the leader (producer) and the RT should be very - * rare, making this a viable workaround. In cases where the time discrepancy is reversed, the follower may complete slightly earlier - * than expected. However, this should not pose a significant issue as the completion of the leader, indicated by the leader - * completed header, is a prerequisite for the follower completion and is expected to occur shortly thereafter. - */ - long producerTimeStamp = - max(consumerRecord.getPubSubMessageTime(), consumerRecord.getValue().producerMetadata.messageTimestamp); - PubSubTopicPartition topicPartition = - new PubSubTopicPartitionImpl(getVersionTopic(), partitionConsumptionState.getPartition()); - sendIngestionHeartbeatToVT( - partitionConsumptionState, - topicPartition, - callback, - leaderMetadataWrapper, - leaderCompleteState, - producerTimeStamp); - } + // /** + // * Leaders propagate HB SOS message from RT to local VT (to all subpartitions in case if amplification + // * Factor is configured to be more than 1) with updated LeaderCompleteState header: + // * Adding the headers during this phase instead of adding it to RT directly simplifies the logic + // * of how to identify the HB SOS from the correct version or whether the HB SOS is from the local + // * colo or remote colo, as the header inherited from an incorrect version or remote colos might + // * provide incorrect information about the leader state. + // */ + // @Override + // protected final void propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( + // PartitionConsumptionState partitionConsumptionState, + // PubSubMessage consumerRecord, + // LeaderProducedRecordContext leaderProducedRecordContext, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingRecordTimestampNs) { + // LeaderProducerCallback callback = createProducerCallback( + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // partition, + // kafkaUrl, + // beforeProcessingRecordTimestampNs); + // LeaderMetadataWrapper leaderMetadataWrapper = new LeaderMetadataWrapper(consumerRecord.getOffset(), + // kafkaClusterId); + // LeaderCompleteState leaderCompleteState = + // LeaderCompleteState.getLeaderCompleteState(partitionConsumptionState.isCompletionReported()); + // /** + // * The maximum value between the original producer timestamp and the timestamp when the message is added to the RT + // topic is used: + // * This approach addresses scenarios wrt clock drift where the producer's timestamp is consistently delayed by + // several minutes, + // * causing it not to align with the {@link + // com.linkedin.davinci.config.VeniceServerConfig#leaderCompleteStateCheckValidIntervalMs} + // * interval. The likelihood of simultaneous significant time discrepancies between the leader (producer) and the RT + // should be very + // * rare, making this a viable workaround. In cases where the time discrepancy is reversed, the follower may complete + // slightly earlier + // * than expected. However, this should not pose a significant issue as the completion of the leader, indicated by + // the leader + // * completed header, is a prerequisite for the follower completion and is expected to occur shortly thereafter. + // */ + // long producerTimeStamp = + // max(consumerRecord.getPubSubMessageTime(), consumerRecord.getValue().producerMetadata.messageTimestamp); + // PubSubTopicPartition topicPartition = + // new PubSubTopicPartitionImpl(getVersionTopic(), partitionConsumptionState.getPartition()); + // sendIngestionHeartbeatToVT( + // partitionConsumptionState, + // topicPartition, + // callback, + // leaderMetadataWrapper, + // leaderCompleteState, + // producerTimeStamp); + // } @Override protected void recordHeartbeatReceived( @@ -3704,6 +3710,7 @@ protected void processControlMessageForViews( } } + @Override protected LeaderProducerCallback createProducerCallback( PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, @@ -3770,25 +3777,26 @@ CompletableFuture sendIngestionHeartbeatToRT(PubSubTopicPar System.currentTimeMillis()); } - private void sendIngestionHeartbeatToVT( - PartitionConsumptionState partitionConsumptionState, - PubSubTopicPartition topicPartition, - PubSubProducerCallback callback, - LeaderMetadataWrapper leaderMetadataWrapper, - LeaderCompleteState leaderCompleteState, - long originTimeStampMs) { - sendIngestionHeartbeat( - partitionConsumptionState, - topicPartition, - callback, - leaderMetadataWrapper, - true, - true, - leaderCompleteState, - originTimeStampMs); - } + // private void sendIngestionHeartbeatToVT( + // PartitionConsumptionState partitionConsumptionState, + // PubSubTopicPartition topicPartition, + // PubSubProducerCallback callback, + // LeaderMetadataWrapper leaderMetadataWrapper, + // LeaderCompleteState leaderCompleteState, + // long originTimeStampMs) { + // sendIngestionHeartbeat( + // partitionConsumptionState, + // topicPartition, + // callback, + // leaderMetadataWrapper, + // true, + // true, + // leaderCompleteState, + // originTimeStampMs); + // } - private CompletableFuture sendIngestionHeartbeat( + @Override + protected CompletableFuture sendIngestionHeartbeat( PartitionConsumptionState partitionConsumptionState, PubSubTopicPartition topicPartition, PubSubProducerCallback callback, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 2cdea5d2f48..d46b5a31b95 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -87,6 +87,8 @@ import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubMessage; import com.linkedin.venice.pubsub.api.PubSubMessageHeaders; +import com.linkedin.venice.pubsub.api.PubSubProduceResult; +import com.linkedin.venice.pubsub.api.PubSubProducerCallback; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.pubsub.api.exceptions.PubSubUnsubscribedTopicPartitionException; @@ -117,6 +119,7 @@ import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import com.linkedin.venice.utils.lazy.Lazy; import com.linkedin.venice.writer.ChunkAwareCallback; +import com.linkedin.venice.writer.LeaderCompleteState; import com.linkedin.venice.writer.LeaderMetadataWrapper; import it.unimi.dsi.fastutil.objects.Object2IntMap; import java.io.Closeable; @@ -4737,13 +4740,22 @@ protected abstract void produceToLocalKafka( int kafkaClusterId, long beforeProcessingRecordTimestampNs); - protected abstract void propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( + protected abstract CompletableFuture sendIngestionHeartbeat( PartitionConsumptionState partitionConsumptionState, + PubSubTopicPartition topicPartition, + PubSubProducerCallback callback, + LeaderMetadataWrapper leaderMetadataWrapper, + boolean shouldLog, + boolean addLeaderCompleteState, + LeaderCompleteState leaderCompleteState, + long originTimeStampMs); + + protected abstract LeaderProducerCallback createProducerCallback( PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, LeaderProducedRecordContext leaderProducedRecordContext, int partition, String kafkaUrl, - int kafkaClusterId, long beforeProcessingRecordTimestampNs); protected abstract void processMessageAndMaybeProduceToKafka( diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 9e232d31c26..f563394b2be 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -20,7 +20,9 @@ import com.linkedin.venice.kafka.protocol.enums.MessageType; import com.linkedin.venice.kafka.protocol.state.StoreVersionState; import com.linkedin.venice.message.KafkaKey; +import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.api.PubSubMessage; +import com.linkedin.venice.pubsub.api.PubSubProducerCallback; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; @@ -28,6 +30,8 @@ import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.ValueHolder; +import com.linkedin.venice.writer.LeaderCompleteState; +import com.linkedin.venice.writer.LeaderMetadataWrapper; import com.linkedin.venice.writer.VeniceWriter; import java.util.ArrayList; import java.util.Arrays; @@ -818,7 +822,7 @@ DelegateConsumerRecordResult delegateConsumerRecord( final LeaderProducedRecordContext heartbeatLeaderProducedRecordContext = leaderProducedRecordContext; maybeQueueCMWritesToVersionTopic( partitionConsumptionState, - () -> storeIngestionTask.propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( + () -> propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( partitionConsumptionState, consumerRecord, heartbeatLeaderProducedRecordContext, @@ -1029,6 +1033,72 @@ private void recordRegionHybridConsumptionStats( } } + /** + * Leaders propagate HB SOS message from RT to local VT (to all subpartitions in case if amplification + * Factor is configured to be more than 1) with updated LeaderCompleteState header: + * Adding the headers during this phase instead of adding it to RT directly simplifies the logic + * of how to identify the HB SOS from the correct version or whether the HB SOS is from the local + * colo or remote colo, as the header inherited from an incorrect version or remote colos might + * provide incorrect information about the leader state. + */ + private void propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( + PartitionConsumptionState partitionConsumptionState, + PubSubMessage consumerRecord, + LeaderProducedRecordContext leaderProducedRecordContext, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs) { + LeaderProducerCallback callback = storeIngestionTask.createProducerCallback( + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + partition, + kafkaUrl, + beforeProcessingRecordTimestampNs); + LeaderMetadataWrapper leaderMetadataWrapper = new LeaderMetadataWrapper(consumerRecord.getOffset(), kafkaClusterId); + LeaderCompleteState leaderCompleteState = + LeaderCompleteState.getLeaderCompleteState(partitionConsumptionState.isCompletionReported()); + /** + * The maximum value between the original producer timestamp and the timestamp when the message is added to the RT topic is used: + * This approach addresses scenarios wrt clock drift where the producer's timestamp is consistently delayed by several minutes, + * causing it not to align with the {@link VeniceServerConfig#getLeaderCompleteStateCheckInFollowerValidIntervalMs()} + * interval. The likelihood of simultaneous significant time discrepancies between the leader (producer) and the RT should be very + * rare, making this a viable workaround. In cases where the time discrepancy is reversed, the follower may complete slightly earlier + * than expected. However, this should not pose a significant issue as the completion of the leader, indicated by the leader + * completed header, is a prerequisite for the follower completion and is expected to occur shortly thereafter. + */ + long producerTimeStamp = + Long.max(consumerRecord.getPubSubMessageTime(), consumerRecord.getValue().producerMetadata.messageTimestamp); + PubSubTopicPartition topicPartition = + new PubSubTopicPartitionImpl(storeIngestionTask.getVersionTopic(), partitionConsumptionState.getPartition()); + sendIngestionHeartbeatToVT( + partitionConsumptionState, + topicPartition, + callback, + leaderMetadataWrapper, + leaderCompleteState, + producerTimeStamp); + } + + private void sendIngestionHeartbeatToVT( + PartitionConsumptionState partitionConsumptionState, + PubSubTopicPartition topicPartition, + PubSubProducerCallback callback, + LeaderMetadataWrapper leaderMetadataWrapper, + LeaderCompleteState leaderCompleteState, + long originTimeStampMs) { + storeIngestionTask.sendIngestionHeartbeat( + partitionConsumptionState, + topicPartition, + callback, + leaderMetadataWrapper, + true, + true, + leaderCompleteState, + originTimeStampMs); + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); From c9f8a783b9e439d01c035d620e2760ba699caee7 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Wed, 5 Feb 2025 14:22:06 -0800 Subject: [PATCH 10/32] =?UTF-8?q?Refactored=20`processMessageAndMaybeProdu?= =?UTF-8?q?ceToKafka()`=20from=20`LeaderFollowerStoreIngestionTask`=20/=20?= =?UTF-8?q?`ActiveActiveStoreIngestionTask`=20into=20`StorePartitionDataRe?= =?UTF-8?q?ceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 192 ++++---- .../LeaderFollowerStoreIngestionTask.java | 410 ++++++++---------- .../kafka/consumer/StoreIngestionTask.java | 30 +- .../consumer/StorePartitionDataReceiver.java | 332 +++++++++++++- .../ActiveActiveStoreIngestionTaskTest.java | 28 +- .../LeaderFollowerStoreIngestionTaskTest.java | 5 +- 6 files changed, 669 insertions(+), 328 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 7c85d5ee657..73a76ad7c56 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -406,7 +406,8 @@ protected IngestionBatchProcessor getIngestionBatchProcessor() { return ingestionBatchProcessorLazy.get(); } - private PubSubMessageProcessedResult processActiveActiveMessage( + @Override + protected PubSubMessageProcessedResult processActiveActiveMessage( PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, int partition, @@ -585,99 +586,99 @@ private PubSubMessageProcessedResult processActiveActiveMessage( } } - // This function may modify the original record in KME, it is unsafe to use the payload from KME directly after - // this function. - protected void processMessageAndMaybeProduceToKafka( - PubSubMessageProcessedResultWrapper consumerRecordWrapper, - PartitionConsumptionState partitionConsumptionState, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs, - long beforeProcessingBatchRecordsTimestampMs) { - /** - * With {@link BatchConflictResolutionPolicy.BATCH_WRITE_LOSES} there is no need - * to perform DCR before EOP and L/F DIV passthrough mode should be used. If the version is going through data - * recovery then there is no need to perform DCR until we completed data recovery and switched to consume from RT. - * TODO. We need to refactor this logic when we support other batch conflict resolution policy. - */ - if (!partitionConsumptionState.isEndOfPushReceived() - || isDataRecovery && partitionConsumptionState.getTopicSwitch() != null) { - super.processMessageAndMaybeProduceToKafka( - consumerRecordWrapper, - partitionConsumptionState, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs); - return; - } - PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); - KafkaKey kafkaKey = consumerRecord.getKey(); - byte[] keyBytes = kafkaKey.getKey(); - final MergeConflictResultWrapper mergeConflictResultWrapper; - if (consumerRecordWrapper.getProcessedResult() != null - && consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper() != null) { - mergeConflictResultWrapper = consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper(); - } else { - mergeConflictResultWrapper = processActiveActiveMessage( - consumerRecord, - partitionConsumptionState, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs).getMergeConflictResultWrapper(); - } - - MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); - if (!mergeConflictResult.isUpdateIgnored()) { - // Apply this update to any views for this store - // TODO: It'd be good to be able to do this in LeaderFollowerStoreIngestionTask instead, however, AA currently is - // the - // only extension of IngestionTask which does a read from disk before applying the record. This makes the - // following function - // call in this context much less obtrusive, however, it implies that all views can only work for AA stores - - // Write to views - Runnable produceToVersionTopic = () -> producePutOrDeleteToKafka( - mergeConflictResultWrapper, - partitionConsumptionState, - keyBytes, - consumerRecord, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs); - if (hasViewWriters()) { - /** - * The ordering guarantees we want is the following: - * - * 1. Write to all view topics (in parallel). - * 2. Write to the VT only after we get the ack for all views AND the previous write to VT was queued into the - * producer (but not necessarily acked). - */ - ByteBuffer oldValueBB = mergeConflictResultWrapper.getOldValueByteBufferProvider().get(); - int oldValueSchemaId = - oldValueBB == null ? -1 : mergeConflictResultWrapper.getOldValueProvider().get().writerSchemaId(); - queueUpVersionTopicWritesWithViewWriters( - partitionConsumptionState, - (viewWriter) -> viewWriter.processRecord( - mergeConflictResultWrapper.getUpdatedValueBytes(), - oldValueBB, - keyBytes, - mergeConflictResult.getValueSchemaId(), - oldValueSchemaId, - mergeConflictResult.getRmdRecord()), - produceToVersionTopic); - } else { - // This function may modify the original record in KME and it is unsafe to use the payload from KME directly - // after this call. - produceToVersionTopic.run(); - } - } - } + // // This function may modify the original record in KME, it is unsafe to use the payload from KME directly after + // // this function. + // protected void processMessageAndMaybeProduceToKafka( + // PubSubMessageProcessedResultWrapper consumerRecordWrapper, + // PartitionConsumptionState partitionConsumptionState, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingRecordTimestampNs, + // long beforeProcessingBatchRecordsTimestampMs) { + // /** + // * With {@link BatchConflictResolutionPolicy.BATCH_WRITE_LOSES} there is no need + // * to perform DCR before EOP and L/F DIV passthrough mode should be used. If the version is going through data + // * recovery then there is no need to perform DCR until we completed data recovery and switched to consume from RT. + // * TODO. We need to refactor this logic when we support other batch conflict resolution policy. + // */ + // if (!partitionConsumptionState.isEndOfPushReceived() + // || isDataRecovery && partitionConsumptionState.getTopicSwitch() != null) { + // super.processMessageAndMaybeProduceToKafka( + // consumerRecordWrapper, + // partitionConsumptionState, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs, + // beforeProcessingBatchRecordsTimestampMs); + // return; + // } + // PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); + // KafkaKey kafkaKey = consumerRecord.getKey(); + // byte[] keyBytes = kafkaKey.getKey(); + // final MergeConflictResultWrapper mergeConflictResultWrapper; + // if (consumerRecordWrapper.getProcessedResult() != null + // && consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper() != null) { + // mergeConflictResultWrapper = consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper(); + // } else { + // mergeConflictResultWrapper = processActiveActiveMessage( + // consumerRecord, + // partitionConsumptionState, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs, + // beforeProcessingBatchRecordsTimestampMs).getMergeConflictResultWrapper(); + // } + // + // MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); + // if (!mergeConflictResult.isUpdateIgnored()) { + // // Apply this update to any views for this store + // // TODO: It'd be good to be able to do this in LeaderFollowerStoreIngestionTask instead, however, AA currently is + // // the + // // only extension of IngestionTask which does a read from disk before applying the record. This makes the + // // following function + // // call in this context much less obtrusive, however, it implies that all views can only work for AA stores + // + // // Write to views + // Runnable produceToVersionTopic = () -> producePutOrDeleteToKafka( + // mergeConflictResultWrapper, + // partitionConsumptionState, + // keyBytes, + // consumerRecord, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs); + // if (hasViewWriters()) { + // /** + // * The ordering guarantees we want is the following: + // * + // * 1. Write to all view topics (in parallel). + // * 2. Write to the VT only after we get the ack for all views AND the previous write to VT was queued into the + // * producer (but not necessarily acked). + // */ + // ByteBuffer oldValueBB = mergeConflictResultWrapper.getOldValueByteBufferProvider().get(); + // int oldValueSchemaId = + // oldValueBB == null ? -1 : mergeConflictResultWrapper.getOldValueProvider().get().writerSchemaId(); + // queueUpVersionTopicWritesWithViewWriters( + // partitionConsumptionState, + // (viewWriter) -> viewWriter.processRecord( + // mergeConflictResultWrapper.getUpdatedValueBytes(), + // oldValueBB, + // keyBytes, + // mergeConflictResult.getValueSchemaId(), + // oldValueSchemaId, + // mergeConflictResult.getRmdRecord()), + // produceToVersionTopic); + // } else { + // // This function may modify the original record in KME and it is unsafe to use the payload from KME directly + // // after this call. + // produceToVersionTopic.run(); + // } + // } + // } /** * Package private for testing purposes. @@ -811,7 +812,8 @@ ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.Transien * @param partition * @param kafkaUrl */ - private void producePutOrDeleteToKafka( + @Override + protected void producePutOrDeleteToKafka( MergeConflictResultWrapper mergeConflictResultWrapper, PartitionConsumptionState partitionConsumptionState, byte[] key, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 8ee3deb4f3d..49713e90c4d 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -10,7 +10,6 @@ import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_PUSH; import static com.linkedin.venice.kafka.protocol.enums.MessageType.UPDATE; import static com.linkedin.venice.pubsub.api.PubSubMessageHeaders.VENICE_LEADER_COMPLETION_STATE_HEADER; -import static com.linkedin.venice.writer.VeniceWriter.APP_DEFAULT_LOGICAL_TS; import static java.util.concurrent.TimeUnit.MILLISECONDS; import com.linkedin.davinci.client.DaVinciRecordTransformerConfig; @@ -37,7 +36,6 @@ import com.linkedin.venice.exceptions.VeniceTimeoutException; import com.linkedin.venice.guid.GuidUtils; import com.linkedin.venice.kafka.protocol.ControlMessage; -import com.linkedin.venice.kafka.protocol.Delete; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.Put; import com.linkedin.venice.kafka.protocol.TopicSwitch; @@ -101,7 +99,6 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; -import java.util.function.Function; import java.util.function.LongPredicate; import java.util.function.Predicate; import java.util.function.Supplier; @@ -386,6 +383,11 @@ public void closeVeniceWriters(boolean doFlush) { } } + @Override + public Map getViewWriters() { + return viewWriters; + } + @Override protected void closeVeniceViewWriters() { if (!viewWriters.isEmpty()) { @@ -3210,7 +3212,8 @@ protected boolean shouldCompressData(PartitionConsumptionState partitionConsumpt return !compressionStrategy.equals(CompressionStrategy.NO_OP); } - private PubSubMessageProcessedResult processMessage( + @Override + protected PubSubMessageProcessedResult processMessage( PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, int partition, @@ -3359,191 +3362,191 @@ private PubSubMessageProcessedResult processMessage( } } - @Override - protected void processMessageAndMaybeProduceToKafka( - PubSubMessageProcessedResultWrapper consumerRecordWrapper, - PartitionConsumptionState partitionConsumptionState, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs, - long beforeProcessingBatchRecordsTimestampMs) { - PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); - KafkaKey kafkaKey = consumerRecord.getKey(); - KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); - byte[] keyBytes = kafkaKey.getKey(); - MessageType msgType = MessageType.valueOf(kafkaValue.messageType); - - WriteComputeResultWrapper writeComputeResultWrapper; - if (consumerRecordWrapper.getProcessedResult() != null - && consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper() != null) { - writeComputeResultWrapper = consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper(); - } else { - writeComputeResultWrapper = processMessage( - consumerRecord, - partitionConsumptionState, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs).getWriteComputeResultWrapper(); - } - if (msgType.equals(UPDATE) && writeComputeResultWrapper.isSkipProduce()) { - return; - } - Runnable produceToVersionTopic = () -> produceToLocalKafkaHelper( - consumerRecord, - partitionConsumptionState, - writeComputeResultWrapper, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs); - // Write to views - if (hasViewWriters()) { - Put newPut = writeComputeResultWrapper.getNewPut(); - // keys will be serialized with chunk suffix during pass-through mode in L/F NR if chunking is enabled - boolean isChunkedKey = isChunked() && !partitionConsumptionState.isEndOfPushReceived(); - queueUpVersionTopicWritesWithViewWriters( - partitionConsumptionState, - (viewWriter) -> viewWriter.processRecord(newPut.putValue, keyBytes, newPut.schemaId, isChunkedKey), - produceToVersionTopic); - } else { - produceToVersionTopic.run(); - } - } - - private void produceToLocalKafkaHelper( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - WriteComputeResultWrapper writeComputeResultWrapper, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs) { - KafkaKey kafkaKey = consumerRecord.getKey(); - KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); - byte[] keyBytes = kafkaKey.getKey(); - MessageType msgType = MessageType.valueOf(kafkaValue.messageType); - LeaderProducedRecordContext leaderProducedRecordContext; - Put newPut = writeComputeResultWrapper.getNewPut(); - switch (msgType) { - case PUT: - leaderProducedRecordContext = - LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); - produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - (callback, leaderMetadataWrapper) -> { - /** - * 1. Unfortunately, Kafka does not support fancy array manipulation via {@link ByteBuffer} or otherwise, - * so we may be forced to do a copy here, if the backing array of the {@link putValue} has padding, - * which is the case when using {@link com.linkedin.venice.serialization.avro.OptimizedKafkaValueSerializer}. - * Since this is in a closure, it is not guaranteed to be invoked. - * - * The {@link OnlineOfflineStoreIngestionTask}, which ignores this closure, will not pay this price. - * - * Conversely, the {@link LeaderFollowerStoreIngestionTask}, which does invoke it, will. - * - * TODO: Evaluate holistically what is the best way to optimize GC for the L/F case. - * - * 2. Enable venice writer "pass-through" mode if we haven't received EOP yet. In pass through mode, - * Leader will reuse upstream producer metadata. This would secures the correctness of DIV states in - * followers when the leadership failover happens. - */ - - if (!partitionConsumptionState.isEndOfPushReceived()) { - partitionConsumptionState.getVeniceWriterLazyRef() - .get() - .put( - kafkaKey, - kafkaValue, - callback, - consumerRecord.getTopicPartition().getPartitionNumber(), - leaderMetadataWrapper); - } else { - partitionConsumptionState.getVeniceWriterLazyRef() - .get() - .put( - keyBytes, - ByteUtils.extractByteArray(newPut.putValue), - newPut.schemaId, - callback, - leaderMetadataWrapper); - } - }, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs); - break; - - case UPDATE: - leaderProducedRecordContext = - LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); - BiConsumer produceFunction = - (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() - .get() - .put( - keyBytes, - ByteUtils.extractByteArray(newPut.getPutValue()), - newPut.getSchemaId(), - callback, - leaderMetadataWrapper, - APP_DEFAULT_LOGICAL_TS, - null, - writeComputeResultWrapper.getOldValueManifest(), - null); - - produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - produceFunction, - partitionConsumptionState.getPartition(), - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs); - break; - - case DELETE: - leaderProducedRecordContext = LeaderProducedRecordContext - .newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, (Delete) kafkaValue.payloadUnion); - produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - (callback, leaderMetadataWrapper) -> { - /** - * DIV pass-through for DELETE messages before EOP. - */ - if (!partitionConsumptionState.isEndOfPushReceived()) { - partitionConsumptionState.getVeniceWriterLazyRef() - .get() - .delete( - kafkaKey, - kafkaValue, - callback, - consumerRecord.getTopicPartition().getPartitionNumber(), - leaderMetadataWrapper); - } else { - partitionConsumptionState.getVeniceWriterLazyRef() - .get() - .delete(keyBytes, callback, leaderMetadataWrapper); - } - }, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs); - break; - - default: - throw new VeniceMessageException( - ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); - } - } + // @Override + // protected void processMessageAndMaybeProduceToKafka( + // PubSubMessageProcessedResultWrapper consumerRecordWrapper, + // PartitionConsumptionState partitionConsumptionState, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingRecordTimestampNs, + // long beforeProcessingBatchRecordsTimestampMs) { + // PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); + // KafkaKey kafkaKey = consumerRecord.getKey(); + // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + // byte[] keyBytes = kafkaKey.getKey(); + // MessageType msgType = MessageType.valueOf(kafkaValue.messageType); + // + // WriteComputeResultWrapper writeComputeResultWrapper; + // if (consumerRecordWrapper.getProcessedResult() != null + // && consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper() != null) { + // writeComputeResultWrapper = consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper(); + // } else { + // writeComputeResultWrapper = processMessage( + // consumerRecord, + // partitionConsumptionState, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs, + // beforeProcessingBatchRecordsTimestampMs).getWriteComputeResultWrapper(); + // } + // if (msgType.equals(UPDATE) && writeComputeResultWrapper.isSkipProduce()) { + // return; + // } + // Runnable produceToVersionTopic = () -> produceToLocalKafkaHelper( + // consumerRecord, + // partitionConsumptionState, + // writeComputeResultWrapper, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs); + // // Write to views + // if (hasViewWriters()) { + // Put newPut = writeComputeResultWrapper.getNewPut(); + // // keys will be serialized with chunk suffix during pass-through mode in L/F NR if chunking is enabled + // boolean isChunkedKey = isChunked() && !partitionConsumptionState.isEndOfPushReceived(); + // queueUpVersionTopicWritesWithViewWriters( + // partitionConsumptionState, + // (viewWriter) -> viewWriter.processRecord(newPut.putValue, keyBytes, newPut.schemaId, isChunkedKey), + // produceToVersionTopic); + // } else { + // produceToVersionTopic.run(); + // } + // } + // + // private void produceToLocalKafkaHelper( + // PubSubMessage consumerRecord, + // PartitionConsumptionState partitionConsumptionState, + // WriteComputeResultWrapper writeComputeResultWrapper, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingRecordTimestampNs) { + // KafkaKey kafkaKey = consumerRecord.getKey(); + // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + // byte[] keyBytes = kafkaKey.getKey(); + // MessageType msgType = MessageType.valueOf(kafkaValue.messageType); + // LeaderProducedRecordContext leaderProducedRecordContext; + // Put newPut = writeComputeResultWrapper.getNewPut(); + // switch (msgType) { + // case PUT: + // leaderProducedRecordContext = + // LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); + // produceToLocalKafka( + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // (callback, leaderMetadataWrapper) -> { + // /** + // * 1. Unfortunately, Kafka does not support fancy array manipulation via {@link ByteBuffer} or otherwise, + // * so we may be forced to do a copy here, if the backing array of the {@link putValue} has padding, + // * which is the case when using {@link com.linkedin.venice.serialization.avro.OptimizedKafkaValueSerializer}. + // * Since this is in a closure, it is not guaranteed to be invoked. + // * + // * The {@link OnlineOfflineStoreIngestionTask}, which ignores this closure, will not pay this price. + // * + // * Conversely, the {@link LeaderFollowerStoreIngestionTask}, which does invoke it, will. + // * + // * TODO: Evaluate holistically what is the best way to optimize GC for the L/F case. + // * + // * 2. Enable venice writer "pass-through" mode if we haven't received EOP yet. In pass through mode, + // * Leader will reuse upstream producer metadata. This would secures the correctness of DIV states in + // * followers when the leadership failover happens. + // */ + // + // if (!partitionConsumptionState.isEndOfPushReceived()) { + // partitionConsumptionState.getVeniceWriterLazyRef() + // .get() + // .put( + // kafkaKey, + // kafkaValue, + // callback, + // consumerRecord.getTopicPartition().getPartitionNumber(), + // leaderMetadataWrapper); + // } else { + // partitionConsumptionState.getVeniceWriterLazyRef() + // .get() + // .put( + // keyBytes, + // ByteUtils.extractByteArray(newPut.putValue), + // newPut.schemaId, + // callback, + // leaderMetadataWrapper); + // } + // }, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs); + // break; + // + // case UPDATE: + // leaderProducedRecordContext = + // LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); + // BiConsumer produceFunction = + // (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() + // .get() + // .put( + // keyBytes, + // ByteUtils.extractByteArray(newPut.getPutValue()), + // newPut.getSchemaId(), + // callback, + // leaderMetadataWrapper, + // APP_DEFAULT_LOGICAL_TS, + // null, + // writeComputeResultWrapper.getOldValueManifest(), + // null); + // + // produceToLocalKafka( + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // produceFunction, + // partitionConsumptionState.getPartition(), + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs); + // break; + // + // case DELETE: + // leaderProducedRecordContext = LeaderProducedRecordContext + // .newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, (Delete) kafkaValue.payloadUnion); + // produceToLocalKafka( + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // (callback, leaderMetadataWrapper) -> { + // /** + // * DIV pass-through for DELETE messages before EOP. + // */ + // if (!partitionConsumptionState.isEndOfPushReceived()) { + // partitionConsumptionState.getVeniceWriterLazyRef() + // .get() + // .delete( + // kafkaKey, + // kafkaValue, + // callback, + // consumerRecord.getTopicPartition().getPartitionNumber(), + // leaderMetadataWrapper); + // } else { + // partitionConsumptionState.getVeniceWriterLazyRef() + // .get() + // .delete(keyBytes, callback, leaderMetadataWrapper); + // } + // }, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs); + // break; + // + // default: + // throw new VeniceMessageException( + // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); + // } + // } /** * Read the existing value. If a value for this key is found from the transient map then use that value, otherwise read @@ -3973,35 +3976,6 @@ protected void resubscribeAsLeader(PartitionConsumptionState partitionConsumptio } } - protected void queueUpVersionTopicWritesWithViewWriters( - PartitionConsumptionState partitionConsumptionState, - Function> viewWriterRecordProcessor, - Runnable versionTopicWrite) { - long preprocessingTime = System.currentTimeMillis(); - CompletableFuture currentVersionTopicWrite = new CompletableFuture<>(); - CompletableFuture[] viewWriterFutures = new CompletableFuture[this.viewWriters.size() + 1]; - int index = 0; - // The first future is for the previous write to VT - viewWriterFutures[index++] = partitionConsumptionState.getLastVTProduceCallFuture(); - for (VeniceViewWriter writer: viewWriters.values()) { - viewWriterFutures[index++] = viewWriterRecordProcessor.apply(writer); - } - hostLevelIngestionStats.recordViewProducerLatency(LatencyUtils.getElapsedTimeFromMsToMs(preprocessingTime)); - CompletableFuture.allOf(viewWriterFutures).whenCompleteAsync((value, exception) -> { - hostLevelIngestionStats.recordViewProducerAckLatency(LatencyUtils.getElapsedTimeFromMsToMs(preprocessingTime)); - if (exception == null) { - versionTopicWrite.run(); - currentVersionTopicWrite.complete(null); - } else { - VeniceException veniceException = new VeniceException(exception); - this.setIngestionException(partitionConsumptionState.getPartition(), veniceException); - currentVersionTopicWrite.completeExceptionally(veniceException); - } - }); - - partitionConsumptionState.setLastVTProduceCallFuture(currentVersionTopicWrite); - } - /** * Once leader is marked completed, immediately reset {@link #lastSendIngestionHeartbeatTimestamp} * such that {@link #maybeSendIngestionHeartbeat()} will send HB SOS to the respective RT topics diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index d46b5a31b95..0e8c150f1d4 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -42,6 +42,7 @@ import com.linkedin.davinci.store.StoragePartitionConfig; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; import com.linkedin.davinci.store.record.ValueRecord; +import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.utils.ChunkAssembler; import com.linkedin.davinci.validation.KafkaDataIntegrityValidator; import com.linkedin.davinci.validation.PartitionTracker; @@ -1926,6 +1927,8 @@ public void closeVeniceWriters(boolean doFlush) { protected void closeVeniceViewWriters() { } + public abstract Map getViewWriters(); + /** * Consumes the kafka actions messages in the queue. */ @@ -4758,8 +4761,8 @@ protected abstract LeaderProducerCallback createProducerCallback( String kafkaUrl, long beforeProcessingRecordTimestampNs); - protected abstract void processMessageAndMaybeProduceToKafka( - PubSubMessageProcessedResultWrapper consumerRecordWrapper, + protected abstract PubSubMessageProcessedResult processMessage( + PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, int partition, String kafkaUrl, @@ -4767,6 +4770,29 @@ protected abstract void processMessageAndMaybeProduceToKafka( long beforeProcessingRecordTimestampNs, long beforeProcessingBatchRecordsTimestampMs); + protected PubSubMessageProcessedResult processActiveActiveMessage( + PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs) { + throw new VeniceException("processActiveActiveMessage() should only be called in active active mode"); + } + + protected void producePutOrDeleteToKafka( + MergeConflictResultWrapper mergeConflictResultWrapper, + PartitionConsumptionState partitionConsumptionState, + byte[] key, + PubSubMessage consumerRecord, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs) { + throw new VeniceException("producePutOrDeleteToKafka() should only be called in active active mode"); + } + protected abstract void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState); protected abstract boolean hasViewWriters(); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index f563394b2be..d59f92feb96 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -4,7 +4,9 @@ import com.linkedin.avroutil1.compatibility.shaded.org.apache.commons.lang3.Validate; import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; +import com.linkedin.davinci.replication.merge.MergeConflictResult; import com.linkedin.davinci.stats.HostLevelIngestionStats; +import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.utils.ByteArrayKey; import com.linkedin.davinci.validation.PartitionTracker; import com.linkedin.davinci.validation.PartitionTracker.TopicType; @@ -13,6 +15,7 @@ import com.linkedin.venice.exceptions.validation.DuplicateDataException; import com.linkedin.venice.exceptions.validation.FatalDataValidationException; import com.linkedin.venice.kafka.protocol.ControlMessage; +import com.linkedin.venice.kafka.protocol.Delete; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.Put; import com.linkedin.venice.kafka.protocol.Update; @@ -22,27 +25,34 @@ import com.linkedin.venice.message.KafkaKey; import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.api.PubSubMessage; +import com.linkedin.venice.pubsub.api.PubSubProduceResult; import com.linkedin.venice.pubsub.api.PubSubProducerCallback; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; +import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.ExceptionUtils; import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.ValueHolder; +import com.linkedin.venice.writer.ChunkAwareCallback; import com.linkedin.venice.writer.LeaderCompleteState; import com.linkedin.venice.writer.LeaderMetadataWrapper; import com.linkedin.venice.writer.VeniceWriter; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.NavigableMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.locks.ReentrantLock; +import java.util.function.BiConsumer; +import java.util.function.Function; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -935,7 +945,7 @@ DelegateConsumerRecordResult delegateConsumerRecord( } else { // This function may modify the original record in KME and it is unsafe to use the payload from KME directly // after this call. - storeIngestionTask.processMessageAndMaybeProduceToKafka( + processMessageAndMaybeProduceToKafka( consumerRecordWrapper, partitionConsumptionState, partition, @@ -984,6 +994,326 @@ private void maybeQueueCMWritesToVersionTopic( * * Extend this function when there is new check needed. */ + void processMessageAndMaybeProduceToKafka( + PubSubMessageProcessedResultWrapper consumerRecordWrapper, + PartitionConsumptionState partitionConsumptionState, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs) { + /** + * With {@link BatchConflictResolutionPolicy.BATCH_WRITE_LOSES} there is no need + * to perform DCR before EOP and L/F DIV passthrough mode should be used. If the version is going through data + * recovery then there is no need to perform DCR until we completed data recovery and switched to consume from RT. + * TODO. We need to refactor this logic when we support other batch conflict resolution policy. + */ + if (storeIngestionTask.isActiveActiveReplicationEnabled() && partitionConsumptionState.isEndOfPushReceived() + && (!storeIngestionTask.isDataRecovery() || partitionConsumptionState.getTopicSwitch() == null)) { + processActiveActiveMessageAndMaybeProduceToKafka( + consumerRecordWrapper, + partitionConsumptionState, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); + return; + } + + PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); + KafkaKey kafkaKey = consumerRecord.getKey(); + KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + byte[] keyBytes = kafkaKey.getKey(); + MessageType msgType = MessageType.valueOf(kafkaValue.messageType); + + WriteComputeResultWrapper writeComputeResultWrapper; + if (consumerRecordWrapper.getProcessedResult() != null + && consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper() != null) { + writeComputeResultWrapper = consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper(); + } else { + writeComputeResultWrapper = + storeIngestionTask + .processMessage( + consumerRecord, + partitionConsumptionState, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs) + .getWriteComputeResultWrapper(); + } + + if (msgType.equals(MessageType.UPDATE) && writeComputeResultWrapper.isSkipProduce()) { + return; + } + Runnable produceToVersionTopic = () -> produceToLocalKafkaHelper( + consumerRecord, + partitionConsumptionState, + writeComputeResultWrapper, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs); + // Write to views + if (storeIngestionTask.hasViewWriters()) { + Put newPut = writeComputeResultWrapper.getNewPut(); + // keys will be serialized with chunk suffix during pass-through mode in L/F NR if chunking is enabled + boolean isChunkedKey = storeIngestionTask.isChunked() && !partitionConsumptionState.isEndOfPushReceived(); + queueUpVersionTopicWritesWithViewWriters( + partitionConsumptionState, + (viewWriter) -> viewWriter.processRecord(newPut.putValue, keyBytes, newPut.schemaId, isChunkedKey), + produceToVersionTopic); + } else { + produceToVersionTopic.run(); + } + } + + private void produceToLocalKafkaHelper( + PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, + WriteComputeResultWrapper writeComputeResultWrapper, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs) { + KafkaKey kafkaKey = consumerRecord.getKey(); + KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + byte[] keyBytes = kafkaKey.getKey(); + MessageType msgType = MessageType.valueOf(kafkaValue.messageType); + LeaderProducedRecordContext leaderProducedRecordContext; + Put newPut = writeComputeResultWrapper.getNewPut(); + switch (msgType) { + case PUT: + leaderProducedRecordContext = + LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); + storeIngestionTask.produceToLocalKafka( + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + (callback, leaderMetadataWrapper) -> { + /** + * 1. Unfortunately, Kafka does not support fancy array manipulation via {@link ByteBuffer} or otherwise, + * so we may be forced to do a copy here, if the backing array of the {@link putValue} has padding, + * which is the case when using {@link com.linkedin.venice.serialization.avro.OptimizedKafkaValueSerializer}. + * Since this is in a closure, it is not guaranteed to be invoked. + * + * The {@link OnlineOfflineStoreIngestionTask}, which ignores this closure, will not pay this price. + * + * Conversely, the {@link LeaderFollowerStoreIngestionTask}, which does invoke it, will. + * + * TODO: Evaluate holistically what is the best way to optimize GC for the L/F case. + * + * 2. Enable venice writer "pass-through" mode if we haven't received EOP yet. In pass through mode, + * Leader will reuse upstream producer metadata. This would secures the correctness of DIV states in + * followers when the leadership failover happens. + */ + + if (!partitionConsumptionState.isEndOfPushReceived()) { + partitionConsumptionState.getVeniceWriterLazyRef() + .get() + .put( + kafkaKey, + kafkaValue, + callback, + consumerRecord.getTopicPartition().getPartitionNumber(), + leaderMetadataWrapper); + } else { + partitionConsumptionState.getVeniceWriterLazyRef() + .get() + .put( + keyBytes, + ByteUtils.extractByteArray(newPut.putValue), + newPut.schemaId, + callback, + leaderMetadataWrapper); + } + }, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs); + break; + + case UPDATE: + leaderProducedRecordContext = + LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); + BiConsumer produceFunction = + (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() + .get() + .put( + keyBytes, + ByteUtils.extractByteArray(newPut.getPutValue()), + newPut.getSchemaId(), + callback, + leaderMetadataWrapper, + VeniceWriter.APP_DEFAULT_LOGICAL_TS, + null, + writeComputeResultWrapper.getOldValueManifest(), + null); + + storeIngestionTask.produceToLocalKafka( + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + produceFunction, + partitionConsumptionState.getPartition(), + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs); + break; + + case DELETE: + leaderProducedRecordContext = LeaderProducedRecordContext + .newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, (Delete) kafkaValue.payloadUnion); + storeIngestionTask.produceToLocalKafka( + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + (callback, leaderMetadataWrapper) -> { + /** + * DIV pass-through for DELETE messages before EOP. + */ + if (!partitionConsumptionState.isEndOfPushReceived()) { + partitionConsumptionState.getVeniceWriterLazyRef() + .get() + .delete( + kafkaKey, + kafkaValue, + callback, + consumerRecord.getTopicPartition().getPartitionNumber(), + leaderMetadataWrapper); + } else { + partitionConsumptionState.getVeniceWriterLazyRef() + .get() + .delete(keyBytes, callback, leaderMetadataWrapper); + } + }, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs); + break; + + default: + throw new VeniceMessageException( + storeIngestionTask.getIngestionTaskName() + " : Invalid/Unrecognized operation type submitted: " + + kafkaValue.messageType); + } + } + + // This function may modify the original record in KME, it is unsafe to use the payload from KME directly after + // this function. + private void processActiveActiveMessageAndMaybeProduceToKafka( + PubSubMessageProcessedResultWrapper consumerRecordWrapper, + PartitionConsumptionState partitionConsumptionState, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs) { + PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); + KafkaKey kafkaKey = consumerRecord.getKey(); + byte[] keyBytes = kafkaKey.getKey(); + final MergeConflictResultWrapper mergeConflictResultWrapper; + if (consumerRecordWrapper.getProcessedResult() != null + && consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper() != null) { + mergeConflictResultWrapper = consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper(); + } else { + mergeConflictResultWrapper = + storeIngestionTask + .processActiveActiveMessage( + consumerRecord, + partitionConsumptionState, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs) + .getMergeConflictResultWrapper(); + } + + MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); + if (!mergeConflictResult.isUpdateIgnored()) { + // Apply this update to any views for this store + // TODO: It'd be good to be able to do this in LeaderFollowerStoreIngestionTask instead, however, AA currently is + // the + // only extension of IngestionTask which does a read from disk before applying the record. This makes the + // following function + // call in this context much less obtrusive, however, it implies that all views can only work for AA stores + + // Write to views + Runnable produceToVersionTopic = () -> storeIngestionTask.producePutOrDeleteToKafka( + mergeConflictResultWrapper, + partitionConsumptionState, + keyBytes, + consumerRecord, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs); + if (storeIngestionTask.hasViewWriters()) { + /** + * The ordering guarantees we want is the following: + * + * 1. Write to all view topics (in parallel). + * 2. Write to the VT only after we get the ack for all views AND the previous write to VT was queued into the + * producer (but not necessarily acked). + */ + ByteBuffer oldValueBB = mergeConflictResultWrapper.getOldValueByteBufferProvider().get(); + int oldValueSchemaId = + oldValueBB == null ? -1 : mergeConflictResultWrapper.getOldValueProvider().get().writerSchemaId(); + queueUpVersionTopicWritesWithViewWriters( + partitionConsumptionState, + (viewWriter) -> viewWriter.processRecord( + mergeConflictResultWrapper.getUpdatedValueBytes(), + oldValueBB, + keyBytes, + mergeConflictResult.getValueSchemaId(), + oldValueSchemaId, + mergeConflictResult.getRmdRecord()), + produceToVersionTopic); + } else { + // This function may modify the original record in KME and it is unsafe to use the payload from KME directly + // after this call. + produceToVersionTopic.run(); + } + } + } + + void queueUpVersionTopicWritesWithViewWriters( + PartitionConsumptionState partitionConsumptionState, + Function> viewWriterRecordProcessor, + Runnable versionTopicWrite) { + long preprocessingTime = System.currentTimeMillis(); + CompletableFuture currentVersionTopicWrite = new CompletableFuture<>(); + Map viewWriters = storeIngestionTask.getViewWriters(); + CompletableFuture[] viewWriterFutures = new CompletableFuture[viewWriters.size() + 1]; + int index = 0; + // The first future is for the previous write to VT + viewWriterFutures[index++] = partitionConsumptionState.getLastVTProduceCallFuture(); + for (VeniceViewWriter writer: viewWriters.values()) { + viewWriterFutures[index++] = viewWriterRecordProcessor.apply(writer); + } + + HostLevelIngestionStats hostLevelIngestionStats = storeIngestionTask.getHostLevelIngestionStats(); + hostLevelIngestionStats.recordViewProducerLatency(LatencyUtils.getElapsedTimeFromMsToMs(preprocessingTime)); + CompletableFuture.allOf(viewWriterFutures).whenCompleteAsync((value, exception) -> { + hostLevelIngestionStats.recordViewProducerAckLatency(LatencyUtils.getElapsedTimeFromMsToMs(preprocessingTime)); + if (exception == null) { + versionTopicWrite.run(); + currentVersionTopicWrite.complete(null); + } else { + VeniceException veniceException = new VeniceException(exception); + storeIngestionTask.setIngestionException(partitionConsumptionState.getPartition(), veniceException); + currentVersionTopicWrite.completeExceptionally(veniceException); + } + }); + + partitionConsumptionState.setLastVTProduceCallFuture(currentVersionTopicWrite); + } + private void validateRecordBeforeProducingToLocalKafka( PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java index 945615055f1..13516cb877c 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java @@ -12,6 +12,7 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -130,29 +131,34 @@ public static Object[] compressionStrategyProvider() { @Test public void testHandleDeleteBeforeEOP() { ActiveActiveStoreIngestionTask ingestionTask = mock(ActiveActiveStoreIngestionTask.class); - doCallRealMethod().when(ingestionTask) - .processMessageAndMaybeProduceToKafka(any(), any(), anyInt(), anyString(), anyInt(), anyLong(), anyLong()); + PubSubTopicPartition topicPartition = mock(PubSubTopicPartition.class); + VeniceServerConfig serverConfig = mock(VeniceServerConfig.class); + when(serverConfig.isComputeFastAvroEnabled()).thenReturn(false); + when(ingestionTask.getServerConfig()).thenReturn(serverConfig); + when(ingestionTask.getHostLevelIngestionStats()).thenReturn(mock(HostLevelIngestionStats.class)); + StorePartitionDataReceiver storePartitionDataReceiver = + spy(new StorePartitionDataReceiver(ingestionTask, topicPartition, "dummyUrl", 0)); PartitionConsumptionState pcs = mock(PartitionConsumptionState.class); when(pcs.isEndOfPushReceived()).thenReturn(false); + when(pcs.getVeniceWriterLazyRef()).thenReturn(Lazy.of(() -> mock(VeniceWriter.class))); PubSubMessage consumerRecord = mock(PubSubMessage.class); - KafkaKey kafkaKey = mock(KafkaKey.class); + KafkaKey kafkaKey = new KafkaKey(MessageType.DELETE, new byte[] { 1 }); when(consumerRecord.getKey()).thenReturn(kafkaKey); KafkaMessageEnvelope kafkaValue = new KafkaMessageEnvelope(); when(consumerRecord.getValue()).thenReturn(kafkaValue); when(consumerRecord.getOffset()).thenReturn(1L); + when(consumerRecord.getTopicPartition()).thenReturn(topicPartition); kafkaValue.messageType = MessageType.DELETE.getValue(); Delete deletePayload = new Delete(); kafkaValue.payloadUnion = deletePayload; + PubSubMessageProcessedResult result = + new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, true)); + PubSubMessageProcessedResultWrapper resultWrapper = + new PubSubMessageProcessedResultWrapper<>(consumerRecord); + resultWrapper.setProcessedResult(result); ArgumentCaptor leaderProducedRecordContextArgumentCaptor = ArgumentCaptor.forClass(LeaderProducedRecordContext.class); - ingestionTask.processMessageAndMaybeProduceToKafka( - new PubSubMessageProcessedResultWrapper<>(consumerRecord), - pcs, - 0, - "dummyUrl", - 0, - 0L, - 0L); + storePartitionDataReceiver.processMessageAndMaybeProduceToKafka(resultWrapper, pcs, 0, "dummyUrl", 0, 0L, 0L); verify(ingestionTask, times(1)).produceToLocalKafka( any(), any(), diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTaskTest.java index bb1f7a318b4..2a5ad46db17 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTaskTest.java @@ -7,6 +7,7 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -289,7 +290,9 @@ public void testQueueUpVersionTopicWritesWithViewWriters() throws InterruptedExc AtomicBoolean writeToVersionTopic = new AtomicBoolean(false); when(mockPartitionConsumptionState.getLastVTProduceCallFuture()) .thenReturn(CompletableFuture.completedFuture(null)); - leaderFollowerStoreIngestionTask.queueUpVersionTopicWritesWithViewWriters( + StorePartitionDataReceiver storePartitionDataReceiver = + spy(new StorePartitionDataReceiver(leaderFollowerStoreIngestionTask, mockTopicPartition, "dummyUrl", 0)); + storePartitionDataReceiver.queueUpVersionTopicWritesWithViewWriters( mockPartitionConsumptionState, (viewWriter) -> viewWriter.processRecord(mock(ByteBuffer.class), new byte[1], 1, false), () -> writeToVersionTopic.set(true)); From 983792d3a0e001cf65faad948fbcaff0bbfb8a27 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 00:30:29 -0700 Subject: [PATCH 11/32] =?UTF-8?q?Refactored=20`producePutOrDeleteToKafka()?= =?UTF-8?q?`=20from=20`ActiveActiveStoreIngestionTask`=20into=20`StorePart?= =?UTF-8?q?itionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 192 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 24 ++- .../consumer/StorePartitionDataReceiver.java | 100 ++++++++- 3 files changed, 213 insertions(+), 103 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 73a76ad7c56..eb11f5c9c11 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -51,7 +51,6 @@ import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.lazy.Lazy; import com.linkedin.venice.writer.ChunkAwareCallback; -import com.linkedin.venice.writer.DeleteMetadata; import com.linkedin.venice.writer.LeaderMetadataWrapper; import com.linkedin.venice.writer.PutMetadata; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; @@ -798,97 +797,97 @@ ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.Transien } } - /** - * This function parses the {@link MergeConflictResult} and decides if the update should be ignored or emit a PUT or a - * DELETE record to VT. - *

- * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after - * this function. - * - * @param mergeConflictResultWrapper The result of conflict resolution. - * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition - * @param key The key bytes of the incoming record. - * @param consumerRecord The {@link PubSubMessage} for the current record. - * @param partition - * @param kafkaUrl - */ - @Override - protected void producePutOrDeleteToKafka( - MergeConflictResultWrapper mergeConflictResultWrapper, - PartitionConsumptionState partitionConsumptionState, - byte[] key, - PubSubMessage consumerRecord, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs) { - MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); - ByteBuffer updatedValueBytes = mergeConflictResultWrapper.getUpdatedValueBytes(); - ByteBuffer updatedRmdBytes = mergeConflictResultWrapper.getUpdatedRmdBytes(); - final int valueSchemaId = mergeConflictResult.getValueSchemaId(); - - ChunkedValueManifest oldValueManifest = mergeConflictResultWrapper.getOldValueManifestContainer().getManifest(); - ChunkedValueManifest oldRmdManifest = mergeConflictResultWrapper.getOldRmdWithValueSchemaId() == null - ? null - : mergeConflictResultWrapper.getOldRmdWithValueSchemaId().getRmdManifest(); - // finally produce - if (mergeConflictResultWrapper.getUpdatedValueBytes() == null) { - hostLevelIngestionStats.recordTombstoneCreatedDCR(); - aggVersionedIngestionStats.recordTombStoneCreationDCR(storeName, versionNumber); - Delete deletePayload = new Delete(); - deletePayload.schemaId = valueSchemaId; - deletePayload.replicationMetadataVersionId = rmdProtocolVersionId; - deletePayload.replicationMetadataPayload = mergeConflictResultWrapper.getUpdatedRmdBytes(); - BiConsumer produceToTopicFunction = - (callback, sourceTopicOffset) -> partitionConsumptionState.getVeniceWriterLazyRef() - .get() - .delete( - key, - callback, - sourceTopicOffset, - APP_DEFAULT_LOGICAL_TS, - new DeleteMetadata(valueSchemaId, rmdProtocolVersionId, updatedRmdBytes), - oldValueManifest, - oldRmdManifest); - LeaderProducedRecordContext leaderProducedRecordContext = - LeaderProducedRecordContext.newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), key, deletePayload); - produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - produceToTopicFunction, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs); - } else { - Put updatedPut = new Put(); - updatedPut.putValue = ByteUtils - .prependIntHeaderToByteBuffer(updatedValueBytes, valueSchemaId, mergeConflictResult.doesResultReuseInput()); - updatedPut.schemaId = valueSchemaId; - updatedPut.replicationMetadataVersionId = rmdProtocolVersionId; - updatedPut.replicationMetadataPayload = updatedRmdBytes; - - BiConsumer produceToTopicFunction = getProduceToTopicFunction( - partitionConsumptionState, - key, - updatedValueBytes, - updatedRmdBytes, - oldValueManifest, - oldRmdManifest, - valueSchemaId, - mergeConflictResult.doesResultReuseInput()); - produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), key, updatedPut), - produceToTopicFunction, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs); - } - } +// /** +// * This function parses the {@link MergeConflictResult} and decides if the update should be ignored or emit a PUT or a +// * DELETE record to VT. +// *

+// * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after +// * this function. +// * +// * @param mergeConflictResultWrapper The result of conflict resolution. +// * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition +// * @param key The key bytes of the incoming record. +// * @param consumerRecord The {@link PubSubMessage} for the current record. +// * @param partition +// * @param kafkaUrl +// */ +// @Override +// protected void producePutOrDeleteToKafka( +// MergeConflictResultWrapper mergeConflictResultWrapper, +// PartitionConsumptionState partitionConsumptionState, +// byte[] key, +// PubSubMessage consumerRecord, +// int partition, +// String kafkaUrl, +// int kafkaClusterId, +// long beforeProcessingRecordTimestampNs) { +// MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); +// ByteBuffer updatedValueBytes = mergeConflictResultWrapper.getUpdatedValueBytes(); +// ByteBuffer updatedRmdBytes = mergeConflictResultWrapper.getUpdatedRmdBytes(); +// final int valueSchemaId = mergeConflictResult.getValueSchemaId(); +// +// ChunkedValueManifest oldValueManifest = mergeConflictResultWrapper.getOldValueManifestContainer().getManifest(); +// ChunkedValueManifest oldRmdManifest = mergeConflictResultWrapper.getOldRmdWithValueSchemaId() == null +// ? null +// : mergeConflictResultWrapper.getOldRmdWithValueSchemaId().getRmdManifest(); +// // finally produce +// if (mergeConflictResultWrapper.getUpdatedValueBytes() == null) { +// hostLevelIngestionStats.recordTombstoneCreatedDCR(); +// aggVersionedIngestionStats.recordTombStoneCreationDCR(storeName, versionNumber); +// Delete deletePayload = new Delete(); +// deletePayload.schemaId = valueSchemaId; +// deletePayload.replicationMetadataVersionId = rmdProtocolVersionId; +// deletePayload.replicationMetadataPayload = mergeConflictResultWrapper.getUpdatedRmdBytes(); +// BiConsumer produceToTopicFunction = +// (callback, sourceTopicOffset) -> partitionConsumptionState.getVeniceWriterLazyRef() +// .get() +// .delete( +// key, +// callback, +// sourceTopicOffset, +// APP_DEFAULT_LOGICAL_TS, +// new DeleteMetadata(valueSchemaId, rmdProtocolVersionId, updatedRmdBytes), +// oldValueManifest, +// oldRmdManifest); +// LeaderProducedRecordContext leaderProducedRecordContext = +// LeaderProducedRecordContext.newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), key, deletePayload); +// produceToLocalKafka( +// consumerRecord, +// partitionConsumptionState, +// leaderProducedRecordContext, +// produceToTopicFunction, +// partition, +// kafkaUrl, +// kafkaClusterId, +// beforeProcessingRecordTimestampNs); +// } else { +// Put updatedPut = new Put(); +// updatedPut.putValue = ByteUtils +// .prependIntHeaderToByteBuffer(updatedValueBytes, valueSchemaId, mergeConflictResult.doesResultReuseInput()); +// updatedPut.schemaId = valueSchemaId; +// updatedPut.replicationMetadataVersionId = rmdProtocolVersionId; +// updatedPut.replicationMetadataPayload = updatedRmdBytes; +// +// BiConsumer produceToTopicFunction = getProduceToTopicFunction( +// partitionConsumptionState, +// key, +// updatedValueBytes, +// updatedRmdBytes, +// oldValueManifest, +// oldRmdManifest, +// valueSchemaId, +// mergeConflictResult.doesResultReuseInput()); +// produceToLocalKafka( +// consumerRecord, +// partitionConsumptionState, +// LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), key, updatedPut), +// produceToTopicFunction, +// partition, +// kafkaUrl, +// kafkaClusterId, +// beforeProcessingRecordTimestampNs); +// } +// } @Override protected void produceToLocalKafka( @@ -1442,7 +1441,8 @@ Runnable buildRepairTask( }; } - int getRmdProtocolVersionId() { + @Override + public int getRmdProtocolVersionId() { return rmdProtocolVersionId; } @@ -1451,6 +1451,12 @@ public final Lazy getKeyLevelLocksManager() { return keyLevelLocksManager; } + @Override + public AggVersionedIngestionStats getAggVersionedIngestionStats() { + return aggVersionedIngestionStats; + } + + @Override protected BiConsumer getProduceToTopicFunction( PartitionConsumptionState partitionConsumptionState, byte[] key, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 0e8c150f1d4..3c91099d43e 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -4781,16 +4781,24 @@ protected PubSubMessageProcessedResult processActiveActiveMessage( throw new VeniceException("processActiveActiveMessage() should only be called in active active mode"); } - protected void producePutOrDeleteToKafka( - MergeConflictResultWrapper mergeConflictResultWrapper, + protected BiConsumer getProduceToTopicFunction( PartitionConsumptionState partitionConsumptionState, byte[] key, - PubSubMessage consumerRecord, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs) { - throw new VeniceException("producePutOrDeleteToKafka() should only be called in active active mode"); + ByteBuffer updatedValueBytes, + ByteBuffer updatedRmdBytes, + ChunkedValueManifest oldValueManifest, + ChunkedValueManifest oldRmdManifest, + int valueSchemaId, + boolean resultReuseInput) { + throw new VeniceException("getProduceToTopicFunction() should only be called in active active mode"); + } + + public AggVersionedIngestionStats getAggVersionedIngestionStats() { + throw new VeniceException("getAggVersionedIngestionStats() should only be called in active active mode"); + } + + public int getRmdProtocolVersionId() { + throw new VeniceException("getRmdProtocolVersionId() should only be called in active active mode"); } protected abstract void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index d59f92feb96..f159dcdb170 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -1,6 +1,7 @@ package com.linkedin.davinci.kafka.consumer; import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_PUSH; +import static com.linkedin.venice.writer.VeniceWriter.APP_DEFAULT_LOGICAL_TS; import com.linkedin.avroutil1.compatibility.shaded.org.apache.commons.lang3.Validate; import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; @@ -30,12 +31,14 @@ import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; +import com.linkedin.venice.storage.protocol.ChunkedValueManifest; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.ExceptionUtils; import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.ValueHolder; import com.linkedin.venice.writer.ChunkAwareCallback; +import com.linkedin.venice.writer.DeleteMetadata; import com.linkedin.venice.writer.LeaderCompleteState; import com.linkedin.venice.writer.LeaderMetadataWrapper; import com.linkedin.venice.writer.VeniceWriter; @@ -1148,7 +1151,7 @@ private void produceToLocalKafkaHelper( newPut.getSchemaId(), callback, leaderMetadataWrapper, - VeniceWriter.APP_DEFAULT_LOGICAL_TS, + APP_DEFAULT_LOGICAL_TS, null, writeComputeResultWrapper.getOldValueManifest(), null); @@ -1244,7 +1247,7 @@ private void processActiveActiveMessageAndMaybeProduceToKafka( // call in this context much less obtrusive, however, it implies that all views can only work for AA stores // Write to views - Runnable produceToVersionTopic = () -> storeIngestionTask.producePutOrDeleteToKafka( + Runnable produceToVersionTopic = () -> producePutOrDeleteToKafka( mergeConflictResultWrapper, partitionConsumptionState, keyBytes, @@ -1314,6 +1317,99 @@ void queueUpVersionTopicWritesWithViewWriters( partitionConsumptionState.setLastVTProduceCallFuture(currentVersionTopicWrite); } + /** + * This function parses the {@link MergeConflictResult} and decides if the update should be ignored or emit a PUT or a + * DELETE record to VT. + *

+ * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after + * this function. + * + * @param mergeConflictResultWrapper The result of conflict resolution. + * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition + * @param key The key bytes of the incoming record. + * @param consumerRecord The {@link PubSubMessage} for the current record. + * @param partition + * @param kafkaUrl + */ + private void producePutOrDeleteToKafka( + MergeConflictResultWrapper mergeConflictResultWrapper, + PartitionConsumptionState partitionConsumptionState, + byte[] key, + PubSubMessage consumerRecord, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs) { + MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); + ByteBuffer updatedValueBytes = mergeConflictResultWrapper.getUpdatedValueBytes(); + ByteBuffer updatedRmdBytes = mergeConflictResultWrapper.getUpdatedRmdBytes(); + final int valueSchemaId = mergeConflictResult.getValueSchemaId(); + + ChunkedValueManifest oldValueManifest = mergeConflictResultWrapper.getOldValueManifestContainer().getManifest(); + ChunkedValueManifest oldRmdManifest = mergeConflictResultWrapper.getOldRmdWithValueSchemaId() == null + ? null + : mergeConflictResultWrapper.getOldRmdWithValueSchemaId().getRmdManifest(); + // finally produce + if (mergeConflictResultWrapper.getUpdatedValueBytes() == null) { + storeIngestionTask.getHostLevelIngestionStats().recordTombstoneCreatedDCR(); + storeIngestionTask.getAggVersionedIngestionStats() + .recordTombStoneCreationDCR(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); + Delete deletePayload = new Delete(); + deletePayload.schemaId = valueSchemaId; + deletePayload.replicationMetadataVersionId = storeIngestionTask.getRmdProtocolVersionId(); + deletePayload.replicationMetadataPayload = mergeConflictResultWrapper.getUpdatedRmdBytes(); + BiConsumer produceToTopicFunction = + (callback, sourceTopicOffset) -> partitionConsumptionState.getVeniceWriterLazyRef() + .get() + .delete( + key, + callback, + sourceTopicOffset, + APP_DEFAULT_LOGICAL_TS, + new DeleteMetadata(valueSchemaId, storeIngestionTask.getRmdProtocolVersionId(), updatedRmdBytes), + oldValueManifest, + oldRmdManifest); + LeaderProducedRecordContext leaderProducedRecordContext = + LeaderProducedRecordContext.newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), key, deletePayload); + storeIngestionTask.produceToLocalKafka( + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + produceToTopicFunction, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs); + } else { + Put updatedPut = new Put(); + updatedPut.putValue = ByteUtils + .prependIntHeaderToByteBuffer(updatedValueBytes, valueSchemaId, mergeConflictResult.doesResultReuseInput()); + updatedPut.schemaId = valueSchemaId; + updatedPut.replicationMetadataVersionId = storeIngestionTask.getRmdProtocolVersionId(); + updatedPut.replicationMetadataPayload = updatedRmdBytes; + + BiConsumer produceToTopicFunction = + storeIngestionTask.getProduceToTopicFunction( + partitionConsumptionState, + key, + updatedValueBytes, + updatedRmdBytes, + oldValueManifest, + oldRmdManifest, + valueSchemaId, + mergeConflictResult.doesResultReuseInput()); + storeIngestionTask.produceToLocalKafka( + consumerRecord, + partitionConsumptionState, + LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), key, updatedPut), + produceToTopicFunction, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs); + } + } + private void validateRecordBeforeProducingToLocalKafka( PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, From 9435ddcd45f8c91558a4e4b764730e5020ea8046 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 00:38:45 -0700 Subject: [PATCH 12/32] =?UTF-8?q?Refactored=20`getProduceToTopicFunction()?= =?UTF-8?q?`=20from=20`ActiveActiveStoreIngestionTask`=20into=20`StorePart?= =?UTF-8?q?itionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 252 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 5 +- .../consumer/StorePartitionDataReceiver.java | 56 +++- .../ActiveActiveStoreIngestionTaskTest.java | 11 +- 4 files changed, 181 insertions(+), 143 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index eb11f5c9c11..53cd8c9d406 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -2,7 +2,6 @@ import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.LEADER; import static com.linkedin.venice.VeniceConstants.REWIND_TIME_DECIDED_BY_SERVER; -import static com.linkedin.venice.writer.VeniceWriter.APP_DEFAULT_LOGICAL_TS; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.davinci.client.DaVinciRecordTransformerConfig; @@ -44,7 +43,6 @@ import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.schema.rmd.RmdUtils; import com.linkedin.venice.serialization.RawBytesStoreDeserializerCache; -import com.linkedin.venice.storage.protocol.ChunkedValueManifest; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Time; @@ -52,7 +50,6 @@ import com.linkedin.venice.utils.lazy.Lazy; import com.linkedin.venice.writer.ChunkAwareCallback; import com.linkedin.venice.writer.LeaderMetadataWrapper; -import com.linkedin.venice.writer.PutMetadata; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import java.io.IOException; import java.nio.ByteBuffer; @@ -797,97 +794,98 @@ ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.Transien } } -// /** -// * This function parses the {@link MergeConflictResult} and decides if the update should be ignored or emit a PUT or a -// * DELETE record to VT. -// *

-// * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after -// * this function. -// * -// * @param mergeConflictResultWrapper The result of conflict resolution. -// * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition -// * @param key The key bytes of the incoming record. -// * @param consumerRecord The {@link PubSubMessage} for the current record. -// * @param partition -// * @param kafkaUrl -// */ -// @Override -// protected void producePutOrDeleteToKafka( -// MergeConflictResultWrapper mergeConflictResultWrapper, -// PartitionConsumptionState partitionConsumptionState, -// byte[] key, -// PubSubMessage consumerRecord, -// int partition, -// String kafkaUrl, -// int kafkaClusterId, -// long beforeProcessingRecordTimestampNs) { -// MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); -// ByteBuffer updatedValueBytes = mergeConflictResultWrapper.getUpdatedValueBytes(); -// ByteBuffer updatedRmdBytes = mergeConflictResultWrapper.getUpdatedRmdBytes(); -// final int valueSchemaId = mergeConflictResult.getValueSchemaId(); -// -// ChunkedValueManifest oldValueManifest = mergeConflictResultWrapper.getOldValueManifestContainer().getManifest(); -// ChunkedValueManifest oldRmdManifest = mergeConflictResultWrapper.getOldRmdWithValueSchemaId() == null -// ? null -// : mergeConflictResultWrapper.getOldRmdWithValueSchemaId().getRmdManifest(); -// // finally produce -// if (mergeConflictResultWrapper.getUpdatedValueBytes() == null) { -// hostLevelIngestionStats.recordTombstoneCreatedDCR(); -// aggVersionedIngestionStats.recordTombStoneCreationDCR(storeName, versionNumber); -// Delete deletePayload = new Delete(); -// deletePayload.schemaId = valueSchemaId; -// deletePayload.replicationMetadataVersionId = rmdProtocolVersionId; -// deletePayload.replicationMetadataPayload = mergeConflictResultWrapper.getUpdatedRmdBytes(); -// BiConsumer produceToTopicFunction = -// (callback, sourceTopicOffset) -> partitionConsumptionState.getVeniceWriterLazyRef() -// .get() -// .delete( -// key, -// callback, -// sourceTopicOffset, -// APP_DEFAULT_LOGICAL_TS, -// new DeleteMetadata(valueSchemaId, rmdProtocolVersionId, updatedRmdBytes), -// oldValueManifest, -// oldRmdManifest); -// LeaderProducedRecordContext leaderProducedRecordContext = -// LeaderProducedRecordContext.newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), key, deletePayload); -// produceToLocalKafka( -// consumerRecord, -// partitionConsumptionState, -// leaderProducedRecordContext, -// produceToTopicFunction, -// partition, -// kafkaUrl, -// kafkaClusterId, -// beforeProcessingRecordTimestampNs); -// } else { -// Put updatedPut = new Put(); -// updatedPut.putValue = ByteUtils -// .prependIntHeaderToByteBuffer(updatedValueBytes, valueSchemaId, mergeConflictResult.doesResultReuseInput()); -// updatedPut.schemaId = valueSchemaId; -// updatedPut.replicationMetadataVersionId = rmdProtocolVersionId; -// updatedPut.replicationMetadataPayload = updatedRmdBytes; -// -// BiConsumer produceToTopicFunction = getProduceToTopicFunction( -// partitionConsumptionState, -// key, -// updatedValueBytes, -// updatedRmdBytes, -// oldValueManifest, -// oldRmdManifest, -// valueSchemaId, -// mergeConflictResult.doesResultReuseInput()); -// produceToLocalKafka( -// consumerRecord, -// partitionConsumptionState, -// LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), key, updatedPut), -// produceToTopicFunction, -// partition, -// kafkaUrl, -// kafkaClusterId, -// beforeProcessingRecordTimestampNs); -// } -// } + // /** + // * This function parses the {@link MergeConflictResult} and decides if the update should be ignored or emit a PUT or + // a + // * DELETE record to VT. + // *

+ // * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after + // * this function. + // * + // * @param mergeConflictResultWrapper The result of conflict resolution. + // * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition + // * @param key The key bytes of the incoming record. + // * @param consumerRecord The {@link PubSubMessage} for the current record. + // * @param partition + // * @param kafkaUrl + // */ + // @Override + // protected void producePutOrDeleteToKafka( + // MergeConflictResultWrapper mergeConflictResultWrapper, + // PartitionConsumptionState partitionConsumptionState, + // byte[] key, + // PubSubMessage consumerRecord, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingRecordTimestampNs) { + // MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); + // ByteBuffer updatedValueBytes = mergeConflictResultWrapper.getUpdatedValueBytes(); + // ByteBuffer updatedRmdBytes = mergeConflictResultWrapper.getUpdatedRmdBytes(); + // final int valueSchemaId = mergeConflictResult.getValueSchemaId(); + // + // ChunkedValueManifest oldValueManifest = mergeConflictResultWrapper.getOldValueManifestContainer().getManifest(); + // ChunkedValueManifest oldRmdManifest = mergeConflictResultWrapper.getOldRmdWithValueSchemaId() == null + // ? null + // : mergeConflictResultWrapper.getOldRmdWithValueSchemaId().getRmdManifest(); + // // finally produce + // if (mergeConflictResultWrapper.getUpdatedValueBytes() == null) { + // hostLevelIngestionStats.recordTombstoneCreatedDCR(); + // aggVersionedIngestionStats.recordTombStoneCreationDCR(storeName, versionNumber); + // Delete deletePayload = new Delete(); + // deletePayload.schemaId = valueSchemaId; + // deletePayload.replicationMetadataVersionId = rmdProtocolVersionId; + // deletePayload.replicationMetadataPayload = mergeConflictResultWrapper.getUpdatedRmdBytes(); + // BiConsumer produceToTopicFunction = + // (callback, sourceTopicOffset) -> partitionConsumptionState.getVeniceWriterLazyRef() + // .get() + // .delete( + // key, + // callback, + // sourceTopicOffset, + // APP_DEFAULT_LOGICAL_TS, + // new DeleteMetadata(valueSchemaId, rmdProtocolVersionId, updatedRmdBytes), + // oldValueManifest, + // oldRmdManifest); + // LeaderProducedRecordContext leaderProducedRecordContext = + // LeaderProducedRecordContext.newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), key, deletePayload); + // produceToLocalKafka( + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // produceToTopicFunction, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs); + // } else { + // Put updatedPut = new Put(); + // updatedPut.putValue = ByteUtils + // .prependIntHeaderToByteBuffer(updatedValueBytes, valueSchemaId, mergeConflictResult.doesResultReuseInput()); + // updatedPut.schemaId = valueSchemaId; + // updatedPut.replicationMetadataVersionId = rmdProtocolVersionId; + // updatedPut.replicationMetadataPayload = updatedRmdBytes; + // + // BiConsumer produceToTopicFunction = getProduceToTopicFunction( + // partitionConsumptionState, + // key, + // updatedValueBytes, + // updatedRmdBytes, + // oldValueManifest, + // oldRmdManifest, + // valueSchemaId, + // mergeConflictResult.doesResultReuseInput()); + // produceToLocalKafka( + // consumerRecord, + // partitionConsumptionState, + // LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), key, updatedPut), + // produceToTopicFunction, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs); + // } + // } @Override protected void produceToLocalKafka( @@ -1456,39 +1454,39 @@ public AggVersionedIngestionStats getAggVersionedIngestionStats() { return aggVersionedIngestionStats; } - @Override - protected BiConsumer getProduceToTopicFunction( - PartitionConsumptionState partitionConsumptionState, - byte[] key, - ByteBuffer updatedValueBytes, - ByteBuffer updatedRmdBytes, - ChunkedValueManifest oldValueManifest, - ChunkedValueManifest oldRmdManifest, - int valueSchemaId, - boolean resultReuseInput) { - return (callback, leaderMetadataWrapper) -> { - if (resultReuseInput) { - // Restore the original header so this function is eventually idempotent as the original KME ByteBuffer - // will be recovered after producing the message to Kafka or if the production failing. - ((ActiveActiveProducerCallback) callback).setOnCompletionFunction( - () -> ByteUtils.prependIntHeaderToByteBuffer( - updatedValueBytes, - ByteUtils.getIntHeaderFromByteBuffer(updatedValueBytes), - true)); - } - getVeniceWriter(partitionConsumptionState).get() - .put( - key, - ByteUtils.extractByteArray(updatedValueBytes), - valueSchemaId, - callback, - leaderMetadataWrapper, - APP_DEFAULT_LOGICAL_TS, - new PutMetadata(getRmdProtocolVersionId(), updatedRmdBytes), - oldValueManifest, - oldRmdManifest); - }; - } + // @Override + // protected BiConsumer getProduceToTopicFunction( + // PartitionConsumptionState partitionConsumptionState, + // byte[] key, + // ByteBuffer updatedValueBytes, + // ByteBuffer updatedRmdBytes, + // ChunkedValueManifest oldValueManifest, + // ChunkedValueManifest oldRmdManifest, + // int valueSchemaId, + // boolean resultReuseInput) { + // return (callback, leaderMetadataWrapper) -> { + // if (resultReuseInput) { + // // Restore the original header so this function is eventually idempotent as the original KME ByteBuffer + // // will be recovered after producing the message to Kafka or if the production failing. + // ((ActiveActiveProducerCallback) callback).setOnCompletionFunction( + // () -> ByteUtils.prependIntHeaderToByteBuffer( + // updatedValueBytes, + // ByteUtils.getIntHeaderFromByteBuffer(updatedValueBytes), + // true)); + // } + // getVeniceWriter(partitionConsumptionState).get() + // .put( + // key, + // ByteUtils.extractByteArray(updatedValueBytes), + // valueSchemaId, + // callback, + // leaderMetadataWrapper, + // APP_DEFAULT_LOGICAL_TS, + // new PutMetadata(getRmdProtocolVersionId(), updatedRmdBytes), + // oldValueManifest, + // oldRmdManifest); + // }; + // } protected LeaderProducerCallback createProducerCallback( PubSubMessage consumerRecord, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 3c91099d43e..84b676a60ae 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -122,6 +122,7 @@ import com.linkedin.venice.writer.ChunkAwareCallback; import com.linkedin.venice.writer.LeaderCompleteState; import com.linkedin.venice.writer.LeaderMetadataWrapper; +import com.linkedin.venice.writer.VeniceWriter; import it.unimi.dsi.fastutil.objects.Object2IntMap; import java.io.Closeable; import java.io.IOException; @@ -4782,7 +4783,6 @@ protected PubSubMessageProcessedResult processActiveActiveMessage( } protected BiConsumer getProduceToTopicFunction( - PartitionConsumptionState partitionConsumptionState, byte[] key, ByteBuffer updatedValueBytes, ByteBuffer updatedRmdBytes, @@ -4801,6 +4801,9 @@ public int getRmdProtocolVersionId() { throw new VeniceException("getRmdProtocolVersionId() should only be called in active active mode"); } + protected abstract Lazy> getVeniceWriter( + PartitionConsumptionState partitionConsumptionState); + protected abstract void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState); protected abstract boolean hasViewWriters(); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index f159dcdb170..4c22178a066 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -41,6 +41,7 @@ import com.linkedin.venice.writer.DeleteMetadata; import com.linkedin.venice.writer.LeaderCompleteState; import com.linkedin.venice.writer.LeaderMetadataWrapper; +import com.linkedin.venice.writer.PutMetadata; import com.linkedin.venice.writer.VeniceWriter; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -1365,7 +1366,7 @@ private void producePutOrDeleteToKafka( key, callback, sourceTopicOffset, - APP_DEFAULT_LOGICAL_TS, + VeniceWriter.APP_DEFAULT_LOGICAL_TS, new DeleteMetadata(valueSchemaId, storeIngestionTask.getRmdProtocolVersionId(), updatedRmdBytes), oldValueManifest, oldRmdManifest); @@ -1388,16 +1389,15 @@ private void producePutOrDeleteToKafka( updatedPut.replicationMetadataVersionId = storeIngestionTask.getRmdProtocolVersionId(); updatedPut.replicationMetadataPayload = updatedRmdBytes; - BiConsumer produceToTopicFunction = - storeIngestionTask.getProduceToTopicFunction( - partitionConsumptionState, - key, - updatedValueBytes, - updatedRmdBytes, - oldValueManifest, - oldRmdManifest, - valueSchemaId, - mergeConflictResult.doesResultReuseInput()); + BiConsumer produceToTopicFunction = getProduceToTopicFunction( + partitionConsumptionState, + key, + updatedValueBytes, + updatedRmdBytes, + oldValueManifest, + oldRmdManifest, + valueSchemaId, + mergeConflictResult.doesResultReuseInput()); storeIngestionTask.produceToLocalKafka( consumerRecord, partitionConsumptionState, @@ -1410,6 +1410,40 @@ private void producePutOrDeleteToKafka( } } + BiConsumer getProduceToTopicFunction( + PartitionConsumptionState partitionConsumptionState, + byte[] key, + ByteBuffer updatedValueBytes, + ByteBuffer updatedRmdBytes, + ChunkedValueManifest oldValueManifest, + ChunkedValueManifest oldRmdManifest, + int valueSchemaId, + boolean resultReuseInput) { + return (callback, leaderMetadataWrapper) -> { + if (resultReuseInput) { + // Restore the original header so this function is eventually idempotent as the original KME ByteBuffer + // will be recovered after producing the message to Kafka or if the production failing. + ((ActiveActiveProducerCallback) callback).setOnCompletionFunction( + () -> ByteUtils.prependIntHeaderToByteBuffer( + updatedValueBytes, + ByteUtils.getIntHeaderFromByteBuffer(updatedValueBytes), + true)); + } + storeIngestionTask.getVeniceWriter(partitionConsumptionState) + .get() + .put( + key, + ByteUtils.extractByteArray(updatedValueBytes), + valueSchemaId, + callback, + leaderMetadataWrapper, + VeniceWriter.APP_DEFAULT_LOGICAL_TS, + new PutMetadata(storeIngestionTask.getRmdProtocolVersionId(), updatedRmdBytes), + oldValueManifest, + oldRmdManifest); + }; + } + private void validateRecordBeforeProducingToLocalKafka( PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java index 13516cb877c..5951061eac2 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java @@ -4,7 +4,6 @@ import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; import static com.linkedin.venice.ConfigKeys.ZOOKEEPER_ADDRESS; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyDouble; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; @@ -333,11 +332,15 @@ public void testLeaderCanSendValueChunksIntoDrainer() throws InterruptedExceptio when(ingestionTask.getKafkaVersionTopic()).thenReturn(testTopic); when(ingestionTask.createProducerCallback(any(), any(), any(), anyInt(), anyString(), anyLong())) .thenCallRealMethod(); - when(ingestionTask.getProduceToTopicFunction(any(), any(), any(), any(), any(), any(), anyInt(), anyBoolean())) - .thenCallRealMethod(); when(ingestionTask.getRmdProtocolVersionId()).thenReturn(rmdProtocolVersionID); doCallRealMethod().when(ingestionTask) .produceToLocalKafka(any(), any(), any(), any(), anyInt(), anyString(), anyInt(), anyLong()); + PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository(); + PubSubTopicPartition topicPartition = + new PubSubTopicPartitionImpl(pubSubTopicRepository.getTopic(testTopic), partition); + StorePartitionDataReceiver storePartitionDataReceiver = + new StorePartitionDataReceiver(ingestionTask, topicPartition, kafkaUrl, kafkaClusterId); + byte[] key = "foo".getBytes(); byte[] updatedKeyBytes = ChunkingUtils.KEY_WITH_CHUNKING_SUFFIX_SERIALIZER.serializeNonChunkedKey(key); @@ -410,7 +413,7 @@ public void testLeaderCanSendValueChunksIntoDrainer() throws InterruptedExceptio consumerRecord, partitionConsumptionState, leaderProducedRecordContext, - ingestionTask.getProduceToTopicFunction( + storePartitionDataReceiver.getProduceToTopicFunction( partitionConsumptionState, updatedKeyBytes, updatedValueBytes, From 6f6c6eb44be0248f8d55f270d4f2538cf7b0ab48 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sun, 20 Oct 2024 19:40:41 -0700 Subject: [PATCH 13/32] =?UTF-8?q?Refactored=20`produceToLocalKafka()`=20fr?= =?UTF-8?q?om=20`LeaderFollowerStoreIngestionTask`=20/=20`ActiveActiveStor?= =?UTF-8?q?eIngestionTask`=20into=20`StorePartitionDataReceiver`.=20?= =?UTF-8?q?=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 57 +++++++++---------- .../LeaderFollowerStoreIngestionTask.java | 50 ++++++++-------- .../kafka/consumer/StoreIngestionTask.java | 23 -------- .../consumer/StorePartitionDataReceiver.java | 51 ++++++++++++++--- .../ActiveActiveStoreIngestionTaskTest.java | 7 +-- 5 files changed, 96 insertions(+), 92 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 53cd8c9d406..44fc5e444de 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -48,8 +48,6 @@ import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.lazy.Lazy; -import com.linkedin.venice.writer.ChunkAwareCallback; -import com.linkedin.venice.writer.LeaderMetadataWrapper; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import java.io.IOException; import java.nio.ByteBuffer; @@ -62,7 +60,6 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; -import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.BinaryDecoder; @@ -886,33 +883,33 @@ ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.Transien // beforeProcessingRecordTimestampNs); // } // } - - @Override - protected void produceToLocalKafka( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - LeaderProducedRecordContext leaderProducedRecordContext, - BiConsumer produceFunction, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs) { - super.produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - produceFunction, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs); - // Update the partition consumption state to say that we've transmitted the message to kafka (but haven't - // necessarily received an ack back yet). - if (partitionConsumptionState.getLeaderFollowerState() == LEADER && partitionConsumptionState.isHybrid() - && consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { - partitionConsumptionState.updateLatestRTOffsetTriedToProduceToVTMap(kafkaUrl, consumerRecord.getOffset()); - } - } +// +// @Override +// protected void produceToLocalKafka( +// PubSubMessage consumerRecord, +// PartitionConsumptionState partitionConsumptionState, +// LeaderProducedRecordContext leaderProducedRecordContext, +// BiConsumer produceFunction, +// int partition, +// String kafkaUrl, +// int kafkaClusterId, +// long beforeProcessingRecordTimestampNs) { +// super.produceToLocalKafka( +// consumerRecord, +// partitionConsumptionState, +// leaderProducedRecordContext, +// produceFunction, +// partition, +// kafkaUrl, +// kafkaClusterId, +// beforeProcessingRecordTimestampNs); +// // Update the partition consumption state to say that we've transmitted the message to kafka (but haven't +// // necessarily received an ack back yet). +// if (partitionConsumptionState.getLeaderFollowerState() == LEADER && partitionConsumptionState.isHybrid() +// && consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { +// partitionConsumptionState.updateLatestRTOffsetTriedToProduceToVTMap(kafkaUrl, consumerRecord.getOffset()); +// } +// } @Override protected Map calculateLeaderUpstreamOffsetWithTopicSwitch( diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 49713e90c4d..65093165c1c 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -71,7 +71,6 @@ import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import com.linkedin.venice.utils.lazy.Lazy; -import com.linkedin.venice.writer.ChunkAwareCallback; import com.linkedin.venice.writer.LeaderCompleteState; import com.linkedin.venice.writer.LeaderMetadataWrapper; import com.linkedin.venice.writer.VeniceWriter; @@ -97,7 +96,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; import java.util.function.LongPredicate; import java.util.function.Predicate; @@ -1689,30 +1687,30 @@ protected static void checkAndHandleUpstreamOffsetRewind( } } - protected void produceToLocalKafka( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - LeaderProducedRecordContext leaderProducedRecordContext, - BiConsumer produceFunction, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs) { - LeaderProducerCallback callback = createProducerCallback( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - partition, - kafkaUrl, - beforeProcessingRecordTimestampNs); - long sourceTopicOffset = consumerRecord.getOffset(); - LeaderMetadataWrapper leaderMetadataWrapper = new LeaderMetadataWrapper(sourceTopicOffset, kafkaClusterId); - partitionConsumptionState.setLastLeaderPersistFuture(leaderProducedRecordContext.getPersistedToDBFuture()); - long beforeProduceTimestampNS = System.nanoTime(); - produceFunction.accept(callback, leaderMetadataWrapper); - getHostLevelIngestionStats() - .recordLeaderProduceLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeProduceTimestampNS)); - } + // protected void produceToLocalKafka( + // PubSubMessage consumerRecord, + // PartitionConsumptionState partitionConsumptionState, + // LeaderProducedRecordContext leaderProducedRecordContext, + // BiConsumer produceFunction, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingRecordTimestampNs) { + // LeaderProducerCallback callback = createProducerCallback( + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // partition, + // kafkaUrl, + // beforeProcessingRecordTimestampNs); + // long sourceTopicOffset = consumerRecord.getOffset(); + // LeaderMetadataWrapper leaderMetadataWrapper = new LeaderMetadataWrapper(sourceTopicOffset, kafkaClusterId); + // partitionConsumptionState.setLastLeaderPersistFuture(leaderProducedRecordContext.getPersistedToDBFuture()); + // long beforeProduceTimestampNS = System.nanoTime(); + // produceFunction.accept(callback, leaderMetadataWrapper); + // getHostLevelIngestionStats() + // .recordLeaderProduceLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeProduceTimestampNS)); + // } @Override protected boolean isRealTimeBufferReplayStarted(PartitionConsumptionState partitionConsumptionState) { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 84b676a60ae..cf295cbeb5b 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -119,7 +119,6 @@ import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import com.linkedin.venice.utils.lazy.Lazy; -import com.linkedin.venice.writer.ChunkAwareCallback; import com.linkedin.venice.writer.LeaderCompleteState; import com.linkedin.venice.writer.LeaderMetadataWrapper; import com.linkedin.venice.writer.VeniceWriter; @@ -153,7 +152,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; @@ -4734,16 +4732,6 @@ protected abstract void updateLatestInMemoryLeaderConsumedRTOffset( String ignoredKafkaUrl, long offset); - protected abstract void produceToLocalKafka( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - LeaderProducedRecordContext leaderProducedRecordContext, - BiConsumer produceFunction, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs); - protected abstract CompletableFuture sendIngestionHeartbeat( PartitionConsumptionState partitionConsumptionState, PubSubTopicPartition topicPartition, @@ -4782,17 +4770,6 @@ protected PubSubMessageProcessedResult processActiveActiveMessage( throw new VeniceException("processActiveActiveMessage() should only be called in active active mode"); } - protected BiConsumer getProduceToTopicFunction( - byte[] key, - ByteBuffer updatedValueBytes, - ByteBuffer updatedRmdBytes, - ChunkedValueManifest oldValueManifest, - ChunkedValueManifest oldRmdManifest, - int valueSchemaId, - boolean resultReuseInput) { - throw new VeniceException("getProduceToTopicFunction() should only be called in active active mode"); - } - public AggVersionedIngestionStats getAggVersionedIngestionStats() { throw new VeniceException("getAggVersionedIngestionStats() should only be called in active active mode"); } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 4c22178a066..ea82cff9442 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -764,7 +764,7 @@ DelegateConsumerRecordResult delegateConsumerRecord( * Simply produce this EOP to local VT. It will be processed in order in the drainer queue later * after successfully producing to kafka. */ - storeIngestionTask.produceToLocalKafka( + produceToLocalKafka( consumerRecord, partitionConsumptionState, leaderProducedRecordContext, @@ -814,7 +814,7 @@ DelegateConsumerRecordResult delegateConsumerRecord( final LeaderProducedRecordContext segmentCMLeaderProduceRecordContext = leaderProducedRecordContext; maybeQueueCMWritesToVersionTopic( partitionConsumptionState, - () -> storeIngestionTask.produceToLocalKafka( + () -> produceToLocalKafka( consumerRecord, partitionConsumptionState, segmentCMLeaderProduceRecordContext, @@ -878,7 +878,7 @@ DelegateConsumerRecordResult delegateConsumerRecord( * to calculate DIV for this message but keeping the ControlMessage content unchanged. {@link VeniceWriter#put()} does not * allow that. */ - storeIngestionTask.produceToLocalKafka( + produceToLocalKafka( consumerRecord, partitionConsumptionState, leaderProducedRecordContext, @@ -909,7 +909,7 @@ DelegateConsumerRecordResult delegateConsumerRecord( } leaderProducedRecordContext = LeaderProducedRecordContext.newControlMessageRecord(kafkaKey.getKey(), controlMessage); - storeIngestionTask.produceToLocalKafka( + produceToLocalKafka( consumerRecord, partitionConsumptionState, leaderProducedRecordContext, @@ -1092,7 +1092,7 @@ private void produceToLocalKafkaHelper( case PUT: leaderProducedRecordContext = LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); - storeIngestionTask.produceToLocalKafka( + produceToLocalKafka( consumerRecord, partitionConsumptionState, leaderProducedRecordContext, @@ -1157,7 +1157,7 @@ private void produceToLocalKafkaHelper( writeComputeResultWrapper.getOldValueManifest(), null); - storeIngestionTask.produceToLocalKafka( + produceToLocalKafka( consumerRecord, partitionConsumptionState, leaderProducedRecordContext, @@ -1171,7 +1171,7 @@ private void produceToLocalKafkaHelper( case DELETE: leaderProducedRecordContext = LeaderProducedRecordContext .newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, (Delete) kafkaValue.payloadUnion); - storeIngestionTask.produceToLocalKafka( + produceToLocalKafka( consumerRecord, partitionConsumptionState, leaderProducedRecordContext, @@ -1372,7 +1372,7 @@ private void producePutOrDeleteToKafka( oldRmdManifest); LeaderProducedRecordContext leaderProducedRecordContext = LeaderProducedRecordContext.newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), key, deletePayload); - storeIngestionTask.produceToLocalKafka( + produceToLocalKafka( consumerRecord, partitionConsumptionState, leaderProducedRecordContext, @@ -1398,7 +1398,7 @@ private void producePutOrDeleteToKafka( oldRmdManifest, valueSchemaId, mergeConflictResult.doesResultReuseInput()); - storeIngestionTask.produceToLocalKafka( + produceToLocalKafka( consumerRecord, partitionConsumptionState, LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), key, updatedPut), @@ -1474,6 +1474,39 @@ private void validateRecordBeforeProducingToLocalKafka( } } + void produceToLocalKafka( + PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, + LeaderProducedRecordContext leaderProducedRecordContext, + BiConsumer produceFunction, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs) { + LeaderProducerCallback callback = storeIngestionTask.createProducerCallback( + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + partition, + kafkaUrl, + beforeProcessingRecordTimestampNs); + long sourceTopicOffset = consumerRecord.getOffset(); + LeaderMetadataWrapper leaderMetadataWrapper = new LeaderMetadataWrapper(sourceTopicOffset, kafkaClusterId); + partitionConsumptionState.setLastLeaderPersistFuture(leaderProducedRecordContext.getPersistedToDBFuture()); + long beforeProduceTimestampNS = System.nanoTime(); + produceFunction.accept(callback, leaderMetadataWrapper); + storeIngestionTask.getHostLevelIngestionStats() + .recordLeaderProduceLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeProduceTimestampNS)); + + // Update the partition consumption state to say that we've transmitted the message to kafka (but haven't + // necessarily received an ack back yet). + if (storeIngestionTask.isActiveActiveReplicationEnabled() + && partitionConsumptionState.getLeaderFollowerState() == LeaderFollowerStateType.LEADER + && partitionConsumptionState.isHybrid() && consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { + partitionConsumptionState.updateLatestRTOffsetTriedToProduceToVTMap(kafkaUrl, consumerRecord.getOffset()); + } + } + private void recordRegionHybridConsumptionStats( int kafkaClusterId, int producedRecordSize, diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java index 5951061eac2..4d9db60bab1 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java @@ -158,7 +158,7 @@ public void testHandleDeleteBeforeEOP() { ArgumentCaptor leaderProducedRecordContextArgumentCaptor = ArgumentCaptor.forClass(LeaderProducedRecordContext.class); storePartitionDataReceiver.processMessageAndMaybeProduceToKafka(resultWrapper, pcs, 0, "dummyUrl", 0, 0L, 0L); - verify(ingestionTask, times(1)).produceToLocalKafka( + verify(storePartitionDataReceiver, times(1)).produceToLocalKafka( any(), any(), leaderProducedRecordContextArgumentCaptor.capture(), @@ -326,6 +326,7 @@ public void testLeaderCanSendValueChunksIntoDrainer() throws InterruptedExceptio HostLevelIngestionStats mockHostLevelIngestionStats = mock(HostLevelIngestionStats.class); ActiveActiveStoreIngestionTask ingestionTask = mock(ActiveActiveStoreIngestionTask.class); + when(ingestionTask.isActiveActiveReplicationEnabled()).thenReturn(true); when(ingestionTask.getHostLevelIngestionStats()).thenReturn(mockHostLevelIngestionStats); when(ingestionTask.getVersionIngestionStats()).thenReturn(mock(AggVersionedIngestionStats.class)); when(ingestionTask.getVersionedDIVStats()).thenReturn(mock(AggVersionedDIVStats.class)); @@ -333,8 +334,6 @@ public void testLeaderCanSendValueChunksIntoDrainer() throws InterruptedExceptio when(ingestionTask.createProducerCallback(any(), any(), any(), anyInt(), anyString(), anyLong())) .thenCallRealMethod(); when(ingestionTask.getRmdProtocolVersionId()).thenReturn(rmdProtocolVersionID); - doCallRealMethod().when(ingestionTask) - .produceToLocalKafka(any(), any(), any(), any(), anyInt(), anyString(), anyInt(), anyLong()); PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository(); PubSubTopicPartition topicPartition = new PubSubTopicPartitionImpl(pubSubTopicRepository.getTopic(testTopic), partition); @@ -409,7 +408,7 @@ public void testLeaderCanSendValueChunksIntoDrainer() throws InterruptedExceptio KafkaKey kafkaKey = mock(KafkaKey.class); when(consumerRecord.getKey()).thenReturn(kafkaKey); when(kafkaKey.getKey()).thenReturn(new byte[] { 0xa }); - ingestionTask.produceToLocalKafka( + storePartitionDataReceiver.produceToLocalKafka( consumerRecord, partitionConsumptionState, leaderProducedRecordContext, From dc08f499ec0987e5456b545f1558b57308f19026 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 00:53:11 -0700 Subject: [PATCH 14/32] =?UTF-8?q?Refactored=20`createProducerCallback()`?= =?UTF-8?q?=20from=20`LeaderFollowerStoreIngestionTask`=20/=20`ActiveActiv?= =?UTF-8?q?eStoreIngestionTask`=20into=20`StorePartitionDataReceiver`.=20?= =?UTF-8?q?=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 34 +++++++++---------- .../LeaderFollowerStoreIngestionTask.java | 34 +++++++++---------- .../kafka/consumer/StoreIngestionTask.java | 8 ----- .../consumer/StorePartitionDataReceiver.java | 30 ++++++++++++++-- .../ActiveActiveStoreIngestionTaskTest.java | 2 -- 5 files changed, 62 insertions(+), 46 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 44fc5e444de..d751a189133 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -1484,23 +1484,23 @@ public AggVersionedIngestionStats getAggVersionedIngestionStats() { // oldRmdManifest); // }; // } - - protected LeaderProducerCallback createProducerCallback( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - LeaderProducedRecordContext leaderProducedRecordContext, - int partition, - String kafkaUrl, - long beforeProcessingRecordTimestampNs) { - return new ActiveActiveProducerCallback( - this, - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - partition, - kafkaUrl, - beforeProcessingRecordTimestampNs); - } +// +// protected LeaderProducerCallback createProducerCallback( +// PubSubMessage consumerRecord, +// PartitionConsumptionState partitionConsumptionState, +// LeaderProducedRecordContext leaderProducedRecordContext, +// int partition, +// String kafkaUrl, +// long beforeProcessingRecordTimestampNs) { +// return new ActiveActiveProducerCallback( +// this, +// consumerRecord, +// partitionConsumptionState, +// leaderProducedRecordContext, +// partition, +// kafkaUrl, +// beforeProcessingRecordTimestampNs); +// } /** * This method does a few things for leader topic-partition subscription: diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 65093165c1c..8f4fc2b63c0 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -3711,23 +3711,23 @@ protected void processControlMessageForViews( } } - @Override - protected LeaderProducerCallback createProducerCallback( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - LeaderProducedRecordContext leaderProducedRecordContext, - int partition, - String kafkaUrl, - long beforeProcessingRecordTimestampNs) { - return new LeaderProducerCallback( - this, - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - partition, - kafkaUrl, - beforeProcessingRecordTimestampNs); - } + // @Override + // protected LeaderProducerCallback createProducerCallback( + // PubSubMessage consumerRecord, + // PartitionConsumptionState partitionConsumptionState, + // LeaderProducedRecordContext leaderProducedRecordContext, + // int partition, + // String kafkaUrl, + // long beforeProcessingRecordTimestampNs) { + // return new LeaderProducerCallback( + // this, + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // partition, + // kafkaUrl, + // beforeProcessingRecordTimestampNs); + // } protected Lazy> getVeniceWriter( PartitionConsumptionState partitionConsumptionState) { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index cf295cbeb5b..51b5f604f20 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -4742,14 +4742,6 @@ protected abstract CompletableFuture sendIngestionHeartbeat LeaderCompleteState leaderCompleteState, long originTimeStampMs); - protected abstract LeaderProducerCallback createProducerCallback( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - LeaderProducedRecordContext leaderProducedRecordContext, - int partition, - String kafkaUrl, - long beforeProcessingRecordTimestampNs); - protected abstract PubSubMessageProcessedResult processMessage( PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index ea82cff9442..fa14a45d0f1 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -1483,7 +1483,7 @@ void produceToLocalKafka( String kafkaUrl, int kafkaClusterId, long beforeProcessingRecordTimestampNs) { - LeaderProducerCallback callback = storeIngestionTask.createProducerCallback( + LeaderProducerCallback callback = createProducerCallback( consumerRecord, partitionConsumptionState, leaderProducedRecordContext, @@ -1542,7 +1542,7 @@ private void propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( String kafkaUrl, int kafkaClusterId, long beforeProcessingRecordTimestampNs) { - LeaderProducerCallback callback = storeIngestionTask.createProducerCallback( + LeaderProducerCallback callback = createProducerCallback( consumerRecord, partitionConsumptionState, leaderProducedRecordContext, @@ -1592,6 +1592,32 @@ private void sendIngestionHeartbeatToVT( originTimeStampMs); } + private LeaderProducerCallback createProducerCallback( + PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, + LeaderProducedRecordContext leaderProducedRecordContext, + int partition, + String kafkaUrl, + long beforeProcessingRecordTimestampNs) { + return storeIngestionTask.isActiveActiveReplicationEnabled() + ? new ActiveActiveProducerCallback( + (ActiveActiveStoreIngestionTask) storeIngestionTask, + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + partition, + kafkaUrl, + beforeProcessingRecordTimestampNs) + : new LeaderProducerCallback( + (LeaderFollowerStoreIngestionTask) storeIngestionTask, + consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + partition, + kafkaUrl, + beforeProcessingRecordTimestampNs); + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java index 4d9db60bab1..994a7266924 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java @@ -331,8 +331,6 @@ public void testLeaderCanSendValueChunksIntoDrainer() throws InterruptedExceptio when(ingestionTask.getVersionIngestionStats()).thenReturn(mock(AggVersionedIngestionStats.class)); when(ingestionTask.getVersionedDIVStats()).thenReturn(mock(AggVersionedDIVStats.class)); when(ingestionTask.getKafkaVersionTopic()).thenReturn(testTopic); - when(ingestionTask.createProducerCallback(any(), any(), any(), anyInt(), anyString(), anyLong())) - .thenCallRealMethod(); when(ingestionTask.getRmdProtocolVersionId()).thenReturn(rmdProtocolVersionID); PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository(); PubSubTopicPartition topicPartition = From 98c00232bbad76d674d4756d554ce796dc405e94 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 02:03:37 -0700 Subject: [PATCH 15/32] =?UTF-8?q?Moved=20`IngestionBatchProcessor`=20from?= =?UTF-8?q?=20`LeaderFollowerStoreIngestionTask`=20/=20`ActiveActiveStoreI?= =?UTF-8?q?ngestionTask`=20into=20`StorePartitionDataReceiver`.=20?= =?UTF-8?q?=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 130 +++++++++--------- .../LeaderFollowerStoreIngestionTask.java | 42 +++--- .../kafka/consumer/StoreIngestionTask.java | 6 +- .../consumer/StorePartitionDataReceiver.java | 32 ++++- .../consumer/StoreIngestionTaskTest.java | 12 +- 5 files changed, 128 insertions(+), 94 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index d751a189133..3a7739aa712 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -81,7 +81,7 @@ public class ActiveActiveStoreIngestionTask extends LeaderFollowerStoreIngestion private final Lazy keyLevelLocksManager; private final AggVersionedIngestionStats aggVersionedIngestionStats; private final RemoteIngestionRepairService remoteIngestionRepairService; - private final Lazy ingestionBatchProcessorLazy; + // private final Lazy ingestionBatchProcessorLazy; private static class ReusableObjects { // reuse buffer for rocksDB value object @@ -145,22 +145,22 @@ public ActiveActiveStoreIngestionTask( isWriteComputationEnabled, getServerConfig().isComputeFastAvroEnabled()); this.remoteIngestionRepairService = builder.getRemoteIngestionRepairService(); - this.ingestionBatchProcessorLazy = Lazy.of(() -> { - if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { - LOGGER.info("AA/WC workload parallel processing is disabled for store version: {}", getKafkaVersionTopic()); - return null; - } - LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", getKafkaVersionTopic()); - return new IngestionBatchProcessor( - kafkaVersionTopic, - parallelProcessingThreadPool, - keyLevelLocksManager.get(), - this::processActiveActiveMessage, - isWriteComputationEnabled, - isActiveActiveReplicationEnabled(), - aggVersionedIngestionStats, - getHostLevelIngestionStats()); - }); + // this.ingestionBatchProcessorLazy = Lazy.of(() -> { + // if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { + // LOGGER.info("AA/WC workload parallel processing is disabled for store version: {}", getKafkaVersionTopic()); + // return null; + // } + // LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", getKafkaVersionTopic()); + // return new IngestionBatchProcessor( + // kafkaVersionTopic, + // parallelProcessingThreadPool, + // keyLevelLocksManager.get(), + // this::processActiveActiveMessage, + // isWriteComputationEnabled, + // isActiveActiveReplicationEnabled(), + // aggVersionedIngestionStats, + // getHostLevelIngestionStats()); + // }); } public static int getKeyLevelLockMaxPoolSizeBasedOnServerConfig(VeniceServerConfig serverConfig, int partitionCount) { @@ -394,10 +394,10 @@ byte[] getRmdWithValueSchemaByteBufferFromStorage( return result.serialize(); } - @Override - protected IngestionBatchProcessor getIngestionBatchProcessor() { - return ingestionBatchProcessorLazy.get(); - } + // @Override + // protected IngestionBatchProcessor getIngestionBatchProcessor() { + // return ingestionBatchProcessorLazy.get(); + // } @Override protected PubSubMessageProcessedResult processActiveActiveMessage( @@ -883,33 +883,33 @@ ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.Transien // beforeProcessingRecordTimestampNs); // } // } -// -// @Override -// protected void produceToLocalKafka( -// PubSubMessage consumerRecord, -// PartitionConsumptionState partitionConsumptionState, -// LeaderProducedRecordContext leaderProducedRecordContext, -// BiConsumer produceFunction, -// int partition, -// String kafkaUrl, -// int kafkaClusterId, -// long beforeProcessingRecordTimestampNs) { -// super.produceToLocalKafka( -// consumerRecord, -// partitionConsumptionState, -// leaderProducedRecordContext, -// produceFunction, -// partition, -// kafkaUrl, -// kafkaClusterId, -// beforeProcessingRecordTimestampNs); -// // Update the partition consumption state to say that we've transmitted the message to kafka (but haven't -// // necessarily received an ack back yet). -// if (partitionConsumptionState.getLeaderFollowerState() == LEADER && partitionConsumptionState.isHybrid() -// && consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { -// partitionConsumptionState.updateLatestRTOffsetTriedToProduceToVTMap(kafkaUrl, consumerRecord.getOffset()); -// } -// } + // + // @Override + // protected void produceToLocalKafka( + // PubSubMessage consumerRecord, + // PartitionConsumptionState partitionConsumptionState, + // LeaderProducedRecordContext leaderProducedRecordContext, + // BiConsumer produceFunction, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingRecordTimestampNs) { + // super.produceToLocalKafka( + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // produceFunction, + // partition, + // kafkaUrl, + // kafkaClusterId, + // beforeProcessingRecordTimestampNs); + // // Update the partition consumption state to say that we've transmitted the message to kafka (but haven't + // // necessarily received an ack back yet). + // if (partitionConsumptionState.getLeaderFollowerState() == LEADER && partitionConsumptionState.isHybrid() + // && consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { + // partitionConsumptionState.updateLatestRTOffsetTriedToProduceToVTMap(kafkaUrl, consumerRecord.getOffset()); + // } + // } @Override protected Map calculateLeaderUpstreamOffsetWithTopicSwitch( @@ -1484,23 +1484,23 @@ public AggVersionedIngestionStats getAggVersionedIngestionStats() { // oldRmdManifest); // }; // } -// -// protected LeaderProducerCallback createProducerCallback( -// PubSubMessage consumerRecord, -// PartitionConsumptionState partitionConsumptionState, -// LeaderProducedRecordContext leaderProducedRecordContext, -// int partition, -// String kafkaUrl, -// long beforeProcessingRecordTimestampNs) { -// return new ActiveActiveProducerCallback( -// this, -// consumerRecord, -// partitionConsumptionState, -// leaderProducedRecordContext, -// partition, -// kafkaUrl, -// beforeProcessingRecordTimestampNs); -// } + // + // protected LeaderProducerCallback createProducerCallback( + // PubSubMessage consumerRecord, + // PartitionConsumptionState partitionConsumptionState, + // LeaderProducedRecordContext leaderProducedRecordContext, + // int partition, + // String kafkaUrl, + // long beforeProcessingRecordTimestampNs) { + // return new ActiveActiveProducerCallback( + // this, + // consumerRecord, + // partitionConsumptionState, + // leaderProducedRecordContext, + // partition, + // kafkaUrl, + // beforeProcessingRecordTimestampNs); + // } /** * This method does a few things for leader topic-partition subscription: diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 8f4fc2b63c0..a72c23abf67 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -191,7 +191,7 @@ public class LeaderFollowerStoreIngestionTask extends StoreIngestionTask { private final AtomicLong lastSendIngestionHeartbeatTimestamp = new AtomicLong(0); - private final Lazy ingestionBatchProcessingLazy; + // private final Lazy ingestionBatchProcessingLazy; private final Version version; public LeaderFollowerStoreIngestionTask( @@ -333,22 +333,22 @@ public LeaderFollowerStoreIngestionTask( builder.getSchemaRepo(), getStoreName(), serverConfig.isComputeFastAvroEnabled()); - this.ingestionBatchProcessingLazy = Lazy.of(() -> { - if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { - LOGGER.info("AA/WC workload parallel processing is disabled for store version: {}", getKafkaVersionTopic()); - return null; - } - LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", getKafkaVersionTopic()); - return new IngestionBatchProcessor( - kafkaVersionTopic, - parallelProcessingThreadPool, - null, - this::processMessage, - isWriteComputationEnabled, - isActiveActiveReplicationEnabled(), - builder.getVersionedStorageIngestionStats(), - getHostLevelIngestionStats()); - }); + // this.ingestionBatchProcessingLazy = Lazy.of(() -> { + // if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { + // LOGGER.info("AA/WC workload parallel processing is disabled for store version: {}", getKafkaVersionTopic()); + // return null; + // } + // LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", getKafkaVersionTopic()); + // return new IngestionBatchProcessor( + // kafkaVersionTopic, + // parallelProcessingThreadPool, + // null, + // this::processMessage, + // isWriteComputationEnabled, + // isActiveActiveReplicationEnabled(), + // builder.getVersionedStorageIngestionStats(), + // getHostLevelIngestionStats()); + // }); } public static VeniceWriter constructVeniceWriter( @@ -393,10 +393,10 @@ protected void closeVeniceViewWriters() { } } - @Override - protected IngestionBatchProcessor getIngestionBatchProcessor() { - return ingestionBatchProcessingLazy.get(); - } + // @Override + // protected IngestionBatchProcessor getIngestionBatchProcessor() { + // return ingestionBatchProcessingLazy.get(); + // } @Override public synchronized void promoteToLeader( diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 51b5f604f20..479ac55d017 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -574,8 +574,6 @@ void setPurgeTransientRecordBuffer(boolean purgeTransientRecordBuffer) { this.purgeTransientRecordBuffer = purgeTransientRecordBuffer; } - protected abstract IngestionBatchProcessor getIngestionBatchProcessor(); - public AbstractStorageEngine getStorageEngine() { return storageEngine; } @@ -4777,6 +4775,10 @@ protected abstract Lazy> getVeniceWriter( protected abstract boolean hasViewWriters(); + public ExecutorService getParallelProcessingThreadPool() { + return parallelProcessingThreadPool; + } + public boolean isDataRecovery() { return isDataRecovery; } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index fa14a45d0f1..63990274de4 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -37,6 +37,7 @@ import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.ValueHolder; +import com.linkedin.venice.utils.lazy.Lazy; import com.linkedin.venice.writer.ChunkAwareCallback; import com.linkedin.venice.writer.DeleteMetadata; import com.linkedin.venice.writer.LeaderCompleteState; @@ -68,6 +69,7 @@ public class StorePartitionDataReceiver private final String kafkaUrl; private final String kafkaUrlForLogger; private final int kafkaClusterId; + private final Lazy ingestionBatchProcessorLazy; private final Logger LOGGER; private long receivedRecordsCount; @@ -84,6 +86,30 @@ public StorePartitionDataReceiver( this.kafkaClusterId = kafkaClusterId; this.LOGGER = LogManager.getLogger(this.getClass().getSimpleName() + " [" + kafkaUrlForLogger + "]"); this.receivedRecordsCount = 0L; + this.ingestionBatchProcessorLazy = Lazy.of(() -> { + final String kafkaVersionTopic = storeIngestionTask.getKafkaVersionTopic(); + if (!storeIngestionTask.getServerConfig().isAAWCWorkloadParallelProcessingEnabled()) { + LOGGER.info("AA/WC workload parallel processing is disabled for store version: {}", kafkaVersionTopic); + return null; + } + IngestionBatchProcessor.ProcessingFunction processingFunction = + (storeIngestionTask.isActiveActiveReplicationEnabled()) + ? storeIngestionTask::processActiveActiveMessage + : storeIngestionTask::processMessage; + KeyLevelLocksManager lockManager = (storeIngestionTask.isActiveActiveReplicationEnabled()) + ? storeIngestionTask.getKeyLevelLocksManager().get() + : null; + LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", kafkaVersionTopic); + return new IngestionBatchProcessor( + storeIngestionTask.getKafkaVersionTopic(), + storeIngestionTask.getParallelProcessingThreadPool(), + lockManager, + processingFunction, + storeIngestionTask.isTransientRecordBufferUsed(), + storeIngestionTask.isActiveActiveReplicationEnabled(), + storeIngestionTask.getAggVersionedIngestionStats(), + storeIngestionTask.getHostLevelIngestionStats()); + }); } @Override @@ -239,7 +265,7 @@ public void produceToStoreBufferServiceOrKafkaInBatch( if (batches.isEmpty()) { return; } - IngestionBatchProcessor ingestionBatchProcessor = storeIngestionTask.getIngestionBatchProcessor(); + IngestionBatchProcessor ingestionBatchProcessor = ingestionBatchProcessorLazy.get(); if (ingestionBatchProcessor == null) { throw new VeniceException( "IngestionBatchProcessor object should present for store version: " @@ -1677,4 +1703,8 @@ public String toString() { int getKafkaClusterId() { return this.kafkaClusterId; } + + IngestionBatchProcessor getIngestionBatchProcessor() { + return ingestionBatchProcessorLazy.get(); + } } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java index bc157da75f5..25c6ef09656 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java @@ -3393,11 +3393,13 @@ public void testActiveActiveStoreIsReadyToServe(HybridConfig hybridConfig, NodeT null); if (hybridConfig.equals(HYBRID) && nodeType.equals(LEADER) && isAaWCParallelProcessingEnabled()) { - assertTrue(storeIngestionTaskUnderTest instanceof ActiveActiveStoreIngestionTask); - ActiveActiveStoreIngestionTask activeActiveStoreIngestionTask = - (ActiveActiveStoreIngestionTask) storeIngestionTaskUnderTest; - assertNotNull(activeActiveStoreIngestionTask.getIngestionBatchProcessor()); - assertNotNull(activeActiveStoreIngestionTask.getIngestionBatchProcessor().getLockManager()); + localConsumedDataReceiver = new StorePartitionDataReceiver( + storeIngestionTaskUnderTest, + fooTopicPartition, + inMemoryLocalKafkaBroker.getKafkaBootstrapServer(), + 1); + assertNotNull(localConsumedDataReceiver.getIngestionBatchProcessor()); + assertNotNull(localConsumedDataReceiver.getIngestionBatchProcessor().getLockManager()); } String rtTopicName = Utils.getRealTimeTopicName(mockStore); From 46fb7e81bd907cbf18196c54f0d697fdbdecd29b Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 02:17:33 -0700 Subject: [PATCH 16/32] =?UTF-8?q?Refactored=20`processMessage()`=20from=20?= =?UTF-8?q?`LeaderFollowerStoreIngestionTask`=20into=20`StorePartitionData?= =?UTF-8?q?Receiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 322 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 25 +- .../consumer/StorePartitionDataReceiver.java | 181 +++++++++- 3 files changed, 350 insertions(+), 178 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index a72c23abf67..0a24816d6ff 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -29,7 +29,6 @@ import com.linkedin.davinci.store.view.ChangeCaptureViewWriter; import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.validation.KafkaDataIntegrityValidator; -import com.linkedin.venice.common.VeniceSystemStoreUtils; import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceMessageException; @@ -39,7 +38,6 @@ import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.Put; import com.linkedin.venice.kafka.protocol.TopicSwitch; -import com.linkedin.venice.kafka.protocol.Update; import com.linkedin.venice.kafka.protocol.enums.ControlMessageType; import com.linkedin.venice.kafka.protocol.enums.MessageType; import com.linkedin.venice.kafka.protocol.state.StoreVersionState; @@ -60,11 +58,8 @@ import com.linkedin.venice.pubsub.api.PubSubProducerCallback; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; -import com.linkedin.venice.schema.SchemaEntry; -import com.linkedin.venice.schema.writecompute.DerivedSchemaEntry; import com.linkedin.venice.serialization.AvroStoreDeserializerCache; import com.linkedin.venice.stats.StatsErrorCode; -import com.linkedin.venice.storage.protocol.ChunkedValueManifest; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.PartitionUtils; @@ -3170,6 +3165,7 @@ public void updateLeaderTopicOnFollower(PartitionConsumptionState partitionConsu * @param data the data that we might compress * @return a bytebuffer that's either the original bytebuffer or a new one depending on if we compressed it. */ + @Override protected ByteBuffer maybeCompressData( int partition, ByteBuffer data, @@ -3210,156 +3206,159 @@ protected boolean shouldCompressData(PartitionConsumptionState partitionConsumpt return !compressionStrategy.equals(CompressionStrategy.NO_OP); } - @Override - protected PubSubMessageProcessedResult processMessage( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs, - long beforeProcessingBatchRecordsTimestampMs) { - KafkaKey kafkaKey = consumerRecord.getKey(); - KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); - byte[] keyBytes = kafkaKey.getKey(); - MessageType msgType = MessageType.valueOf(kafkaValue.messageType); - switch (msgType) { - case PUT: - Put put = (Put) kafkaValue.payloadUnion; - put.putValue = maybeCompressData( - consumerRecord.getTopicPartition().getPartitionNumber(), - put.putValue, - partitionConsumptionState); - ByteBuffer putValue = put.putValue; - - /** - * For WC enabled stores update the transient record map with the latest {key,value}. This is needed only for messages - * received from RT. Messages received from VT have been persisted to disk already before switching to RT topic. - */ - if (isWriteComputationEnabled && partitionConsumptionState.isEndOfPushReceived()) { - partitionConsumptionState.setTransientRecord( - kafkaClusterId, - consumerRecord.getOffset(), - keyBytes, - putValue.array(), - putValue.position(), - putValue.remaining(), - put.schemaId, - null); - } - - return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(put, null, false)); - - case UPDATE: - /** - * 1. Currently, we support chunking only for messages produced on VT topic during batch part of the ingestion - * for hybrid stores. Chunking is NOT supported for messages produced to RT topics during streaming ingestion. - * - * So the assumption here is that the PUT/UPDATE messages stored in transientRecord should always be a full value - * (non chunked). Decoding should succeed using the simplified API - * {@link ChunkingAdapter#constructValue} - * - * 2. We always use the latest value schema to deserialize stored value bytes. - * 3. We always use the partial update schema with an ID combination of the latest value schema ID + update schema ID - * to deserialize the incoming Update request payload bytes. - * - * The reason for 2 and 3 is that we depend on the fact that the latest value schema must be a superset schema - * that contains all value fields that ever existed in a store value schema. So, always using a superset schema - * as the reader schema avoids data loss where the serialized bytes contain data for a field, however, the - * deserialized record does not contain that field because the reader schema does not contain that field. - */ - Update update = (Update) kafkaValue.payloadUnion; - final int readerValueSchemaId; - final int readerUpdateProtocolVersion; - if (isIngestingSystemStore()) { - DerivedSchemaEntry latestDerivedSchemaEntry = schemaRepository.getLatestDerivedSchema(storeName); - readerValueSchemaId = latestDerivedSchemaEntry.getValueSchemaID(); - readerUpdateProtocolVersion = latestDerivedSchemaEntry.getId(); - } else { - SchemaEntry supersetSchemaEntry = schemaRepository.getSupersetSchema(storeName); - if (supersetSchemaEntry == null) { - throw new IllegalStateException("Cannot find superset schema for store: " + storeName); - } - readerValueSchemaId = supersetSchemaEntry.getId(); - readerUpdateProtocolVersion = update.updateSchemaId; - } - ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); - final GenericRecord currValue = readStoredValueRecord( - partitionConsumptionState, - keyBytes, - readerValueSchemaId, - consumerRecord.getTopicPartition(), - valueManifestContainer); - - final byte[] updatedValueBytes; - final ChunkedValueManifest oldValueManifest = valueManifestContainer.getManifest(); - - try { - long writeComputeStartTimeInNS = System.nanoTime(); - // Leader nodes are the only ones which process UPDATES, so it's valid to always compress and not call - // 'maybeCompress'. - updatedValueBytes = compressor.get() - .compress( - storeWriteComputeHandler.applyWriteCompute( - currValue, - update.schemaId, - readerValueSchemaId, - update.updateValue, - update.updateSchemaId, - readerUpdateProtocolVersion)); - hostLevelIngestionStats - .recordWriteComputeUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(writeComputeStartTimeInNS)); - } catch (Exception e) { - writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_UPDATE_FAILURE.code; - throw new RuntimeException(e); - } - - if (updatedValueBytes == null) { - if (currValue != null) { - throw new IllegalStateException( - "Detect a situation where the current value exists and the Write Compute request" - + "deletes the current value. It is unexpected because Write Compute only supports partial update and does " - + "not support record value deletion."); - } else { - // No-op. The fact that currValue does not exist on the leader means currValue does not exist on the - // follower - // either. So, there is no need to tell the follower replica to do anything. - return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, true)); - } - } else { - partitionConsumptionState.setTransientRecord( - kafkaClusterId, - consumerRecord.getOffset(), - keyBytes, - updatedValueBytes, - 0, - updatedValueBytes.length, - readerValueSchemaId, - null); - - ByteBuffer updateValueWithSchemaId = - ByteUtils.prependIntHeaderToByteBuffer(ByteBuffer.wrap(updatedValueBytes), readerValueSchemaId, false); - - Put updatedPut = new Put(); - updatedPut.putValue = updateValueWithSchemaId; - updatedPut.schemaId = readerValueSchemaId; - return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(updatedPut, oldValueManifest, false)); - } - case DELETE: - /** - * For WC enabled stores update the transient record map with the latest {key,null} for similar reason as mentioned in PUT above. - */ - if (isWriteComputationEnabled && partitionConsumptionState.isEndOfPushReceived()) { - partitionConsumptionState.setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, -1, null); - } - return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, false)); - - default: - throw new VeniceMessageException( - ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); - } - } - + // @Override + // protected PubSubMessageProcessedResult processMessage( + // PubSubMessage consumerRecord, + // PartitionConsumptionState partitionConsumptionState, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingRecordTimestampNs, + // long beforeProcessingBatchRecordsTimestampMs) { + // KafkaKey kafkaKey = consumerRecord.getKey(); + // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + // byte[] keyBytes = kafkaKey.getKey(); + // MessageType msgType = MessageType.valueOf(kafkaValue.messageType); + // switch (msgType) { + // case PUT: + // Put put = (Put) kafkaValue.payloadUnion; + // put.putValue = maybeCompressData( + // consumerRecord.getTopicPartition().getPartitionNumber(), + // put.putValue, + // partitionConsumptionState); + // ByteBuffer putValue = put.putValue; + // + // /** + // * For WC enabled stores update the transient record map with the latest {key,value}. This is needed only for + // messages + // * received from RT. Messages received from VT have been persisted to disk already before switching to RT topic. + // */ + // if (isWriteComputationEnabled && partitionConsumptionState.isEndOfPushReceived()) { + // partitionConsumptionState.setTransientRecord( + // kafkaClusterId, + // consumerRecord.getOffset(), + // keyBytes, + // putValue.array(), + // putValue.position(), + // putValue.remaining(), + // put.schemaId, + // null); + // } + // + // return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(put, null, false)); + // + // case UPDATE: + // /** + // * 1. Currently, we support chunking only for messages produced on VT topic during batch part of the ingestion + // * for hybrid stores. Chunking is NOT supported for messages produced to RT topics during streaming ingestion. + // * + // * So the assumption here is that the PUT/UPDATE messages stored in transientRecord should always be a full value + // * (non chunked). Decoding should succeed using the simplified API + // * {@link ChunkingAdapter#constructValue} + // * + // * 2. We always use the latest value schema to deserialize stored value bytes. + // * 3. We always use the partial update schema with an ID combination of the latest value schema ID + update schema + // ID + // * to deserialize the incoming Update request payload bytes. + // * + // * The reason for 2 and 3 is that we depend on the fact that the latest value schema must be a superset schema + // * that contains all value fields that ever existed in a store value schema. So, always using a superset schema + // * as the reader schema avoids data loss where the serialized bytes contain data for a field, however, the + // * deserialized record does not contain that field because the reader schema does not contain that field. + // */ + // Update update = (Update) kafkaValue.payloadUnion; + // final int readerValueSchemaId; + // final int readerUpdateProtocolVersion; + // if (isIngestingSystemStore()) { + // DerivedSchemaEntry latestDerivedSchemaEntry = schemaRepository.getLatestDerivedSchema(storeName); + // readerValueSchemaId = latestDerivedSchemaEntry.getValueSchemaID(); + // readerUpdateProtocolVersion = latestDerivedSchemaEntry.getId(); + // } else { + // SchemaEntry supersetSchemaEntry = schemaRepository.getSupersetSchema(storeName); + // if (supersetSchemaEntry == null) { + // throw new IllegalStateException("Cannot find superset schema for store: " + storeName); + // } + // readerValueSchemaId = supersetSchemaEntry.getId(); + // readerUpdateProtocolVersion = update.updateSchemaId; + // } + // ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); + // final GenericRecord currValue = readStoredValueRecord( + // partitionConsumptionState, + // keyBytes, + // readerValueSchemaId, + // consumerRecord.getTopicPartition(), + // valueManifestContainer); + // + // final byte[] updatedValueBytes; + // final ChunkedValueManifest oldValueManifest = valueManifestContainer.getManifest(); + // + // try { + // long writeComputeStartTimeInNS = System.nanoTime(); + // // Leader nodes are the only ones which process UPDATES, so it's valid to always compress and not call + // // 'maybeCompress'. + // updatedValueBytes = compressor.get() + // .compress( + // storeWriteComputeHandler.applyWriteCompute( + // currValue, + // update.schemaId, + // readerValueSchemaId, + // update.updateValue, + // update.updateSchemaId, + // readerUpdateProtocolVersion)); + // hostLevelIngestionStats + // .recordWriteComputeUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(writeComputeStartTimeInNS)); + // } catch (Exception e) { + // writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_UPDATE_FAILURE.code; + // throw new RuntimeException(e); + // } + // + // if (updatedValueBytes == null) { + // if (currValue != null) { + // throw new IllegalStateException( + // "Detect a situation where the current value exists and the Write Compute request" + // + "deletes the current value. It is unexpected because Write Compute only supports partial update and does " + // + "not support record value deletion."); + // } else { + // // No-op. The fact that currValue does not exist on the leader means currValue does not exist on the + // // follower + // // either. So, there is no need to tell the follower replica to do anything. + // return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, true)); + // } + // } else { + // partitionConsumptionState.setTransientRecord( + // kafkaClusterId, + // consumerRecord.getOffset(), + // keyBytes, + // updatedValueBytes, + // 0, + // updatedValueBytes.length, + // readerValueSchemaId, + // null); + // + // ByteBuffer updateValueWithSchemaId = + // ByteUtils.prependIntHeaderToByteBuffer(ByteBuffer.wrap(updatedValueBytes), readerValueSchemaId, false); + // + // Put updatedPut = new Put(); + // updatedPut.putValue = updateValueWithSchemaId; + // updatedPut.schemaId = readerValueSchemaId; + // return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(updatedPut, oldValueManifest, false)); + // } + // case DELETE: + // /** + // * For WC enabled stores update the transient record map with the latest {key,null} for similar reason as mentioned + // in PUT above. + // */ + // if (isWriteComputationEnabled && partitionConsumptionState.isEndOfPushReceived()) { + // partitionConsumptionState.setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, -1, null); + // } + // return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, false)); + // + // default: + // throw new VeniceMessageException( + // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); + // } + // } + // // @Override // protected void processMessageAndMaybeProduceToKafka( // PubSubMessageProcessedResultWrapper consumerRecordWrapper, @@ -3551,7 +3550,7 @@ protected PubSubMessageProcessedResult processMessage( * it from the storage engine. * @return {@link Optional#empty} if the value */ - private GenericRecord readStoredValueRecord( + protected GenericRecord readStoredValueRecord( PartitionConsumptionState partitionConsumptionState, byte[] keyBytes, int readerValueSchemaID, @@ -3643,9 +3642,9 @@ interface GetLastKnownUpstreamTopicOffset { long apply(String sourceKafkaUrl, PubSubTopic upstreamTopic); } - private boolean isIngestingSystemStore() { - return VeniceSystemStoreUtils.isSystemStore(storeName); - } + // private boolean isIngestingSystemStore() { + // return VeniceSystemStoreUtils.isSystemStore(storeName); + // } /** * This method fetches/calculates latest leader persisted offset and last offset in RT topic. The method relies on @@ -4013,4 +4012,9 @@ public KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders() { public Lazy getKeyLevelLocksManager() { throw new VeniceException("getKeyLevelLocksManager() should only be called in active-active replication mode"); } + + @Override + public StoreWriteComputeProcessor getStoreWriteComputeHandler() { + return storeWriteComputeHandler; + } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 479ac55d017..ca5d5fe6d66 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -38,6 +38,7 @@ import com.linkedin.davinci.storage.StorageEngineRepository; import com.linkedin.davinci.storage.StorageMetadataService; import com.linkedin.davinci.storage.StorageService; +import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.StoragePartitionConfig; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; @@ -159,6 +160,7 @@ import java.util.function.Supplier; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -4740,14 +4742,17 @@ protected abstract CompletableFuture sendIngestionHeartbeat LeaderCompleteState leaderCompleteState, long originTimeStampMs); - protected abstract PubSubMessageProcessedResult processMessage( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, + protected abstract ByteBuffer maybeCompressData( int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs, - long beforeProcessingBatchRecordsTimestampMs); + ByteBuffer data, + PartitionConsumptionState partitionConsumptionState); + + protected abstract GenericRecord readStoredValueRecord( + PartitionConsumptionState partitionConsumptionState, + byte[] keyBytes, + int readerValueSchemaID, + PubSubTopicPartition topicPartition, + ChunkedValueManifestContainer manifestContainer); protected PubSubMessageProcessedResult processActiveActiveMessage( PubSubMessage consumerRecord, @@ -4768,6 +4773,8 @@ public int getRmdProtocolVersionId() { throw new VeniceException("getRmdProtocolVersionId() should only be called in active active mode"); } + public abstract StoreWriteComputeProcessor getStoreWriteComputeHandler(); + protected abstract Lazy> getVeniceWriter( PartitionConsumptionState partitionConsumptionState); @@ -4775,6 +4782,10 @@ protected abstract Lazy> getVeniceWriter( protected abstract boolean hasViewWriters(); + void setWriteComputeFailureCode(int code) { + this.writeComputeFailureCode = code; + } + public ExecutorService getParallelProcessingThreadPool() { return parallelProcessingThreadPool; } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 63990274de4..e8b415cc8ad 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -7,10 +7,12 @@ import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; import com.linkedin.davinci.replication.merge.MergeConflictResult; import com.linkedin.davinci.stats.HostLevelIngestionStats; +import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.utils.ByteArrayKey; import com.linkedin.davinci.validation.PartitionTracker; import com.linkedin.davinci.validation.PartitionTracker.TopicType; +import com.linkedin.venice.common.VeniceSystemStoreUtils; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceMessageException; import com.linkedin.venice.exceptions.validation.DuplicateDataException; @@ -30,7 +32,10 @@ import com.linkedin.venice.pubsub.api.PubSubProducerCallback; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.schema.SchemaEntry; +import com.linkedin.venice.schema.writecompute.DerivedSchemaEntry; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; +import com.linkedin.venice.stats.StatsErrorCode; import com.linkedin.venice.storage.protocol.ChunkedValueManifest; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.ExceptionUtils; @@ -58,6 +63,7 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiConsumer; import java.util.function.Function; +import org.apache.avro.generic.GenericRecord; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -95,7 +101,7 @@ public StorePartitionDataReceiver( IngestionBatchProcessor.ProcessingFunction processingFunction = (storeIngestionTask.isActiveActiveReplicationEnabled()) ? storeIngestionTask::processActiveActiveMessage - : storeIngestionTask::processMessage; + : this::processMessage; KeyLevelLocksManager lockManager = (storeIngestionTask.isActiveActiveReplicationEnabled()) ? storeIngestionTask.getKeyLevelLocksManager().get() : null; @@ -1062,17 +1068,14 @@ void processMessageAndMaybeProduceToKafka( && consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper() != null) { writeComputeResultWrapper = consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper(); } else { - writeComputeResultWrapper = - storeIngestionTask - .processMessage( - consumerRecord, - partitionConsumptionState, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs) - .getWriteComputeResultWrapper(); + writeComputeResultWrapper = processMessage( + consumerRecord, + partitionConsumptionState, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs).getWriteComputeResultWrapper(); } if (msgType.equals(MessageType.UPDATE) && writeComputeResultWrapper.isSkipProduce()) { @@ -1233,6 +1236,160 @@ private void produceToLocalKafkaHelper( } } + private PubSubMessageProcessedResult processMessage( + PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs) { + KafkaKey kafkaKey = consumerRecord.getKey(); + KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + byte[] keyBytes = kafkaKey.getKey(); + MessageType msgType = MessageType.valueOf(kafkaValue.messageType); + switch (msgType) { + case PUT: + Put put = (Put) kafkaValue.payloadUnion; + put.putValue = storeIngestionTask.maybeCompressData( + consumerRecord.getTopicPartition().getPartitionNumber(), + put.putValue, + partitionConsumptionState); + ByteBuffer putValue = put.putValue; + + /** + * For WC enabled stores update the transient record map with the latest {key,value}. This is needed only for messages + * received from RT. Messages received from VT have been persisted to disk already before switching to RT topic. + */ + if (storeIngestionTask.isTransientRecordBufferUsed() && partitionConsumptionState.isEndOfPushReceived()) { + partitionConsumptionState.setTransientRecord( + kafkaClusterId, + consumerRecord.getOffset(), + keyBytes, + putValue.array(), + putValue.position(), + putValue.remaining(), + put.schemaId, + null); + } + + return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(put, null, false)); + + case UPDATE: + /** + * 1. Currently, we support chunking only for messages produced on VT topic during batch part of the ingestion + * for hybrid stores. Chunking is NOT supported for messages produced to RT topics during streaming ingestion. + * + * So the assumption here is that the PUT/UPDATE messages stored in transientRecord should always be a full value + * (non chunked). Decoding should succeed using the simplified API + * {@link ChunkingAdapter#constructValue} + * + * 2. We always use the latest value schema to deserialize stored value bytes. + * 3. We always use the partial update schema with an ID combination of the latest value schema ID + update schema ID + * to deserialize the incoming Update request payload bytes. + * + * The reason for 2 and 3 is that we depend on the fact that the latest value schema must be a superset schema + * that contains all value fields that ever existed in a store value schema. So, always using a superset schema + * as the reader schema avoids data loss where the serialized bytes contain data for a field, however, the + * deserialized record does not contain that field because the reader schema does not contain that field. + */ + Update update = (Update) kafkaValue.payloadUnion; + final int readerValueSchemaId; + final int readerUpdateProtocolVersion; + final String storeName = storeIngestionTask.getStoreName(); + if (VeniceSystemStoreUtils.isSystemStore(storeName)) { + DerivedSchemaEntry latestDerivedSchemaEntry = + storeIngestionTask.getSchemaRepo().getLatestDerivedSchema(storeName); + readerValueSchemaId = latestDerivedSchemaEntry.getValueSchemaID(); + readerUpdateProtocolVersion = latestDerivedSchemaEntry.getId(); + } else { + SchemaEntry supersetSchemaEntry = storeIngestionTask.getSchemaRepo().getSupersetSchema(storeName); + if (supersetSchemaEntry == null) { + throw new IllegalStateException("Cannot find superset schema for store: " + storeName); + } + readerValueSchemaId = supersetSchemaEntry.getId(); + readerUpdateProtocolVersion = update.updateSchemaId; + } + ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); + final GenericRecord currValue = storeIngestionTask.readStoredValueRecord( + partitionConsumptionState, + keyBytes, + readerValueSchemaId, + consumerRecord.getTopicPartition(), + valueManifestContainer); + + final byte[] updatedValueBytes; + final ChunkedValueManifest oldValueManifest = valueManifestContainer.getManifest(); + + try { + long writeComputeStartTimeInNS = System.nanoTime(); + // Leader nodes are the only ones which process UPDATES, so it's valid to always compress and not call + // 'maybeCompress'. + updatedValueBytes = storeIngestionTask.getCompressor() + .get() + .compress( + storeIngestionTask.getStoreWriteComputeHandler() + .applyWriteCompute( + currValue, + update.schemaId, + readerValueSchemaId, + update.updateValue, + update.updateSchemaId, + readerUpdateProtocolVersion)); + storeIngestionTask.getHostLevelIngestionStats() + .recordWriteComputeUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(writeComputeStartTimeInNS)); + } catch (Exception e) { + storeIngestionTask.setWriteComputeFailureCode(StatsErrorCode.WRITE_COMPUTE_UPDATE_FAILURE.code); + throw new RuntimeException(e); + } + + if (updatedValueBytes == null) { + if (currValue != null) { + throw new IllegalStateException( + "Detect a situation where the current value exists and the Write Compute request" + + "deletes the current value. It is unexpected because Write Compute only supports partial update and does " + + "not support record value deletion."); + } else { + // No-op. The fact that currValue does not exist on the leader means currValue does not exist on the + // follower + // either. So, there is no need to tell the follower replica to do anything. + return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, true)); + } + } else { + partitionConsumptionState.setTransientRecord( + kafkaClusterId, + consumerRecord.getOffset(), + keyBytes, + updatedValueBytes, + 0, + updatedValueBytes.length, + readerValueSchemaId, + null); + + ByteBuffer updateValueWithSchemaId = + ByteUtils.prependIntHeaderToByteBuffer(ByteBuffer.wrap(updatedValueBytes), readerValueSchemaId, false); + + Put updatedPut = new Put(); + updatedPut.putValue = updateValueWithSchemaId; + updatedPut.schemaId = readerValueSchemaId; + return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(updatedPut, oldValueManifest, false)); + } + case DELETE: + /** + * For WC enabled stores update the transient record map with the latest {key,null} for similar reason as mentioned in PUT above. + */ + if (storeIngestionTask.isTransientRecordBufferUsed() && partitionConsumptionState.isEndOfPushReceived()) { + partitionConsumptionState.setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, -1, null); + } + return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, false)); + + default: + throw new VeniceMessageException( + storeIngestionTask.getIngestionTaskName() + " : Invalid/Unrecognized operation type submitted: " + + kafkaValue.messageType); + } + } + // This function may modify the original record in KME, it is unsafe to use the payload from KME directly after // this function. private void processActiveActiveMessageAndMaybeProduceToKafka( From d311658c195dd84e532f4ed58a9dbb0c236e6ba9 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 02:33:34 -0700 Subject: [PATCH 17/32] =?UTF-8?q?Refactored=20`processActiveActiveMessage(?= =?UTF-8?q?)`=20from=20`ActiveActiveStoreIngestionTask`=20into=20`StorePar?= =?UTF-8?q?titionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 381 +++++++++--------- .../LeaderFollowerStoreIngestionTask.java | 10 + .../kafka/consumer/StoreIngestionTask.java | 53 ++- .../consumer/StorePartitionDataReceiver.java | 214 +++++++++- 4 files changed, 450 insertions(+), 208 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 3a7739aa712..82f7b1fd778 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -23,13 +23,11 @@ import com.linkedin.davinci.store.record.ValueRecord; import com.linkedin.venice.exceptions.PersistenceFailureException; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.exceptions.VeniceMessageException; import com.linkedin.venice.kafka.protocol.ControlMessage; import com.linkedin.venice.kafka.protocol.Delete; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.Put; import com.linkedin.venice.kafka.protocol.TopicSwitch; -import com.linkedin.venice.kafka.protocol.Update; import com.linkedin.venice.kafka.protocol.enums.ControlMessageType; import com.linkedin.venice.kafka.protocol.enums.MessageType; import com.linkedin.venice.kafka.protocol.state.StoreVersionState; @@ -52,7 +50,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -341,6 +338,7 @@ private byte[] prependReplicationMetadataBytesWithValueSchemaId(ByteBuffer metad * @param partition The partition to fetch the replication metadata from storage engine * @return The object containing RMD and value schema id. If nothing is found, return null */ + @Override RmdWithValueSchemaId getReplicationMetadataAndSchemaId( PartitionConsumptionState partitionConsumptionState, byte[] key, @@ -369,6 +367,7 @@ RmdWithValueSchemaId getReplicationMetadataAndSchemaId( return rmdWithValueSchemaId; } + @Override public RmdSerDe getRmdSerDe() { return rmdSerDe; } @@ -398,187 +397,187 @@ byte[] getRmdWithValueSchemaByteBufferFromStorage( // protected IngestionBatchProcessor getIngestionBatchProcessor() { // return ingestionBatchProcessorLazy.get(); // } - - @Override - protected PubSubMessageProcessedResult processActiveActiveMessage( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs, - long beforeProcessingBatchRecordsTimestampMs) { - KafkaKey kafkaKey = consumerRecord.getKey(); - KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); - byte[] keyBytes = kafkaKey.getKey(); - MessageType msgType = MessageType.valueOf(kafkaValue.messageType); - final int incomingValueSchemaId; - final int incomingWriteComputeSchemaId; - - switch (msgType) { - case PUT: - incomingValueSchemaId = ((Put) kafkaValue.payloadUnion).schemaId; - incomingWriteComputeSchemaId = -1; - break; - case UPDATE: - Update incomingUpdate = (Update) kafkaValue.payloadUnion; - incomingValueSchemaId = incomingUpdate.schemaId; - incomingWriteComputeSchemaId = incomingUpdate.updateSchemaId; - break; - case DELETE: - incomingValueSchemaId = -1; // Ignored since we don't need the schema id for DELETE operations. - incomingWriteComputeSchemaId = -1; - break; - default: - throw new VeniceMessageException( - ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); - } - final ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); - Lazy> oldValueProvider = Lazy.of( - () -> getValueBytesForKey( - partitionConsumptionState, - keyBytes, - consumerRecord.getTopicPartition(), - valueManifestContainer, - beforeProcessingBatchRecordsTimestampMs)); - if (hasChangeCaptureView) { - /** - * Since this function will update the transient cache before writing the view, and if there is - * a change capture view writer, we need to lookup first, otherwise the transient cache will be populated - * when writing to the view after this function. - */ - oldValueProvider.get(); - } - - final RmdWithValueSchemaId rmdWithValueSchemaID = getReplicationMetadataAndSchemaId( - partitionConsumptionState, - keyBytes, - partition, - beforeProcessingBatchRecordsTimestampMs); - - final long writeTimestamp = getWriteTimestampFromKME(kafkaValue); - final long offsetSumPreOperation = - rmdWithValueSchemaID != null ? RmdUtils.extractOffsetVectorSumFromRmd(rmdWithValueSchemaID.getRmdRecord()) : 0; - List recordTimestampsPreOperation = rmdWithValueSchemaID != null - ? RmdUtils.extractTimestampFromRmd(rmdWithValueSchemaID.getRmdRecord()) - : Collections.singletonList(0L); - - // get the source offset and the id - long sourceOffset = consumerRecord.getOffset(); - final MergeConflictResult mergeConflictResult; - - aggVersionedIngestionStats.recordTotalDCR(storeName, versionNumber); - - Lazy oldValueByteBufferProvider = unwrapByteBufferFromOldValueProvider(oldValueProvider); - - long beforeDCRTimestampInNs = System.nanoTime(); - switch (msgType) { - case PUT: - mergeConflictResult = mergeConflictResolver.put( - oldValueByteBufferProvider, - rmdWithValueSchemaID, - ((Put) kafkaValue.payloadUnion).putValue, - writeTimestamp, - incomingValueSchemaId, - sourceOffset, - kafkaClusterId, - kafkaClusterId // Use the kafka cluster ID as the colo ID for now because one colo/fabric has only one - // Kafka cluster. TODO: evaluate whether it is enough this way, or we need to add a new - // config to represent the mapping from Kafka server URLs to colo ID. - ); - getHostLevelIngestionStats() - .recordIngestionActiveActivePutLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); - break; - - case DELETE: - mergeConflictResult = mergeConflictResolver.delete( - oldValueByteBufferProvider, - rmdWithValueSchemaID, - writeTimestamp, - sourceOffset, - kafkaClusterId, - kafkaClusterId); - getHostLevelIngestionStats() - .recordIngestionActiveActiveDeleteLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); - break; - - case UPDATE: - mergeConflictResult = mergeConflictResolver.update( - oldValueByteBufferProvider, - rmdWithValueSchemaID, - ((Update) kafkaValue.payloadUnion).updateValue, - incomingValueSchemaId, - incomingWriteComputeSchemaId, - writeTimestamp, - sourceOffset, - kafkaClusterId, - kafkaClusterId, - valueManifestContainer); - getHostLevelIngestionStats() - .recordIngestionActiveActiveUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); - break; - default: - throw new VeniceMessageException( - ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); - } - - if (mergeConflictResult.isUpdateIgnored()) { - hostLevelIngestionStats.recordUpdateIgnoredDCR(); - // Record the last ignored offset - partitionConsumptionState - .updateLatestIgnoredUpstreamRTOffset(kafkaClusterIdToUrlMap.get(kafkaClusterId), sourceOffset); - return new PubSubMessageProcessedResult( - new MergeConflictResultWrapper( - mergeConflictResult, - oldValueProvider, - oldValueByteBufferProvider, - rmdWithValueSchemaID, - valueManifestContainer, - null, - null)); - } else { - validatePostOperationResultsAndRecord(mergeConflictResult, offsetSumPreOperation, recordTimestampsPreOperation); - - final ByteBuffer updatedValueBytes = maybeCompressData( - consumerRecord.getTopicPartition().getPartitionNumber(), - mergeConflictResult.getNewValue(), - partitionConsumptionState); - - final int valueSchemaId = mergeConflictResult.getValueSchemaId(); - - GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); - final ByteBuffer updatedRmdBytes = - rmdSerDe.serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); - - if (updatedValueBytes == null) { - hostLevelIngestionStats.recordTombstoneCreatedDCR(); - aggVersionedIngestionStats.recordTombStoneCreationDCR(storeName, versionNumber); - partitionConsumptionState - .setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, valueSchemaId, rmdRecord); - } else { - int valueLen = updatedValueBytes.remaining(); - partitionConsumptionState.setTransientRecord( - kafkaClusterId, - consumerRecord.getOffset(), - keyBytes, - updatedValueBytes.array(), - updatedValueBytes.position(), - valueLen, - valueSchemaId, - rmdRecord); - } - return new PubSubMessageProcessedResult( - new MergeConflictResultWrapper( - mergeConflictResult, - oldValueProvider, - oldValueByteBufferProvider, - rmdWithValueSchemaID, - valueManifestContainer, - updatedValueBytes, - updatedRmdBytes)); - } - } - + // + // @Override + // protected PubSubMessageProcessedResult processActiveActiveMessage( + // PubSubMessage consumerRecord, + // PartitionConsumptionState partitionConsumptionState, + // int partition, + // String kafkaUrl, + // int kafkaClusterId, + // long beforeProcessingRecordTimestampNs, + // long beforeProcessingBatchRecordsTimestampMs) { + // KafkaKey kafkaKey = consumerRecord.getKey(); + // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + // byte[] keyBytes = kafkaKey.getKey(); + // MessageType msgType = MessageType.valueOf(kafkaValue.messageType); + // final int incomingValueSchemaId; + // final int incomingWriteComputeSchemaId; + // + // switch (msgType) { + // case PUT: + // incomingValueSchemaId = ((Put) kafkaValue.payloadUnion).schemaId; + // incomingWriteComputeSchemaId = -1; + // break; + // case UPDATE: + // Update incomingUpdate = (Update) kafkaValue.payloadUnion; + // incomingValueSchemaId = incomingUpdate.schemaId; + // incomingWriteComputeSchemaId = incomingUpdate.updateSchemaId; + // break; + // case DELETE: + // incomingValueSchemaId = -1; // Ignored since we don't need the schema id for DELETE operations. + // incomingWriteComputeSchemaId = -1; + // break; + // default: + // throw new VeniceMessageException( + // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); + // } + // final ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); + // Lazy> oldValueProvider = Lazy.of( + // () -> getValueBytesForKey( + // partitionConsumptionState, + // keyBytes, + // consumerRecord.getTopicPartition(), + // valueManifestContainer, + // beforeProcessingBatchRecordsTimestampMs)); + // if (hasChangeCaptureView) { + // /** + // * Since this function will update the transient cache before writing the view, and if there is + // * a change capture view writer, we need to lookup first, otherwise the transient cache will be populated + // * when writing to the view after this function. + // */ + // oldValueProvider.get(); + // } + // + // final RmdWithValueSchemaId rmdWithValueSchemaID = getReplicationMetadataAndSchemaId( + // partitionConsumptionState, + // keyBytes, + // partition, + // beforeProcessingBatchRecordsTimestampMs); + // + // final long writeTimestamp = getWriteTimestampFromKME(kafkaValue); + // final long offsetSumPreOperation = + // rmdWithValueSchemaID != null ? RmdUtils.extractOffsetVectorSumFromRmd(rmdWithValueSchemaID.getRmdRecord()) : 0; + // List recordTimestampsPreOperation = rmdWithValueSchemaID != null + // ? RmdUtils.extractTimestampFromRmd(rmdWithValueSchemaID.getRmdRecord()) + // : Collections.singletonList(0L); + // + // // get the source offset and the id + // long sourceOffset = consumerRecord.getOffset(); + // final MergeConflictResult mergeConflictResult; + // + // aggVersionedIngestionStats.recordTotalDCR(storeName, versionNumber); + // + // Lazy oldValueByteBufferProvider = unwrapByteBufferFromOldValueProvider(oldValueProvider); + // + // long beforeDCRTimestampInNs = System.nanoTime(); + // switch (msgType) { + // case PUT: + // mergeConflictResult = mergeConflictResolver.put( + // oldValueByteBufferProvider, + // rmdWithValueSchemaID, + // ((Put) kafkaValue.payloadUnion).putValue, + // writeTimestamp, + // incomingValueSchemaId, + // sourceOffset, + // kafkaClusterId, + // kafkaClusterId // Use the kafka cluster ID as the colo ID for now because one colo/fabric has only one + // // Kafka cluster. TODO: evaluate whether it is enough this way, or we need to add a new + // // config to represent the mapping from Kafka server URLs to colo ID. + // ); + // getHostLevelIngestionStats() + // .recordIngestionActiveActivePutLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); + // break; + // + // case DELETE: + // mergeConflictResult = mergeConflictResolver.delete( + // oldValueByteBufferProvider, + // rmdWithValueSchemaID, + // writeTimestamp, + // sourceOffset, + // kafkaClusterId, + // kafkaClusterId); + // getHostLevelIngestionStats() + // .recordIngestionActiveActiveDeleteLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); + // break; + // + // case UPDATE: + // mergeConflictResult = mergeConflictResolver.update( + // oldValueByteBufferProvider, + // rmdWithValueSchemaID, + // ((Update) kafkaValue.payloadUnion).updateValue, + // incomingValueSchemaId, + // incomingWriteComputeSchemaId, + // writeTimestamp, + // sourceOffset, + // kafkaClusterId, + // kafkaClusterId, + // valueManifestContainer); + // getHostLevelIngestionStats() + // .recordIngestionActiveActiveUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); + // break; + // default: + // throw new VeniceMessageException( + // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); + // } + // + // if (mergeConflictResult.isUpdateIgnored()) { + // hostLevelIngestionStats.recordUpdateIgnoredDCR(); + // // Record the last ignored offset + // partitionConsumptionState + // .updateLatestIgnoredUpstreamRTOffset(kafkaClusterIdToUrlMap.get(kafkaClusterId), sourceOffset); + // return new PubSubMessageProcessedResult( + // new MergeConflictResultWrapper( + // mergeConflictResult, + // oldValueProvider, + // oldValueByteBufferProvider, + // rmdWithValueSchemaID, + // valueManifestContainer, + // null, + // null)); + // } else { + // validatePostOperationResultsAndRecord(mergeConflictResult, offsetSumPreOperation, recordTimestampsPreOperation); + // + // final ByteBuffer updatedValueBytes = maybeCompressData( + // consumerRecord.getTopicPartition().getPartitionNumber(), + // mergeConflictResult.getNewValue(), + // partitionConsumptionState); + // + // final int valueSchemaId = mergeConflictResult.getValueSchemaId(); + // + // GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); + // final ByteBuffer updatedRmdBytes = + // rmdSerDe.serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); + // + // if (updatedValueBytes == null) { + // hostLevelIngestionStats.recordTombstoneCreatedDCR(); + // aggVersionedIngestionStats.recordTombStoneCreationDCR(storeName, versionNumber); + // partitionConsumptionState + // .setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, valueSchemaId, rmdRecord); + // } else { + // int valueLen = updatedValueBytes.remaining(); + // partitionConsumptionState.setTransientRecord( + // kafkaClusterId, + // consumerRecord.getOffset(), + // keyBytes, + // updatedValueBytes.array(), + // updatedValueBytes.position(), + // valueLen, + // valueSchemaId, + // rmdRecord); + // } + // return new PubSubMessageProcessedResult( + // new MergeConflictResultWrapper( + // mergeConflictResult, + // oldValueProvider, + // oldValueByteBufferProvider, + // rmdWithValueSchemaID, + // valueManifestContainer, + // updatedValueBytes, + // updatedRmdBytes)); + // } + // } + // // // This function may modify the original record in KME, it is unsafe to use the payload from KME directly after // // this function. // protected void processMessageAndMaybeProduceToKafka( @@ -684,7 +683,8 @@ static Lazy unwrapByteBufferFromOldValueProvider( }); } - private long getWriteTimestampFromKME(KafkaMessageEnvelope kme) { + @Override + public long getWriteTimestampFromKME(KafkaMessageEnvelope kme) { if (kme.producerMetadata.logicalTimestamp >= 0) { return kme.producerMetadata.logicalTimestamp; } else { @@ -692,7 +692,8 @@ private long getWriteTimestampFromKME(KafkaMessageEnvelope kme) { } } - private void validatePostOperationResultsAndRecord( + @Override + void validatePostOperationResultsAndRecord( MergeConflictResult mergeConflictResult, Long offsetSumPreOperation, List timestampsPreOperation) { @@ -735,7 +736,8 @@ private void validatePostOperationResultsAndRecord( * @param topicPartition The {@link PubSubTopicPartition} from which the incoming record was consumed * @return */ - private ByteBufferValueRecord getValueBytesForKey( + @Override + public ByteBufferValueRecord getValueBytesForKey( PartitionConsumptionState partitionConsumptionState, byte[] key, PubSubTopicPartition topicPartition, @@ -1451,6 +1453,11 @@ public AggVersionedIngestionStats getAggVersionedIngestionStats() { return aggVersionedIngestionStats; } + @Override + public MergeConflictResolver getMergeConflictResolver() { + return mergeConflictResolver; + } + // @Override // protected BiConsumer getProduceToTopicFunction( // PartitionConsumptionState partitionConsumptionState, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 0a24816d6ff..c0f69e598de 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -4017,4 +4017,14 @@ public Lazy getKeyLevelLocksManager() { public StoreWriteComputeProcessor getStoreWriteComputeHandler() { return storeWriteComputeHandler; } + + @Override + public Int2ObjectMap getKafkaClusterIdToUrlMap() { + return kafkaClusterIdToUrlMap; + } + + @Override + public boolean hasChangeCaptureView() { + return hasChangeCaptureView; + } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index ca5d5fe6d66..dda9c94d602 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -32,6 +32,10 @@ import com.linkedin.davinci.ingestion.LagType; import com.linkedin.davinci.listener.response.AdminResponse; import com.linkedin.davinci.notifier.VeniceNotifier; +import com.linkedin.davinci.replication.RmdWithValueSchemaId; +import com.linkedin.davinci.replication.merge.MergeConflictResolver; +import com.linkedin.davinci.replication.merge.MergeConflictResult; +import com.linkedin.davinci.replication.merge.RmdSerDe; import com.linkedin.davinci.stats.AggVersionedDIVStats; import com.linkedin.davinci.stats.AggVersionedIngestionStats; import com.linkedin.davinci.stats.HostLevelIngestionStats; @@ -42,6 +46,7 @@ import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.StoragePartitionConfig; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; +import com.linkedin.davinci.store.record.ByteBufferValueRecord; import com.linkedin.davinci.store.record.ValueRecord; import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.utils.ChunkAssembler; @@ -123,6 +128,7 @@ import com.linkedin.venice.writer.LeaderCompleteState; import com.linkedin.venice.writer.LeaderMetadataWrapper; import com.linkedin.venice.writer.VeniceWriter; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; import java.io.Closeable; import java.io.IOException; @@ -4754,15 +4760,11 @@ protected abstract GenericRecord readStoredValueRecord( PubSubTopicPartition topicPartition, ChunkedValueManifestContainer manifestContainer); - protected PubSubMessageProcessedResult processActiveActiveMessage( - PubSubMessage consumerRecord, - PartitionConsumptionState partitionConsumptionState, - int partition, - String kafkaUrl, - int kafkaClusterId, - long beforeProcessingRecordTimestampNs, - long beforeProcessingBatchRecordsTimestampMs) { - throw new VeniceException("processActiveActiveMessage() should only be called in active active mode"); + void validatePostOperationResultsAndRecord( + MergeConflictResult mergeConflictResult, + Long offsetSumPreOperation, + List timestampsPreOperation) { + throw new VeniceException("validatePostOperationResultsAndRecord() should only be called in active active mode"); } public AggVersionedIngestionStats getAggVersionedIngestionStats() { @@ -4773,6 +4775,39 @@ public int getRmdProtocolVersionId() { throw new VeniceException("getRmdProtocolVersionId() should only be called in active active mode"); } + public ByteBufferValueRecord getValueBytesForKey( + PartitionConsumptionState partitionConsumptionState, + byte[] key, + PubSubTopicPartition topicPartition, + ChunkedValueManifestContainer valueManifestContainer, + long currentTimeForMetricsMs) { + throw new VeniceException("getValueBytesForKey() should only be called in active active mode"); + } + + RmdWithValueSchemaId getReplicationMetadataAndSchemaId( + PartitionConsumptionState partitionConsumptionState, + byte[] key, + int partition, + long currentTimeForMetricsMs) { + throw new VeniceException("getReplicationMetadataAndSchemaId() should only be called in active active mode"); + } + + public long getWriteTimestampFromKME(KafkaMessageEnvelope kme) { + throw new VeniceException("getWriteTimestampFromKME() should only be called in active active mode"); + } + + public MergeConflictResolver getMergeConflictResolver() { + throw new VeniceException("getMergeConflictResolver() should only be called in active active mode"); + } + + public RmdSerDe getRmdSerDe() { + throw new VeniceException("getRmdSerDe() should only be called in active active mode"); + } + + public abstract Int2ObjectMap getKafkaClusterIdToUrlMap(); + + public abstract boolean hasChangeCaptureView(); + public abstract StoreWriteComputeProcessor getStoreWriteComputeHandler(); protected abstract Lazy> getVeniceWriter( diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index e8b415cc8ad..cdebecf2a2e 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -5,9 +5,12 @@ import com.linkedin.avroutil1.compatibility.shaded.org.apache.commons.lang3.Validate; import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; +import com.linkedin.davinci.replication.RmdWithValueSchemaId; +import com.linkedin.davinci.replication.merge.MergeConflictResolver; import com.linkedin.davinci.replication.merge.MergeConflictResult; import com.linkedin.davinci.stats.HostLevelIngestionStats; import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; +import com.linkedin.davinci.store.record.ByteBufferValueRecord; import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.utils.ByteArrayKey; import com.linkedin.davinci.validation.PartitionTracker; @@ -33,6 +36,7 @@ import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.schema.SchemaEntry; +import com.linkedin.venice.schema.rmd.RmdUtils; import com.linkedin.venice.schema.writecompute.DerivedSchemaEntry; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.stats.StatsErrorCode; @@ -100,7 +104,7 @@ public StorePartitionDataReceiver( } IngestionBatchProcessor.ProcessingFunction processingFunction = (storeIngestionTask.isActiveActiveReplicationEnabled()) - ? storeIngestionTask::processActiveActiveMessage + ? this::processActiveActiveMessage : this::processMessage; KeyLevelLocksManager lockManager = (storeIngestionTask.isActiveActiveReplicationEnabled()) ? storeIngestionTask.getKeyLevelLocksManager().get() @@ -1408,17 +1412,14 @@ private void processActiveActiveMessageAndMaybeProduceToKafka( && consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper() != null) { mergeConflictResultWrapper = consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper(); } else { - mergeConflictResultWrapper = - storeIngestionTask - .processActiveActiveMessage( - consumerRecord, - partitionConsumptionState, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs) - .getMergeConflictResultWrapper(); + mergeConflictResultWrapper = processActiveActiveMessage( + consumerRecord, + partitionConsumptionState, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs).getMergeConflictResultWrapper(); } MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); @@ -1501,6 +1502,195 @@ void queueUpVersionTopicWritesWithViewWriters( partitionConsumptionState.setLastVTProduceCallFuture(currentVersionTopicWrite); } + private PubSubMessageProcessedResult processActiveActiveMessage( + PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs) { + KafkaKey kafkaKey = consumerRecord.getKey(); + KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + byte[] keyBytes = kafkaKey.getKey(); + MessageType msgType = MessageType.valueOf(kafkaValue.messageType); + final int incomingValueSchemaId; + final int incomingWriteComputeSchemaId; + + switch (msgType) { + case PUT: + incomingValueSchemaId = ((Put) kafkaValue.payloadUnion).schemaId; + incomingWriteComputeSchemaId = -1; + break; + case UPDATE: + Update incomingUpdate = (Update) kafkaValue.payloadUnion; + incomingValueSchemaId = incomingUpdate.schemaId; + incomingWriteComputeSchemaId = incomingUpdate.updateSchemaId; + break; + case DELETE: + incomingValueSchemaId = -1; // Ignored since we don't need the schema id for DELETE operations. + incomingWriteComputeSchemaId = -1; + break; + default: + throw new VeniceMessageException( + storeIngestionTask.getIngestionTaskName() + " : Invalid/Unrecognized operation type submitted: " + + kafkaValue.messageType); + } + final ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); + Lazy> oldValueProvider = Lazy.of( + () -> storeIngestionTask.getValueBytesForKey( + partitionConsumptionState, + keyBytes, + consumerRecord.getTopicPartition(), + valueManifestContainer, + beforeProcessingBatchRecordsTimestampMs)); + if (storeIngestionTask.hasChangeCaptureView()) { + /** + * Since this function will update the transient cache before writing the view, and if there is + * a change capture view writer, we need to lookup first, otherwise the transient cache will be populated + * when writing to the view after this function. + */ + oldValueProvider.get(); + } + + final RmdWithValueSchemaId rmdWithValueSchemaID = storeIngestionTask.getReplicationMetadataAndSchemaId( + partitionConsumptionState, + keyBytes, + partition, + beforeProcessingBatchRecordsTimestampMs); + + final long writeTimestamp = storeIngestionTask.getWriteTimestampFromKME(kafkaValue); + final long offsetSumPreOperation = + rmdWithValueSchemaID != null ? RmdUtils.extractOffsetVectorSumFromRmd(rmdWithValueSchemaID.getRmdRecord()) : 0; + List recordTimestampsPreOperation = rmdWithValueSchemaID != null + ? RmdUtils.extractTimestampFromRmd(rmdWithValueSchemaID.getRmdRecord()) + : Collections.singletonList(0L); + + // get the source offset and the id + long sourceOffset = consumerRecord.getOffset(); + final MergeConflictResult mergeConflictResult; + + storeIngestionTask.getAggVersionedIngestionStats() + .recordTotalDCR(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); + + Lazy oldValueByteBufferProvider = + ActiveActiveStoreIngestionTask.unwrapByteBufferFromOldValueProvider(oldValueProvider); + + long beforeDCRTimestampInNs = System.nanoTime(); + final MergeConflictResolver mergeConflictResolver = storeIngestionTask.getMergeConflictResolver(); + switch (msgType) { + case PUT: + mergeConflictResult = mergeConflictResolver.put( + oldValueByteBufferProvider, + rmdWithValueSchemaID, + ((Put) kafkaValue.payloadUnion).putValue, + writeTimestamp, + incomingValueSchemaId, + sourceOffset, + kafkaClusterId, + kafkaClusterId // Use the kafka cluster ID as the colo ID for now because one colo/fabric has only one + // Kafka cluster. TODO: evaluate whether it is enough this way, or we need to add a new + // config to represent the mapping from Kafka server URLs to colo ID. + ); + storeIngestionTask.getHostLevelIngestionStats() + .recordIngestionActiveActivePutLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); + break; + + case DELETE: + mergeConflictResult = mergeConflictResolver.delete( + oldValueByteBufferProvider, + rmdWithValueSchemaID, + writeTimestamp, + sourceOffset, + kafkaClusterId, + kafkaClusterId); + storeIngestionTask.getHostLevelIngestionStats() + .recordIngestionActiveActiveDeleteLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); + break; + + case UPDATE: + mergeConflictResult = mergeConflictResolver.update( + oldValueByteBufferProvider, + rmdWithValueSchemaID, + ((Update) kafkaValue.payloadUnion).updateValue, + incomingValueSchemaId, + incomingWriteComputeSchemaId, + writeTimestamp, + sourceOffset, + kafkaClusterId, + kafkaClusterId, + valueManifestContainer); + storeIngestionTask.getHostLevelIngestionStats() + .recordIngestionActiveActiveUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); + break; + default: + throw new VeniceMessageException( + storeIngestionTask.getIngestionTaskName() + " : Invalid/Unrecognized operation type submitted: " + + kafkaValue.messageType); + } + + if (mergeConflictResult.isUpdateIgnored()) { + storeIngestionTask.getHostLevelIngestionStats().recordUpdateIgnoredDCR(); + // Record the last ignored offset + partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset( + storeIngestionTask.getKafkaClusterIdToUrlMap().get(kafkaClusterId), + sourceOffset); + return new PubSubMessageProcessedResult( + new MergeConflictResultWrapper( + mergeConflictResult, + oldValueProvider, + oldValueByteBufferProvider, + rmdWithValueSchemaID, + valueManifestContainer, + null, + null)); + } else { + storeIngestionTask.validatePostOperationResultsAndRecord( + mergeConflictResult, + offsetSumPreOperation, + recordTimestampsPreOperation); + + final ByteBuffer updatedValueBytes = storeIngestionTask.maybeCompressData( + consumerRecord.getTopicPartition().getPartitionNumber(), + mergeConflictResult.getNewValue(), + partitionConsumptionState); + + final int valueSchemaId = mergeConflictResult.getValueSchemaId(); + + GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); + final ByteBuffer updatedRmdBytes = storeIngestionTask.getRmdSerDe() + .serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); + + if (updatedValueBytes == null) { + storeIngestionTask.getHostLevelIngestionStats().recordTombstoneCreatedDCR(); + storeIngestionTask.getAggVersionedIngestionStats() + .recordTombStoneCreationDCR(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); + partitionConsumptionState + .setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, valueSchemaId, rmdRecord); + } else { + int valueLen = updatedValueBytes.remaining(); + partitionConsumptionState.setTransientRecord( + kafkaClusterId, + consumerRecord.getOffset(), + keyBytes, + updatedValueBytes.array(), + updatedValueBytes.position(), + valueLen, + valueSchemaId, + rmdRecord); + } + return new PubSubMessageProcessedResult( + new MergeConflictResultWrapper( + mergeConflictResult, + oldValueProvider, + oldValueByteBufferProvider, + rmdWithValueSchemaID, + valueManifestContainer, + updatedValueBytes, + updatedRmdBytes)); + } + } + /** * This function parses the {@link MergeConflictResult} and decides if the update should be ignored or emit a PUT or a * DELETE record to VT. From 1270f39564755c901c356a551abe6ac10ac7dacb Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 02:43:43 -0700 Subject: [PATCH 18/32] =?UTF-8?q?Refactored=20`validatePostOperationResult?= =?UTF-8?q?sAndRecord()`=20from=20`ActiveActiveStoreIngestionTask`=20into?= =?UTF-8?q?=20`StorePartitionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 115 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 12 -- .../consumer/StorePartitionDataReceiver.java | 68 +++++++++-- .../ActiveActiveStoreIngestionTaskTest.java | 4 +- 4 files changed, 119 insertions(+), 80 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 82f7b1fd778..53601b0373d 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -10,7 +10,6 @@ import com.linkedin.davinci.replication.RmdWithValueSchemaId; import com.linkedin.davinci.replication.merge.MergeConflictResolver; import com.linkedin.davinci.replication.merge.MergeConflictResolverFactory; -import com.linkedin.davinci.replication.merge.MergeConflictResult; import com.linkedin.davinci.replication.merge.RmdSerDe; import com.linkedin.davinci.replication.merge.StringAnnotatedStoreSchemaCache; import com.linkedin.davinci.stats.AggVersionedIngestionStats; @@ -39,7 +38,6 @@ import com.linkedin.venice.pubsub.api.PubSubMessage; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; -import com.linkedin.venice.schema.rmd.RmdUtils; import com.linkedin.venice.serialization.RawBytesStoreDeserializerCache; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.LatencyUtils; @@ -58,7 +56,6 @@ import java.util.Properties; import java.util.Set; import java.util.function.BooleanSupplier; -import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.BinaryDecoder; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.logging.log4j.LogManager; @@ -671,62 +668,62 @@ byte[] getRmdWithValueSchemaByteBufferFromStorage( // } // } // } - - /** - * Package private for testing purposes. - */ - static Lazy unwrapByteBufferFromOldValueProvider( - Lazy> oldValueProvider) { - return Lazy.of(() -> { - ByteBufferValueRecord bbValueRecord = oldValueProvider.get(); - return bbValueRecord == null ? null : bbValueRecord.value(); - }); - } - - @Override - public long getWriteTimestampFromKME(KafkaMessageEnvelope kme) { - if (kme.producerMetadata.logicalTimestamp >= 0) { - return kme.producerMetadata.logicalTimestamp; - } else { - return kme.producerMetadata.messageTimestamp; - } - } - - @Override - void validatePostOperationResultsAndRecord( - MergeConflictResult mergeConflictResult, - Long offsetSumPreOperation, - List timestampsPreOperation) { - // Nothing was applied, no harm no foul - if (mergeConflictResult.isUpdateIgnored()) { - return; - } - // Post Validation checks on resolution - GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); - if (offsetSumPreOperation > RmdUtils.extractOffsetVectorSumFromRmd(rmdRecord)) { - // offsets went backwards, raise an alert! - hostLevelIngestionStats.recordOffsetRegressionDCRError(); - aggVersionedIngestionStats.recordOffsetRegressionDCRError(storeName, versionNumber); - LOGGER - .error("Offset vector found to have gone backwards!! New invalid replication metadata result: {}", rmdRecord); - } - - // TODO: This comparison doesn't work well for write compute+schema evolution (can spike up). VENG-8129 - // this works fine for now however as we do not fully support A/A write compute operations (as we only do root - // timestamp comparisons). - - List timestampsPostOperation = RmdUtils.extractTimestampFromRmd(rmdRecord); - for (int i = 0; i < timestampsPreOperation.size(); i++) { - if (timestampsPreOperation.get(i) > timestampsPostOperation.get(i)) { - // timestamps went backwards, raise an alert! - hostLevelIngestionStats.recordTimestampRegressionDCRError(); - aggVersionedIngestionStats.recordTimestampRegressionDCRError(storeName, versionNumber); - LOGGER.error( - "Timestamp found to have gone backwards!! Invalid replication metadata result: {}", - mergeConflictResult.getRmdRecord()); - } - } - } + // + // /** + // * Package private for testing purposes. + // */ + // static Lazy unwrapByteBufferFromOldValueProvider( + // Lazy> oldValueProvider) { + // return Lazy.of(() -> { + // ByteBufferValueRecord bbValueRecord = oldValueProvider.get(); + // return bbValueRecord == null ? null : bbValueRecord.value(); + // }); + // } + // + // @Override + // public long getWriteTimestampFromKME(KafkaMessageEnvelope kme) { + // if (kme.producerMetadata.logicalTimestamp >= 0) { + // return kme.producerMetadata.logicalTimestamp; + // } else { + // return kme.producerMetadata.messageTimestamp; + // } + // } + // + // @Override + // void validatePostOperationResultsAndRecord( + // MergeConflictResult mergeConflictResult, + // Long offsetSumPreOperation, + // List timestampsPreOperation) { + // // Nothing was applied, no harm no foul + // if (mergeConflictResult.isUpdateIgnored()) { + // return; + // } + // // Post Validation checks on resolution + // GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); + // if (offsetSumPreOperation > RmdUtils.extractOffsetVectorSumFromRmd(rmdRecord)) { + // // offsets went backwards, raise an alert! + // hostLevelIngestionStats.recordOffsetRegressionDCRError(); + // aggVersionedIngestionStats.recordOffsetRegressionDCRError(storeName, versionNumber); + // LOGGER + // .error("Offset vector found to have gone backwards!! New invalid replication metadata result: {}", rmdRecord); + // } + // + // // TODO: This comparison doesn't work well for write compute+schema evolution (can spike up). VENG-8129 + // // this works fine for now however as we do not fully support A/A write compute operations (as we only do root + // // timestamp comparisons). + // + // List timestampsPostOperation = RmdUtils.extractTimestampFromRmd(rmdRecord); + // for (int i = 0; i < timestampsPreOperation.size(); i++) { + // if (timestampsPreOperation.get(i) > timestampsPostOperation.get(i)) { + // // timestamps went backwards, raise an alert! + // hostLevelIngestionStats.recordTimestampRegressionDCRError(); + // aggVersionedIngestionStats.recordTimestampRegressionDCRError(storeName, versionNumber); + // LOGGER.error( + // "Timestamp found to have gone backwards!! Invalid replication metadata result: {}", + // mergeConflictResult.getRmdRecord()); + // } + // } + // } /** * Get the value bytes for a key from {@link PartitionConsumptionState.TransientRecord} or from disk. The assumption diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index dda9c94d602..72625604d2e 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -34,7 +34,6 @@ import com.linkedin.davinci.notifier.VeniceNotifier; import com.linkedin.davinci.replication.RmdWithValueSchemaId; import com.linkedin.davinci.replication.merge.MergeConflictResolver; -import com.linkedin.davinci.replication.merge.MergeConflictResult; import com.linkedin.davinci.replication.merge.RmdSerDe; import com.linkedin.davinci.stats.AggVersionedDIVStats; import com.linkedin.davinci.stats.AggVersionedIngestionStats; @@ -4760,13 +4759,6 @@ protected abstract GenericRecord readStoredValueRecord( PubSubTopicPartition topicPartition, ChunkedValueManifestContainer manifestContainer); - void validatePostOperationResultsAndRecord( - MergeConflictResult mergeConflictResult, - Long offsetSumPreOperation, - List timestampsPreOperation) { - throw new VeniceException("validatePostOperationResultsAndRecord() should only be called in active active mode"); - } - public AggVersionedIngestionStats getAggVersionedIngestionStats() { throw new VeniceException("getAggVersionedIngestionStats() should only be called in active active mode"); } @@ -4792,10 +4784,6 @@ RmdWithValueSchemaId getReplicationMetadataAndSchemaId( throw new VeniceException("getReplicationMetadataAndSchemaId() should only be called in active active mode"); } - public long getWriteTimestampFromKME(KafkaMessageEnvelope kme) { - throw new VeniceException("getWriteTimestampFromKME() should only be called in active active mode"); - } - public MergeConflictResolver getMergeConflictResolver() { throw new VeniceException("getMergeConflictResolver() should only be called in active active mode"); } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index cdebecf2a2e..1109b6977f5 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -1559,7 +1559,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( partition, beforeProcessingBatchRecordsTimestampMs); - final long writeTimestamp = storeIngestionTask.getWriteTimestampFromKME(kafkaValue); + final long writeTimestamp = getWriteTimestampFromKME(kafkaValue); final long offsetSumPreOperation = rmdWithValueSchemaID != null ? RmdUtils.extractOffsetVectorSumFromRmd(rmdWithValueSchemaID.getRmdRecord()) : 0; List recordTimestampsPreOperation = rmdWithValueSchemaID != null @@ -1573,8 +1573,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( storeIngestionTask.getAggVersionedIngestionStats() .recordTotalDCR(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); - Lazy oldValueByteBufferProvider = - ActiveActiveStoreIngestionTask.unwrapByteBufferFromOldValueProvider(oldValueProvider); + Lazy oldValueByteBufferProvider = unwrapByteBufferFromOldValueProvider(oldValueProvider); long beforeDCRTimestampInNs = System.nanoTime(); final MergeConflictResolver mergeConflictResolver = storeIngestionTask.getMergeConflictResolver(); @@ -1645,10 +1644,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( null, null)); } else { - storeIngestionTask.validatePostOperationResultsAndRecord( - mergeConflictResult, - offsetSumPreOperation, - recordTimestampsPreOperation); + validatePostOperationResultsAndRecord(mergeConflictResult, offsetSumPreOperation, recordTimestampsPreOperation); final ByteBuffer updatedValueBytes = storeIngestionTask.maybeCompressData( consumerRecord.getTopicPartition().getPartitionNumber(), @@ -1991,6 +1987,64 @@ private LeaderProducerCallback createProducerCallback( beforeProcessingRecordTimestampNs); } + /** + * Package private for testing purposes. + */ + static Lazy unwrapByteBufferFromOldValueProvider( + Lazy> oldValueProvider) { + return Lazy.of(() -> { + ByteBufferValueRecord bbValueRecord = oldValueProvider.get(); + return bbValueRecord == null ? null : bbValueRecord.value(); + }); + } + + public long getWriteTimestampFromKME(KafkaMessageEnvelope kme) { + if (kme.producerMetadata.logicalTimestamp >= 0) { + return kme.producerMetadata.logicalTimestamp; + } else { + return kme.producerMetadata.messageTimestamp; + } + } + + private void validatePostOperationResultsAndRecord( + MergeConflictResult mergeConflictResult, + Long offsetSumPreOperation, + List timestampsPreOperation) { + // Nothing was applied, no harm no foul + if (mergeConflictResult.isUpdateIgnored()) { + return; + } + // Post Validation checks on resolution + GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); + if (offsetSumPreOperation > RmdUtils.extractOffsetVectorSumFromRmd(rmdRecord)) { + // offsets went backwards, raise an alert! + storeIngestionTask.getHostLevelIngestionStats().recordOffsetRegressionDCRError(); + storeIngestionTask.getAggVersionedIngestionStats() + .recordOffsetRegressionDCRError(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); + LOGGER + .error("Offset vector found to have gone backwards!! New invalid replication metadata result: {}", rmdRecord); + } + + // TODO: This comparison doesn't work well for write compute+schema evolution (can spike up). VENG-8129 + // this works fine for now however as we do not fully support A/A write compute operations (as we only do root + // timestamp comparisons). + + List timestampsPostOperation = RmdUtils.extractTimestampFromRmd(rmdRecord); + for (int i = 0; i < timestampsPreOperation.size(); i++) { + if (timestampsPreOperation.get(i) > timestampsPostOperation.get(i)) { + // timestamps went backwards, raise an alert! + storeIngestionTask.getHostLevelIngestionStats().recordTimestampRegressionDCRError(); + storeIngestionTask.getAggVersionedIngestionStats() + .recordTimestampRegressionDCRError( + storeIngestionTask.getStoreName(), + storeIngestionTask.getVersionNumber()); + LOGGER.error( + "Timestamp found to have gone backwards!! Invalid replication metadata result: {}", + mergeConflictResult.getRmdRecord()); + } + } + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java index 994a7266924..864f9b907e9 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java @@ -608,11 +608,11 @@ public void testReadingChunkedRmdFromStorage() { @Test public void testUnwrapByteBufferFromOldValueProvider() { - Lazy lazyBB = ActiveActiveStoreIngestionTask.unwrapByteBufferFromOldValueProvider(Lazy.of(() -> null)); + Lazy lazyBB = StorePartitionDataReceiver.unwrapByteBufferFromOldValueProvider(Lazy.of(() -> null)); assertNotNull(lazyBB); assertNull(lazyBB.get()); - lazyBB = ActiveActiveStoreIngestionTask.unwrapByteBufferFromOldValueProvider( + lazyBB = StorePartitionDataReceiver.unwrapByteBufferFromOldValueProvider( Lazy.of(() -> new ByteBufferValueRecord<>(ByteBuffer.wrap(new byte[1]), 1))); assertNotNull(lazyBB); assertNotNull(lazyBB.get()); From 1f9f4c27494c33e3c8ddfd753e9f314b8196b974 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 02:51:36 -0700 Subject: [PATCH 19/32] =?UTF-8?q?Refactored=20`getValueBytesForKey()`=20an?= =?UTF-8?q?d=20`getCurrentValueFromTransientRecord()`=20from=20`ActiveActi?= =?UTF-8?q?veStoreIngestionTask`=20into=20`StorePartitionDataReceiver`.=20?= =?UTF-8?q?=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 159 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 10 -- .../consumer/StorePartitionDataReceiver.java | 86 +++++++++- .../ActiveActiveStoreIngestionTaskTest.java | 6 +- 4 files changed, 166 insertions(+), 95 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 53601b0373d..0afb045bdea 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -5,6 +5,7 @@ import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.davinci.client.DaVinciRecordTransformerConfig; +import com.linkedin.davinci.client.DaVinciRecordTransformerFunctionalInterface; import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.config.VeniceStoreVersionConfig; import com.linkedin.davinci.replication.RmdWithValueSchemaId; @@ -15,10 +16,8 @@ import com.linkedin.davinci.stats.AggVersionedIngestionStats; import com.linkedin.davinci.storage.StorageService; import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; -import com.linkedin.davinci.storage.chunking.RawBytesChunkingAdapter; import com.linkedin.davinci.storage.chunking.SingleGetChunkingAdapter; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; -import com.linkedin.davinci.store.record.ByteBufferValueRecord; import com.linkedin.davinci.store.record.ValueRecord; import com.linkedin.venice.exceptions.PersistenceFailureException; import com.linkedin.venice.exceptions.VeniceException; @@ -38,14 +37,12 @@ import com.linkedin.venice.pubsub.api.PubSubMessage; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; -import com.linkedin.venice.serialization.RawBytesStoreDeserializerCache; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.lazy.Lazy; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; -import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; @@ -56,7 +53,6 @@ import java.util.Properties; import java.util.Set; import java.util.function.BooleanSupplier; -import org.apache.avro.io.BinaryDecoder; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -67,7 +63,7 @@ */ public class ActiveActiveStoreIngestionTask extends LeaderFollowerStoreIngestionTask { private static final Logger LOGGER = LogManager.getLogger(ActiveActiveStoreIngestionTask.class); - private static final byte[] BINARY_DECODER_PARAM = new byte[16]; + // private static final byte[] BINARY_DECODER_PARAM = new byte[16]; private final int rmdProtocolVersionId; private final MergeConflictResolver mergeConflictResolver; @@ -76,15 +72,16 @@ public class ActiveActiveStoreIngestionTask extends LeaderFollowerStoreIngestion private final AggVersionedIngestionStats aggVersionedIngestionStats; private final RemoteIngestionRepairService remoteIngestionRepairService; // private final Lazy ingestionBatchProcessorLazy; - - private static class ReusableObjects { - // reuse buffer for rocksDB value object - final ByteBuffer reusedByteBuffer = ByteBuffer.allocate(1024 * 1024); - final BinaryDecoder binaryDecoder = - AvroCompatibilityHelper.newBinaryDecoder(BINARY_DECODER_PARAM, 0, BINARY_DECODER_PARAM.length, null); - } - - private final ThreadLocal threadLocalReusableObjects = ThreadLocal.withInitial(ReusableObjects::new); + // + // private static class ReusableObjects { + // // reuse buffer for rocksDB value object + // final ByteBuffer reusedByteBuffer = ByteBuffer.allocate(1024 * 1024); + // final BinaryDecoder binaryDecoder = + // AvroCompatibilityHelper.newBinaryDecoder(BINARY_DECODER_PARAM, 0, BINARY_DECODER_PARAM.length, null); + // } + // + // private final ThreadLocal threadLocalReusableObjects = + // ThreadLocal.withInitial(ReusableObjects::new); public ActiveActiveStoreIngestionTask( StorageService storageService, @@ -724,72 +721,72 @@ byte[] getRmdWithValueSchemaByteBufferFromStorage( // } // } // } - - /** - * Get the value bytes for a key from {@link PartitionConsumptionState.TransientRecord} or from disk. The assumption - * is that the {@link PartitionConsumptionState.TransientRecord} only contains the full value. - * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition - * @param key The key bytes of the incoming record. - * @param topicPartition The {@link PubSubTopicPartition} from which the incoming record was consumed - * @return - */ - @Override - public ByteBufferValueRecord getValueBytesForKey( - PartitionConsumptionState partitionConsumptionState, - byte[] key, - PubSubTopicPartition topicPartition, - ChunkedValueManifestContainer valueManifestContainer, - long currentTimeForMetricsMs) { - ByteBufferValueRecord originalValue = null; - // Find the existing value. If a value for this key is found from the transient map then use that value, otherwise - // get it from DB. - PartitionConsumptionState.TransientRecord transientRecord = partitionConsumptionState.getTransientRecord(key); - if (transientRecord == null) { - long lookupStartTimeInNS = System.nanoTime(); - ReusableObjects reusableObjects = threadLocalReusableObjects.get(); - ByteBuffer reusedRawValue = reusableObjects.reusedByteBuffer; - BinaryDecoder binaryDecoder = reusableObjects.binaryDecoder; - originalValue = RawBytesChunkingAdapter.INSTANCE.getWithSchemaId( - storageEngine, - topicPartition.getPartitionNumber(), - ByteBuffer.wrap(key), - isChunked, - reusedRawValue, - binaryDecoder, - RawBytesStoreDeserializerCache.getInstance(), - compressor.get(), - valueManifestContainer); - hostLevelIngestionStats.recordIngestionValueBytesLookUpLatency( - LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS), - currentTimeForMetricsMs); - } else { - hostLevelIngestionStats.recordIngestionValueBytesCacheHitCount(currentTimeForMetricsMs); - // construct originalValue from this transient record only if it's not null. - if (transientRecord.getValue() != null) { - if (valueManifestContainer != null) { - valueManifestContainer.setManifest(transientRecord.getValueManifest()); - } - originalValue = new ByteBufferValueRecord<>( - getCurrentValueFromTransientRecord(transientRecord), - transientRecord.getValueSchemaId()); - } - } - return originalValue; - } - - ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.TransientRecord transientRecord) { - ByteBuffer compressedValue = - ByteBuffer.wrap(transientRecord.getValue(), transientRecord.getValueOffset(), transientRecord.getValueLen()); - try { - return getCompressionStrategy().isCompressionEnabled() - ? getCompressor().get() - .decompress(compressedValue.array(), compressedValue.position(), compressedValue.remaining()) - : compressedValue; - } catch (IOException e) { - throw new VeniceException(e); - } - } - + // + // /** + // * Get the value bytes for a key from {@link PartitionConsumptionState.TransientRecord} or from disk. The assumption + // * is that the {@link PartitionConsumptionState.TransientRecord} only contains the full value. + // * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition + // * @param key The key bytes of the incoming record. + // * @param topicPartition The {@link PubSubTopicPartition} from which the incoming record was consumed + // * @return + // */ + // @Override + // public ByteBufferValueRecord getValueBytesForKey( + // PartitionConsumptionState partitionConsumptionState, + // byte[] key, + // PubSubTopicPartition topicPartition, + // ChunkedValueManifestContainer valueManifestContainer, + // long currentTimeForMetricsMs) { + // ByteBufferValueRecord originalValue = null; + // // Find the existing value. If a value for this key is found from the transient map then use that value, otherwise + // // get it from DB. + // PartitionConsumptionState.TransientRecord transientRecord = partitionConsumptionState.getTransientRecord(key); + // if (transientRecord == null) { + // long lookupStartTimeInNS = System.nanoTime(); + // ReusableObjects reusableObjects = threadLocalReusableObjects.get(); + // ByteBuffer reusedRawValue = reusableObjects.reusedByteBuffer; + // BinaryDecoder binaryDecoder = reusableObjects.binaryDecoder; + // originalValue = RawBytesChunkingAdapter.INSTANCE.getWithSchemaId( + // storageEngine, + // topicPartition.getPartitionNumber(), + // ByteBuffer.wrap(key), + // isChunked, + // reusedRawValue, + // binaryDecoder, + // RawBytesStoreDeserializerCache.getInstance(), + // compressor.get(), + // valueManifestContainer); + // hostLevelIngestionStats.recordIngestionValueBytesLookUpLatency( + // LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS), + // currentTimeForMetricsMs); + // } else { + // hostLevelIngestionStats.recordIngestionValueBytesCacheHitCount(currentTimeForMetricsMs); + // // construct originalValue from this transient record only if it's not null. + // if (transientRecord.getValue() != null) { + // if (valueManifestContainer != null) { + // valueManifestContainer.setManifest(transientRecord.getValueManifest()); + // } + // originalValue = new ByteBufferValueRecord<>( + // getCurrentValueFromTransientRecord(transientRecord), + // transientRecord.getValueSchemaId()); + // } + // } + // return originalValue; + // } + // + // ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.TransientRecord transientRecord) { + // ByteBuffer compressedValue = + // ByteBuffer.wrap(transientRecord.getValue(), transientRecord.getValueOffset(), transientRecord.getValueLen()); + // try { + // return getCompressionStrategy().isCompressionEnabled() + // ? getCompressor().get() + // .decompress(compressedValue.array(), compressedValue.position(), compressedValue.remaining()) + // : compressedValue; + // } catch (IOException e) { + // throw new VeniceException(e); + // } + // } + // // /** // * This function parses the {@link MergeConflictResult} and decides if the update should be ignored or emit a PUT or // a diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 72625604d2e..73aa323de29 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -45,7 +45,6 @@ import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.StoragePartitionConfig; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; -import com.linkedin.davinci.store.record.ByteBufferValueRecord; import com.linkedin.davinci.store.record.ValueRecord; import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.utils.ChunkAssembler; @@ -4767,15 +4766,6 @@ public int getRmdProtocolVersionId() { throw new VeniceException("getRmdProtocolVersionId() should only be called in active active mode"); } - public ByteBufferValueRecord getValueBytesForKey( - PartitionConsumptionState partitionConsumptionState, - byte[] key, - PubSubTopicPartition topicPartition, - ChunkedValueManifestContainer valueManifestContainer, - long currentTimeForMetricsMs) { - throw new VeniceException("getValueBytesForKey() should only be called in active active mode"); - } - RmdWithValueSchemaId getReplicationMetadataAndSchemaId( PartitionConsumptionState partitionConsumptionState, byte[] key, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 1109b6977f5..8444cf08180 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -3,6 +3,7 @@ import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_PUSH; import static com.linkedin.venice.writer.VeniceWriter.APP_DEFAULT_LOGICAL_TS; +import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.avroutil1.compatibility.shaded.org.apache.commons.lang3.Validate; import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; import com.linkedin.davinci.replication.RmdWithValueSchemaId; @@ -10,6 +11,7 @@ import com.linkedin.davinci.replication.merge.MergeConflictResult; import com.linkedin.davinci.stats.HostLevelIngestionStats; import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; +import com.linkedin.davinci.storage.chunking.RawBytesChunkingAdapter; import com.linkedin.davinci.store.record.ByteBufferValueRecord; import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.utils.ByteArrayKey; @@ -38,6 +40,7 @@ import com.linkedin.venice.schema.SchemaEntry; import com.linkedin.venice.schema.rmd.RmdUtils; import com.linkedin.venice.schema.writecompute.DerivedSchemaEntry; +import com.linkedin.venice.serialization.RawBytesStoreDeserializerCache; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.stats.StatsErrorCode; import com.linkedin.venice.storage.protocol.ChunkedValueManifest; @@ -53,6 +56,7 @@ import com.linkedin.venice.writer.LeaderMetadataWrapper; import com.linkedin.venice.writer.PutMetadata; import com.linkedin.venice.writer.VeniceWriter; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -68,22 +72,34 @@ import java.util.function.BiConsumer; import java.util.function.Function; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; public class StorePartitionDataReceiver implements ConsumedDataReceiver>> { + private final Logger LOGGER; + private static final byte[] BINARY_DECODER_PARAM = new byte[16]; + private final StoreIngestionTask storeIngestionTask; private final PubSubTopicPartition topicPartition; private final String kafkaUrl; private final String kafkaUrlForLogger; private final int kafkaClusterId; private final Lazy ingestionBatchProcessorLazy; - private final Logger LOGGER; private long receivedRecordsCount; + private static class ReusableObjects { + // reuse buffer for rocksDB value object + final ByteBuffer reusedByteBuffer = ByteBuffer.allocate(1024 * 1024); + final BinaryDecoder binaryDecoder = + AvroCompatibilityHelper.newBinaryDecoder(BINARY_DECODER_PARAM, 0, BINARY_DECODER_PARAM.length, null); + } + + private final ThreadLocal threadLocalReusableObjects = ThreadLocal.withInitial(ReusableObjects::new); + public StorePartitionDataReceiver( StoreIngestionTask storeIngestionTask, PubSubTopicPartition topicPartition, @@ -1538,7 +1554,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( } final ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); Lazy> oldValueProvider = Lazy.of( - () -> storeIngestionTask.getValueBytesForKey( + () -> getValueBytesForKey( partitionConsumptionState, keyBytes, consumerRecord.getTopicPartition(), @@ -2045,6 +2061,72 @@ private void validatePostOperationResultsAndRecord( } } + /** + * Get the value bytes for a key from {@link PartitionConsumptionState.TransientRecord} or from disk. The assumption + * is that the {@link PartitionConsumptionState.TransientRecord} only contains the full value. + * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition + * @param key The key bytes of the incoming record. + * @param topicPartition The {@link PubSubTopicPartition} from which the incoming record was consumed + * @return + */ + private ByteBufferValueRecord getValueBytesForKey( + PartitionConsumptionState partitionConsumptionState, + byte[] key, + PubSubTopicPartition topicPartition, + ChunkedValueManifestContainer valueManifestContainer, + long currentTimeForMetricsMs) { + ByteBufferValueRecord originalValue = null; + // Find the existing value. If a value for this key is found from the transient map then use that value, otherwise + // get it from DB. + PartitionConsumptionState.TransientRecord transientRecord = partitionConsumptionState.getTransientRecord(key); + if (transientRecord == null) { + long lookupStartTimeInNS = System.nanoTime(); + ReusableObjects reusableObjects = threadLocalReusableObjects.get(); + ByteBuffer reusedRawValue = reusableObjects.reusedByteBuffer; + BinaryDecoder binaryDecoder = reusableObjects.binaryDecoder; + originalValue = RawBytesChunkingAdapter.INSTANCE.getWithSchemaId( + storeIngestionTask.getStorageEngine(), + topicPartition.getPartitionNumber(), + ByteBuffer.wrap(key), + storeIngestionTask.isChunked(), + reusedRawValue, + binaryDecoder, + RawBytesStoreDeserializerCache.getInstance(), + storeIngestionTask.getCompressor().get(), + valueManifestContainer); + storeIngestionTask.getHostLevelIngestionStats() + .recordIngestionValueBytesLookUpLatency( + LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS), + currentTimeForMetricsMs); + } else { + storeIngestionTask.getHostLevelIngestionStats().recordIngestionValueBytesCacheHitCount(currentTimeForMetricsMs); + // construct originalValue from this transient record only if it's not null. + if (transientRecord.getValue() != null) { + if (valueManifestContainer != null) { + valueManifestContainer.setManifest(transientRecord.getValueManifest()); + } + originalValue = new ByteBufferValueRecord<>( + getCurrentValueFromTransientRecord(transientRecord), + transientRecord.getValueSchemaId()); + } + } + return originalValue; + } + + ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.TransientRecord transientRecord) { + ByteBuffer compressedValue = + ByteBuffer.wrap(transientRecord.getValue(), transientRecord.getValueOffset(), transientRecord.getValueLen()); + try { + return storeIngestionTask.getCompressionStrategy().isCompressionEnabled() + ? storeIngestionTask.getCompressor() + .get() + .decompress(compressedValue.array(), compressedValue.position(), compressedValue.remaining()) + : compressedValue; + } catch (IOException e) { + throw new VeniceException(e); + } + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java index 864f9b907e9..0fdef9ff5ea 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java @@ -177,7 +177,9 @@ public void testGetValueBytesFromTransientRecords(CompressionStrategy strategy) VeniceCompressor compressor = getCompressor(strategy); when(ingestionTask.getCompressor()).thenReturn(Lazy.of(() -> compressor)); when(ingestionTask.getCompressionStrategy()).thenReturn(strategy); - when(ingestionTask.getCurrentValueFromTransientRecord(any())).thenCallRealMethod(); + PubSubTopicPartition topicPartition = mock(PubSubTopicPartition.class); + StorePartitionDataReceiver storePartitionDataReceiver = + new StorePartitionDataReceiver(ingestionTask, topicPartition, "dummyUrl", 0); byte[] dataBytes = "Hello World".getBytes(); byte[] transientRecordValueBytes = dataBytes; @@ -192,7 +194,7 @@ public void testGetValueBytesFromTransientRecords(CompressionStrategy strategy) when(transientRecord.getValue()).thenReturn(transientRecordValueBytes); when(transientRecord.getValueOffset()).thenReturn(startPosition); when(transientRecord.getValueLen()).thenReturn(dataLength); - ByteBuffer result = ingestionTask.getCurrentValueFromTransientRecord(transientRecord); + ByteBuffer result = storePartitionDataReceiver.getCurrentValueFromTransientRecord(transientRecord); Assert.assertEquals(result.remaining(), dataBytes.length); byte[] resultByteArray = new byte[result.remaining()]; result.get(resultByteArray); From 23544b6ce7ce823fb336b8f54ade4c94e2c14697 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 14:30:30 -0700 Subject: [PATCH 20/32] =?UTF-8?q?Refactored=20`maybeCompressData()`=20from?= =?UTF-8?q?=20`LeaderFollowerStoreIngestionTask`=20into=20`StorePartitionD?= =?UTF-8?q?ataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 102 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 9 +- .../consumer/StorePartitionDataReceiver.java | 48 ++++++++- 3 files changed, 100 insertions(+), 59 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index c0f69e598de..96bdf8b5fc7 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -29,7 +29,6 @@ import com.linkedin.davinci.store.view.ChangeCaptureViewWriter; import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.validation.KafkaDataIntegrityValidator; -import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceMessageException; import com.linkedin.venice.exceptions.VeniceTimeoutException; @@ -72,7 +71,6 @@ import com.linkedin.venice.writer.VeniceWriterFactory; import com.linkedin.venice.writer.VeniceWriterOptions; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; -import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; @@ -1256,7 +1254,7 @@ protected boolean shouldProduceToVersionTopic(PartitionConsumptionState partitio return (!versionTopic.equals(leaderTopic) || partitionConsumptionState.consumeRemotely()); } - protected static boolean isLeader(PartitionConsumptionState partitionConsumptionState) { + public static boolean isLeader(PartitionConsumptionState partitionConsumptionState) { return Objects.equals(partitionConsumptionState.getLeaderFollowerState(), LEADER); } @@ -3157,55 +3155,55 @@ public void updateLeaderTopicOnFollower(PartitionConsumptionState partitionConsu } } - /** - * Compresses data in a bytebuffer when consuming from rt as a leader node and compression is enabled for the store - * version for which we're consuming data. - * - * @param partition which partition we're acting on so as to determine the PartitionConsumptionState - * @param data the data that we might compress - * @return a bytebuffer that's either the original bytebuffer or a new one depending on if we compressed it. - */ - @Override - protected ByteBuffer maybeCompressData( - int partition, - ByteBuffer data, - PartitionConsumptionState partitionConsumptionState) { - // To handle delete operations - if (data == null) { - return null; - } - if (shouldCompressData(partitionConsumptionState)) { - try { - long startTimeInNS = System.nanoTime(); - // We need to expand the front of the returned bytebuffer to make room for schema header insertion - ByteBuffer result = compressor.get().compress(data, ByteUtils.SIZE_OF_INT); - hostLevelIngestionStats.recordLeaderCompressLatency(LatencyUtils.getElapsedTimeFromNSToMS(startTimeInNS)); - return result; - } catch (IOException e) { - // throw a loud exception if something goes wrong here - throw new RuntimeException( - String.format( - "Failed to compress value in venice writer! Aborting write! partition: %d, leader topic: %s, compressor: %s", - partition, - partitionConsumptionState.getOffsetRecord().getLeaderTopic(pubSubTopicRepository), - compressor.getClass().getName()), - e); - } - } - return data; - } - - protected boolean shouldCompressData(PartitionConsumptionState partitionConsumptionState) { - if (!isLeader(partitionConsumptionState)) { - return false; // Not leader, don't compress - } - PubSubTopic leaderTopic = partitionConsumptionState.getOffsetRecord().getLeaderTopic(pubSubTopicRepository); - if (!realTimeTopic.equals(leaderTopic)) { - return false; // We're consuming from version topic (don't compress it) - } - return !compressionStrategy.equals(CompressionStrategy.NO_OP); - } - + // /** + // * Compresses data in a bytebuffer when consuming from rt as a leader node and compression is enabled for the store + // * version for which we're consuming data. + // * + // * @param partition which partition we're acting on so as to determine the PartitionConsumptionState + // * @param data the data that we might compress + // * @return a bytebuffer that's either the original bytebuffer or a new one depending on if we compressed it. + // */ + // @Override + // protected ByteBuffer maybeCompressData( + // int partition, + // ByteBuffer data, + // PartitionConsumptionState partitionConsumptionState) { + // // To handle delete operations + // if (data == null) { + // return null; + // } + // if (shouldCompressData(partitionConsumptionState)) { + // try { + // long startTimeInNS = System.nanoTime(); + // // We need to expand the front of the returned bytebuffer to make room for schema header insertion + // ByteBuffer result = compressor.get().compress(data, ByteUtils.SIZE_OF_INT); + // hostLevelIngestionStats.recordLeaderCompressLatency(LatencyUtils.getElapsedTimeFromNSToMS(startTimeInNS)); + // return result; + // } catch (IOException e) { + // // throw a loud exception if something goes wrong here + // throw new RuntimeException( + // String.format( + // "Failed to compress value in venice writer! Aborting write! partition: %d, leader topic: %s, compressor: %s", + // partition, + // partitionConsumptionState.getOffsetRecord().getLeaderTopic(pubSubTopicRepository), + // compressor.getClass().getName()), + // e); + // } + // } + // return data; + // } + // + // protected boolean shouldCompressData(PartitionConsumptionState partitionConsumptionState) { + // if (!isLeader(partitionConsumptionState)) { + // return false; // Not leader, don't compress + // } + // PubSubTopic leaderTopic = partitionConsumptionState.getOffsetRecord().getLeaderTopic(pubSubTopicRepository); + // if (!realTimeTopic.equals(leaderTopic)) { + // return false; // We're consuming from version topic (don't compress it) + // } + // return !compressionStrategy.equals(CompressionStrategy.NO_OP); + // } + // // @Override // protected PubSubMessageProcessedResult processMessage( // PubSubMessage consumerRecord, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 73aa323de29..df0355ad4c5 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -4198,6 +4198,10 @@ public PubSubTopic getVersionTopic() { return versionTopic; } + public PubSubTopic getRealTimeTopic() { + return realTimeTopic; + } + public boolean isMetricsEmissionEnabled() { return emitMetrics.get(); } @@ -4746,11 +4750,6 @@ protected abstract CompletableFuture sendIngestionHeartbeat LeaderCompleteState leaderCompleteState, long originTimeStampMs); - protected abstract ByteBuffer maybeCompressData( - int partition, - ByteBuffer data, - PartitionConsumptionState partitionConsumptionState); - protected abstract GenericRecord readStoredValueRecord( PartitionConsumptionState partitionConsumptionState, byte[] keyBytes, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 8444cf08180..fea064ddd61 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -18,6 +18,7 @@ import com.linkedin.davinci.validation.PartitionTracker; import com.linkedin.davinci.validation.PartitionTracker.TopicType; import com.linkedin.venice.common.VeniceSystemStoreUtils; +import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceMessageException; import com.linkedin.venice.exceptions.validation.DuplicateDataException; @@ -1271,7 +1272,7 @@ private PubSubMessageProcessedResult processMessage( switch (msgType) { case PUT: Put put = (Put) kafkaValue.payloadUnion; - put.putValue = storeIngestionTask.maybeCompressData( + put.putValue = maybeCompressData( consumerRecord.getTopicPartition().getPartitionNumber(), put.putValue, partitionConsumptionState); @@ -1662,7 +1663,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( } else { validatePostOperationResultsAndRecord(mergeConflictResult, offsetSumPreOperation, recordTimestampsPreOperation); - final ByteBuffer updatedValueBytes = storeIngestionTask.maybeCompressData( + final ByteBuffer updatedValueBytes = maybeCompressData( consumerRecord.getTopicPartition().getPartitionNumber(), mergeConflictResult.getNewValue(), partitionConsumptionState); @@ -2061,6 +2062,49 @@ private void validatePostOperationResultsAndRecord( } } + private ByteBuffer maybeCompressData( + int partition, + ByteBuffer data, + PartitionConsumptionState partitionConsumptionState) { + // To handle delete operations + if (data == null) { + return null; + } + if (shouldCompressData(partitionConsumptionState)) { + try { + long startTimeInNS = System.nanoTime(); + // We need to expand the front of the returned bytebuffer to make room for schema header insertion + ByteBuffer result = storeIngestionTask.getCompressor().get().compress(data, ByteUtils.SIZE_OF_INT); + storeIngestionTask.getHostLevelIngestionStats() + .recordLeaderCompressLatency(LatencyUtils.getElapsedTimeFromNSToMS(startTimeInNS)); + return result; + } catch (IOException e) { + // throw a loud exception if something goes wrong here + throw new RuntimeException( + String.format( + "Failed to compress value in venice writer! Aborting write! partition: %d, leader topic: %s, compressor: %s", + partition, + partitionConsumptionState.getOffsetRecord() + .getLeaderTopic(storeIngestionTask.getPubSubTopicRepository()), + storeIngestionTask.getCompressor().getClass().getName()), + e); + } + } + return data; + } + + private boolean shouldCompressData(PartitionConsumptionState partitionConsumptionState) { + if (!LeaderFollowerStoreIngestionTask.isLeader(partitionConsumptionState)) { + return false; // Not leader, don't compress + } + PubSubTopic leaderTopic = + partitionConsumptionState.getOffsetRecord().getLeaderTopic(storeIngestionTask.getPubSubTopicRepository()); + if (!storeIngestionTask.getRealTimeTopic().equals(leaderTopic)) { + return false; // We're consuming from version topic (don't compress it) + } + return !storeIngestionTask.getCompressionStrategy().equals(CompressionStrategy.NO_OP); + } + /** * Get the value bytes for a key from {@link PartitionConsumptionState.TransientRecord} or from disk. The assumption * is that the {@link PartitionConsumptionState.TransientRecord} only contains the full value. From b7130119980aa6ec03314bc27b0035cb3f8d447d Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 15:02:17 -0700 Subject: [PATCH 21/32] =?UTF-8?q?Refactored=20`readStoredValueRecord()`=20?= =?UTF-8?q?from=20`LeaderFollowerStoreIngestionTask`=20into=20`StorePartit?= =?UTF-8?q?ionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 130 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 12 +- .../consumer/StorePartitionDataReceiver.java | 66 ++++++++- 3 files changed, 134 insertions(+), 74 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 96bdf8b5fc7..9ca5a9203da 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -16,13 +16,10 @@ import com.linkedin.davinci.config.VeniceStoreVersionConfig; import com.linkedin.davinci.helix.LeaderFollowerPartitionStateModel; import com.linkedin.davinci.ingestion.LagType; -import com.linkedin.davinci.listener.response.NoOpReadResponseStats; import com.linkedin.davinci.schema.merge.CollectionTimestampMergeRecordHelper; import com.linkedin.davinci.schema.merge.MergeRecordHelper; import com.linkedin.davinci.stats.ingestion.heartbeat.HeartbeatMonitoringService; import com.linkedin.davinci.storage.StorageService; -import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; -import com.linkedin.davinci.storage.chunking.GenericRecordChunkingAdapter; import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; import com.linkedin.davinci.store.record.ValueRecord; @@ -93,7 +90,6 @@ import java.util.function.LongPredicate; import java.util.function.Predicate; import java.util.function.Supplier; -import org.apache.avro.generic.GenericRecord; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -3542,66 +3538,67 @@ public void updateLeaderTopicOnFollower(PartitionConsumptionState partitionConsu // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); // } // } - - /** - * Read the existing value. If a value for this key is found from the transient map then use that value, otherwise read - * it from the storage engine. - * @return {@link Optional#empty} if the value - */ - protected GenericRecord readStoredValueRecord( - PartitionConsumptionState partitionConsumptionState, - byte[] keyBytes, - int readerValueSchemaID, - PubSubTopicPartition topicPartition, - ChunkedValueManifestContainer manifestContainer) { - final GenericRecord currValue; - PartitionConsumptionState.TransientRecord transientRecord = partitionConsumptionState.getTransientRecord(keyBytes); - if (transientRecord == null) { - try { - long lookupStartTimeInNS = System.nanoTime(); - currValue = GenericRecordChunkingAdapter.INSTANCE.get( - storageEngine, - topicPartition.getPartitionNumber(), - ByteBuffer.wrap(keyBytes), - isChunked, - null, - null, - NoOpReadResponseStats.SINGLETON, - readerValueSchemaID, - storeDeserializerCache, - compressor.get(), - manifestContainer); - hostLevelIngestionStats - .recordWriteComputeLookUpLatency(LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS)); - } catch (Exception e) { - writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_DESERIALIZATION_FAILURE.code; - throw e; - } - } else { - hostLevelIngestionStats.recordWriteComputeCacheHitCount(); - // construct currValue from this transient record only if it's not null. - if (transientRecord.getValue() != null) { - try { - currValue = GenericRecordChunkingAdapter.INSTANCE.constructValue( - transientRecord.getValue(), - transientRecord.getValueOffset(), - transientRecord.getValueLen(), - storeDeserializerCache.getDeserializer(transientRecord.getValueSchemaId(), readerValueSchemaID), - compressor.get()); - } catch (Exception e) { - writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_DESERIALIZATION_FAILURE.code; - throw e; - } - if (manifestContainer != null) { - manifestContainer.setManifest(transientRecord.getValueManifest()); - } - - } else { - currValue = null; - } - } - return currValue; - } + // + // /** + // * Read the existing value. If a value for this key is found from the transient map then use that value, otherwise + // read + // * it from the storage engine. + // * @return {@link Optional#empty} if the value + // */ + // protected GenericRecord readStoredValueRecord( + // PartitionConsumptionState partitionConsumptionState, + // byte[] keyBytes, + // int readerValueSchemaID, + // PubSubTopicPartition topicPartition, + // ChunkedValueManifestContainer manifestContainer) { + // final GenericRecord currValue; + // PartitionConsumptionState.TransientRecord transientRecord = partitionConsumptionState.getTransientRecord(keyBytes); + // if (transientRecord == null) { + // try { + // long lookupStartTimeInNS = System.nanoTime(); + // currValue = GenericRecordChunkingAdapter.INSTANCE.get( + // storageEngine, + // topicPartition.getPartitionNumber(), + // ByteBuffer.wrap(keyBytes), + // isChunked, + // null, + // null, + // NoOpReadResponseStats.SINGLETON, + // readerValueSchemaID, + // storeDeserializerCache, + // compressor.get(), + // manifestContainer); + // hostLevelIngestionStats + // .recordWriteComputeLookUpLatency(LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS)); + // } catch (Exception e) { + // writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_DESERIALIZATION_FAILURE.code; + // throw e; + // } + // } else { + // hostLevelIngestionStats.recordWriteComputeCacheHitCount(); + // // construct currValue from this transient record only if it's not null. + // if (transientRecord.getValue() != null) { + // try { + // currValue = GenericRecordChunkingAdapter.INSTANCE.constructValue( + // transientRecord.getValue(), + // transientRecord.getValueOffset(), + // transientRecord.getValueLen(), + // storeDeserializerCache.getDeserializer(transientRecord.getValueSchemaId(), readerValueSchemaID), + // compressor.get()); + // } catch (Exception e) { + // writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_DESERIALIZATION_FAILURE.code; + // throw e; + // } + // if (manifestContainer != null) { + // manifestContainer.setManifest(transientRecord.getValueManifest()); + // } + // + // } else { + // currValue = null; + // } + // } + // return currValue; + // } /** * Clone DIV check results from OffsetRecord to the DIV validator that is used for leader consumption thread. @@ -4016,6 +4013,11 @@ public StoreWriteComputeProcessor getStoreWriteComputeHandler() { return storeWriteComputeHandler; } + @Override + public AvroStoreDeserializerCache getStoreDeserializerCache() { + return storeDeserializerCache; + } + @Override public Int2ObjectMap getKafkaClusterIdToUrlMap() { return kafkaClusterIdToUrlMap; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index df0355ad4c5..14456b26c71 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -41,7 +41,6 @@ import com.linkedin.davinci.storage.StorageEngineRepository; import com.linkedin.davinci.storage.StorageMetadataService; import com.linkedin.davinci.storage.StorageService; -import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.StoragePartitionConfig; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; @@ -100,6 +99,7 @@ import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.schema.SchemaEntry; +import com.linkedin.venice.serialization.AvroStoreDeserializerCache; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.serialization.avro.ChunkedValueManifestSerializer; import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; @@ -164,7 +164,6 @@ import java.util.function.Supplier; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -4750,13 +4749,6 @@ protected abstract CompletableFuture sendIngestionHeartbeat LeaderCompleteState leaderCompleteState, long originTimeStampMs); - protected abstract GenericRecord readStoredValueRecord( - PartitionConsumptionState partitionConsumptionState, - byte[] keyBytes, - int readerValueSchemaID, - PubSubTopicPartition topicPartition, - ChunkedValueManifestContainer manifestContainer); - public AggVersionedIngestionStats getAggVersionedIngestionStats() { throw new VeniceException("getAggVersionedIngestionStats() should only be called in active active mode"); } @@ -4787,6 +4779,8 @@ public RmdSerDe getRmdSerDe() { public abstract StoreWriteComputeProcessor getStoreWriteComputeHandler(); + public abstract AvroStoreDeserializerCache getStoreDeserializerCache(); + protected abstract Lazy> getVeniceWriter( PartitionConsumptionState partitionConsumptionState); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index fea064ddd61..68ce4de6b88 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -6,11 +6,13 @@ import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.avroutil1.compatibility.shaded.org.apache.commons.lang3.Validate; import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; +import com.linkedin.davinci.listener.response.NoOpReadResponseStats; import com.linkedin.davinci.replication.RmdWithValueSchemaId; import com.linkedin.davinci.replication.merge.MergeConflictResolver; import com.linkedin.davinci.replication.merge.MergeConflictResult; import com.linkedin.davinci.stats.HostLevelIngestionStats; import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; +import com.linkedin.davinci.storage.chunking.GenericRecordChunkingAdapter; import com.linkedin.davinci.storage.chunking.RawBytesChunkingAdapter; import com.linkedin.davinci.store.record.ByteBufferValueRecord; import com.linkedin.davinci.store.view.VeniceViewWriter; @@ -67,6 +69,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.locks.ReentrantLock; @@ -1332,7 +1335,7 @@ private PubSubMessageProcessedResult processMessage( readerUpdateProtocolVersion = update.updateSchemaId; } ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); - final GenericRecord currValue = storeIngestionTask.readStoredValueRecord( + final GenericRecord currValue = readStoredValueRecord( partitionConsumptionState, keyBytes, readerValueSchemaId, @@ -2105,6 +2108,67 @@ private boolean shouldCompressData(PartitionConsumptionState partitionConsumptio return !storeIngestionTask.getCompressionStrategy().equals(CompressionStrategy.NO_OP); } + /** + * Read the existing value. If a value for this key is found from the transient map then use that value, otherwise read + * it from the storage engine. + * @return {@link Optional#empty} if the value + */ + private GenericRecord readStoredValueRecord( + PartitionConsumptionState partitionConsumptionState, + byte[] keyBytes, + int readerValueSchemaID, + PubSubTopicPartition topicPartition, + ChunkedValueManifestContainer manifestContainer) { + final GenericRecord currValue; + PartitionConsumptionState.TransientRecord transientRecord = partitionConsumptionState.getTransientRecord(keyBytes); + if (transientRecord == null) { + try { + long lookupStartTimeInNS = System.nanoTime(); + currValue = GenericRecordChunkingAdapter.INSTANCE.get( + storeIngestionTask.getStorageEngine(), + topicPartition.getPartitionNumber(), + ByteBuffer.wrap(keyBytes), + storeIngestionTask.isChunked(), + null, + null, + NoOpReadResponseStats.SINGLETON, + readerValueSchemaID, + storeIngestionTask.getStoreDeserializerCache(), + storeIngestionTask.getCompressor().get(), + manifestContainer); + storeIngestionTask.getHostLevelIngestionStats() + .recordWriteComputeLookUpLatency(LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS)); + } catch (Exception e) { + storeIngestionTask.setWriteComputeFailureCode(StatsErrorCode.WRITE_COMPUTE_DESERIALIZATION_FAILURE.code); + throw e; + } + } else { + storeIngestionTask.getHostLevelIngestionStats().recordWriteComputeCacheHitCount(); + // construct currValue from this transient record only if it's not null. + if (transientRecord.getValue() != null) { + try { + currValue = GenericRecordChunkingAdapter.INSTANCE.constructValue( + transientRecord.getValue(), + transientRecord.getValueOffset(), + transientRecord.getValueLen(), + storeIngestionTask.getStoreDeserializerCache() + .getDeserializer(transientRecord.getValueSchemaId(), readerValueSchemaID), + storeIngestionTask.getCompressor().get()); + } catch (Exception e) { + storeIngestionTask.setWriteComputeFailureCode(StatsErrorCode.WRITE_COMPUTE_DESERIALIZATION_FAILURE.code); + throw e; + } + if (manifestContainer != null) { + manifestContainer.setManifest(transientRecord.getValueManifest()); + } + + } else { + currValue = null; + } + } + return currValue; + } + /** * Get the value bytes for a key from {@link PartitionConsumptionState.TransientRecord} or from disk. The assumption * is that the {@link PartitionConsumptionState.TransientRecord} only contains the full value. From 6b7493deb73af0b3b80c86a33b4dcd67f4054736 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sun, 20 Oct 2024 19:42:54 -0700 Subject: [PATCH 22/32] =?UTF-8?q?Refactored=20`getAndUpdateLeaderCompleted?= =?UTF-8?q?State()`=20from=20`LeaderFollowerStoreIngestionTask`=20into=20`?= =?UTF-8?q?StorePartitionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 92 +++++++++---------- .../kafka/consumer/StoreIngestionTask.java | 8 -- .../consumer/StorePartitionDataReceiver.java | 46 +++++++++- .../consumer/StoreIngestionTaskTest.java | 9 +- 4 files changed, 95 insertions(+), 60 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 9ca5a9203da..c4197e57a1a 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -8,8 +8,6 @@ import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.PAUSE_TRANSITION_FROM_STANDBY_TO_LEADER; import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.STANDBY; import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_PUSH; -import static com.linkedin.venice.kafka.protocol.enums.MessageType.UPDATE; -import static com.linkedin.venice.pubsub.api.PubSubMessageHeaders.VENICE_LEADER_COMPLETION_STATE_HEADER; import static java.util.concurrent.TimeUnit.MILLISECONDS; import com.linkedin.davinci.client.DaVinciRecordTransformerConfig; @@ -48,8 +46,6 @@ import com.linkedin.venice.pubsub.PubSubConstants; import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.api.PubSubMessage; -import com.linkedin.venice.pubsub.api.PubSubMessageHeader; -import com.linkedin.venice.pubsub.api.PubSubMessageHeaders; import com.linkedin.venice.pubsub.api.PubSubProduceResult; import com.linkedin.venice.pubsub.api.PubSubProducerCallback; import com.linkedin.venice.pubsub.api.PubSubTopic; @@ -69,7 +65,6 @@ import com.linkedin.venice.writer.VeniceWriterOptions; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import java.nio.ByteBuffer; -import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -2136,49 +2131,50 @@ protected boolean checkAndLogIfLagIsAcceptableForHybridStore( return isLagAcceptable; } - /** - * HeartBeat SOS messages carry the leader completion state in the header. This function extracts the leader completion - * state from that header and updates the {@param partitionConsumptionState} accordingly. - */ - @Override - protected void getAndUpdateLeaderCompletedState( - KafkaKey kafkaKey, - KafkaMessageEnvelope kafkaValue, - ControlMessage controlMessage, - PubSubMessageHeaders pubSubMessageHeaders, - PartitionConsumptionState partitionConsumptionState) { - if (isHybridFollower(partitionConsumptionState)) { - ControlMessageType controlMessageType = ControlMessageType.valueOf(controlMessage); - if (controlMessageType == ControlMessageType.START_OF_SEGMENT - && Arrays.equals(kafkaKey.getKey(), KafkaKey.HEART_BEAT.getKey())) { - LeaderCompleteState oldState = partitionConsumptionState.getLeaderCompleteState(); - LeaderCompleteState newState = oldState; - for (PubSubMessageHeader header: pubSubMessageHeaders) { - if (header.key().equals(VENICE_LEADER_COMPLETION_STATE_HEADER)) { - newState = LeaderCompleteState.valueOf(header.value()[0]); - partitionConsumptionState - .setLastLeaderCompleteStateUpdateInMs(kafkaValue.producerMetadata.messageTimestamp); - break; // only interested in this header here - } - } - - if (oldState != newState) { - LOGGER.info( - "LeaderCompleteState for replica: {} changed from {} to {}", - partitionConsumptionState.getReplicaId(), - oldState, - newState); - partitionConsumptionState.setLeaderCompleteState(newState); - } else { - LOGGER.debug( - "LeaderCompleteState for replica: {} received from leader: {} and is unchanged from the previous state", - partitionConsumptionState.getReplicaId(), - newState); - } - } - } - } - + // /** + // * HeartBeat SOS messages carry the leader completion state in the header. This function extracts the leader + // completion + // * state from that header and updates the {@param partitionConsumptionState} accordingly. + // */ + // @Override + // protected void getAndUpdateLeaderCompletedState( + // KafkaKey kafkaKey, + // KafkaMessageEnvelope kafkaValue, + // ControlMessage controlMessage, + // PubSubMessageHeaders pubSubMessageHeaders, + // PartitionConsumptionState partitionConsumptionState) { + // if (isHybridFollower(partitionConsumptionState)) { + // ControlMessageType controlMessageType = ControlMessageType.valueOf(controlMessage); + // if (controlMessageType == ControlMessageType.START_OF_SEGMENT + // && Arrays.equals(kafkaKey.getKey(), KafkaKey.HEART_BEAT.getKey())) { + // LeaderCompleteState oldState = partitionConsumptionState.getLeaderCompleteState(); + // LeaderCompleteState newState = oldState; + // for (PubSubMessageHeader header: pubSubMessageHeaders) { + // if (header.key().equals(VENICE_LEADER_COMPLETION_STATE_HEADER)) { + // newState = LeaderCompleteState.valueOf(header.value()[0]); + // partitionConsumptionState + // .setLastLeaderCompleteStateUpdateInMs(kafkaValue.producerMetadata.messageTimestamp); + // break; // only interested in this header here + // } + // } + // + // if (oldState != newState) { + // LOGGER.info( + // "LeaderCompleteState for replica: {} changed from {} to {}", + // partitionConsumptionState.getReplicaId(), + // oldState, + // newState); + // partitionConsumptionState.setLeaderCompleteState(newState); + // } else { + // LOGGER.debug( + // "LeaderCompleteState for replica: {} received from leader: {} and is unchanged from the previous state", + // partitionConsumptionState.getReplicaId(), + // newState); + // } + // } + // } + // } + // // /** // * Leaders propagate HB SOS message from RT to local VT (to all subpartitions in case if amplification // * Factor is configured to be more than 1) with updated LeaderCompleteState header: diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 14456b26c71..c478b94efc0 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -90,7 +90,6 @@ import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubMessage; -import com.linkedin.venice.pubsub.api.PubSubMessageHeaders; import com.linkedin.venice.pubsub.api.PubSubProduceResult; import com.linkedin.venice.pubsub.api.PubSubProducerCallback; import com.linkedin.venice.pubsub.api.PubSubTopic; @@ -4727,13 +4726,6 @@ public Consumer getDivErrorMetricCallback() { public abstract KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders(); - protected abstract void getAndUpdateLeaderCompletedState( - KafkaKey kafkaKey, - KafkaMessageEnvelope kafkaValue, - ControlMessage controlMessage, - PubSubMessageHeaders pubSubMessageHeaders, - PartitionConsumptionState partitionConsumptionState); - protected abstract void updateLatestInMemoryLeaderConsumedRTOffset( PartitionConsumptionState pcs, String ignoredKafkaUrl, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 68ce4de6b88..f4a5a2da22f 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -36,6 +36,8 @@ import com.linkedin.venice.message.KafkaKey; import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.api.PubSubMessage; +import com.linkedin.venice.pubsub.api.PubSubMessageHeader; +import com.linkedin.venice.pubsub.api.PubSubMessageHeaders; import com.linkedin.venice.pubsub.api.PubSubProduceResult; import com.linkedin.venice.pubsub.api.PubSubProducerCallback; import com.linkedin.venice.pubsub.api.PubSubTopic; @@ -721,7 +723,7 @@ DelegateConsumerRecordResult delegateConsumerRecord( + consumerRecord.getTopicPartition() + " Offset " + consumerRecord.getOffset()); } else if (msgType == MessageType.CONTROL_MESSAGE) { ControlMessage controlMessage = (ControlMessage) kafkaValue.payloadUnion; - storeIngestionTask.getAndUpdateLeaderCompletedState( + getAndUpdateLeaderCompletedState( kafkaKey, kafkaValue, controlMessage, @@ -2169,6 +2171,48 @@ private GenericRecord readStoredValueRecord( return currValue; } + /** + * HeartBeat SOS messages carry the leader completion state in the header. This function extracts the leader completion + * state from that header and updates the {@param partitionConsumptionState} accordingly. + */ + void getAndUpdateLeaderCompletedState( + KafkaKey kafkaKey, + KafkaMessageEnvelope kafkaValue, + ControlMessage controlMessage, + PubSubMessageHeaders pubSubMessageHeaders, + PartitionConsumptionState partitionConsumptionState) { + if (storeIngestionTask.isHybridFollower(partitionConsumptionState)) { + ControlMessageType controlMessageType = ControlMessageType.valueOf(controlMessage); + if (controlMessageType == ControlMessageType.START_OF_SEGMENT + && Arrays.equals(kafkaKey.getKey(), KafkaKey.HEART_BEAT.getKey())) { + LeaderCompleteState oldState = partitionConsumptionState.getLeaderCompleteState(); + LeaderCompleteState newState = oldState; + for (PubSubMessageHeader header: pubSubMessageHeaders) { + if (header.key().equals(PubSubMessageHeaders.VENICE_LEADER_COMPLETION_STATE_HEADER)) { + newState = LeaderCompleteState.valueOf(header.value()[0]); + partitionConsumptionState + .setLastLeaderCompleteStateUpdateInMs(kafkaValue.producerMetadata.messageTimestamp); + break; // only interested in this header here + } + } + + if (oldState != newState) { + LOGGER.info( + "LeaderCompleteState for replica: {} changed from {} to {}", + partitionConsumptionState.getReplicaId(), + oldState, + newState); + partitionConsumptionState.setLeaderCompleteState(newState); + } else { + LOGGER.debug( + "LeaderCompleteState for replica: {} received from leader: {} and is unchanged from the previous state", + partitionConsumptionState.getReplicaId(), + newState); + } + } + } + } + /** * Get the value bytes for a key from {@link PartitionConsumptionState.TransientRecord} or from disk. The assumption * is that the {@link PartitionConsumptionState.TransientRecord} only contains the full value. diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java index 25c6ef09656..797e230a5cf 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java @@ -3707,16 +3707,19 @@ public void testGetAndUpdateLeaderCompletedState(HybridConfig hybridConfig, Node KafkaMessageEnvelope kafkaMessageEnvelope = getHeartbeatKME(producerTimestamp, mockLeaderMetadataWrapper, generateHeartbeatMessage(CheckSumType.NONE), "0"); + PubSubTopicPartitionImpl topicPartition = new PubSubTopicPartitionImpl(pubSubTopic, PARTITION_FOO); PubSubMessageHeaders pubSubMessageHeaders = new PubSubMessageHeaders(); pubSubMessageHeaders.add(VeniceWriter.getLeaderCompleteStateHeader(LEADER_COMPLETED)); PubSubMessage pubSubMessage = new ImmutablePubSubMessage( KafkaKey.HEART_BEAT, kafkaMessageEnvelope, - new PubSubTopicPartitionImpl(pubSubTopic, PARTITION_FOO), + topicPartition, 0, 0, 0, pubSubMessageHeaders); + StorePartitionDataReceiver storePartitionDataReceiver = + new StorePartitionDataReceiver(ingestionTask, topicPartition, "dummyUrl", 0); assertEquals(partitionConsumptionState.getLeaderCompleteState(), LEADER_NOT_COMPLETED); assertEquals(partitionConsumptionState.getLastLeaderCompleteStateUpdateInMs(), 0L); @@ -3727,7 +3730,7 @@ public void testGetAndUpdateLeaderCompletedState(HybridConfig hybridConfig, Node if (nodeType != DA_VINCI) { partitionConsumptionState.setLeaderFollowerState(LeaderFollowerStateType.LEADER); - ingestionTask.getAndUpdateLeaderCompletedState( + storePartitionDataReceiver.getAndUpdateLeaderCompletedState( kafkaKey, kafkaValue, controlMessage, @@ -3738,7 +3741,7 @@ public void testGetAndUpdateLeaderCompletedState(HybridConfig hybridConfig, Node } partitionConsumptionState.setLeaderFollowerState(STANDBY); - ingestionTask.getAndUpdateLeaderCompletedState( + storePartitionDataReceiver.getAndUpdateLeaderCompletedState( kafkaKey, kafkaValue, controlMessage, From 5293304da2ae9c931eace4faf4cffcd3fd14078b Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 15:33:07 -0700 Subject: [PATCH 23/32] =?UTF-8?q?Refactored=20`shouldProduceInBatch()`=20f?= =?UTF-8?q?rom=20`StoreIngestionTask`=20into=20`StorePartitionDataReceiver?= =?UTF-8?q?`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../davinci/kafka/consumer/StoreIngestionTask.java | 14 +++++++------- .../kafka/consumer/StorePartitionDataReceiver.java | 8 +++++++- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index c478b94efc0..ca4438c29ad 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -1301,13 +1301,13 @@ protected void produceToStoreBufferService( // hostLevelIngestionStats.recordStorageQuotaUsed(storageUtilizationManager.getDiskQuotaUsage()); // } // } - - public boolean shouldProduceInBatch(Iterable> records) { - return (isActiveActiveReplicationEnabled || isWriteComputationEnabled) - && serverConfig.isAAWCWorkloadParallelProcessingEnabled() - && IngestionBatchProcessor.isAllMessagesFromRTTopic(records); - } - + // + // public boolean shouldProduceInBatch(Iterable> records) { + // return (isActiveActiveReplicationEnabled || isWriteComputationEnabled) + // && serverConfig.isAAWCWorkloadParallelProcessingEnabled() + // && IngestionBatchProcessor.isAllMessagesFromRTTopic(records); + // } + // // public void produceToStoreBufferServiceOrKafkaInBatch( // Iterable> records, // PubSubTopicPartition topicPartition, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index f4a5a2da22f..63eb5029070 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -211,7 +211,7 @@ protected void produceToStoreBufferServiceOrKafka( */ records = validateAndFilterOutDuplicateMessagesFromLeaderTopic(records, kafkaUrl, topicPartition); - if (storeIngestionTask.shouldProduceInBatch(records)) { + if (shouldProduceInBatch(records)) { produceToStoreBufferServiceOrKafkaInBatch( records, topicPartition, @@ -344,6 +344,12 @@ public void produceToStoreBufferServiceOrKafkaInBatch( beforeProcessingBatchRecordsTimestampMs); } + private boolean shouldProduceInBatch(Iterable> records) { + return (storeIngestionTask.isActiveActiveReplicationEnabled() || storeIngestionTask.isTransientRecordBufferUsed()) + && storeIngestionTask.getServerConfig().isAAWCWorkloadParallelProcessingEnabled() + && IngestionBatchProcessor.isAllMessagesFromRTTopic(records); + } + private void updateMetricsAndEnforceQuota( long totalBytesRead, int partition, From cee37059f9af5c614fb8f6088f0c59ae7c8714bb Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 15:46:23 -0700 Subject: [PATCH 24/32] =?UTF-8?q?Refactored=20`shouldProcessRecord()`=20fr?= =?UTF-8?q?om=20`StoreIngestionTask`=20/=20`LeaderFollowerStoreIngestionTa?= =?UTF-8?q?sk`=20into=20`StorePartitionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 177 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 110 +++++------ .../consumer/StorePartitionDataReceiver.java | 150 ++++++++++++++- 3 files changed, 288 insertions(+), 149 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index c4197e57a1a..edb4069baa2 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -7,7 +7,6 @@ import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.LEADER; import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.PAUSE_TRANSITION_FROM_STANDBY_TO_LEADER; import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.STANDBY; -import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_PUSH; import static java.util.concurrent.TimeUnit.MILLISECONDS; import com.linkedin.davinci.client.DaVinciRecordTransformerConfig; @@ -25,7 +24,6 @@ import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.validation.KafkaDataIntegrityValidator; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.exceptions.VeniceMessageException; import com.linkedin.venice.exceptions.VeniceTimeoutException; import com.linkedin.venice.guid.GuidUtils; import com.linkedin.venice.kafka.protocol.ControlMessage; @@ -686,7 +684,7 @@ protected void checkLongRunningTaskState() throws InterruptedException { currentLeaderTopic = versionTopic; } /** - * The flag is turned on in {@link LeaderFollowerStoreIngestionTask#shouldProcessRecord} avoid consuming + * The flag is turned on in {@link StorePartitionDataReceiver#shouldProcessRecord} avoid consuming * unwanted messages after EOP in remote VT, such as SOBR. Now that the leader switches to consume locally, * it should not skip any message. */ @@ -1841,93 +1839,92 @@ protected void reportIfCatchUpVersionTopicOffset(PartitionConsumptionState pcs) } } - /** - * For Leader/Follower model, the follower should have the same kind of check as the Online/Offline model; - * for leader, it's possible that it consumers from real-time topic or GF topic. - */ - @Override - protected boolean shouldProcessRecord(PubSubMessage record) { - PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(record.getPartition()); - if (partitionConsumptionState == null) { - LOGGER.info( - "Skipping message as partition is no longer actively subscribed. Replica: {}", - Utils.getReplicaId(versionTopic, record.getPartition())); - return false; - } - switch (partitionConsumptionState.getLeaderFollowerState()) { - case LEADER: - PubSubTopic currentLeaderTopic = - partitionConsumptionState.getOffsetRecord().getLeaderTopic(pubSubTopicRepository); - if (partitionConsumptionState.consumeRemotely() - && currentLeaderTopic.isVersionTopicOrStreamReprocessingTopic()) { - if (partitionConsumptionState.skipKafkaMessage()) { - String msg = "Skipping messages after EOP in remote version topic. Replica: " - + partitionConsumptionState.getReplicaId(); - if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { - LOGGER.info(msg); - } - return false; - } - if (record.getKey().isControlMessage()) { - ControlMessageType controlMessageType = - ControlMessageType.valueOf((ControlMessage) record.getValue().payloadUnion); - if (controlMessageType == END_OF_PUSH) { - /** - * The flag is turned on to avoid consuming unwanted messages after EOP in remote VT, such as SOBR. In - * {@link LeaderFollowerStoreIngestionTask#checkLongRunningTaskState()}, once leader notices that EOP is - * received, it will unsubscribe from the remote VT and turn off this flag. However, if data recovery is - * in progress and the store is hybrid then we actually want to consume messages after EOP. In that case - * remote TS will be skipped but with a different method. - */ - if (!(isDataRecovery && isHybridMode())) { - partitionConsumptionState.setSkipKafkaMessage(true); - } - } - } - } - - if (!Utils.resolveLeaderTopicFromPubSubTopic(pubSubTopicRepository, record.getTopicPartition().getPubSubTopic()) - .equals(currentLeaderTopic)) { - String errorMsg = - "Leader replica: {} received a pubsub message that doesn't belong to the leader topic. Leader topic: " - + currentLeaderTopic + ", topic of incoming message: " - + record.getTopicPartition().getPubSubTopic().getName(); - if (!REDUNDANT_LOGGING_FILTER.isRedundantException(errorMsg)) { - LOGGER.error(errorMsg, partitionConsumptionState.getReplicaId()); - } - return false; - } - break; - default: - PubSubTopic pubSubTopic = record.getTopicPartition().getPubSubTopic(); - String topicName = pubSubTopic.getName(); - if (!versionTopic.equals(pubSubTopic)) { - String errorMsg = partitionConsumptionState.getLeaderFollowerState() + " replica: " - + partitionConsumptionState.getReplicaId() + " received message from non version topic: " + topicName; - if (consumerHasSubscription(pubSubTopic, partitionConsumptionState)) { - throw new VeniceMessageException( - errorMsg + ". Throwing exception as the node still subscribes to " + topicName); - } - if (!REDUNDANT_LOGGING_FILTER.isRedundantException(errorMsg)) { - LOGGER.error("{}. Skipping the message as the node does not subscribe to {}", errorMsg, topicName); - } - return false; - } - - long lastOffset = partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset(); - if (lastOffset >= record.getOffset()) { - String message = partitionConsumptionState.getLeaderFollowerState() + " replica: " - + partitionConsumptionState.getReplicaId() + " had already processed the record"; - if (!REDUNDANT_LOGGING_FILTER.isRedundantException(message)) { - LOGGER.info("{}; LastKnownOffset: {}; OffsetOfIncomingRecord: {}", message, lastOffset, record.getOffset()); - } - return false; - } - break; - } - - return super.shouldProcessRecord(record); - } + // /** + // * For Leader/Follower model, the follower should have the same kind of check as the Online/Offline model; + // * for leader, it's possible that it consumers from real-time topic or GF topic. + // */ + // @Override + // protected boolean shouldProcessRecord(PubSubMessage record) { + // PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(record.getPartition()); + // if (partitionConsumptionState == null) { + // LOGGER.info( + // "Skipping message as partition is no longer actively subscribed. Replica: {}", + // Utils.getReplicaId(versionTopic, record.getPartition())); + // return false; + // } + // switch (partitionConsumptionState.getLeaderFollowerState()) { + // case LEADER: + // PubSubTopic currentLeaderTopic = + // partitionConsumptionState.getOffsetRecord().getLeaderTopic(pubSubTopicRepository); + // if (partitionConsumptionState.consumeRemotely() + // && currentLeaderTopic.isVersionTopicOrStreamReprocessingTopic()) { + // if (partitionConsumptionState.skipKafkaMessage()) { + // String msg = "Skipping messages after EOP in remote version topic. Replica: " + // + partitionConsumptionState.getReplicaId(); + // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { + // LOGGER.info(msg); + // } + // return false; + // } + // if (record.getKey().isControlMessage()) { + // ControlMessageType controlMessageType = + // ControlMessageType.valueOf((ControlMessage) record.getValue().payloadUnion); + // if (controlMessageType == END_OF_PUSH) { + // /** + // * The flag is turned on to avoid consuming unwanted messages after EOP in remote VT, such as SOBR. In + // * {@link LeaderFollowerStoreIngestionTask#checkLongRunningTaskState()}, once leader notices that EOP is + // * received, it will unsubscribe from the remote VT and turn off this flag. However, if data recovery is + // * in progress and the store is hybrid then we actually want to consume messages after EOP. In that case + // * remote TS will be skipped but with a different method. + // */ + // if (!(isDataRecovery && isHybridMode())) { + // partitionConsumptionState.setSkipKafkaMessage(true); + // } + // } + // } + // } + // if (!Utils.resolveLeaderTopicFromPubSubTopic(pubSubTopicRepository, record.getTopicPartition().getPubSubTopic()) + // .equals(currentLeaderTopic)) { + // String errorMsg = + // "Leader replica: {} received a pubsub message that doesn't belong to the leader topic. Leader topic: " + // + currentLeaderTopic + ", topic of incoming message: " + // + record.getTopicPartition().getPubSubTopic().getName(); + // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(errorMsg)) { + // LOGGER.error(errorMsg, partitionConsumptionState.getReplicaId()); + // } + // return false; + // } + // break; + // default: + // PubSubTopic pubSubTopic = record.getTopicPartition().getPubSubTopic(); + // String topicName = pubSubTopic.getName(); + // if (!versionTopic.equals(pubSubTopic)) { + // String errorMsg = partitionConsumptionState.getLeaderFollowerState() + " replica: " + // + partitionConsumptionState.getReplicaId() + " received message from non version topic: " + topicName; + // if (consumerHasSubscription(pubSubTopic, partitionConsumptionState)) { + // throw new VeniceMessageException( + // errorMsg + ". Throwing exception as the node still subscribes to " + topicName); + // } + // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(errorMsg)) { + // LOGGER.error("{}. Skipping the message as the node does not subscribe to {}", errorMsg, topicName); + // } + // return false; + // } + // + // long lastOffset = partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset(); + // if (lastOffset >= record.getOffset()) { + // String message = partitionConsumptionState.getLeaderFollowerState() + " replica: " + // + partitionConsumptionState.getReplicaId() + " had already processed the record"; + // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(message)) { + // LOGGER.info("{}; LastKnownOffset: {}; OffsetOfIncomingRecord: {}", message, lastOffset, record.getOffset()); + // } + // return false; + // } + // break; + // } + // + // return super.shouldProcessRecord(record); + // } /** * Additional safeguards in Leader/Follower ingestion: diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index ca4438c29ad..e204c0d1c2d 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -2445,61 +2445,61 @@ public int getFailedIngestionPartitionCount() { return failedPartitions.size(); } - /** - * Common record check for different state models: - * check whether server continues receiving messages after EOP for a batch-only store. - */ - protected boolean shouldProcessRecord(PubSubMessage record) { - PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(record.getPartition()); - - if (partitionConsumptionState == null) { - String msg = "PCS for replica: " + Utils.getReplicaId(kafkaVersionTopic, record.getPartition()) - + " is null. Skipping incoming record with topic-partition: {} and offset: {}"; - if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { - LOGGER.info(msg, record.getTopicPartition(), record.getOffset()); - } - return false; - } - - if (partitionConsumptionState.isErrorReported()) { - String msg = "Replica: " + partitionConsumptionState.getReplicaId() - + " is already errored. Skipping incoming record with topic-partition: {} and offset: {}"; - if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { - LOGGER.info(msg, record.getTopicPartition(), record.getOffset()); - } - return false; - } - - if (partitionConsumptionState.isEndOfPushReceived() && partitionConsumptionState.isBatchOnly()) { - KafkaKey key = record.getKey(); - KafkaMessageEnvelope value = record.getValue(); - if (key.isControlMessage() - && ControlMessageType.valueOf((ControlMessage) value.payloadUnion) == ControlMessageType.END_OF_SEGMENT) { - // Still allow END_OF_SEGMENT control message - return true; - } - // emit metric for unexpected messages - if (emitMetrics.get()) { - hostLevelIngestionStats.recordUnexpectedMessage(); - } - - // Report such kind of message once per minute to reduce logging volume - /* - * TODO: right now, if we update a store to enable hybrid, {@link StoreIngestionTask} for the existing versions - * won't know it since {@link #hybridStoreConfig} parameter is passed during construction. - * - * So far, to make hybrid store/incremental store work, customer needs to do a new push after enabling hybrid/ - * incremental push feature of the store. - */ - String message = "The record was received after 'EOP', but the store: " + kafkaVersionTopic - + " is neither hybrid nor incremental push enabled, so will skip it. Current records replica: {}"; - if (!REDUNDANT_LOGGING_FILTER.isRedundantException(message)) { - LOGGER.warn(message, partitionConsumptionState.getReplicaId()); - } - return false; - } - return true; - } + // /** + // * Common record check for different state models: + // * check whether server continues receiving messages after EOP for a batch-only store. + // */ + // protected boolean shouldProcessRecord(PubSubMessage record) { + // PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(record.getPartition()); + // + // if (partitionConsumptionState == null) { + // String msg = "PCS for replica: " + Utils.getReplicaId(kafkaVersionTopic, record.getPartition()) + // + " is null. Skipping incoming record with topic-partition: {} and offset: {}"; + // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { + // LOGGER.info(msg, record.getTopicPartition(), record.getOffset()); + // } + // return false; + // } + // + // if (partitionConsumptionState.isErrorReported()) { + // String msg = "Replica: " + partitionConsumptionState.getReplicaId() + // + " is already errored. Skipping incoming record with topic-partition: {} and offset: {}"; + // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { + // LOGGER.info(msg, record.getTopicPartition(), record.getOffset()); + // } + // return false; + // } + // + // if (partitionConsumptionState.isEndOfPushReceived() && partitionConsumptionState.isBatchOnly()) { + // KafkaKey key = record.getKey(); + // KafkaMessageEnvelope value = record.getValue(); + // if (key.isControlMessage() + // && ControlMessageType.valueOf((ControlMessage) value.payloadUnion) == ControlMessageType.END_OF_SEGMENT) { + // // Still allow END_OF_SEGMENT control message + // return true; + // } + // // emit metric for unexpected messages + // if (emitMetrics.get()) { + // hostLevelIngestionStats.recordUnexpectedMessage(); + // } + // + // // Report such kind of message once per minute to reduce logging volume + // /* + // * TODO: right now, if we update a store to enable hybrid, {@link StoreIngestionTask} for the existing versions + // * won't know it since {@link #hybridStoreConfig} parameter is passed during construction. + // * + // * So far, to make hybrid store/incremental store work, customer needs to do a new push after enabling hybrid/ + // * incremental push feature of the store. + // */ + // String message = "The record was received after 'EOP', but the store: " + kafkaVersionTopic + // + " is neither hybrid nor incremental push enabled, so will skip it. Current records replica: {}"; + // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(message)) { + // LOGGER.warn(message, partitionConsumptionState.getReplicaId()); + // } + // return false; + // } + // return true; + // } protected boolean shouldPersistRecord( PubSubMessage record, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 63eb5029070..08b3bc8e071 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -229,7 +229,7 @@ protected void produceToStoreBufferServiceOrKafka( for (PubSubMessage record: records) { long beforeProcessingPerRecordTimestampNs = System.nanoTime(); partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); - if (!storeIngestionTask.shouldProcessRecord(record)) { + if (!shouldProcessRecord(record)) { partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); continue; } @@ -278,7 +278,7 @@ public void produceToStoreBufferServiceOrKafkaInBatch( if (partitionConsumptionState != null) { partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); } - if (!storeIngestionTask.shouldProcessRecord(record)) { + if (!shouldProcessRecord(record)) { if (partitionConsumptionState != null) { partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); } @@ -350,6 +350,148 @@ private boolean shouldProduceInBatch(Iterable record) { + PartitionConsumptionState partitionConsumptionState = + storeIngestionTask.getPartitionConsumptionState(record.getPartition()); + if (partitionConsumptionState == null) { + LOGGER.info( + "Skipping message as partition is no longer actively subscribed. Replica: {}", + Utils.getReplicaId(storeIngestionTask.getVersionTopic(), record.getPartition())); + return false; + } + switch (partitionConsumptionState.getLeaderFollowerState()) { + case LEADER: + PubSubTopic currentLeaderTopic = + partitionConsumptionState.getOffsetRecord().getLeaderTopic(storeIngestionTask.getPubSubTopicRepository()); + if (partitionConsumptionState.consumeRemotely() + && currentLeaderTopic.isVersionTopicOrStreamReprocessingTopic()) { + if (partitionConsumptionState.skipKafkaMessage()) { + String msg = "Skipping messages after EOP in remote version topic. Replica: " + + partitionConsumptionState.getReplicaId(); + if (!StoreIngestionTask.REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { + LOGGER.info(msg); + } + return false; + } + if (record.getKey().isControlMessage()) { + ControlMessageType controlMessageType = + ControlMessageType.valueOf((ControlMessage) record.getValue().payloadUnion); + if (controlMessageType == ControlMessageType.END_OF_PUSH) { + /** + * The flag is turned on to avoid consuming unwanted messages after EOP in remote VT, such as SOBR. In + * {@link LeaderFollowerStoreIngestionTask#checkLongRunningTaskState()}, once leader notices that EOP is + * received, it will unsubscribe from the remote VT and turn off this flag. However, if data recovery is + * in progress and the store is hybrid then we actually want to consume messages after EOP. In that case + * remote TS will be skipped but with a different method. + */ + if (!(storeIngestionTask.isDataRecovery() && storeIngestionTask.isHybridMode())) { + partitionConsumptionState.setSkipKafkaMessage(true); + } + } + } + } + if (!Utils + .resolveLeaderTopicFromPubSubTopic( + storeIngestionTask.getPubSubTopicRepository(), + record.getTopicPartition().getPubSubTopic()) + .equals(currentLeaderTopic)) { + String errorMsg = + "Leader replica: {} received a pubsub message that doesn't belong to the leader topic. Leader topic: " + + currentLeaderTopic + ", topic of incoming message: " + + record.getTopicPartition().getPubSubTopic().getName(); + if (!StoreIngestionTask.REDUNDANT_LOGGING_FILTER.isRedundantException(errorMsg)) { + LOGGER.error(errorMsg, partitionConsumptionState.getReplicaId()); + } + return false; + } + break; + default: + PubSubTopic pubSubTopic = record.getTopicPartition().getPubSubTopic(); + String topicName = pubSubTopic.getName(); + if (!storeIngestionTask.getVersionTopic().equals(pubSubTopic)) { + String errorMsg = partitionConsumptionState.getLeaderFollowerState() + " replica: " + + partitionConsumptionState.getReplicaId() + " received message from non version topic: " + topicName; + if (storeIngestionTask.consumerHasSubscription(pubSubTopic, partitionConsumptionState)) { + throw new VeniceMessageException( + errorMsg + ". Throwing exception as the node still subscribes to " + topicName); + } + if (!StoreIngestionTask.REDUNDANT_LOGGING_FILTER.isRedundantException(errorMsg)) { + LOGGER.error("{}. Skipping the message as the node does not subscribe to {}", errorMsg, topicName); + } + return false; + } + + long lastOffset = partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset(); + if (lastOffset >= record.getOffset()) { + String message = partitionConsumptionState.getLeaderFollowerState() + " replica: " + + partitionConsumptionState.getReplicaId() + " had already processed the record"; + if (!StoreIngestionTask.REDUNDANT_LOGGING_FILTER.isRedundantException(message)) { + LOGGER.info("{}; LastKnownOffset: {}; OffsetOfIncomingRecord: {}", message, lastOffset, record.getOffset()); + } + return false; + } + break; + } + + /** + * Common record check for different state models: + * check whether server continues receiving messages after EOP for a batch-only store. + */ + // PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(record.getPartition()); + // if (partitionConsumptionState == null) { + // String msg = "PCS for replica: " + Utils.getReplicaId(kafkaVersionTopic, record.getPartition()) + // + " is null. Skipping incoming record with topic-partition: {} and offset: {}"; + // if (!StoreIngestionTask.REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { + // LOGGER.info(msg, record.getTopicPartition(), record.getOffset()); + // } + // return false; + // } + + if (partitionConsumptionState.isErrorReported()) { + String msg = "Replica: " + partitionConsumptionState.getReplicaId() + + " is already errored. Skipping incoming record with topic-partition: {} and offset: {}"; + if (!StoreIngestionTask.REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { + LOGGER.info(msg, record.getTopicPartition(), record.getOffset()); + } + return false; + } + + if (partitionConsumptionState.isEndOfPushReceived() && partitionConsumptionState.isBatchOnly()) { + KafkaKey key = record.getKey(); + KafkaMessageEnvelope value = record.getValue(); + if (key.isControlMessage() + && ControlMessageType.valueOf((ControlMessage) value.payloadUnion) == ControlMessageType.END_OF_SEGMENT) { + // Still allow END_OF_SEGMENT control message + return true; + } + // emit metric for unexpected messages + if (storeIngestionTask.isMetricsEmissionEnabled()) { + storeIngestionTask.getHostLevelIngestionStats().recordUnexpectedMessage(); + } + + // Report such kind of message once per minute to reduce logging volume + /* + * TODO: right now, if we update a store to enable hybrid, {@link StoreIngestionTask} for the existing versions + * won't know it since {@link #hybridStoreConfig} parameter is passed during construction. + * + * So far, to make hybrid store/incremental store work, customer needs to do a new push after enabling hybrid/ + * incremental push feature of the store. + */ + String message = + "The record was received after 'EOP', but the store: " + storeIngestionTask.getKafkaVersionTopic() + + " is neither hybrid nor incremental push enabled, so will skip it. Current records replica: {}"; + if (!StoreIngestionTask.REDUNDANT_LOGGING_FILTER.isRedundantException(message)) { + LOGGER.warn(message, partitionConsumptionState.getReplicaId()); + } + return false; + } + return true; + } + private void updateMetricsAndEnforceQuota( long totalBytesRead, int partition, @@ -682,8 +824,8 @@ private DelegateConsumerRecordResult delegateConsumerRecordMaybeWithLock( * The caller of this function should only process this {@param consumerRecord} further if the return is * {@link DelegateConsumerRecordResult#QUEUED_TO_DRAINER}. * - * This function assumes {@link LeaderFollowerStoreIngestionTask#shouldProcessRecord(PubSubMessage)} has been called which happens in - * {@link StorePartitionDataReceiver#produceToStoreBufferServiceOrKafka(Iterable, PubSubTopicPartition, String, int)} + * This function assumes {@link #shouldProcessRecord(PubSubMessage)} has been called which happens in + * {@link #produceToStoreBufferServiceOrKafka(Iterable, PubSubTopicPartition, String, int)} * before calling this and the it was decided that this record needs to be processed. It does not perform any * validation check on the PartitionConsumptionState object to keep the goal of the function simple and not overload. * From a0b92c9f6e8a683e6dfef65975be2de4e3f3cfe5 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 23:19:00 -0700 Subject: [PATCH 25/32] =?UTF-8?q?Refactored=20`getReplicationMetadataAndSc?= =?UTF-8?q?hemaId()`=20and=20`getRmdWithValueSchemaByteBufferFromStorage()?= =?UTF-8?q?`=20from=20`ActiveActiveStoreIngestionTask`=20into=20`StorePart?= =?UTF-8?q?itionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 123 +++++++++--------- .../kafka/consumer/StoreIngestionTask.java | 9 -- .../consumer/StorePartitionDataReceiver.java | 68 +++++++++- .../ActiveActiveStoreIngestionTaskTest.java | 14 +- 4 files changed, 136 insertions(+), 78 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 0afb045bdea..19047279da4 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -8,17 +8,13 @@ import com.linkedin.davinci.client.DaVinciRecordTransformerFunctionalInterface; import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.config.VeniceStoreVersionConfig; -import com.linkedin.davinci.replication.RmdWithValueSchemaId; import com.linkedin.davinci.replication.merge.MergeConflictResolver; import com.linkedin.davinci.replication.merge.MergeConflictResolverFactory; import com.linkedin.davinci.replication.merge.RmdSerDe; import com.linkedin.davinci.replication.merge.StringAnnotatedStoreSchemaCache; import com.linkedin.davinci.stats.AggVersionedIngestionStats; import com.linkedin.davinci.storage.StorageService; -import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; -import com.linkedin.davinci.storage.chunking.SingleGetChunkingAdapter; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; -import com.linkedin.davinci.store.record.ValueRecord; import com.linkedin.venice.exceptions.PersistenceFailureException; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.kafka.protocol.ControlMessage; @@ -38,7 +34,6 @@ import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.utils.ByteUtils; -import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.lazy.Lazy; @@ -323,70 +318,72 @@ private byte[] prependReplicationMetadataBytesWithValueSchemaId(ByteBuffer metad return replicationMetadataBytesWithValueSchemaId; } - /** - * Get the existing value schema ID and RMD associated with the given key. If information for this key is found from - * the transient map then use that, otherwise get it from storage engine. - * - * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition - * @param key Bytes of key. - * @param partition The partition to fetch the replication metadata from storage engine - * @return The object containing RMD and value schema id. If nothing is found, return null - */ - @Override - RmdWithValueSchemaId getReplicationMetadataAndSchemaId( - PartitionConsumptionState partitionConsumptionState, - byte[] key, - int partition, - long currentTimeForMetricsMs) { - PartitionConsumptionState.TransientRecord cachedRecord = partitionConsumptionState.getTransientRecord(key); - if (cachedRecord != null) { - getHostLevelIngestionStats().recordIngestionReplicationMetadataCacheHitCount(currentTimeForMetricsMs); - return new RmdWithValueSchemaId( - cachedRecord.getValueSchemaId(), - getRmdProtocolVersionId(), - cachedRecord.getReplicationMetadataRecord(), - cachedRecord.getRmdManifest()); - } - ChunkedValueManifestContainer rmdManifestContainer = new ChunkedValueManifestContainer(); - byte[] replicationMetadataWithValueSchemaBytes = - getRmdWithValueSchemaByteBufferFromStorage(partition, key, rmdManifestContainer, currentTimeForMetricsMs); - if (replicationMetadataWithValueSchemaBytes == null) { - return null; // No RMD for this key - } - RmdWithValueSchemaId rmdWithValueSchemaId = new RmdWithValueSchemaId(); - // Get old RMD manifest value from RMD Manifest container object. - rmdWithValueSchemaId.setRmdManifest(rmdManifestContainer.getManifest()); - getRmdSerDe() - .deserializeValueSchemaIdPrependedRmdBytes(replicationMetadataWithValueSchemaBytes, rmdWithValueSchemaId); - return rmdWithValueSchemaId; - } + // /** + // * Get the existing value schema ID and RMD associated with the given key. If information for this key is found from + // * the transient map then use that, otherwise get it from storage engine. + // * + // * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition + // * @param key Bytes of key. + // * @param partition The partition to fetch the replication metadata from storage engine + // * @return The object containing RMD and value schema id. If nothing is found, return null + // */ + // @Override + // RmdWithValueSchemaId getReplicationMetadataAndSchemaId( + // PartitionConsumptionState partitionConsumptionState, + // byte[] key, + // int partition, + // long currentTimeForMetricsMs) { + // PartitionConsumptionState.TransientRecord cachedRecord = partitionConsumptionState.getTransientRecord(key); + // if (cachedRecord != null) { + // getHostLevelIngestionStats().recordIngestionReplicationMetadataCacheHitCount(currentTimeForMetricsMs); + // return new RmdWithValueSchemaId( + // cachedRecord.getValueSchemaId(), + // getRmdProtocolVersionId(), + // cachedRecord.getReplicationMetadataRecord(), + // cachedRecord.getRmdManifest()); + // } + // ChunkedValueManifestContainer rmdManifestContainer = new ChunkedValueManifestContainer(); + // byte[] replicationMetadataWithValueSchemaBytes = + // getRmdWithValueSchemaByteBufferFromStorage(partition, key, rmdManifestContainer, currentTimeForMetricsMs); + // if (replicationMetadataWithValueSchemaBytes == null) { + // return null; // No RMD for this key + // } + // RmdWithValueSchemaId rmdWithValueSchemaId = new RmdWithValueSchemaId(); + // // Get old RMD manifest value from RMD Manifest container object. + // rmdWithValueSchemaId.setRmdManifest(rmdManifestContainer.getManifest()); + // getRmdSerDe() + // .deserializeValueSchemaIdPrependedRmdBytes(replicationMetadataWithValueSchemaBytes, rmdWithValueSchemaId); + // return rmdWithValueSchemaId; + // } @Override public RmdSerDe getRmdSerDe() { return rmdSerDe; } - /** - * This method tries to retrieve the RMD bytes with prepended value schema ID from storage engine. It will also store - * RMD manifest into passed-in {@link ChunkedValueManifestContainer} container object if current RMD value is chunked. - */ - byte[] getRmdWithValueSchemaByteBufferFromStorage( - int partition, - byte[] key, - ChunkedValueManifestContainer rmdManifestContainer, - long currentTimeForMetricsMs) { - final long lookupStartTimeInNS = System.nanoTime(); - ValueRecord result = SingleGetChunkingAdapter - .getReplicationMetadata(getStorageEngine(), partition, key, isChunked(), rmdManifestContainer); - getHostLevelIngestionStats().recordIngestionReplicationMetadataLookUpLatency( - LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS), - currentTimeForMetricsMs); - if (result == null) { - return null; - } - return result.serialize(); - } - + // /** + // * This method tries to retrieve the RMD bytes with prepended value schema ID from storage engine. It will also + // store + // * RMD manifest into passed-in {@link ChunkedValueManifestContainer} container object if current RMD value is + // chunked. + // */ + // byte[] getRmdWithValueSchemaByteBufferFromStorage( + // int partition, + // byte[] key, + // ChunkedValueManifestContainer rmdManifestContainer, + // long currentTimeForMetricsMs) { + // final long lookupStartTimeInNS = System.nanoTime(); + // ValueRecord result = SingleGetChunkingAdapter + // .getReplicationMetadata(getStorageEngine(), partition, key, isChunked(), rmdManifestContainer); + // getHostLevelIngestionStats().recordIngestionReplicationMetadataLookUpLatency( + // LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS), + // currentTimeForMetricsMs); + // if (result == null) { + // return null; + // } + // return result.serialize(); + // } + // // @Override // protected IngestionBatchProcessor getIngestionBatchProcessor() { // return ingestionBatchProcessorLazy.get(); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index e204c0d1c2d..2257493077c 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -32,7 +32,6 @@ import com.linkedin.davinci.ingestion.LagType; import com.linkedin.davinci.listener.response.AdminResponse; import com.linkedin.davinci.notifier.VeniceNotifier; -import com.linkedin.davinci.replication.RmdWithValueSchemaId; import com.linkedin.davinci.replication.merge.MergeConflictResolver; import com.linkedin.davinci.replication.merge.RmdSerDe; import com.linkedin.davinci.stats.AggVersionedDIVStats; @@ -4749,14 +4748,6 @@ public int getRmdProtocolVersionId() { throw new VeniceException("getRmdProtocolVersionId() should only be called in active active mode"); } - RmdWithValueSchemaId getReplicationMetadataAndSchemaId( - PartitionConsumptionState partitionConsumptionState, - byte[] key, - int partition, - long currentTimeForMetricsMs) { - throw new VeniceException("getReplicationMetadataAndSchemaId() should only be called in active active mode"); - } - public MergeConflictResolver getMergeConflictResolver() { throw new VeniceException("getMergeConflictResolver() should only be called in active active mode"); } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 08b3bc8e071..2ea2465d215 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -14,7 +14,9 @@ import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; import com.linkedin.davinci.storage.chunking.GenericRecordChunkingAdapter; import com.linkedin.davinci.storage.chunking.RawBytesChunkingAdapter; +import com.linkedin.davinci.storage.chunking.SingleGetChunkingAdapter; import com.linkedin.davinci.store.record.ByteBufferValueRecord; +import com.linkedin.davinci.store.record.ValueRecord; import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.utils.ByteArrayKey; import com.linkedin.davinci.validation.PartitionTracker; @@ -1723,7 +1725,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( oldValueProvider.get(); } - final RmdWithValueSchemaId rmdWithValueSchemaID = storeIngestionTask.getReplicationMetadataAndSchemaId( + final RmdWithValueSchemaId rmdWithValueSchemaID = getReplicationMetadataAndSchemaId( partitionConsumptionState, keyBytes, partition, @@ -2427,6 +2429,70 @@ ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.Transien } } + /** + * Get the existing value schema ID and RMD associated with the given key. If information for this key is found from + * the transient map then use that, otherwise get it from storage engine. + * + * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition + * @param key Bytes of key. + * @param partition The partition to fetch the replication metadata from storage engine + * @return The object containing RMD and value schema id. If nothing is found, return null + */ + private RmdWithValueSchemaId getReplicationMetadataAndSchemaId( + PartitionConsumptionState partitionConsumptionState, + byte[] key, + int partition, + long currentTimeForMetricsMs) { + PartitionConsumptionState.TransientRecord cachedRecord = partitionConsumptionState.getTransientRecord(key); + if (cachedRecord != null) { + storeIngestionTask.getHostLevelIngestionStats() + .recordIngestionReplicationMetadataCacheHitCount(currentTimeForMetricsMs); + return new RmdWithValueSchemaId( + cachedRecord.getValueSchemaId(), + storeIngestionTask.getRmdProtocolVersionId(), + cachedRecord.getReplicationMetadataRecord(), + cachedRecord.getRmdManifest()); + } + ChunkedValueManifestContainer rmdManifestContainer = new ChunkedValueManifestContainer(); + byte[] replicationMetadataWithValueSchemaBytes = + getRmdWithValueSchemaByteBufferFromStorage(partition, key, rmdManifestContainer, currentTimeForMetricsMs); + if (replicationMetadataWithValueSchemaBytes == null) { + return null; // No RMD for this key + } + RmdWithValueSchemaId rmdWithValueSchemaId = new RmdWithValueSchemaId(); + // Get old RMD manifest value from RMD Manifest container object. + rmdWithValueSchemaId.setRmdManifest(rmdManifestContainer.getManifest()); + storeIngestionTask.getRmdSerDe() + .deserializeValueSchemaIdPrependedRmdBytes(replicationMetadataWithValueSchemaBytes, rmdWithValueSchemaId); + return rmdWithValueSchemaId; + } + + /** + * This method tries to retrieve the RMD bytes with prepended value schema ID from storage engine. It will also store + * RMD manifest into passed-in {@link ChunkedValueManifestContainer} container object if current RMD value is chunked. + */ + byte[] getRmdWithValueSchemaByteBufferFromStorage( + int partition, + byte[] key, + ChunkedValueManifestContainer rmdManifestContainer, + long currentTimeForMetricsMs) { + final long lookupStartTimeInNS = System.nanoTime(); + ValueRecord result = SingleGetChunkingAdapter.getReplicationMetadata( + storeIngestionTask.getStorageEngine(), + partition, + key, + storeIngestionTask.isChunked(), + rmdManifestContainer); + storeIngestionTask.getHostLevelIngestionStats() + .recordIngestionReplicationMetadataLookUpLatency( + LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS), + currentTimeForMetricsMs); + if (result == null) { + return null; + } + return result.serialize(); + } + @Override public PubSubTopic destinationIdentifier() { return storeIngestionTask.getVersionTopic(); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java index 0fdef9ff5ea..ccdda58c217 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java @@ -528,14 +528,16 @@ public void testReadingChunkedRmdFromStorage() { when(ingestionTask.getStorageEngine()).thenReturn(storageEngine); when(ingestionTask.getSchemaRepo()).thenReturn(schemaRepository); when(ingestionTask.getServerConfig()).thenReturn(serverConfig); - when(ingestionTask.getRmdWithValueSchemaByteBufferFromStorage(anyInt(), any(), any(), anyLong())) - .thenCallRealMethod(); when(ingestionTask.isChunked()).thenReturn(true); when(ingestionTask.getHostLevelIngestionStats()).thenReturn(mock(HostLevelIngestionStats.class)); ChunkedValueManifestContainer container = new ChunkedValueManifestContainer(); when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(topLevelKey1))) .thenReturn(expectedNonChunkedValue); - byte[] result = ingestionTask.getRmdWithValueSchemaByteBufferFromStorage(partition, key1, container, 0L); + PubSubTopicPartition topicPartition = mock(PubSubTopicPartition.class); + StorePartitionDataReceiver storePartitionDataReceiver = + new StorePartitionDataReceiver(ingestionTask, topicPartition, "dummyUrl", 0); + byte[] result = + storePartitionDataReceiver.getRmdWithValueSchemaByteBufferFromStorage(partition, key1, container, 0L); Assert.assertNotNull(result); Assert.assertNull(container.getManifest()); Assert.assertEquals(result, expectedNonChunkedValue); @@ -565,7 +567,8 @@ public void testReadingChunkedRmdFromStorage() { when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(topLevelKey2))) .thenReturn(chunkedManifestBytes.array()); when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(chunkedKey1InKey2))).thenReturn(chunkedValue1); - byte[] result2 = ingestionTask.getRmdWithValueSchemaByteBufferFromStorage(partition, key2, container, 0L); + byte[] result2 = + storePartitionDataReceiver.getRmdWithValueSchemaByteBufferFromStorage(partition, key2, container, 0L); Assert.assertNotNull(result2); Assert.assertNotNull(container.getManifest()); Assert.assertEquals(container.getManifest().getKeysWithChunkIdSuffix().size(), 1); @@ -601,7 +604,8 @@ public void testReadingChunkedRmdFromStorage() { .thenReturn(chunkedManifestBytes.array()); when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(chunkedKey1InKey3))).thenReturn(chunkedValue1); when(storageEngine.getReplicationMetadata(partition, ByteBuffer.wrap(chunkedKey2InKey3))).thenReturn(chunkedValue2); - byte[] result3 = ingestionTask.getRmdWithValueSchemaByteBufferFromStorage(partition, key3, container, 0L); + byte[] result3 = + storePartitionDataReceiver.getRmdWithValueSchemaByteBufferFromStorage(partition, key3, container, 0L); Assert.assertNotNull(result3); Assert.assertNotNull(container.getManifest()); Assert.assertEquals(container.getManifest().getKeysWithChunkIdSuffix().size(), 2); From bb6b5e209c2fbceaf340ca83f29027787c963ae9 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sat, 19 Oct 2024 23:59:41 -0700 Subject: [PATCH 26/32] =?UTF-8?q?Refactored=20fields=20`mergeConflictResol?= =?UTF-8?q?ver`,=20`rmdSerDe`,=20`keyLevelLocksManager`=20from=20`ActiveAc?= =?UTF-8?q?tiveStoreIngestionTask`=20into=20`StorePartitionDataReceiver`.?= =?UTF-8?q?=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 76 +++++------- .../LeaderFollowerStoreIngestionTask.java | 5 - .../kafka/consumer/StoreIngestionTask.java | 16 +-- .../consumer/StorePartitionDataReceiver.java | 110 +++++++++++------- 4 files changed, 106 insertions(+), 101 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 19047279da4..db5db137101 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -8,10 +8,6 @@ import com.linkedin.davinci.client.DaVinciRecordTransformerFunctionalInterface; import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.config.VeniceStoreVersionConfig; -import com.linkedin.davinci.replication.merge.MergeConflictResolver; -import com.linkedin.davinci.replication.merge.MergeConflictResolverFactory; -import com.linkedin.davinci.replication.merge.RmdSerDe; -import com.linkedin.davinci.replication.merge.StringAnnotatedStoreSchemaCache; import com.linkedin.davinci.stats.AggVersionedIngestionStats; import com.linkedin.davinci.storage.StorageService; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; @@ -61,9 +57,9 @@ public class ActiveActiveStoreIngestionTask extends LeaderFollowerStoreIngestion // private static final byte[] BINARY_DECODER_PARAM = new byte[16]; private final int rmdProtocolVersionId; - private final MergeConflictResolver mergeConflictResolver; - private final RmdSerDe rmdSerDe; - private final Lazy keyLevelLocksManager; + // private final MergeConflictResolver mergeConflictResolver; + // private final RmdSerDe rmdSerDe; + // private final Lazy keyLevelLocksManager; private final AggVersionedIngestionStats aggVersionedIngestionStats; private final RemoteIngestionRepairService remoteIngestionRepairService; // private final Lazy ingestionBatchProcessorLazy; @@ -108,28 +104,28 @@ public ActiveActiveStoreIngestionTask( this.rmdProtocolVersionId = version.getRmdVersionId(); this.aggVersionedIngestionStats = versionedIngestionStats; - int knownKafkaClusterNumber = serverConfig.getKafkaClusterIdToUrlMap().size(); - - int initialPoolSize = knownKafkaClusterNumber + 1; - this.keyLevelLocksManager = Lazy.of( - () -> new KeyLevelLocksManager( - getVersionTopic().getName(), - initialPoolSize, - getKeyLevelLockMaxPoolSizeBasedOnServerConfig(serverConfig, storeVersionPartitionCount))); - StringAnnotatedStoreSchemaCache annotatedReadOnlySchemaRepository = - new StringAnnotatedStoreSchemaCache(storeName, schemaRepository); - - this.rmdSerDe = new RmdSerDe( - annotatedReadOnlySchemaRepository, - rmdProtocolVersionId, - getServerConfig().isComputeFastAvroEnabled()); - this.mergeConflictResolver = MergeConflictResolverFactory.getInstance() - .createMergeConflictResolver( - annotatedReadOnlySchemaRepository, - rmdSerDe, - getStoreName(), - isWriteComputationEnabled, - getServerConfig().isComputeFastAvroEnabled()); + // int knownKafkaClusterNumber = serverConfig.getKafkaClusterIdToUrlMap().size(); + // + // int initialPoolSize = knownKafkaClusterNumber + 1; + // this.keyLevelLocksManager = Lazy.of( + // () -> new KeyLevelLocksManager( + // getVersionTopic().getName(), + // initialPoolSize, + // getKeyLevelLockMaxPoolSizeBasedOnServerConfig(serverConfig, storeVersionPartitionCount))); + // StringAnnotatedStoreSchemaCache annotatedReadOnlySchemaRepository = + // new StringAnnotatedStoreSchemaCache(storeName, schemaRepository); + // + // this.rmdSerDe = new RmdSerDe( + // annotatedReadOnlySchemaRepository, + // rmdProtocolVersionId, + // getServerConfig().isComputeFastAvroEnabled()); + // this.mergeConflictResolver = MergeConflictResolverFactory.getInstance() + // .createMergeConflictResolver( + // annotatedReadOnlySchemaRepository, + // rmdSerDe, + // getStoreName(), + // isWriteComputationEnabled, + // getServerConfig().isComputeFastAvroEnabled()); this.remoteIngestionRepairService = builder.getRemoteIngestionRepairService(); // this.ingestionBatchProcessorLazy = Lazy.of(() -> { // if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { @@ -355,12 +351,12 @@ private byte[] prependReplicationMetadataBytesWithValueSchemaId(ByteBuffer metad // .deserializeValueSchemaIdPrependedRmdBytes(replicationMetadataWithValueSchemaBytes, rmdWithValueSchemaId); // return rmdWithValueSchemaId; // } - - @Override - public RmdSerDe getRmdSerDe() { - return rmdSerDe; - } - + // + // @Override + // public RmdSerDe getRmdSerDe() { + // return rmdSerDe; + // } + // // /** // * This method tries to retrieve the RMD bytes with prepended value schema ID from storage engine. It will also // store @@ -1434,21 +1430,11 @@ public int getRmdProtocolVersionId() { return rmdProtocolVersionId; } - @Override - public final Lazy getKeyLevelLocksManager() { - return keyLevelLocksManager; - } - @Override public AggVersionedIngestionStats getAggVersionedIngestionStats() { return aggVersionedIngestionStats; } - @Override - public MergeConflictResolver getMergeConflictResolver() { - return mergeConflictResolver; - } - // @Override // protected BiConsumer getProduceToTopicFunction( // PartitionConsumptionState partitionConsumptionState, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index edb4069baa2..5f03c2c0e47 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -3996,11 +3996,6 @@ public KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders() { return kafkaDataIntegrityValidatorForLeaders; } - @Override - public Lazy getKeyLevelLocksManager() { - throw new VeniceException("getKeyLevelLocksManager() should only be called in active-active replication mode"); - } - @Override public StoreWriteComputeProcessor getStoreWriteComputeHandler() { return storeWriteComputeHandler; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 2257493077c..414c0b8b04e 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -32,8 +32,6 @@ import com.linkedin.davinci.ingestion.LagType; import com.linkedin.davinci.listener.response.AdminResponse; import com.linkedin.davinci.notifier.VeniceNotifier; -import com.linkedin.davinci.replication.merge.MergeConflictResolver; -import com.linkedin.davinci.replication.merge.RmdSerDe; import com.linkedin.davinci.stats.AggVersionedDIVStats; import com.linkedin.davinci.stats.AggVersionedIngestionStats; import com.linkedin.davinci.stats.HostLevelIngestionStats; @@ -4721,8 +4719,6 @@ public Consumer getDivErrorMetricCallback() { return divErrorMetricCallback; } - public abstract Lazy getKeyLevelLocksManager(); - public abstract KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders(); protected abstract void updateLatestInMemoryLeaderConsumedRTOffset( @@ -4748,14 +4744,6 @@ public int getRmdProtocolVersionId() { throw new VeniceException("getRmdProtocolVersionId() should only be called in active active mode"); } - public MergeConflictResolver getMergeConflictResolver() { - throw new VeniceException("getMergeConflictResolver() should only be called in active active mode"); - } - - public RmdSerDe getRmdSerDe() { - throw new VeniceException("getRmdSerDe() should only be called in active active mode"); - } - public abstract Int2ObjectMap getKafkaClusterIdToUrlMap(); public abstract boolean hasChangeCaptureView(); @@ -4791,6 +4779,10 @@ public int getLocalKafkaClusterId() { return localKafkaClusterId; } + public int getStoreVersionPartitionCount() { + return storeVersionPartitionCount; + } + // For unit test purpose. void setVersionRole(PartitionReplicaIngestionContext.VersionRole versionRole) { this.versionRole = versionRole; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 2ea2465d215..2d8d37ea2fa 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -9,7 +9,10 @@ import com.linkedin.davinci.listener.response.NoOpReadResponseStats; import com.linkedin.davinci.replication.RmdWithValueSchemaId; import com.linkedin.davinci.replication.merge.MergeConflictResolver; +import com.linkedin.davinci.replication.merge.MergeConflictResolverFactory; import com.linkedin.davinci.replication.merge.MergeConflictResult; +import com.linkedin.davinci.replication.merge.RmdSerDe; +import com.linkedin.davinci.replication.merge.StringAnnotatedStoreSchemaCache; import com.linkedin.davinci.stats.HostLevelIngestionStats; import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; import com.linkedin.davinci.storage.chunking.GenericRecordChunkingAdapter; @@ -95,6 +98,9 @@ public class StorePartitionDataReceiver private final String kafkaUrl; private final String kafkaUrlForLogger; private final int kafkaClusterId; + private final Lazy rmdSerDe; + private final Lazy mergeConflictResolver; + private final Lazy keyLevelLocksManager; private final Lazy ingestionBatchProcessorLazy; private long receivedRecordsCount; @@ -120,6 +126,31 @@ public StorePartitionDataReceiver( this.kafkaClusterId = kafkaClusterId; this.LOGGER = LogManager.getLogger(this.getClass().getSimpleName() + " [" + kafkaUrlForLogger + "]"); this.receivedRecordsCount = 0L; + final StringAnnotatedStoreSchemaCache annotatedReadOnlySchemaRepository = + new StringAnnotatedStoreSchemaCache(storeIngestionTask.getStoreName(), storeIngestionTask.getSchemaRepo()); + this.rmdSerDe = Lazy.of( + () -> new RmdSerDe( + annotatedReadOnlySchemaRepository, + storeIngestionTask.getRmdProtocolVersionId(), + storeIngestionTask.getServerConfig().isComputeFastAvroEnabled())); + this.mergeConflictResolver = Lazy.of( + () -> MergeConflictResolverFactory.getInstance() + .createMergeConflictResolver( + annotatedReadOnlySchemaRepository, + rmdSerDe.get(), + storeIngestionTask.getStoreName(), + storeIngestionTask.isTransientRecordBufferUsed(), + storeIngestionTask.getServerConfig().isComputeFastAvroEnabled())); + this.keyLevelLocksManager = Lazy.of(() -> { + final int knownKafkaClusterNumber = storeIngestionTask.getServerConfig().getKafkaClusterIdToUrlMap().size(); + final int initialPoolSize = knownKafkaClusterNumber + 1; + return new KeyLevelLocksManager( + storeIngestionTask.getVersionTopic().getName(), + initialPoolSize, + ActiveActiveStoreIngestionTask.getKeyLevelLockMaxPoolSizeBasedOnServerConfig( + storeIngestionTask.getServerConfig(), + storeIngestionTask.getStoreVersionPartitionCount())); + }); this.ingestionBatchProcessorLazy = Lazy.of(() -> { final String kafkaVersionTopic = storeIngestionTask.getKafkaVersionTopic(); if (!storeIngestionTask.getServerConfig().isAAWCWorkloadParallelProcessingEnabled()) { @@ -130,9 +161,8 @@ public StorePartitionDataReceiver( (storeIngestionTask.isActiveActiveReplicationEnabled()) ? this::processActiveActiveMessage : this::processMessage; - KeyLevelLocksManager lockManager = (storeIngestionTask.isActiveActiveReplicationEnabled()) - ? storeIngestionTask.getKeyLevelLocksManager().get() - : null; + KeyLevelLocksManager lockManager = + (storeIngestionTask.isActiveActiveReplicationEnabled()) ? keyLevelLocksManager.get() : null; LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", kafkaVersionTopic); return new IngestionBatchProcessor( storeIngestionTask.getKafkaVersionTopic(), @@ -798,7 +828,7 @@ private DelegateConsumerRecordResult delegateConsumerRecordMaybeWithLock( * -> [fabric A thread]produce to VT */ final ByteArrayKey byteArrayKey = ByteArrayKey.wrap(consumerRecordWrapper.getMessage().getKey().getKey()); - ReentrantLock keyLevelLock = storeIngestionTask.getKeyLevelLocksManager().get().acquireLockByKey(byteArrayKey); + ReentrantLock keyLevelLock = keyLevelLocksManager.get().acquireLockByKey(byteArrayKey); keyLevelLock.lock(); try { return delegateConsumerRecord( @@ -810,7 +840,7 @@ private DelegateConsumerRecordResult delegateConsumerRecordMaybeWithLock( beforeProcessingBatchRecordsTimestampMs); } finally { keyLevelLock.unlock(); - storeIngestionTask.getKeyLevelLocksManager().get().releaseLock(byteArrayKey); + keyLevelLocksManager.get().releaseLock(byteArrayKey); } } } @@ -1748,49 +1778,51 @@ private PubSubMessageProcessedResult processActiveActiveMessage( Lazy oldValueByteBufferProvider = unwrapByteBufferFromOldValueProvider(oldValueProvider); long beforeDCRTimestampInNs = System.nanoTime(); - final MergeConflictResolver mergeConflictResolver = storeIngestionTask.getMergeConflictResolver(); switch (msgType) { case PUT: - mergeConflictResult = mergeConflictResolver.put( - oldValueByteBufferProvider, - rmdWithValueSchemaID, - ((Put) kafkaValue.payloadUnion).putValue, - writeTimestamp, - incomingValueSchemaId, - sourceOffset, - kafkaClusterId, - kafkaClusterId // Use the kafka cluster ID as the colo ID for now because one colo/fabric has only one - // Kafka cluster. TODO: evaluate whether it is enough this way, or we need to add a new - // config to represent the mapping from Kafka server URLs to colo ID. - ); + mergeConflictResult = mergeConflictResolver.get() + .put( + oldValueByteBufferProvider, + rmdWithValueSchemaID, + ((Put) kafkaValue.payloadUnion).putValue, + writeTimestamp, + incomingValueSchemaId, + sourceOffset, + kafkaClusterId, + kafkaClusterId // Use the kafka cluster ID as the colo ID for now because one colo/fabric has only one + // Kafka cluster. TODO: evaluate whether it is enough this way, or we need to add a new + // config to represent the mapping from Kafka server URLs to colo ID. + ); storeIngestionTask.getHostLevelIngestionStats() .recordIngestionActiveActivePutLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); break; case DELETE: - mergeConflictResult = mergeConflictResolver.delete( - oldValueByteBufferProvider, - rmdWithValueSchemaID, - writeTimestamp, - sourceOffset, - kafkaClusterId, - kafkaClusterId); + mergeConflictResult = mergeConflictResolver.get() + .delete( + oldValueByteBufferProvider, + rmdWithValueSchemaID, + writeTimestamp, + sourceOffset, + kafkaClusterId, + kafkaClusterId); storeIngestionTask.getHostLevelIngestionStats() .recordIngestionActiveActiveDeleteLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); break; case UPDATE: - mergeConflictResult = mergeConflictResolver.update( - oldValueByteBufferProvider, - rmdWithValueSchemaID, - ((Update) kafkaValue.payloadUnion).updateValue, - incomingValueSchemaId, - incomingWriteComputeSchemaId, - writeTimestamp, - sourceOffset, - kafkaClusterId, - kafkaClusterId, - valueManifestContainer); + mergeConflictResult = mergeConflictResolver.get() + .update( + oldValueByteBufferProvider, + rmdWithValueSchemaID, + ((Update) kafkaValue.payloadUnion).updateValue, + incomingValueSchemaId, + incomingWriteComputeSchemaId, + writeTimestamp, + sourceOffset, + kafkaClusterId, + kafkaClusterId, + valueManifestContainer); storeIngestionTask.getHostLevelIngestionStats() .recordIngestionActiveActiveUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); break; @@ -1826,8 +1858,8 @@ private PubSubMessageProcessedResult processActiveActiveMessage( final int valueSchemaId = mergeConflictResult.getValueSchemaId(); GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); - final ByteBuffer updatedRmdBytes = storeIngestionTask.getRmdSerDe() - .serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); + final ByteBuffer updatedRmdBytes = + rmdSerDe.get().serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); if (updatedValueBytes == null) { storeIngestionTask.getHostLevelIngestionStats().recordTombstoneCreatedDCR(); @@ -2462,7 +2494,7 @@ private RmdWithValueSchemaId getReplicationMetadataAndSchemaId( RmdWithValueSchemaId rmdWithValueSchemaId = new RmdWithValueSchemaId(); // Get old RMD manifest value from RMD Manifest container object. rmdWithValueSchemaId.setRmdManifest(rmdManifestContainer.getManifest()); - storeIngestionTask.getRmdSerDe() + rmdSerDe.get() .deserializeValueSchemaIdPrependedRmdBytes(replicationMetadataWithValueSchemaBytes, rmdWithValueSchemaId); return rmdWithValueSchemaId; } From 7ff77b66fa4a1705eb44a907d143961bc3c4ecae Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Fri, 3 Jan 2025 11:25:12 -0800 Subject: [PATCH 27/32] =?UTF-8?q?Undo=20refactoring=20fields=20`mergeConfl?= =?UTF-8?q?ictResolver`=20and=20`rmdSerDe`=20from=20`ActiveActiveStoreInge?= =?UTF-8?q?stionTask`=20into=20`StorePartitionDataReceiver`.=20?= =?UTF-8?q?=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 53 +++++++++++-------- .../kafka/consumer/StoreIngestionTask.java | 10 ++++ .../consumer/StorePartitionDataReceiver.java | 33 +++--------- 3 files changed, 47 insertions(+), 49 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index db5db137101..dbaf8f0100f 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -8,6 +8,10 @@ import com.linkedin.davinci.client.DaVinciRecordTransformerFunctionalInterface; import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.config.VeniceStoreVersionConfig; +import com.linkedin.davinci.replication.merge.MergeConflictResolver; +import com.linkedin.davinci.replication.merge.MergeConflictResolverFactory; +import com.linkedin.davinci.replication.merge.RmdSerDe; +import com.linkedin.davinci.replication.merge.StringAnnotatedStoreSchemaCache; import com.linkedin.davinci.stats.AggVersionedIngestionStats; import com.linkedin.davinci.storage.StorageService; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; @@ -57,8 +61,8 @@ public class ActiveActiveStoreIngestionTask extends LeaderFollowerStoreIngestion // private static final byte[] BINARY_DECODER_PARAM = new byte[16]; private final int rmdProtocolVersionId; - // private final MergeConflictResolver mergeConflictResolver; - // private final RmdSerDe rmdSerDe; + private final MergeConflictResolver mergeConflictResolver; + private final RmdSerDe rmdSerDe; // private final Lazy keyLevelLocksManager; private final AggVersionedIngestionStats aggVersionedIngestionStats; private final RemoteIngestionRepairService remoteIngestionRepairService; @@ -112,20 +116,20 @@ public ActiveActiveStoreIngestionTask( // getVersionTopic().getName(), // initialPoolSize, // getKeyLevelLockMaxPoolSizeBasedOnServerConfig(serverConfig, storeVersionPartitionCount))); - // StringAnnotatedStoreSchemaCache annotatedReadOnlySchemaRepository = - // new StringAnnotatedStoreSchemaCache(storeName, schemaRepository); - // - // this.rmdSerDe = new RmdSerDe( - // annotatedReadOnlySchemaRepository, - // rmdProtocolVersionId, - // getServerConfig().isComputeFastAvroEnabled()); - // this.mergeConflictResolver = MergeConflictResolverFactory.getInstance() - // .createMergeConflictResolver( - // annotatedReadOnlySchemaRepository, - // rmdSerDe, - // getStoreName(), - // isWriteComputationEnabled, - // getServerConfig().isComputeFastAvroEnabled()); + StringAnnotatedStoreSchemaCache annotatedReadOnlySchemaRepository = + new StringAnnotatedStoreSchemaCache(storeName, schemaRepository); + + this.rmdSerDe = new RmdSerDe( + annotatedReadOnlySchemaRepository, + rmdProtocolVersionId, + getServerConfig().isComputeFastAvroEnabled()); + this.mergeConflictResolver = MergeConflictResolverFactory.getInstance() + .createMergeConflictResolver( + annotatedReadOnlySchemaRepository, + rmdSerDe, + getStoreName(), + isWriteComputationEnabled, + getServerConfig().isComputeFastAvroEnabled()); this.remoteIngestionRepairService = builder.getRemoteIngestionRepairService(); // this.ingestionBatchProcessorLazy = Lazy.of(() -> { // if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { @@ -351,12 +355,12 @@ private byte[] prependReplicationMetadataBytesWithValueSchemaId(ByteBuffer metad // .deserializeValueSchemaIdPrependedRmdBytes(replicationMetadataWithValueSchemaBytes, rmdWithValueSchemaId); // return rmdWithValueSchemaId; // } - // - // @Override - // public RmdSerDe getRmdSerDe() { - // return rmdSerDe; - // } - // + + @Override + RmdSerDe getRmdSerDe() { + return rmdSerDe; + } + // /** // * This method tries to retrieve the RMD bytes with prepended value schema ID from storage engine. It will also // store @@ -1430,6 +1434,11 @@ public int getRmdProtocolVersionId() { return rmdProtocolVersionId; } + @Override + MergeConflictResolver getMergeConflictResolver() { + return mergeConflictResolver; + } + @Override public AggVersionedIngestionStats getAggVersionedIngestionStats() { return aggVersionedIngestionStats; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 414c0b8b04e..62e4e7c2fbd 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -32,6 +32,8 @@ import com.linkedin.davinci.ingestion.LagType; import com.linkedin.davinci.listener.response.AdminResponse; import com.linkedin.davinci.notifier.VeniceNotifier; +import com.linkedin.davinci.replication.merge.MergeConflictResolver; +import com.linkedin.davinci.replication.merge.RmdSerDe; import com.linkedin.davinci.stats.AggVersionedDIVStats; import com.linkedin.davinci.stats.AggVersionedIngestionStats; import com.linkedin.davinci.stats.HostLevelIngestionStats; @@ -4744,6 +4746,14 @@ public int getRmdProtocolVersionId() { throw new VeniceException("getRmdProtocolVersionId() should only be called in active active mode"); } + MergeConflictResolver getMergeConflictResolver() { + throw new VeniceException("getMergeConflictResolver() should only be called in active active mode"); + } + + RmdSerDe getRmdSerDe() { + throw new VeniceException("getRmdSerDe() should only be called in active active mode"); + } + public abstract Int2ObjectMap getKafkaClusterIdToUrlMap(); public abstract boolean hasChangeCaptureView(); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 2d8d37ea2fa..cf5f97944c8 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -8,11 +8,7 @@ import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; import com.linkedin.davinci.listener.response.NoOpReadResponseStats; import com.linkedin.davinci.replication.RmdWithValueSchemaId; -import com.linkedin.davinci.replication.merge.MergeConflictResolver; -import com.linkedin.davinci.replication.merge.MergeConflictResolverFactory; import com.linkedin.davinci.replication.merge.MergeConflictResult; -import com.linkedin.davinci.replication.merge.RmdSerDe; -import com.linkedin.davinci.replication.merge.StringAnnotatedStoreSchemaCache; import com.linkedin.davinci.stats.HostLevelIngestionStats; import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; import com.linkedin.davinci.storage.chunking.GenericRecordChunkingAdapter; @@ -98,8 +94,6 @@ public class StorePartitionDataReceiver private final String kafkaUrl; private final String kafkaUrlForLogger; private final int kafkaClusterId; - private final Lazy rmdSerDe; - private final Lazy mergeConflictResolver; private final Lazy keyLevelLocksManager; private final Lazy ingestionBatchProcessorLazy; @@ -126,21 +120,6 @@ public StorePartitionDataReceiver( this.kafkaClusterId = kafkaClusterId; this.LOGGER = LogManager.getLogger(this.getClass().getSimpleName() + " [" + kafkaUrlForLogger + "]"); this.receivedRecordsCount = 0L; - final StringAnnotatedStoreSchemaCache annotatedReadOnlySchemaRepository = - new StringAnnotatedStoreSchemaCache(storeIngestionTask.getStoreName(), storeIngestionTask.getSchemaRepo()); - this.rmdSerDe = Lazy.of( - () -> new RmdSerDe( - annotatedReadOnlySchemaRepository, - storeIngestionTask.getRmdProtocolVersionId(), - storeIngestionTask.getServerConfig().isComputeFastAvroEnabled())); - this.mergeConflictResolver = Lazy.of( - () -> MergeConflictResolverFactory.getInstance() - .createMergeConflictResolver( - annotatedReadOnlySchemaRepository, - rmdSerDe.get(), - storeIngestionTask.getStoreName(), - storeIngestionTask.isTransientRecordBufferUsed(), - storeIngestionTask.getServerConfig().isComputeFastAvroEnabled())); this.keyLevelLocksManager = Lazy.of(() -> { final int knownKafkaClusterNumber = storeIngestionTask.getServerConfig().getKafkaClusterIdToUrlMap().size(); final int initialPoolSize = knownKafkaClusterNumber + 1; @@ -1780,7 +1759,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( long beforeDCRTimestampInNs = System.nanoTime(); switch (msgType) { case PUT: - mergeConflictResult = mergeConflictResolver.get() + mergeConflictResult = storeIngestionTask.getMergeConflictResolver() .put( oldValueByteBufferProvider, rmdWithValueSchemaID, @@ -1798,7 +1777,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( break; case DELETE: - mergeConflictResult = mergeConflictResolver.get() + mergeConflictResult = storeIngestionTask.getMergeConflictResolver() .delete( oldValueByteBufferProvider, rmdWithValueSchemaID, @@ -1811,7 +1790,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( break; case UPDATE: - mergeConflictResult = mergeConflictResolver.get() + mergeConflictResult = storeIngestionTask.getMergeConflictResolver() .update( oldValueByteBufferProvider, rmdWithValueSchemaID, @@ -1858,8 +1837,8 @@ private PubSubMessageProcessedResult processActiveActiveMessage( final int valueSchemaId = mergeConflictResult.getValueSchemaId(); GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); - final ByteBuffer updatedRmdBytes = - rmdSerDe.get().serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); + final ByteBuffer updatedRmdBytes = storeIngestionTask.getRmdSerDe() + .serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); if (updatedValueBytes == null) { storeIngestionTask.getHostLevelIngestionStats().recordTombstoneCreatedDCR(); @@ -2494,7 +2473,7 @@ private RmdWithValueSchemaId getReplicationMetadataAndSchemaId( RmdWithValueSchemaId rmdWithValueSchemaId = new RmdWithValueSchemaId(); // Get old RMD manifest value from RMD Manifest container object. rmdWithValueSchemaId.setRmdManifest(rmdManifestContainer.getManifest()); - rmdSerDe.get() + storeIngestionTask.getRmdSerDe() .deserializeValueSchemaIdPrependedRmdBytes(replicationMetadataWithValueSchemaBytes, rmdWithValueSchemaId); return rmdWithValueSchemaId; } From 85ff1af51e71c9769a19e3266ff44c92fbd29d75 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sun, 20 Oct 2024 00:07:57 -0700 Subject: [PATCH 28/32] =?UTF-8?q?Removed=20`aggVersionedIngestionStats`=20?= =?UTF-8?q?from=20`ActiveActiveStoreIngestionTask`,=20because=20it's=20jus?= =?UTF-8?q?t=20`versionedIngestionStats`=3F=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../consumer/ActiveActiveStoreIngestionTask.java | 10 ++-------- .../davinci/kafka/consumer/StoreIngestionTask.java | 4 ---- .../kafka/consumer/StorePartitionDataReceiver.java | 12 ++++++------ 3 files changed, 8 insertions(+), 18 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index dbaf8f0100f..f800dd3a7f8 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -12,7 +12,6 @@ import com.linkedin.davinci.replication.merge.MergeConflictResolverFactory; import com.linkedin.davinci.replication.merge.RmdSerDe; import com.linkedin.davinci.replication.merge.StringAnnotatedStoreSchemaCache; -import com.linkedin.davinci.stats.AggVersionedIngestionStats; import com.linkedin.davinci.storage.StorageService; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; import com.linkedin.venice.exceptions.PersistenceFailureException; @@ -64,7 +63,7 @@ public class ActiveActiveStoreIngestionTask extends LeaderFollowerStoreIngestion private final MergeConflictResolver mergeConflictResolver; private final RmdSerDe rmdSerDe; // private final Lazy keyLevelLocksManager; - private final AggVersionedIngestionStats aggVersionedIngestionStats; + // private final AggVersionedIngestionStats aggVersionedIngestionStats; private final RemoteIngestionRepairService remoteIngestionRepairService; // private final Lazy ingestionBatchProcessorLazy; // @@ -107,7 +106,7 @@ public ActiveActiveStoreIngestionTask( this.rmdProtocolVersionId = version.getRmdVersionId(); - this.aggVersionedIngestionStats = versionedIngestionStats; + // this.aggVersionedIngestionStats = versionedIngestionStats; // int knownKafkaClusterNumber = serverConfig.getKafkaClusterIdToUrlMap().size(); // // int initialPoolSize = knownKafkaClusterNumber + 1; @@ -1439,11 +1438,6 @@ MergeConflictResolver getMergeConflictResolver() { return mergeConflictResolver; } - @Override - public AggVersionedIngestionStats getAggVersionedIngestionStats() { - return aggVersionedIngestionStats; - } - // @Override // protected BiConsumer getProduceToTopicFunction( // PartitionConsumptionState partitionConsumptionState, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 62e4e7c2fbd..56104ff0ffb 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -4738,10 +4738,6 @@ protected abstract CompletableFuture sendIngestionHeartbeat LeaderCompleteState leaderCompleteState, long originTimeStampMs); - public AggVersionedIngestionStats getAggVersionedIngestionStats() { - throw new VeniceException("getAggVersionedIngestionStats() should only be called in active active mode"); - } - public int getRmdProtocolVersionId() { throw new VeniceException("getRmdProtocolVersionId() should only be called in active active mode"); } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index cf5f97944c8..7af53a2f4c0 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -150,7 +150,7 @@ public StorePartitionDataReceiver( processingFunction, storeIngestionTask.isTransientRecordBufferUsed(), storeIngestionTask.isActiveActiveReplicationEnabled(), - storeIngestionTask.getAggVersionedIngestionStats(), + storeIngestionTask.getVersionIngestionStats(), storeIngestionTask.getHostLevelIngestionStats()); }); } @@ -1751,7 +1751,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( long sourceOffset = consumerRecord.getOffset(); final MergeConflictResult mergeConflictResult; - storeIngestionTask.getAggVersionedIngestionStats() + storeIngestionTask.getVersionIngestionStats() .recordTotalDCR(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); Lazy oldValueByteBufferProvider = unwrapByteBufferFromOldValueProvider(oldValueProvider); @@ -1842,7 +1842,7 @@ private PubSubMessageProcessedResult processActiveActiveMessage( if (updatedValueBytes == null) { storeIngestionTask.getHostLevelIngestionStats().recordTombstoneCreatedDCR(); - storeIngestionTask.getAggVersionedIngestionStats() + storeIngestionTask.getVersionIngestionStats() .recordTombStoneCreationDCR(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); partitionConsumptionState .setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, valueSchemaId, rmdRecord); @@ -1905,7 +1905,7 @@ private void producePutOrDeleteToKafka( // finally produce if (mergeConflictResultWrapper.getUpdatedValueBytes() == null) { storeIngestionTask.getHostLevelIngestionStats().recordTombstoneCreatedDCR(); - storeIngestionTask.getAggVersionedIngestionStats() + storeIngestionTask.getVersionIngestionStats() .recordTombStoneCreationDCR(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); Delete deletePayload = new Delete(); deletePayload.schemaId = valueSchemaId; @@ -2202,7 +2202,7 @@ private void validatePostOperationResultsAndRecord( if (offsetSumPreOperation > RmdUtils.extractOffsetVectorSumFromRmd(rmdRecord)) { // offsets went backwards, raise an alert! storeIngestionTask.getHostLevelIngestionStats().recordOffsetRegressionDCRError(); - storeIngestionTask.getAggVersionedIngestionStats() + storeIngestionTask.getVersionIngestionStats() .recordOffsetRegressionDCRError(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); LOGGER .error("Offset vector found to have gone backwards!! New invalid replication metadata result: {}", rmdRecord); @@ -2217,7 +2217,7 @@ private void validatePostOperationResultsAndRecord( if (timestampsPreOperation.get(i) > timestampsPostOperation.get(i)) { // timestamps went backwards, raise an alert! storeIngestionTask.getHostLevelIngestionStats().recordTimestampRegressionDCRError(); - storeIngestionTask.getAggVersionedIngestionStats() + storeIngestionTask.getVersionIngestionStats() .recordTimestampRegressionDCRError( storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); From bb61d5ad8714a9606ef609514b5d8d0f7c3440aa Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sun, 20 Oct 2024 00:14:47 -0700 Subject: [PATCH 29/32] =?UTF-8?q?Removed=20all=20the=20dangling=20commente?= =?UTF-8?q?d=20out=20code=20blocks=20that=20were=20moved=20into=20`StorePa?= =?UTF-8?q?rtitionDataReceiver`.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 723 ---------- .../LeaderFollowerStoreIngestionTask.java | 1231 ----------------- .../kafka/consumer/StoreIngestionTask.java | 450 ------ .../consumer/StorePartitionDataReceiver.java | 10 - 4 files changed, 2414 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index f800dd3a7f8..d5c18049181 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -57,25 +57,11 @@ */ public class ActiveActiveStoreIngestionTask extends LeaderFollowerStoreIngestionTask { private static final Logger LOGGER = LogManager.getLogger(ActiveActiveStoreIngestionTask.class); - // private static final byte[] BINARY_DECODER_PARAM = new byte[16]; private final int rmdProtocolVersionId; private final MergeConflictResolver mergeConflictResolver; private final RmdSerDe rmdSerDe; - // private final Lazy keyLevelLocksManager; - // private final AggVersionedIngestionStats aggVersionedIngestionStats; private final RemoteIngestionRepairService remoteIngestionRepairService; - // private final Lazy ingestionBatchProcessorLazy; - // - // private static class ReusableObjects { - // // reuse buffer for rocksDB value object - // final ByteBuffer reusedByteBuffer = ByteBuffer.allocate(1024 * 1024); - // final BinaryDecoder binaryDecoder = - // AvroCompatibilityHelper.newBinaryDecoder(BINARY_DECODER_PARAM, 0, BINARY_DECODER_PARAM.length, null); - // } - // - // private final ThreadLocal threadLocalReusableObjects = - // ThreadLocal.withInitial(ReusableObjects::new); public ActiveActiveStoreIngestionTask( StorageService storageService, @@ -106,15 +92,6 @@ public ActiveActiveStoreIngestionTask( this.rmdProtocolVersionId = version.getRmdVersionId(); - // this.aggVersionedIngestionStats = versionedIngestionStats; - // int knownKafkaClusterNumber = serverConfig.getKafkaClusterIdToUrlMap().size(); - // - // int initialPoolSize = knownKafkaClusterNumber + 1; - // this.keyLevelLocksManager = Lazy.of( - // () -> new KeyLevelLocksManager( - // getVersionTopic().getName(), - // initialPoolSize, - // getKeyLevelLockMaxPoolSizeBasedOnServerConfig(serverConfig, storeVersionPartitionCount))); StringAnnotatedStoreSchemaCache annotatedReadOnlySchemaRepository = new StringAnnotatedStoreSchemaCache(storeName, schemaRepository); @@ -130,22 +107,6 @@ public ActiveActiveStoreIngestionTask( isWriteComputationEnabled, getServerConfig().isComputeFastAvroEnabled()); this.remoteIngestionRepairService = builder.getRemoteIngestionRepairService(); - // this.ingestionBatchProcessorLazy = Lazy.of(() -> { - // if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { - // LOGGER.info("AA/WC workload parallel processing is disabled for store version: {}", getKafkaVersionTopic()); - // return null; - // } - // LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", getKafkaVersionTopic()); - // return new IngestionBatchProcessor( - // kafkaVersionTopic, - // parallelProcessingThreadPool, - // keyLevelLocksManager.get(), - // this::processActiveActiveMessage, - // isWriteComputationEnabled, - // isActiveActiveReplicationEnabled(), - // aggVersionedIngestionStats, - // getHostLevelIngestionStats()); - // }); } public static int getKeyLevelLockMaxPoolSizeBasedOnServerConfig(VeniceServerConfig serverConfig, int partitionCount) { @@ -169,58 +130,6 @@ public static int getKeyLevelLockMaxPoolSizeBasedOnServerConfig(VeniceServerConf * serverConfig.getKafkaClusterIdToUrlMap().size() * multiplier + 1; } - // @Override - // protected DelegateConsumerRecordResult delegateConsumerRecord( - // PubSubMessageProcessedResultWrapper consumerRecordWrapper, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingPerRecordTimestampNs, - // long beforeProcessingBatchRecordsTimestampMs) { - // if (!consumerRecordWrapper.getMessage().getTopicPartition().getPubSubTopic().isRealTime()) { - // /** - // * We don't need to lock the partition here because during VT consumption there is only one consumption source. - // */ - // return super.delegateConsumerRecord( - // consumerRecordWrapper, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs, - // beforeProcessingBatchRecordsTimestampMs); - // } else { - // /** - // * The below flow must be executed in a critical session for the same key: - // * Read existing value/RMD from transient record cache/disk -> perform DCR and decide incoming value wins - // * -> update transient record cache -> produce to VT (just call send, no need to wait for the produce future in the - // critical session) - // * - // * Otherwise, there could be race conditions: - // * [fabric A thread]Read from transient record cache -> [fabric A thread]perform DCR and decide incoming value wins - // * -> [fabric B thread]read from transient record cache -> [fabric B thread]perform DCR and decide incoming value - // wins - // * -> [fabric B thread]update transient record cache -> [fabric B thread]produce to VT -> [fabric A thread]update - // transient record cache - // * -> [fabric A thread]produce to VT - // */ - // final ByteArrayKey byteArrayKey = ByteArrayKey.wrap(consumerRecordWrapper.getMessage().getKey().getKey()); - // ReentrantLock keyLevelLock = this.keyLevelLocksManager.get().acquireLockByKey(byteArrayKey); - // keyLevelLock.lock(); - // try { - // return super.delegateConsumerRecord( - // consumerRecordWrapper, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs, - // beforeProcessingBatchRecordsTimestampMs); - // } finally { - // keyLevelLock.unlock(); - // this.keyLevelLocksManager.get().releaseLock(byteArrayKey); - // } - // } - // } - @Override protected void putInStorageEngine(int partition, byte[] keyBytes, Put put) { try { @@ -317,592 +226,11 @@ private byte[] prependReplicationMetadataBytesWithValueSchemaId(ByteBuffer metad return replicationMetadataBytesWithValueSchemaId; } - // /** - // * Get the existing value schema ID and RMD associated with the given key. If information for this key is found from - // * the transient map then use that, otherwise get it from storage engine. - // * - // * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition - // * @param key Bytes of key. - // * @param partition The partition to fetch the replication metadata from storage engine - // * @return The object containing RMD and value schema id. If nothing is found, return null - // */ - // @Override - // RmdWithValueSchemaId getReplicationMetadataAndSchemaId( - // PartitionConsumptionState partitionConsumptionState, - // byte[] key, - // int partition, - // long currentTimeForMetricsMs) { - // PartitionConsumptionState.TransientRecord cachedRecord = partitionConsumptionState.getTransientRecord(key); - // if (cachedRecord != null) { - // getHostLevelIngestionStats().recordIngestionReplicationMetadataCacheHitCount(currentTimeForMetricsMs); - // return new RmdWithValueSchemaId( - // cachedRecord.getValueSchemaId(), - // getRmdProtocolVersionId(), - // cachedRecord.getReplicationMetadataRecord(), - // cachedRecord.getRmdManifest()); - // } - // ChunkedValueManifestContainer rmdManifestContainer = new ChunkedValueManifestContainer(); - // byte[] replicationMetadataWithValueSchemaBytes = - // getRmdWithValueSchemaByteBufferFromStorage(partition, key, rmdManifestContainer, currentTimeForMetricsMs); - // if (replicationMetadataWithValueSchemaBytes == null) { - // return null; // No RMD for this key - // } - // RmdWithValueSchemaId rmdWithValueSchemaId = new RmdWithValueSchemaId(); - // // Get old RMD manifest value from RMD Manifest container object. - // rmdWithValueSchemaId.setRmdManifest(rmdManifestContainer.getManifest()); - // getRmdSerDe() - // .deserializeValueSchemaIdPrependedRmdBytes(replicationMetadataWithValueSchemaBytes, rmdWithValueSchemaId); - // return rmdWithValueSchemaId; - // } - @Override RmdSerDe getRmdSerDe() { return rmdSerDe; } - // /** - // * This method tries to retrieve the RMD bytes with prepended value schema ID from storage engine. It will also - // store - // * RMD manifest into passed-in {@link ChunkedValueManifestContainer} container object if current RMD value is - // chunked. - // */ - // byte[] getRmdWithValueSchemaByteBufferFromStorage( - // int partition, - // byte[] key, - // ChunkedValueManifestContainer rmdManifestContainer, - // long currentTimeForMetricsMs) { - // final long lookupStartTimeInNS = System.nanoTime(); - // ValueRecord result = SingleGetChunkingAdapter - // .getReplicationMetadata(getStorageEngine(), partition, key, isChunked(), rmdManifestContainer); - // getHostLevelIngestionStats().recordIngestionReplicationMetadataLookUpLatency( - // LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS), - // currentTimeForMetricsMs); - // if (result == null) { - // return null; - // } - // return result.serialize(); - // } - // - // @Override - // protected IngestionBatchProcessor getIngestionBatchProcessor() { - // return ingestionBatchProcessorLazy.get(); - // } - // - // @Override - // protected PubSubMessageProcessedResult processActiveActiveMessage( - // PubSubMessage consumerRecord, - // PartitionConsumptionState partitionConsumptionState, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingRecordTimestampNs, - // long beforeProcessingBatchRecordsTimestampMs) { - // KafkaKey kafkaKey = consumerRecord.getKey(); - // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); - // byte[] keyBytes = kafkaKey.getKey(); - // MessageType msgType = MessageType.valueOf(kafkaValue.messageType); - // final int incomingValueSchemaId; - // final int incomingWriteComputeSchemaId; - // - // switch (msgType) { - // case PUT: - // incomingValueSchemaId = ((Put) kafkaValue.payloadUnion).schemaId; - // incomingWriteComputeSchemaId = -1; - // break; - // case UPDATE: - // Update incomingUpdate = (Update) kafkaValue.payloadUnion; - // incomingValueSchemaId = incomingUpdate.schemaId; - // incomingWriteComputeSchemaId = incomingUpdate.updateSchemaId; - // break; - // case DELETE: - // incomingValueSchemaId = -1; // Ignored since we don't need the schema id for DELETE operations. - // incomingWriteComputeSchemaId = -1; - // break; - // default: - // throw new VeniceMessageException( - // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); - // } - // final ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); - // Lazy> oldValueProvider = Lazy.of( - // () -> getValueBytesForKey( - // partitionConsumptionState, - // keyBytes, - // consumerRecord.getTopicPartition(), - // valueManifestContainer, - // beforeProcessingBatchRecordsTimestampMs)); - // if (hasChangeCaptureView) { - // /** - // * Since this function will update the transient cache before writing the view, and if there is - // * a change capture view writer, we need to lookup first, otherwise the transient cache will be populated - // * when writing to the view after this function. - // */ - // oldValueProvider.get(); - // } - // - // final RmdWithValueSchemaId rmdWithValueSchemaID = getReplicationMetadataAndSchemaId( - // partitionConsumptionState, - // keyBytes, - // partition, - // beforeProcessingBatchRecordsTimestampMs); - // - // final long writeTimestamp = getWriteTimestampFromKME(kafkaValue); - // final long offsetSumPreOperation = - // rmdWithValueSchemaID != null ? RmdUtils.extractOffsetVectorSumFromRmd(rmdWithValueSchemaID.getRmdRecord()) : 0; - // List recordTimestampsPreOperation = rmdWithValueSchemaID != null - // ? RmdUtils.extractTimestampFromRmd(rmdWithValueSchemaID.getRmdRecord()) - // : Collections.singletonList(0L); - // - // // get the source offset and the id - // long sourceOffset = consumerRecord.getOffset(); - // final MergeConflictResult mergeConflictResult; - // - // aggVersionedIngestionStats.recordTotalDCR(storeName, versionNumber); - // - // Lazy oldValueByteBufferProvider = unwrapByteBufferFromOldValueProvider(oldValueProvider); - // - // long beforeDCRTimestampInNs = System.nanoTime(); - // switch (msgType) { - // case PUT: - // mergeConflictResult = mergeConflictResolver.put( - // oldValueByteBufferProvider, - // rmdWithValueSchemaID, - // ((Put) kafkaValue.payloadUnion).putValue, - // writeTimestamp, - // incomingValueSchemaId, - // sourceOffset, - // kafkaClusterId, - // kafkaClusterId // Use the kafka cluster ID as the colo ID for now because one colo/fabric has only one - // // Kafka cluster. TODO: evaluate whether it is enough this way, or we need to add a new - // // config to represent the mapping from Kafka server URLs to colo ID. - // ); - // getHostLevelIngestionStats() - // .recordIngestionActiveActivePutLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); - // break; - // - // case DELETE: - // mergeConflictResult = mergeConflictResolver.delete( - // oldValueByteBufferProvider, - // rmdWithValueSchemaID, - // writeTimestamp, - // sourceOffset, - // kafkaClusterId, - // kafkaClusterId); - // getHostLevelIngestionStats() - // .recordIngestionActiveActiveDeleteLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); - // break; - // - // case UPDATE: - // mergeConflictResult = mergeConflictResolver.update( - // oldValueByteBufferProvider, - // rmdWithValueSchemaID, - // ((Update) kafkaValue.payloadUnion).updateValue, - // incomingValueSchemaId, - // incomingWriteComputeSchemaId, - // writeTimestamp, - // sourceOffset, - // kafkaClusterId, - // kafkaClusterId, - // valueManifestContainer); - // getHostLevelIngestionStats() - // .recordIngestionActiveActiveUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); - // break; - // default: - // throw new VeniceMessageException( - // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); - // } - // - // if (mergeConflictResult.isUpdateIgnored()) { - // hostLevelIngestionStats.recordUpdateIgnoredDCR(); - // // Record the last ignored offset - // partitionConsumptionState - // .updateLatestIgnoredUpstreamRTOffset(kafkaClusterIdToUrlMap.get(kafkaClusterId), sourceOffset); - // return new PubSubMessageProcessedResult( - // new MergeConflictResultWrapper( - // mergeConflictResult, - // oldValueProvider, - // oldValueByteBufferProvider, - // rmdWithValueSchemaID, - // valueManifestContainer, - // null, - // null)); - // } else { - // validatePostOperationResultsAndRecord(mergeConflictResult, offsetSumPreOperation, recordTimestampsPreOperation); - // - // final ByteBuffer updatedValueBytes = maybeCompressData( - // consumerRecord.getTopicPartition().getPartitionNumber(), - // mergeConflictResult.getNewValue(), - // partitionConsumptionState); - // - // final int valueSchemaId = mergeConflictResult.getValueSchemaId(); - // - // GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); - // final ByteBuffer updatedRmdBytes = - // rmdSerDe.serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); - // - // if (updatedValueBytes == null) { - // hostLevelIngestionStats.recordTombstoneCreatedDCR(); - // aggVersionedIngestionStats.recordTombStoneCreationDCR(storeName, versionNumber); - // partitionConsumptionState - // .setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, valueSchemaId, rmdRecord); - // } else { - // int valueLen = updatedValueBytes.remaining(); - // partitionConsumptionState.setTransientRecord( - // kafkaClusterId, - // consumerRecord.getOffset(), - // keyBytes, - // updatedValueBytes.array(), - // updatedValueBytes.position(), - // valueLen, - // valueSchemaId, - // rmdRecord); - // } - // return new PubSubMessageProcessedResult( - // new MergeConflictResultWrapper( - // mergeConflictResult, - // oldValueProvider, - // oldValueByteBufferProvider, - // rmdWithValueSchemaID, - // valueManifestContainer, - // updatedValueBytes, - // updatedRmdBytes)); - // } - // } - // - // // This function may modify the original record in KME, it is unsafe to use the payload from KME directly after - // // this function. - // protected void processMessageAndMaybeProduceToKafka( - // PubSubMessageProcessedResultWrapper consumerRecordWrapper, - // PartitionConsumptionState partitionConsumptionState, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingRecordTimestampNs, - // long beforeProcessingBatchRecordsTimestampMs) { - // /** - // * With {@link BatchConflictResolutionPolicy.BATCH_WRITE_LOSES} there is no need - // * to perform DCR before EOP and L/F DIV passthrough mode should be used. If the version is going through data - // * recovery then there is no need to perform DCR until we completed data recovery and switched to consume from RT. - // * TODO. We need to refactor this logic when we support other batch conflict resolution policy. - // */ - // if (!partitionConsumptionState.isEndOfPushReceived() - // || isDataRecovery && partitionConsumptionState.getTopicSwitch() != null) { - // super.processMessageAndMaybeProduceToKafka( - // consumerRecordWrapper, - // partitionConsumptionState, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs, - // beforeProcessingBatchRecordsTimestampMs); - // return; - // } - // PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); - // KafkaKey kafkaKey = consumerRecord.getKey(); - // byte[] keyBytes = kafkaKey.getKey(); - // final MergeConflictResultWrapper mergeConflictResultWrapper; - // if (consumerRecordWrapper.getProcessedResult() != null - // && consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper() != null) { - // mergeConflictResultWrapper = consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper(); - // } else { - // mergeConflictResultWrapper = processActiveActiveMessage( - // consumerRecord, - // partitionConsumptionState, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs, - // beforeProcessingBatchRecordsTimestampMs).getMergeConflictResultWrapper(); - // } - // - // MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); - // if (!mergeConflictResult.isUpdateIgnored()) { - // // Apply this update to any views for this store - // // TODO: It'd be good to be able to do this in LeaderFollowerStoreIngestionTask instead, however, AA currently is - // // the - // // only extension of IngestionTask which does a read from disk before applying the record. This makes the - // // following function - // // call in this context much less obtrusive, however, it implies that all views can only work for AA stores - // - // // Write to views - // Runnable produceToVersionTopic = () -> producePutOrDeleteToKafka( - // mergeConflictResultWrapper, - // partitionConsumptionState, - // keyBytes, - // consumerRecord, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs); - // if (hasViewWriters()) { - // /** - // * The ordering guarantees we want is the following: - // * - // * 1. Write to all view topics (in parallel). - // * 2. Write to the VT only after we get the ack for all views AND the previous write to VT was queued into the - // * producer (but not necessarily acked). - // */ - // ByteBuffer oldValueBB = mergeConflictResultWrapper.getOldValueByteBufferProvider().get(); - // int oldValueSchemaId = - // oldValueBB == null ? -1 : mergeConflictResultWrapper.getOldValueProvider().get().writerSchemaId(); - // queueUpVersionTopicWritesWithViewWriters( - // partitionConsumptionState, - // (viewWriter) -> viewWriter.processRecord( - // mergeConflictResultWrapper.getUpdatedValueBytes(), - // oldValueBB, - // keyBytes, - // mergeConflictResult.getValueSchemaId(), - // oldValueSchemaId, - // mergeConflictResult.getRmdRecord()), - // produceToVersionTopic); - // } else { - // // This function may modify the original record in KME and it is unsafe to use the payload from KME directly - // // after this call. - // produceToVersionTopic.run(); - // } - // } - // } - // - // /** - // * Package private for testing purposes. - // */ - // static Lazy unwrapByteBufferFromOldValueProvider( - // Lazy> oldValueProvider) { - // return Lazy.of(() -> { - // ByteBufferValueRecord bbValueRecord = oldValueProvider.get(); - // return bbValueRecord == null ? null : bbValueRecord.value(); - // }); - // } - // - // @Override - // public long getWriteTimestampFromKME(KafkaMessageEnvelope kme) { - // if (kme.producerMetadata.logicalTimestamp >= 0) { - // return kme.producerMetadata.logicalTimestamp; - // } else { - // return kme.producerMetadata.messageTimestamp; - // } - // } - // - // @Override - // void validatePostOperationResultsAndRecord( - // MergeConflictResult mergeConflictResult, - // Long offsetSumPreOperation, - // List timestampsPreOperation) { - // // Nothing was applied, no harm no foul - // if (mergeConflictResult.isUpdateIgnored()) { - // return; - // } - // // Post Validation checks on resolution - // GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); - // if (offsetSumPreOperation > RmdUtils.extractOffsetVectorSumFromRmd(rmdRecord)) { - // // offsets went backwards, raise an alert! - // hostLevelIngestionStats.recordOffsetRegressionDCRError(); - // aggVersionedIngestionStats.recordOffsetRegressionDCRError(storeName, versionNumber); - // LOGGER - // .error("Offset vector found to have gone backwards!! New invalid replication metadata result: {}", rmdRecord); - // } - // - // // TODO: This comparison doesn't work well for write compute+schema evolution (can spike up). VENG-8129 - // // this works fine for now however as we do not fully support A/A write compute operations (as we only do root - // // timestamp comparisons). - // - // List timestampsPostOperation = RmdUtils.extractTimestampFromRmd(rmdRecord); - // for (int i = 0; i < timestampsPreOperation.size(); i++) { - // if (timestampsPreOperation.get(i) > timestampsPostOperation.get(i)) { - // // timestamps went backwards, raise an alert! - // hostLevelIngestionStats.recordTimestampRegressionDCRError(); - // aggVersionedIngestionStats.recordTimestampRegressionDCRError(storeName, versionNumber); - // LOGGER.error( - // "Timestamp found to have gone backwards!! Invalid replication metadata result: {}", - // mergeConflictResult.getRmdRecord()); - // } - // } - // } - // - // /** - // * Get the value bytes for a key from {@link PartitionConsumptionState.TransientRecord} or from disk. The assumption - // * is that the {@link PartitionConsumptionState.TransientRecord} only contains the full value. - // * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition - // * @param key The key bytes of the incoming record. - // * @param topicPartition The {@link PubSubTopicPartition} from which the incoming record was consumed - // * @return - // */ - // @Override - // public ByteBufferValueRecord getValueBytesForKey( - // PartitionConsumptionState partitionConsumptionState, - // byte[] key, - // PubSubTopicPartition topicPartition, - // ChunkedValueManifestContainer valueManifestContainer, - // long currentTimeForMetricsMs) { - // ByteBufferValueRecord originalValue = null; - // // Find the existing value. If a value for this key is found from the transient map then use that value, otherwise - // // get it from DB. - // PartitionConsumptionState.TransientRecord transientRecord = partitionConsumptionState.getTransientRecord(key); - // if (transientRecord == null) { - // long lookupStartTimeInNS = System.nanoTime(); - // ReusableObjects reusableObjects = threadLocalReusableObjects.get(); - // ByteBuffer reusedRawValue = reusableObjects.reusedByteBuffer; - // BinaryDecoder binaryDecoder = reusableObjects.binaryDecoder; - // originalValue = RawBytesChunkingAdapter.INSTANCE.getWithSchemaId( - // storageEngine, - // topicPartition.getPartitionNumber(), - // ByteBuffer.wrap(key), - // isChunked, - // reusedRawValue, - // binaryDecoder, - // RawBytesStoreDeserializerCache.getInstance(), - // compressor.get(), - // valueManifestContainer); - // hostLevelIngestionStats.recordIngestionValueBytesLookUpLatency( - // LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS), - // currentTimeForMetricsMs); - // } else { - // hostLevelIngestionStats.recordIngestionValueBytesCacheHitCount(currentTimeForMetricsMs); - // // construct originalValue from this transient record only if it's not null. - // if (transientRecord.getValue() != null) { - // if (valueManifestContainer != null) { - // valueManifestContainer.setManifest(transientRecord.getValueManifest()); - // } - // originalValue = new ByteBufferValueRecord<>( - // getCurrentValueFromTransientRecord(transientRecord), - // transientRecord.getValueSchemaId()); - // } - // } - // return originalValue; - // } - // - // ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.TransientRecord transientRecord) { - // ByteBuffer compressedValue = - // ByteBuffer.wrap(transientRecord.getValue(), transientRecord.getValueOffset(), transientRecord.getValueLen()); - // try { - // return getCompressionStrategy().isCompressionEnabled() - // ? getCompressor().get() - // .decompress(compressedValue.array(), compressedValue.position(), compressedValue.remaining()) - // : compressedValue; - // } catch (IOException e) { - // throw new VeniceException(e); - // } - // } - // - // /** - // * This function parses the {@link MergeConflictResult} and decides if the update should be ignored or emit a PUT or - // a - // * DELETE record to VT. - // *

- // * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after - // * this function. - // * - // * @param mergeConflictResultWrapper The result of conflict resolution. - // * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition - // * @param key The key bytes of the incoming record. - // * @param consumerRecord The {@link PubSubMessage} for the current record. - // * @param partition - // * @param kafkaUrl - // */ - // @Override - // protected void producePutOrDeleteToKafka( - // MergeConflictResultWrapper mergeConflictResultWrapper, - // PartitionConsumptionState partitionConsumptionState, - // byte[] key, - // PubSubMessage consumerRecord, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingRecordTimestampNs) { - // MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); - // ByteBuffer updatedValueBytes = mergeConflictResultWrapper.getUpdatedValueBytes(); - // ByteBuffer updatedRmdBytes = mergeConflictResultWrapper.getUpdatedRmdBytes(); - // final int valueSchemaId = mergeConflictResult.getValueSchemaId(); - // - // ChunkedValueManifest oldValueManifest = mergeConflictResultWrapper.getOldValueManifestContainer().getManifest(); - // ChunkedValueManifest oldRmdManifest = mergeConflictResultWrapper.getOldRmdWithValueSchemaId() == null - // ? null - // : mergeConflictResultWrapper.getOldRmdWithValueSchemaId().getRmdManifest(); - // // finally produce - // if (mergeConflictResultWrapper.getUpdatedValueBytes() == null) { - // hostLevelIngestionStats.recordTombstoneCreatedDCR(); - // aggVersionedIngestionStats.recordTombStoneCreationDCR(storeName, versionNumber); - // Delete deletePayload = new Delete(); - // deletePayload.schemaId = valueSchemaId; - // deletePayload.replicationMetadataVersionId = rmdProtocolVersionId; - // deletePayload.replicationMetadataPayload = mergeConflictResultWrapper.getUpdatedRmdBytes(); - // BiConsumer produceToTopicFunction = - // (callback, sourceTopicOffset) -> partitionConsumptionState.getVeniceWriterLazyRef() - // .get() - // .delete( - // key, - // callback, - // sourceTopicOffset, - // APP_DEFAULT_LOGICAL_TS, - // new DeleteMetadata(valueSchemaId, rmdProtocolVersionId, updatedRmdBytes), - // oldValueManifest, - // oldRmdManifest); - // LeaderProducedRecordContext leaderProducedRecordContext = - // LeaderProducedRecordContext.newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), key, deletePayload); - // produceToLocalKafka( - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // produceToTopicFunction, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs); - // } else { - // Put updatedPut = new Put(); - // updatedPut.putValue = ByteUtils - // .prependIntHeaderToByteBuffer(updatedValueBytes, valueSchemaId, mergeConflictResult.doesResultReuseInput()); - // updatedPut.schemaId = valueSchemaId; - // updatedPut.replicationMetadataVersionId = rmdProtocolVersionId; - // updatedPut.replicationMetadataPayload = updatedRmdBytes; - // - // BiConsumer produceToTopicFunction = getProduceToTopicFunction( - // partitionConsumptionState, - // key, - // updatedValueBytes, - // updatedRmdBytes, - // oldValueManifest, - // oldRmdManifest, - // valueSchemaId, - // mergeConflictResult.doesResultReuseInput()); - // produceToLocalKafka( - // consumerRecord, - // partitionConsumptionState, - // LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), key, updatedPut), - // produceToTopicFunction, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs); - // } - // } - // - // @Override - // protected void produceToLocalKafka( - // PubSubMessage consumerRecord, - // PartitionConsumptionState partitionConsumptionState, - // LeaderProducedRecordContext leaderProducedRecordContext, - // BiConsumer produceFunction, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingRecordTimestampNs) { - // super.produceToLocalKafka( - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // produceFunction, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs); - // // Update the partition consumption state to say that we've transmitted the message to kafka (but haven't - // // necessarily received an ack back yet). - // if (partitionConsumptionState.getLeaderFollowerState() == LEADER && partitionConsumptionState.isHybrid() - // && consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { - // partitionConsumptionState.updateLatestRTOffsetTriedToProduceToVTMap(kafkaUrl, consumerRecord.getOffset()); - // } - // } - @Override protected Map calculateLeaderUpstreamOffsetWithTopicSwitch( PartitionConsumptionState partitionConsumptionState, @@ -1438,57 +766,6 @@ MergeConflictResolver getMergeConflictResolver() { return mergeConflictResolver; } - // @Override - // protected BiConsumer getProduceToTopicFunction( - // PartitionConsumptionState partitionConsumptionState, - // byte[] key, - // ByteBuffer updatedValueBytes, - // ByteBuffer updatedRmdBytes, - // ChunkedValueManifest oldValueManifest, - // ChunkedValueManifest oldRmdManifest, - // int valueSchemaId, - // boolean resultReuseInput) { - // return (callback, leaderMetadataWrapper) -> { - // if (resultReuseInput) { - // // Restore the original header so this function is eventually idempotent as the original KME ByteBuffer - // // will be recovered after producing the message to Kafka or if the production failing. - // ((ActiveActiveProducerCallback) callback).setOnCompletionFunction( - // () -> ByteUtils.prependIntHeaderToByteBuffer( - // updatedValueBytes, - // ByteUtils.getIntHeaderFromByteBuffer(updatedValueBytes), - // true)); - // } - // getVeniceWriter(partitionConsumptionState).get() - // .put( - // key, - // ByteUtils.extractByteArray(updatedValueBytes), - // valueSchemaId, - // callback, - // leaderMetadataWrapper, - // APP_DEFAULT_LOGICAL_TS, - // new PutMetadata(getRmdProtocolVersionId(), updatedRmdBytes), - // oldValueManifest, - // oldRmdManifest); - // }; - // } - // - // protected LeaderProducerCallback createProducerCallback( - // PubSubMessage consumerRecord, - // PartitionConsumptionState partitionConsumptionState, - // LeaderProducedRecordContext leaderProducedRecordContext, - // int partition, - // String kafkaUrl, - // long beforeProcessingRecordTimestampNs) { - // return new ActiveActiveProducerCallback( - // this, - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // partition, - // kafkaUrl, - // beforeProcessingRecordTimestampNs); - // } - /** * This method does a few things for leader topic-partition subscription: * (1) Calculate Kafka URL to leader subscribe offset map. diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 5f03c2c0e47..affb7df8d19 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -173,7 +173,6 @@ public class LeaderFollowerStoreIngestionTask extends StoreIngestionTask { private final AtomicLong lastSendIngestionHeartbeatTimestamp = new AtomicLong(0); - // private final Lazy ingestionBatchProcessingLazy; private final Version version; public LeaderFollowerStoreIngestionTask( @@ -315,22 +314,6 @@ public LeaderFollowerStoreIngestionTask( builder.getSchemaRepo(), getStoreName(), serverConfig.isComputeFastAvroEnabled()); - // this.ingestionBatchProcessingLazy = Lazy.of(() -> { - // if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { - // LOGGER.info("AA/WC workload parallel processing is disabled for store version: {}", getKafkaVersionTopic()); - // return null; - // } - // LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", getKafkaVersionTopic()); - // return new IngestionBatchProcessor( - // kafkaVersionTopic, - // parallelProcessingThreadPool, - // null, - // this::processMessage, - // isWriteComputationEnabled, - // isActiveActiveReplicationEnabled(), - // builder.getVersionedStorageIngestionStats(), - // getHostLevelIngestionStats()); - // }); } public static VeniceWriter constructVeniceWriter( @@ -375,11 +358,6 @@ protected void closeVeniceViewWriters() { } } - // @Override - // protected IngestionBatchProcessor getIngestionBatchProcessor() { - // return ingestionBatchProcessingLazy.get(); - // } - @Override public synchronized void promoteToLeader( PubSubTopicPartition topicPartition, @@ -1669,31 +1647,6 @@ protected static void checkAndHandleUpstreamOffsetRewind( } } - // protected void produceToLocalKafka( - // PubSubMessage consumerRecord, - // PartitionConsumptionState partitionConsumptionState, - // LeaderProducedRecordContext leaderProducedRecordContext, - // BiConsumer produceFunction, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingRecordTimestampNs) { - // LeaderProducerCallback callback = createProducerCallback( - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // partition, - // kafkaUrl, - // beforeProcessingRecordTimestampNs); - // long sourceTopicOffset = consumerRecord.getOffset(); - // LeaderMetadataWrapper leaderMetadataWrapper = new LeaderMetadataWrapper(sourceTopicOffset, kafkaClusterId); - // partitionConsumptionState.setLastLeaderPersistFuture(leaderProducedRecordContext.getPersistedToDBFuture()); - // long beforeProduceTimestampNS = System.nanoTime(); - // produceFunction.accept(callback, leaderMetadataWrapper); - // getHostLevelIngestionStats() - // .recordLeaderProduceLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeProduceTimestampNS)); - // } - @Override protected boolean isRealTimeBufferReplayStarted(PartitionConsumptionState partitionConsumptionState) { TopicSwitchWrapper topicSwitch = partitionConsumptionState.getTopicSwitch(); @@ -1839,93 +1792,6 @@ protected void reportIfCatchUpVersionTopicOffset(PartitionConsumptionState pcs) } } - // /** - // * For Leader/Follower model, the follower should have the same kind of check as the Online/Offline model; - // * for leader, it's possible that it consumers from real-time topic or GF topic. - // */ - // @Override - // protected boolean shouldProcessRecord(PubSubMessage record) { - // PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(record.getPartition()); - // if (partitionConsumptionState == null) { - // LOGGER.info( - // "Skipping message as partition is no longer actively subscribed. Replica: {}", - // Utils.getReplicaId(versionTopic, record.getPartition())); - // return false; - // } - // switch (partitionConsumptionState.getLeaderFollowerState()) { - // case LEADER: - // PubSubTopic currentLeaderTopic = - // partitionConsumptionState.getOffsetRecord().getLeaderTopic(pubSubTopicRepository); - // if (partitionConsumptionState.consumeRemotely() - // && currentLeaderTopic.isVersionTopicOrStreamReprocessingTopic()) { - // if (partitionConsumptionState.skipKafkaMessage()) { - // String msg = "Skipping messages after EOP in remote version topic. Replica: " - // + partitionConsumptionState.getReplicaId(); - // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { - // LOGGER.info(msg); - // } - // return false; - // } - // if (record.getKey().isControlMessage()) { - // ControlMessageType controlMessageType = - // ControlMessageType.valueOf((ControlMessage) record.getValue().payloadUnion); - // if (controlMessageType == END_OF_PUSH) { - // /** - // * The flag is turned on to avoid consuming unwanted messages after EOP in remote VT, such as SOBR. In - // * {@link LeaderFollowerStoreIngestionTask#checkLongRunningTaskState()}, once leader notices that EOP is - // * received, it will unsubscribe from the remote VT and turn off this flag. However, if data recovery is - // * in progress and the store is hybrid then we actually want to consume messages after EOP. In that case - // * remote TS will be skipped but with a different method. - // */ - // if (!(isDataRecovery && isHybridMode())) { - // partitionConsumptionState.setSkipKafkaMessage(true); - // } - // } - // } - // } - // if (!Utils.resolveLeaderTopicFromPubSubTopic(pubSubTopicRepository, record.getTopicPartition().getPubSubTopic()) - // .equals(currentLeaderTopic)) { - // String errorMsg = - // "Leader replica: {} received a pubsub message that doesn't belong to the leader topic. Leader topic: " - // + currentLeaderTopic + ", topic of incoming message: " - // + record.getTopicPartition().getPubSubTopic().getName(); - // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(errorMsg)) { - // LOGGER.error(errorMsg, partitionConsumptionState.getReplicaId()); - // } - // return false; - // } - // break; - // default: - // PubSubTopic pubSubTopic = record.getTopicPartition().getPubSubTopic(); - // String topicName = pubSubTopic.getName(); - // if (!versionTopic.equals(pubSubTopic)) { - // String errorMsg = partitionConsumptionState.getLeaderFollowerState() + " replica: " - // + partitionConsumptionState.getReplicaId() + " received message from non version topic: " + topicName; - // if (consumerHasSubscription(pubSubTopic, partitionConsumptionState)) { - // throw new VeniceMessageException( - // errorMsg + ". Throwing exception as the node still subscribes to " + topicName); - // } - // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(errorMsg)) { - // LOGGER.error("{}. Skipping the message as the node does not subscribe to {}", errorMsg, topicName); - // } - // return false; - // } - // - // long lastOffset = partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset(); - // if (lastOffset >= record.getOffset()) { - // String message = partitionConsumptionState.getLeaderFollowerState() + " replica: " - // + partitionConsumptionState.getReplicaId() + " had already processed the record"; - // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(message)) { - // LOGGER.info("{}; LastKnownOffset: {}; OffsetOfIncomingRecord: {}", message, lastOffset, record.getOffset()); - // } - // return false; - // } - // break; - // } - // - // return super.shouldProcessRecord(record); - // } - /** * Additional safeguards in Leader/Follower ingestion: * 1. Check whether the incoming messages are from the expected source topics @@ -2043,24 +1909,6 @@ protected final void recordAssembledRecordSizeRatio(double ratio, long currentTi } } - // @Override - // protected void recordRegionHybridConsumptionStats( - // int kafkaClusterId, - // int producedRecordSize, - // long upstreamOffset, - // long currentTimeMs) { - // if (kafkaClusterId >= 0) { - // versionedIngestionStats.recordRegionHybridConsumption( - // storeName, - // versionNumber, - // kafkaClusterId, - // producedRecordSize, - // upstreamOffset, - // currentTimeMs); - // hostLevelIngestionStats.recordTotalRegionHybridBytesConsumed(kafkaClusterId, producedRecordSize, currentTimeMs); - // } - // } - @Override protected boolean isHybridFollower(PartitionConsumptionState partitionConsumptionState) { return isHybridMode() && (isDaVinciClient || partitionConsumptionState.getLeaderFollowerState().equals(STANDBY)); @@ -2128,106 +1976,6 @@ protected boolean checkAndLogIfLagIsAcceptableForHybridStore( return isLagAcceptable; } - // /** - // * HeartBeat SOS messages carry the leader completion state in the header. This function extracts the leader - // completion - // * state from that header and updates the {@param partitionConsumptionState} accordingly. - // */ - // @Override - // protected void getAndUpdateLeaderCompletedState( - // KafkaKey kafkaKey, - // KafkaMessageEnvelope kafkaValue, - // ControlMessage controlMessage, - // PubSubMessageHeaders pubSubMessageHeaders, - // PartitionConsumptionState partitionConsumptionState) { - // if (isHybridFollower(partitionConsumptionState)) { - // ControlMessageType controlMessageType = ControlMessageType.valueOf(controlMessage); - // if (controlMessageType == ControlMessageType.START_OF_SEGMENT - // && Arrays.equals(kafkaKey.getKey(), KafkaKey.HEART_BEAT.getKey())) { - // LeaderCompleteState oldState = partitionConsumptionState.getLeaderCompleteState(); - // LeaderCompleteState newState = oldState; - // for (PubSubMessageHeader header: pubSubMessageHeaders) { - // if (header.key().equals(VENICE_LEADER_COMPLETION_STATE_HEADER)) { - // newState = LeaderCompleteState.valueOf(header.value()[0]); - // partitionConsumptionState - // .setLastLeaderCompleteStateUpdateInMs(kafkaValue.producerMetadata.messageTimestamp); - // break; // only interested in this header here - // } - // } - // - // if (oldState != newState) { - // LOGGER.info( - // "LeaderCompleteState for replica: {} changed from {} to {}", - // partitionConsumptionState.getReplicaId(), - // oldState, - // newState); - // partitionConsumptionState.setLeaderCompleteState(newState); - // } else { - // LOGGER.debug( - // "LeaderCompleteState for replica: {} received from leader: {} and is unchanged from the previous state", - // partitionConsumptionState.getReplicaId(), - // newState); - // } - // } - // } - // } - // - // /** - // * Leaders propagate HB SOS message from RT to local VT (to all subpartitions in case if amplification - // * Factor is configured to be more than 1) with updated LeaderCompleteState header: - // * Adding the headers during this phase instead of adding it to RT directly simplifies the logic - // * of how to identify the HB SOS from the correct version or whether the HB SOS is from the local - // * colo or remote colo, as the header inherited from an incorrect version or remote colos might - // * provide incorrect information about the leader state. - // */ - // @Override - // protected final void propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( - // PartitionConsumptionState partitionConsumptionState, - // PubSubMessage consumerRecord, - // LeaderProducedRecordContext leaderProducedRecordContext, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingRecordTimestampNs) { - // LeaderProducerCallback callback = createProducerCallback( - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // partition, - // kafkaUrl, - // beforeProcessingRecordTimestampNs); - // LeaderMetadataWrapper leaderMetadataWrapper = new LeaderMetadataWrapper(consumerRecord.getOffset(), - // kafkaClusterId); - // LeaderCompleteState leaderCompleteState = - // LeaderCompleteState.getLeaderCompleteState(partitionConsumptionState.isCompletionReported()); - // /** - // * The maximum value between the original producer timestamp and the timestamp when the message is added to the RT - // topic is used: - // * This approach addresses scenarios wrt clock drift where the producer's timestamp is consistently delayed by - // several minutes, - // * causing it not to align with the {@link - // com.linkedin.davinci.config.VeniceServerConfig#leaderCompleteStateCheckValidIntervalMs} - // * interval. The likelihood of simultaneous significant time discrepancies between the leader (producer) and the RT - // should be very - // * rare, making this a viable workaround. In cases where the time discrepancy is reversed, the follower may complete - // slightly earlier - // * than expected. However, this should not pose a significant issue as the completion of the leader, indicated by - // the leader - // * completed header, is a prerequisite for the follower completion and is expected to occur shortly thereafter. - // */ - // long producerTimeStamp = - // max(consumerRecord.getPubSubMessageTime(), consumerRecord.getValue().producerMetadata.messageTimestamp); - // PubSubTopicPartition topicPartition = - // new PubSubTopicPartitionImpl(getVersionTopic(), partitionConsumptionState.getPartition()); - // sendIngestionHeartbeatToVT( - // partitionConsumptionState, - // topicPartition, - // callback, - // leaderMetadataWrapper, - // leaderCompleteState, - // producerTimeStamp); - // } - @Override protected void recordHeartbeatReceived( PartitionConsumptionState partitionConsumptionState, @@ -2264,460 +2012,6 @@ protected void recordHeartbeatReceived( } } - // @Override - // public Iterable> - // validateAndFilterOutDuplicateMessagesFromLeaderTopic( - // Iterable> records, - // String kafkaUrl, - // PubSubTopicPartition topicPartition) { - // PartitionConsumptionState partitionConsumptionState = - // partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); - // if (partitionConsumptionState == null) { - // // The partition is likely unsubscribed, will skip these messages. - // LOGGER.warn( - // "No partition consumption state for store version: {}, partition:{}, will filter out all the messages", - // kafkaVersionTopic, - // topicPartition.getPartitionNumber()); - // return Collections.emptyList(); - // } - // boolean isEndOfPushReceived = partitionConsumptionState.isEndOfPushReceived(); - // if (!shouldProduceToVersionTopic(partitionConsumptionState)) { - // return records; - // } - // /** - // * Just to note this code is getting executed in Leader only. Leader DIV check progress is always ahead of the - // * actual data persisted on disk. Leader DIV check results will not be persisted on disk. - // */ - // Iterator> iter = records.iterator(); - // while (iter.hasNext()) { - // PubSubMessage record = iter.next(); - // boolean isRealTimeMsg = record.getTopicPartition().getPubSubTopic().isRealTime(); - // try { - // /** - // * TODO: An improvement can be made to fail all future versions for fatal DIV exceptions after EOP. - // */ - // if (!isGlobalRtDivEnabled) { - // validateMessage( - // PartitionTracker.VERSION_TOPIC, - // this.kafkaDataIntegrityValidatorForLeaders, - // record, - // isEndOfPushReceived, - // partitionConsumptionState); - // } else { - // validateMessage( - // PartitionTracker.TopicType.of( - // isRealTimeMsg - // ? PartitionTracker.TopicType.REALTIME_TOPIC_TYPE - // : PartitionTracker.TopicType.VERSION_TOPIC_TYPE, - // kafkaUrl), - // this.kafkaDataIntegrityValidatorForLeaders, - // record, - // isEndOfPushReceived, - // partitionConsumptionState); - // } - // versionedDIVStats.recordSuccessMsg(storeName, versionNumber); - // } catch (FatalDataValidationException e) { - // if (!isEndOfPushReceived) { - // throw e; - // } - // } catch (DuplicateDataException e) { - // /** - // * Skip duplicated messages; leader must not produce duplicated messages from RT to VT, because leader will - // * override the DIV info for messages from RT; as a result, both leaders and followers will persisted duplicated - // * messages to disk, and potentially rewind a k/v pair to an old value. - // */ - // divErrorMetricCallback.accept(e); - // LOGGER.debug( - // "Skipping a duplicate record from: {} offset: {} for replica: {}", - // record.getTopicPartition(), - // record.getOffset(), - // partitionConsumptionState.getReplicaId()); - // iter.remove(); - // } - // } - // return records; - // } - // - // /** - // * The goal of this function is to possibly produce the incoming kafka message consumed from local VT, remote VT, RT - // or SR topic to - // * local VT if needed. It's decided based on the function output of {@link #shouldProduceToVersionTopic} and message - // type. - // * It also perform any necessary additional computation operation such as for write-compute/update message. - // * It returns a boolean indicating if it was produced to kafka or not. - // * - // * This function should be called as one of the first steps in processing pipeline for all messages consumed from - // any kafka topic. - // * - // * The caller of this function should only process this {@param consumerRecord} further if the return is - // * {@link DelegateConsumerRecordResult#QUEUED_TO_DRAINER}. - // * - // * This function assumes {@link #shouldProcessRecord(PubSubMessage)} has been called which happens in - // * {@link StorePartitionDataReceiver#produceToStoreBufferServiceOrKafka(Iterable, PubSubTopicPartition, String, - // int)} - // * before calling this and the it was decided that this record needs to be processed. It does not perform any - // * validation check on the PartitionConsumptionState object to keep the goal of the function simple and not - // overload. - // * - // * Also DIV validation is done here if the message is received from RT topic. For more info please see - // * please see {@literal StoreIngestionTask#internalProcessConsumerRecord(PubSubMessage, PartitionConsumptionState, - // LeaderProducedRecordContext, int, String, long)} - // * - // * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after - // this function. - // * - // * @return a {@link DelegateConsumerRecordResult} indicating what to do with the record - // */ - // @Override - // protected DelegateConsumerRecordResult delegateConsumerRecord( - // PubSubMessageProcessedResultWrapper consumerRecordWrapper, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingPerRecordTimestampNs, - // long beforeProcessingBatchRecordsTimestampMs) { - // PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); - // try { - // KafkaKey kafkaKey = consumerRecord.getKey(); - // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); - // /** - // * partitionConsumptionState must be in a valid state and no error reported. This is made sure by calling - // * {@link shouldProcessRecord} before processing any record. - // * - // * ^ This is no longer true because with shared consumer the partitionConsumptionState could have been removed - // * from unsubscribe action in the StoreIngestionTask thread. Today, when unsubscribing - // * {@link StoreIngestionTask.waitForAllMessageToBeProcessedFromTopicPartition} only ensure the buffer queue is - // * drained before unsubscribe. Records being processed by shared consumer may see invalid partitionConsumptionState. - // */ - // PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(partition); - // if (partitionConsumptionState == null) { - // // The partition is likely unsubscribed, will skip these messages. - // return DelegateConsumerRecordResult.SKIPPED_MESSAGE; - // } - // boolean produceToLocalKafka = shouldProduceToVersionTopic(partitionConsumptionState); - // // UPDATE message is only expected in LEADER which must be produced to kafka. - // MessageType msgType = MessageType.valueOf(kafkaValue); - // if (msgType == UPDATE && !produceToLocalKafka) { - // throw new VeniceMessageException( - // ingestionTaskName + " hasProducedToKafka: Received UPDATE message in non-leader for: " - // + consumerRecord.getTopicPartition() + " Offset " + consumerRecord.getOffset()); - // } else if (msgType == MessageType.CONTROL_MESSAGE) { - // ControlMessage controlMessage = (ControlMessage) kafkaValue.payloadUnion; - // getAndUpdateLeaderCompletedState( - // kafkaKey, - // kafkaValue, - // controlMessage, - // consumerRecord.getPubSubMessageHeaders(), - // partitionConsumptionState); - // } - // - // /** - // * return early if it needs not be produced to local VT such as cases like - // * (i) it's a follower or (ii) leader is consuming from VT - // */ - // if (!produceToLocalKafka) { - // /** - // * For the local consumption, the batch data won't be produce to the local VT again, so we will switch - // * to real-time writer upon EOP here and for the remote consumption of VT, the switch will be handled - // * in the following section as it needs to flush the messages and then switch. - // */ - // if (isLeader(partitionConsumptionState) && msgType == MessageType.CONTROL_MESSAGE - // && ControlMessageType.valueOf((ControlMessage) kafkaValue.payloadUnion).equals(END_OF_PUSH)) { - // LOGGER.info( - // "Switching to the VeniceWriter for real-time workload for topic: {}, partition: {}", - // getVersionTopic().getName(), - // partition); - // // Just to be extra safe - // partitionConsumptionState.getVeniceWriterLazyRef().ifPresent(vw -> vw.flush()); - // partitionConsumptionState.setVeniceWriterLazyRef(veniceWriterForRealTime); - // } - // /** - // * Materialized view need to produce to the corresponding view topic for the batch portion of the data. This is - // * achieved in the following ways: - // * 1. Remote fabric(s) will leverage NR where the leader will replicate VT from NR source fabric and produce - // * to local view topic(s). - // * 2. NR source fabric's view topic will be produced by VPJ. This is because there is no checkpointing and - // * easy way to add checkpointing for leaders consuming the local VT. Making it difficult and error prone if - // * we let the leader produce to view topic(s) in NR source fabric. - // */ - // return DelegateConsumerRecordResult.QUEUED_TO_DRAINER; - // } - // - // // If we are here the message must be produced to local kafka or silently consumed. - // LeaderProducedRecordContext leaderProducedRecordContext; - // // No need to resolve cluster id and kafka url because sep topics are real time topic and it's not VT - // validateRecordBeforeProducingToLocalKafka(consumerRecord, partitionConsumptionState, kafkaUrl, kafkaClusterId); - // - // if (consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { - // recordRegionHybridConsumptionStats( - // kafkaClusterId, - // consumerRecord.getPayloadSize(), - // consumerRecord.getOffset(), - // beforeProcessingBatchRecordsTimestampMs); - // updateLatestInMemoryLeaderConsumedRTOffset(partitionConsumptionState, kafkaUrl, consumerRecord.getOffset()); - // } - // - // // heavy leader processing starts here - // versionedIngestionStats.recordLeaderPreprocessingLatency( - // storeName, - // versionNumber, - // LatencyUtils.getElapsedTimeFromNSToMS(beforeProcessingPerRecordTimestampNs), - // beforeProcessingBatchRecordsTimestampMs); - // - // if (kafkaKey.isControlMessage()) { - // boolean producedFinally = true; - // ControlMessage controlMessage = (ControlMessage) kafkaValue.getPayloadUnion(); - // ControlMessageType controlMessageType = ControlMessageType.valueOf(controlMessage); - // leaderProducedRecordContext = LeaderProducedRecordContext - // .newControlMessageRecord(kafkaClusterId, consumerRecord.getOffset(), kafkaKey.getKey(), controlMessage); - // switch (controlMessageType) { - // case START_OF_PUSH: - // /** - // * N.B.: This is expected to be the first time we call {@link veniceWriter#get()}. During this time - // * since startOfPush has not been processed yet, {@link StoreVersionState} is not created yet (unless - // * this is a server restart scenario). So the {@link com.linkedin.venice.writer.VeniceWriter#isChunkingEnabled} - // field - // * will not be set correctly at this point. This is fine as chunking is mostly not applicable for control messages. - // * This chunking flag for the veniceWriter will happen be set correctly in - // * {@link StoreIngestionTask#processStartOfPush(ControlMessage, int, long, PartitionConsumptionState)}, - // * which will be called when this message gets processed in drainer thread after successfully producing - // * to kafka. - // * - // * Note update: the first time we call {@link veniceWriter#get()} is different in various use cases: - // * 1. For hybrid store with L/F enabled, the first time a VeniceWriter is created is after leader switches to RT and - // * consumes the first message; potential message type: SOS, EOS, data message. - // * 2. For store version generated by stream reprocessing push type, the first time is after leader switches to - // * SR topic and consumes the first message; potential message type: SOS, EOS, data message (consider server - // restart). - // * 3. For store with native replication enabled, the first time is after leader switches to remote topic and start - // * consumes the first message; potential message type: SOS, EOS, SOP, EOP, data message (consider server restart). - // */ - // case END_OF_PUSH: - // // CMs that are produced with DIV pass-through mode can break DIV without synchronization with view writers. - // // This is because for data (PUT) records we queue their produceToLocalKafka behind the completion of view - // // writers. The main SIT will move on to subsequent messages and for CMs that don't need to be propagated - // // to view topics we are producing them directly. If we don't check the previous write before producing the - // // CMs then in the VT we might get out of order messages and with pass-through DIV that's going to be an - // // issue. e.g. a PUT record belonging to seg:0 can come after the EOS of seg:0 due to view writer delays. - // // Since SOP and EOP are rare we can simply wait for the last VT produce future. - // checkAndWaitForLastVTProduceFuture(partitionConsumptionState); - // /** - // * Simply produce this EOP to local VT. It will be processed in order in the drainer queue later - // * after successfully producing to kafka. - // */ - // produceToLocalKafka( - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() - // .get() - // .put( - // consumerRecord.getKey(), - // consumerRecord.getValue(), - // callback, - // consumerRecord.getTopicPartition().getPartitionNumber(), - // leaderMetadataWrapper), - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs); - // partitionConsumptionState.getVeniceWriterLazyRef().get().flush(); - // // Switch the writer for real-time workload - // LOGGER.info( - // "Switching to the VeniceWriter for real-time workload for topic: {}, partition: {}", - // getVersionTopic().getName(), - // partition); - // partitionConsumptionState.setVeniceWriterLazyRef(veniceWriterForRealTime); - // break; - // case START_OF_SEGMENT: - // case END_OF_SEGMENT: - // /** - // * SOS and EOS will be produced to the local version topic with DIV pass-through mode by leader in the following - // cases: - // * 1. SOS and EOS are from stream-reprocessing topics (use cases: stream-reprocessing) - // * 2. SOS and EOS are from version topics in a remote fabric (use cases: native replication for remote fabrics) - // * - // * SOS and EOS will not be produced to local version topic in the following cases: - // * 1. SOS and EOS are from real-time topics (use cases: hybrid ingestion, incremental push to RT) - // * 2. SOS and EOS are from version topics in local fabric, which has 2 different scenarios: - // * i. native replication is enabled, but the current fabric is the source fabric (use cases: native repl for source - // fabric) - // * ii. native replication is not enabled; it doesn't matter whether current replica is leader or follower, - // * messages from local VT doesn't need to be reproduced into local VT again (use case: local batch consumption) - // * - // * There is one exception that overrules the above conditions. i.e. if the SOS is a heartbeat from the RT topic. - // * In such case the heartbeat is produced to VT with updated {@link LeaderMetadataWrapper}. - // * - // * We want to ensure correct ordering for any SOS and EOS that we do decide to write to VT. This is done by - // * coordinating with the corresponding {@link PartitionConsumptionState#getLastVTProduceCallFuture}. - // * However, this coordination is only needed if there are view writers. i.e. the VT writes and CM writes - // * need to be in the same mode. Either both coordinate with lastVTProduceCallFuture or neither. - // */ - // if (!consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { - // final LeaderProducedRecordContext segmentCMLeaderProduceRecordContext = leaderProducedRecordContext; - // maybeQueueCMWritesToVersionTopic( - // partitionConsumptionState, - // () -> produceToLocalKafka( - // consumerRecord, - // partitionConsumptionState, - // segmentCMLeaderProduceRecordContext, - // (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() - // .get() - // .put( - // consumerRecord.getKey(), - // consumerRecord.getValue(), - // callback, - // consumerRecord.getTopicPartition().getPartitionNumber(), - // leaderMetadataWrapper), - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs)); - // } else { - // if (controlMessageType == START_OF_SEGMENT - // && Arrays.equals(consumerRecord.getKey().getKey(), KafkaKey.HEART_BEAT.getKey())) { - // final LeaderProducedRecordContext heartbeatLeaderProducedRecordContext = leaderProducedRecordContext; - // maybeQueueCMWritesToVersionTopic( - // partitionConsumptionState, - // () -> propagateHeartbeatFromUpstreamTopicToLocalVersionTopic( - // partitionConsumptionState, - // consumerRecord, - // heartbeatLeaderProducedRecordContext, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs)); - // } else { - // /** - // * Based on current design handling this case (specially EOS) is tricky as we don't produce the SOS/EOS - // * received from RT to local VT. But ideally EOS must be queued in-order (after all previous data message - // * PUT/UPDATE/DELETE) to drainer. But since the queueing of data message to drainer - // * happens in Kafka producer callback there is no way to queue this EOS to drainer in-order. - // * - // * Usually following processing in Leader for other control message. - // * 1. DIV: - // * 2. updateOffset: - // * 3. stats maintenance as in {@link StoreIngestionTask#processKafkaDataMessage(PubSubMessage, - // PartitionConsumptionState, LeaderProducedRecordContext)} - // * - // * For #1 Since we have moved the DIV validation in this function, We are good with DIV part which is the most - // critical one. - // * For #2 Leader will not update the offset for SOS/EOS. From Server restart point of view this is tolerable. This - // was the case in previous design also. So there is no change in behaviour. - // * For #3 stat counter update will not happen for SOS/EOS message. This should not be a big issue. If needed we can - // copy some of the stats maintenance - // * work here. - // * - // * So in summary NO further processing is needed SOS/EOS received from RT topics. Just silently drop the message - // here. - // * We should not return false here. - // */ - // producedFinally = false; - // } - // } - // break; - // case START_OF_INCREMENTAL_PUSH: - // case END_OF_INCREMENTAL_PUSH: - // // For inc push to RT policy, the control msg is written in RT topic, we will need to calculate the - // // destination partition in VT correctly. - // int versionTopicPartitionToBeProduced = consumerRecord.getTopicPartition().getPartitionNumber(); - // /** - // * We are using {@link VeniceWriter#asyncSendControlMessage()} api instead of {@link VeniceWriter#put()} since we - // have - // * to calculate DIV for this message but keeping the ControlMessage content unchanged. {@link VeniceWriter#put()} - // does not - // * allow that. - // */ - // produceToLocalKafka( - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() - // .get() - // .asyncSendControlMessage( - // controlMessage, - // versionTopicPartitionToBeProduced, - // new HashMap<>(), - // callback, - // leaderMetadataWrapper), - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs); - // break; - // case TOPIC_SWITCH: - // /** - // * For TOPIC_SWITCH message we should use -1 as consumedOffset. This will ensure that it does not update the - // * setLeaderUpstreamOffset in: - // * {@link #updateOffsetsAsRemoteConsumeLeader(PartitionConsumptionState, LeaderProducedRecordContext, String, - // PubSubMessage, UpdateVersionTopicOffset, UpdateUpstreamTopicOffset)} - // * The leaderUpstreamOffset is set from the TS message config itself. We should not override it. - // */ - // if (isDataRecovery && !partitionConsumptionState.isBatchOnly()) { - // // Ignore remote VT's TS message since we might need to consume more RT or incremental push data from VT - // // that's no longer in the local/remote RT due to retention. - // return DelegateConsumerRecordResult.SKIPPED_MESSAGE; - // } - // leaderProducedRecordContext = - // LeaderProducedRecordContext.newControlMessageRecord(kafkaKey.getKey(), controlMessage); - // produceToLocalKafka( - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() - // .get() - // .asyncSendControlMessage( - // controlMessage, - // consumerRecord.getTopicPartition().getPartitionNumber(), - // new HashMap<>(), - // callback, - // DEFAULT_LEADER_METADATA_WRAPPER), - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs); - // break; - // case VERSION_SWAP: - // return DelegateConsumerRecordResult.QUEUED_TO_DRAINER; - // default: - // // do nothing - // break; - // } - // if (!isSegmentControlMsg(controlMessageType)) { - // LOGGER.info( - // "Replica: {} hasProducedToKafka: {}; ControlMessage: {}; Incoming record topic-partition: {}; offset: {}", - // partitionConsumptionState.getReplicaId(), - // producedFinally, - // controlMessageType.name(), - // consumerRecord.getTopicPartition(), - // consumerRecord.getOffset()); - // } - // } else if (kafkaValue == null) { - // throw new VeniceMessageException( - // partitionConsumptionState.getReplicaId() - // + " hasProducedToKafka: Given null Venice Message. TopicPartition: " - // + consumerRecord.getTopicPartition() + " Offset " + consumerRecord.getOffset()); - // } else { - // // This function may modify the original record in KME and it is unsafe to use the payload from KME directly - // // after this call. - // processMessageAndMaybeProduceToKafka( - // consumerRecordWrapper, - // partitionConsumptionState, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs, - // beforeProcessingBatchRecordsTimestampMs); - // } - // return DelegateConsumerRecordResult.PRODUCED_TO_KAFKA; - // } catch (Exception e) { - // throw new VeniceException( - // ingestionTaskName + " hasProducedToKafka: exception for message received from: " - // + consumerRecord.getTopicPartition() + ", Offset: " + consumerRecord.getOffset() + ". Bubbling up.", - // e); - // } - // } - /** * Besides draining messages in the drainer queue, wait for the last producer future. */ @@ -2802,42 +2096,6 @@ protected void waitForAllMessageToBeProcessedFromTopicPartition( } } - // /** - // * Checks before producing local version topic. - // * - // * Extend this function when there is new check needed. - // */ - // @Override - // protected void validateRecordBeforeProducingToLocalKafka( - // PubSubMessage consumerRecord, - // PartitionConsumptionState partitionConsumptionState, - // String kafkaUrl, - // int kafkaClusterId) { - // // Check whether the message is from local version topic; leader shouldn't consume from local VT and then produce - // // back to VT again - // // localKafkaClusterId will always be the regular one without "_sep" suffix so kafkaClusterId should be converted - // // for comparison. Like-wise for the kafkaUrl. - // if (kafkaClusterId == localKafkaClusterId - // && consumerRecord.getTopicPartition().getPubSubTopic().equals(this.versionTopic) - // && kafkaUrl.equals(this.localKafkaServer)) { - // // N.B.: Ideally, the first two conditions should be sufficient, but for some reasons, in certain tests, the - // // third condition also ends up being necessary. In any case, doing the cluster ID check should be a - // // fast short-circuit in normal cases. - // try { - // int partitionId = partitionConsumptionState.getPartition(); - // setIngestionException( - // partitionId, - // new VeniceException( - // "Store version " + this.kafkaVersionTopic + " partition " + partitionId - // + " is consuming from local version topic and producing back to local version topic" - // + ", kafkaClusterId = " + kafkaClusterId + ", kafkaUrl = " + kafkaUrl + ", this.localKafkaServer = " - // + this.localKafkaServer)); - // } catch (VeniceException offerToQueueException) { - // setLastStoreIngestionException(offerToQueueException); - // } - // } - // } - // calculate the replication once per partition, checking Leader instance will make sure we calculate it just once // per partition. private static final Predicate BATCH_REPLICATION_LAG_FILTER = @@ -3144,455 +2402,6 @@ public void updateLeaderTopicOnFollower(PartitionConsumptionState partitionConsu } } - // /** - // * Compresses data in a bytebuffer when consuming from rt as a leader node and compression is enabled for the store - // * version for which we're consuming data. - // * - // * @param partition which partition we're acting on so as to determine the PartitionConsumptionState - // * @param data the data that we might compress - // * @return a bytebuffer that's either the original bytebuffer or a new one depending on if we compressed it. - // */ - // @Override - // protected ByteBuffer maybeCompressData( - // int partition, - // ByteBuffer data, - // PartitionConsumptionState partitionConsumptionState) { - // // To handle delete operations - // if (data == null) { - // return null; - // } - // if (shouldCompressData(partitionConsumptionState)) { - // try { - // long startTimeInNS = System.nanoTime(); - // // We need to expand the front of the returned bytebuffer to make room for schema header insertion - // ByteBuffer result = compressor.get().compress(data, ByteUtils.SIZE_OF_INT); - // hostLevelIngestionStats.recordLeaderCompressLatency(LatencyUtils.getElapsedTimeFromNSToMS(startTimeInNS)); - // return result; - // } catch (IOException e) { - // // throw a loud exception if something goes wrong here - // throw new RuntimeException( - // String.format( - // "Failed to compress value in venice writer! Aborting write! partition: %d, leader topic: %s, compressor: %s", - // partition, - // partitionConsumptionState.getOffsetRecord().getLeaderTopic(pubSubTopicRepository), - // compressor.getClass().getName()), - // e); - // } - // } - // return data; - // } - // - // protected boolean shouldCompressData(PartitionConsumptionState partitionConsumptionState) { - // if (!isLeader(partitionConsumptionState)) { - // return false; // Not leader, don't compress - // } - // PubSubTopic leaderTopic = partitionConsumptionState.getOffsetRecord().getLeaderTopic(pubSubTopicRepository); - // if (!realTimeTopic.equals(leaderTopic)) { - // return false; // We're consuming from version topic (don't compress it) - // } - // return !compressionStrategy.equals(CompressionStrategy.NO_OP); - // } - // - // @Override - // protected PubSubMessageProcessedResult processMessage( - // PubSubMessage consumerRecord, - // PartitionConsumptionState partitionConsumptionState, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingRecordTimestampNs, - // long beforeProcessingBatchRecordsTimestampMs) { - // KafkaKey kafkaKey = consumerRecord.getKey(); - // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); - // byte[] keyBytes = kafkaKey.getKey(); - // MessageType msgType = MessageType.valueOf(kafkaValue.messageType); - // switch (msgType) { - // case PUT: - // Put put = (Put) kafkaValue.payloadUnion; - // put.putValue = maybeCompressData( - // consumerRecord.getTopicPartition().getPartitionNumber(), - // put.putValue, - // partitionConsumptionState); - // ByteBuffer putValue = put.putValue; - // - // /** - // * For WC enabled stores update the transient record map with the latest {key,value}. This is needed only for - // messages - // * received from RT. Messages received from VT have been persisted to disk already before switching to RT topic. - // */ - // if (isWriteComputationEnabled && partitionConsumptionState.isEndOfPushReceived()) { - // partitionConsumptionState.setTransientRecord( - // kafkaClusterId, - // consumerRecord.getOffset(), - // keyBytes, - // putValue.array(), - // putValue.position(), - // putValue.remaining(), - // put.schemaId, - // null); - // } - // - // return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(put, null, false)); - // - // case UPDATE: - // /** - // * 1. Currently, we support chunking only for messages produced on VT topic during batch part of the ingestion - // * for hybrid stores. Chunking is NOT supported for messages produced to RT topics during streaming ingestion. - // * - // * So the assumption here is that the PUT/UPDATE messages stored in transientRecord should always be a full value - // * (non chunked). Decoding should succeed using the simplified API - // * {@link ChunkingAdapter#constructValue} - // * - // * 2. We always use the latest value schema to deserialize stored value bytes. - // * 3. We always use the partial update schema with an ID combination of the latest value schema ID + update schema - // ID - // * to deserialize the incoming Update request payload bytes. - // * - // * The reason for 2 and 3 is that we depend on the fact that the latest value schema must be a superset schema - // * that contains all value fields that ever existed in a store value schema. So, always using a superset schema - // * as the reader schema avoids data loss where the serialized bytes contain data for a field, however, the - // * deserialized record does not contain that field because the reader schema does not contain that field. - // */ - // Update update = (Update) kafkaValue.payloadUnion; - // final int readerValueSchemaId; - // final int readerUpdateProtocolVersion; - // if (isIngestingSystemStore()) { - // DerivedSchemaEntry latestDerivedSchemaEntry = schemaRepository.getLatestDerivedSchema(storeName); - // readerValueSchemaId = latestDerivedSchemaEntry.getValueSchemaID(); - // readerUpdateProtocolVersion = latestDerivedSchemaEntry.getId(); - // } else { - // SchemaEntry supersetSchemaEntry = schemaRepository.getSupersetSchema(storeName); - // if (supersetSchemaEntry == null) { - // throw new IllegalStateException("Cannot find superset schema for store: " + storeName); - // } - // readerValueSchemaId = supersetSchemaEntry.getId(); - // readerUpdateProtocolVersion = update.updateSchemaId; - // } - // ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); - // final GenericRecord currValue = readStoredValueRecord( - // partitionConsumptionState, - // keyBytes, - // readerValueSchemaId, - // consumerRecord.getTopicPartition(), - // valueManifestContainer); - // - // final byte[] updatedValueBytes; - // final ChunkedValueManifest oldValueManifest = valueManifestContainer.getManifest(); - // - // try { - // long writeComputeStartTimeInNS = System.nanoTime(); - // // Leader nodes are the only ones which process UPDATES, so it's valid to always compress and not call - // // 'maybeCompress'. - // updatedValueBytes = compressor.get() - // .compress( - // storeWriteComputeHandler.applyWriteCompute( - // currValue, - // update.schemaId, - // readerValueSchemaId, - // update.updateValue, - // update.updateSchemaId, - // readerUpdateProtocolVersion)); - // hostLevelIngestionStats - // .recordWriteComputeUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(writeComputeStartTimeInNS)); - // } catch (Exception e) { - // writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_UPDATE_FAILURE.code; - // throw new RuntimeException(e); - // } - // - // if (updatedValueBytes == null) { - // if (currValue != null) { - // throw new IllegalStateException( - // "Detect a situation where the current value exists and the Write Compute request" - // + "deletes the current value. It is unexpected because Write Compute only supports partial update and does " - // + "not support record value deletion."); - // } else { - // // No-op. The fact that currValue does not exist on the leader means currValue does not exist on the - // // follower - // // either. So, there is no need to tell the follower replica to do anything. - // return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, true)); - // } - // } else { - // partitionConsumptionState.setTransientRecord( - // kafkaClusterId, - // consumerRecord.getOffset(), - // keyBytes, - // updatedValueBytes, - // 0, - // updatedValueBytes.length, - // readerValueSchemaId, - // null); - // - // ByteBuffer updateValueWithSchemaId = - // ByteUtils.prependIntHeaderToByteBuffer(ByteBuffer.wrap(updatedValueBytes), readerValueSchemaId, false); - // - // Put updatedPut = new Put(); - // updatedPut.putValue = updateValueWithSchemaId; - // updatedPut.schemaId = readerValueSchemaId; - // return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(updatedPut, oldValueManifest, false)); - // } - // case DELETE: - // /** - // * For WC enabled stores update the transient record map with the latest {key,null} for similar reason as mentioned - // in PUT above. - // */ - // if (isWriteComputationEnabled && partitionConsumptionState.isEndOfPushReceived()) { - // partitionConsumptionState.setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, -1, null); - // } - // return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, false)); - // - // default: - // throw new VeniceMessageException( - // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); - // } - // } - // - // @Override - // protected void processMessageAndMaybeProduceToKafka( - // PubSubMessageProcessedResultWrapper consumerRecordWrapper, - // PartitionConsumptionState partitionConsumptionState, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingRecordTimestampNs, - // long beforeProcessingBatchRecordsTimestampMs) { - // PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); - // KafkaKey kafkaKey = consumerRecord.getKey(); - // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); - // byte[] keyBytes = kafkaKey.getKey(); - // MessageType msgType = MessageType.valueOf(kafkaValue.messageType); - // - // WriteComputeResultWrapper writeComputeResultWrapper; - // if (consumerRecordWrapper.getProcessedResult() != null - // && consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper() != null) { - // writeComputeResultWrapper = consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper(); - // } else { - // writeComputeResultWrapper = processMessage( - // consumerRecord, - // partitionConsumptionState, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs, - // beforeProcessingBatchRecordsTimestampMs).getWriteComputeResultWrapper(); - // } - // if (msgType.equals(UPDATE) && writeComputeResultWrapper.isSkipProduce()) { - // return; - // } - // Runnable produceToVersionTopic = () -> produceToLocalKafkaHelper( - // consumerRecord, - // partitionConsumptionState, - // writeComputeResultWrapper, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs); - // // Write to views - // if (hasViewWriters()) { - // Put newPut = writeComputeResultWrapper.getNewPut(); - // // keys will be serialized with chunk suffix during pass-through mode in L/F NR if chunking is enabled - // boolean isChunkedKey = isChunked() && !partitionConsumptionState.isEndOfPushReceived(); - // queueUpVersionTopicWritesWithViewWriters( - // partitionConsumptionState, - // (viewWriter) -> viewWriter.processRecord(newPut.putValue, keyBytes, newPut.schemaId, isChunkedKey), - // produceToVersionTopic); - // } else { - // produceToVersionTopic.run(); - // } - // } - // - // private void produceToLocalKafkaHelper( - // PubSubMessage consumerRecord, - // PartitionConsumptionState partitionConsumptionState, - // WriteComputeResultWrapper writeComputeResultWrapper, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingRecordTimestampNs) { - // KafkaKey kafkaKey = consumerRecord.getKey(); - // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); - // byte[] keyBytes = kafkaKey.getKey(); - // MessageType msgType = MessageType.valueOf(kafkaValue.messageType); - // LeaderProducedRecordContext leaderProducedRecordContext; - // Put newPut = writeComputeResultWrapper.getNewPut(); - // switch (msgType) { - // case PUT: - // leaderProducedRecordContext = - // LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); - // produceToLocalKafka( - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // (callback, leaderMetadataWrapper) -> { - // /** - // * 1. Unfortunately, Kafka does not support fancy array manipulation via {@link ByteBuffer} or otherwise, - // * so we may be forced to do a copy here, if the backing array of the {@link putValue} has padding, - // * which is the case when using {@link com.linkedin.venice.serialization.avro.OptimizedKafkaValueSerializer}. - // * Since this is in a closure, it is not guaranteed to be invoked. - // * - // * The {@link OnlineOfflineStoreIngestionTask}, which ignores this closure, will not pay this price. - // * - // * Conversely, the {@link LeaderFollowerStoreIngestionTask}, which does invoke it, will. - // * - // * TODO: Evaluate holistically what is the best way to optimize GC for the L/F case. - // * - // * 2. Enable venice writer "pass-through" mode if we haven't received EOP yet. In pass through mode, - // * Leader will reuse upstream producer metadata. This would secures the correctness of DIV states in - // * followers when the leadership failover happens. - // */ - // - // if (!partitionConsumptionState.isEndOfPushReceived()) { - // partitionConsumptionState.getVeniceWriterLazyRef() - // .get() - // .put( - // kafkaKey, - // kafkaValue, - // callback, - // consumerRecord.getTopicPartition().getPartitionNumber(), - // leaderMetadataWrapper); - // } else { - // partitionConsumptionState.getVeniceWriterLazyRef() - // .get() - // .put( - // keyBytes, - // ByteUtils.extractByteArray(newPut.putValue), - // newPut.schemaId, - // callback, - // leaderMetadataWrapper); - // } - // }, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs); - // break; - // - // case UPDATE: - // leaderProducedRecordContext = - // LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); - // BiConsumer produceFunction = - // (callback, leaderMetadataWrapper) -> partitionConsumptionState.getVeniceWriterLazyRef() - // .get() - // .put( - // keyBytes, - // ByteUtils.extractByteArray(newPut.getPutValue()), - // newPut.getSchemaId(), - // callback, - // leaderMetadataWrapper, - // APP_DEFAULT_LOGICAL_TS, - // null, - // writeComputeResultWrapper.getOldValueManifest(), - // null); - // - // produceToLocalKafka( - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // produceFunction, - // partitionConsumptionState.getPartition(), - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs); - // break; - // - // case DELETE: - // leaderProducedRecordContext = LeaderProducedRecordContext - // .newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, (Delete) kafkaValue.payloadUnion); - // produceToLocalKafka( - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // (callback, leaderMetadataWrapper) -> { - // /** - // * DIV pass-through for DELETE messages before EOP. - // */ - // if (!partitionConsumptionState.isEndOfPushReceived()) { - // partitionConsumptionState.getVeniceWriterLazyRef() - // .get() - // .delete( - // kafkaKey, - // kafkaValue, - // callback, - // consumerRecord.getTopicPartition().getPartitionNumber(), - // leaderMetadataWrapper); - // } else { - // partitionConsumptionState.getVeniceWriterLazyRef() - // .get() - // .delete(keyBytes, callback, leaderMetadataWrapper); - // } - // }, - // partition, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingRecordTimestampNs); - // break; - // - // default: - // throw new VeniceMessageException( - // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); - // } - // } - // - // /** - // * Read the existing value. If a value for this key is found from the transient map then use that value, otherwise - // read - // * it from the storage engine. - // * @return {@link Optional#empty} if the value - // */ - // protected GenericRecord readStoredValueRecord( - // PartitionConsumptionState partitionConsumptionState, - // byte[] keyBytes, - // int readerValueSchemaID, - // PubSubTopicPartition topicPartition, - // ChunkedValueManifestContainer manifestContainer) { - // final GenericRecord currValue; - // PartitionConsumptionState.TransientRecord transientRecord = partitionConsumptionState.getTransientRecord(keyBytes); - // if (transientRecord == null) { - // try { - // long lookupStartTimeInNS = System.nanoTime(); - // currValue = GenericRecordChunkingAdapter.INSTANCE.get( - // storageEngine, - // topicPartition.getPartitionNumber(), - // ByteBuffer.wrap(keyBytes), - // isChunked, - // null, - // null, - // NoOpReadResponseStats.SINGLETON, - // readerValueSchemaID, - // storeDeserializerCache, - // compressor.get(), - // manifestContainer); - // hostLevelIngestionStats - // .recordWriteComputeLookUpLatency(LatencyUtils.getElapsedTimeFromNSToMS(lookupStartTimeInNS)); - // } catch (Exception e) { - // writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_DESERIALIZATION_FAILURE.code; - // throw e; - // } - // } else { - // hostLevelIngestionStats.recordWriteComputeCacheHitCount(); - // // construct currValue from this transient record only if it's not null. - // if (transientRecord.getValue() != null) { - // try { - // currValue = GenericRecordChunkingAdapter.INSTANCE.constructValue( - // transientRecord.getValue(), - // transientRecord.getValueOffset(), - // transientRecord.getValueLen(), - // storeDeserializerCache.getDeserializer(transientRecord.getValueSchemaId(), readerValueSchemaID), - // compressor.get()); - // } catch (Exception e) { - // writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_DESERIALIZATION_FAILURE.code; - // throw e; - // } - // if (manifestContainer != null) { - // manifestContainer.setManifest(transientRecord.getValueManifest()); - // } - // - // } else { - // currValue = null; - // } - // } - // return currValue; - // } - /** * Clone DIV check results from OffsetRecord to the DIV validator that is used for leader consumption thread. * @@ -3630,10 +2439,6 @@ interface GetLastKnownUpstreamTopicOffset { long apply(String sourceKafkaUrl, PubSubTopic upstreamTopic); } - // private boolean isIngestingSystemStore() { - // return VeniceSystemStoreUtils.isSystemStore(storeName); - // } - /** * This method fetches/calculates latest leader persisted offset and last offset in RT topic. The method relies on * {@link #getLatestPersistedUpstreamOffsetForHybridOffsetLagMeasurement(PartitionConsumptionState, String)} to fetch @@ -3698,24 +2503,6 @@ protected void processControlMessageForViews( } } - // @Override - // protected LeaderProducerCallback createProducerCallback( - // PubSubMessage consumerRecord, - // PartitionConsumptionState partitionConsumptionState, - // LeaderProducedRecordContext leaderProducedRecordContext, - // int partition, - // String kafkaUrl, - // long beforeProcessingRecordTimestampNs) { - // return new LeaderProducerCallback( - // this, - // consumerRecord, - // partitionConsumptionState, - // leaderProducedRecordContext, - // partition, - // kafkaUrl, - // beforeProcessingRecordTimestampNs); - // } - protected Lazy> getVeniceWriter( PartitionConsumptionState partitionConsumptionState) { return partitionConsumptionState.getVeniceWriterLazyRef(); @@ -3765,24 +2552,6 @@ CompletableFuture sendIngestionHeartbeatToRT(PubSubTopicPar System.currentTimeMillis()); } - // private void sendIngestionHeartbeatToVT( - // PartitionConsumptionState partitionConsumptionState, - // PubSubTopicPartition topicPartition, - // PubSubProducerCallback callback, - // LeaderMetadataWrapper leaderMetadataWrapper, - // LeaderCompleteState leaderCompleteState, - // long originTimeStampMs) { - // sendIngestionHeartbeat( - // partitionConsumptionState, - // topicPartition, - // callback, - // leaderMetadataWrapper, - // true, - // true, - // leaderCompleteState, - // originTimeStampMs); - // } - @Override protected CompletableFuture sendIngestionHeartbeat( PartitionConsumptionState partitionConsumptionState, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 56104ff0ffb..3e0f1a7aaad 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -1136,276 +1136,6 @@ protected void produceToStoreBufferService( } } - // public abstract Iterable> - // validateAndFilterOutDuplicateMessagesFromLeaderTopic( - // Iterable> records, - // String kafkaUrl, - // PubSubTopicPartition topicPartition); - // - // public int handleSingleMessage( - // PubSubMessageProcessedResultWrapper consumerRecordWrapper, - // PubSubTopicPartition topicPartition, - // PartitionConsumptionState partitionConsumptionState, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingPerRecordTimestampNs, - // long beforeProcessingBatchRecordsTimestampMs, - // boolean metricsEnabled, - // ValueHolder elapsedTimeForPuttingIntoQueue) throws InterruptedException { - // PubSubMessage record = consumerRecordWrapper.getMessage(); - // if (record.getKey().isControlMessage()) { - // ControlMessage controlMessage = (ControlMessage) record.getValue().payloadUnion; - // if (ControlMessageType.valueOf(controlMessage.controlMessageType) == ControlMessageType.START_OF_PUSH) { - // /** - // * N.B.: The rest of the {@link ControlMessage} types are handled by: - // * {@link #processControlMessage(KafkaMessageEnvelope, ControlMessage, int, long, PartitionConsumptionState)} - // * - // * But for the SOP in particular, we want to process it here, at the start of the pipeline, to ensure that the - // * {@link StoreVersionState} is properly primed, as other functions below this point, but prior to being - // * enqueued into the {@link StoreBufferService} rely on this state to be there. - // */ - // processStartOfPush( - // record.getValue(), - // controlMessage, - // record.getTopicPartition().getPartitionNumber(), - // partitionConsumptionStateMap.get(topicPartition.getPartitionNumber())); - // } - // } - // - // // This function may modify the original record in KME and it is unsafe to use the payload from KME directly after - // // this call. - // DelegateConsumerRecordResult delegateConsumerRecordResult = delegateConsumerRecord( - // consumerRecordWrapper, - // topicPartition.getPartitionNumber(), - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs, - // beforeProcessingBatchRecordsTimestampMs); - // - // switch (delegateConsumerRecordResult) { - // case QUEUED_TO_DRAINER: - // long queuePutStartTimeInNS = metricsEnabled ? System.nanoTime() : 0; - // - // // blocking call - // storeBufferService.putConsumerRecord( - // record, - // this, - // null, - // topicPartition.getPartitionNumber(), - // kafkaUrl, - // beforeProcessingPerRecordTimestampNs); - // - // if (metricsEnabled) { - // elapsedTimeForPuttingIntoQueue.setValue( - // elapsedTimeForPuttingIntoQueue.getValue() + LatencyUtils.getElapsedTimeFromNSToMS(queuePutStartTimeInNS)); - // } - // break; - // case PRODUCED_TO_KAFKA: - // case SKIPPED_MESSAGE: - // break; - // default: - // throw new VeniceException( - // ingestionTaskName + " received unknown DelegateConsumerRecordResult enum for " - // + record.getTopicPartition()); - // } - // // Update the latest message consumed time - // partitionConsumptionState.setLatestMessageConsumedTimestampInMs(beforeProcessingBatchRecordsTimestampMs); - // - // return record.getPayloadSize(); - // } - - // /** - // * This function is in charge of producing the consumer records to the writer buffers maintained by {@link - // StoreBufferService}. - // * - // * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after - // this call. - // * - // * @param records : received consumer records - // * @param topicPartition - // * @throws InterruptedException - // */ - // protected void produceToStoreBufferServiceOrKafka( - // Iterable> records, - // PubSubTopicPartition topicPartition, - // String kafkaUrl, - // int kafkaClusterId) throws InterruptedException { - // PartitionConsumptionState partitionConsumptionState = - // partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); - // if (partitionConsumptionState == null) { - // throw new VeniceException( - // "PartitionConsumptionState should present for store version: " + kafkaVersionTopic + ", partition: " - // + topicPartition.getPartitionNumber()); - // } - // /** - // * Validate and filter out duplicate messages from the real-time topic as early as possible, so that - // * the following batch processing logic won't spend useless efforts on duplicate messages. - // */ - // records = validateAndFilterOutDuplicateMessagesFromLeaderTopic(records, kafkaUrl, topicPartition); - // - // if (shouldProduceInBatch(records)) { - // produceToStoreBufferServiceOrKafkaInBatch( - // records, - // topicPartition, - // partitionConsumptionState, - // kafkaUrl, - // kafkaClusterId); - // return; - // } - // - // long totalBytesRead = 0; - // ValueHolder elapsedTimeForPuttingIntoQueue = new ValueHolder<>(0d); - // boolean metricsEnabled = emitMetrics.get(); - // long beforeProcessingBatchRecordsTimestampMs = System.currentTimeMillis(); - // - // partitionConsumptionState = partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); - // for (PubSubMessage record: records) { - // long beforeProcessingPerRecordTimestampNs = System.nanoTime(); - // partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); - // if (!shouldProcessRecord(record)) { - // partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); - // continue; - // } - // - // // Check schema id availability before putting consumer record to drainer queue - // waitReadyToProcessRecord(record); - // - // totalBytesRead += handleSingleMessage( - // new PubSubMessageProcessedResultWrapper<>(record), - // topicPartition, - // partitionConsumptionState, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs, - // beforeProcessingBatchRecordsTimestampMs, - // metricsEnabled, - // elapsedTimeForPuttingIntoQueue); - // } - // - // /** - // * Even if the records list is empty, we still need to check quota to potentially resume partition - // */ - // storageUtilizationManager.enforcePartitionQuota(topicPartition.getPartitionNumber(), totalBytesRead); - // - // if (metricsEnabled) { - // if (totalBytesRead > 0) { - // hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); - // } - // if (elapsedTimeForPuttingIntoQueue.getValue() > 0) { - // hostLevelIngestionStats.recordConsumerRecordsQueuePutLatency( - // elapsedTimeForPuttingIntoQueue.getValue(), - // beforeProcessingBatchRecordsTimestampMs); - // } - // - // hostLevelIngestionStats.recordStorageQuotaUsed(storageUtilizationManager.getDiskQuotaUsage()); - // } - // } - // - // public boolean shouldProduceInBatch(Iterable> records) { - // return (isActiveActiveReplicationEnabled || isWriteComputationEnabled) - // && serverConfig.isAAWCWorkloadParallelProcessingEnabled() - // && IngestionBatchProcessor.isAllMessagesFromRTTopic(records); - // } - // - // public void produceToStoreBufferServiceOrKafkaInBatch( - // Iterable> records, - // PubSubTopicPartition topicPartition, - // PartitionConsumptionState partitionConsumptionState, - // String kafkaUrl, - // int kafkaClusterId) throws InterruptedException { - // long totalBytesRead = 0; - // ValueHolder elapsedTimeForPuttingIntoQueue = new ValueHolder<>(0d); - // boolean metricsEnabled = emitMetrics.get(); - // long beforeProcessingBatchRecordsTimestampMs = System.currentTimeMillis(); - // /** - // * Split the records into mini batches. - // */ - // int batchSize = serverConfig.getAAWCWorkloadParallelProcessingThreadPoolSize(); - // List>> batches = new ArrayList<>(); - // List> ongoingBatch = new ArrayList<>(batchSize); - // Iterator> iter = records.iterator(); - // while (iter.hasNext()) { - // PubSubMessage record = iter.next(); - // if (partitionConsumptionState != null) { - // partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); - // } - // if (!shouldProcessRecord(record)) { - // if (partitionConsumptionState != null) { - // partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); - // } - // continue; - // } - // waitReadyToProcessRecord(record); - // ongoingBatch.add(record); - // if (ongoingBatch.size() == batchSize) { - // batches.add(ongoingBatch); - // ongoingBatch = new ArrayList<>(batchSize); - // } - // } - // if (!ongoingBatch.isEmpty()) { - // batches.add(ongoingBatch); - // } - // if (batches.isEmpty()) { - // return; - // } - // IngestionBatchProcessor ingestionBatchProcessor = getIngestionBatchProcessor(); - // if (ingestionBatchProcessor == null) { - // throw new VeniceException( - // "IngestionBatchProcessor object should present for store version: " + kafkaVersionTopic); - // } - // /** - // * Process records batch by batch. - // */ - // for (List> batch: batches) { - // NavigableMap keyLockMap = ingestionBatchProcessor.lockKeys(batch); - // try { - // long beforeProcessingPerRecordTimestampNs = System.nanoTime(); - // List> processedResults = - // ingestionBatchProcessor.process( - // batch, - // partitionConsumptionState, - // topicPartition.getPartitionNumber(), - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs, - // beforeProcessingBatchRecordsTimestampMs); - // - // for (PubSubMessageProcessedResultWrapper processedRecord: processedResults) { - // totalBytesRead += handleSingleMessage( - // processedRecord, - // topicPartition, - // partitionConsumptionState, - // kafkaUrl, - // kafkaClusterId, - // beforeProcessingPerRecordTimestampNs, - // beforeProcessingBatchRecordsTimestampMs, - // metricsEnabled, - // elapsedTimeForPuttingIntoQueue); - // } - // } finally { - // ingestionBatchProcessor.unlockKeys(keyLockMap); - // } - // } - // - // /** - // * Even if the records list is empty, we still need to check quota to potentially resume partition - // */ - // storageUtilizationManager.enforcePartitionQuota(topicPartition.getPartitionNumber(), totalBytesRead); - // - // if (metricsEnabled) { - // if (totalBytesRead > 0) { - // hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); - // } - // if (elapsedTimeForPuttingIntoQueue.getValue() > 0) { - // hostLevelIngestionStats.recordConsumerRecordsQueuePutLatency( - // elapsedTimeForPuttingIntoQueue.getValue(), - // beforeProcessingBatchRecordsTimestampMs); - // } - // - // hostLevelIngestionStats.recordStorageQuotaUsed(storageUtilizationManager.getDiskQuotaUsage()); - // } - // } - // For testing purpose List getPartitionIngestionExceptionList() { return this.partitionIngestionExceptionList; @@ -2444,62 +2174,6 @@ public int getFailedIngestionPartitionCount() { return failedPartitions.size(); } - // /** - // * Common record check for different state models: - // * check whether server continues receiving messages after EOP for a batch-only store. - // */ - // protected boolean shouldProcessRecord(PubSubMessage record) { - // PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(record.getPartition()); - // - // if (partitionConsumptionState == null) { - // String msg = "PCS for replica: " + Utils.getReplicaId(kafkaVersionTopic, record.getPartition()) - // + " is null. Skipping incoming record with topic-partition: {} and offset: {}"; - // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { - // LOGGER.info(msg, record.getTopicPartition(), record.getOffset()); - // } - // return false; - // } - // - // if (partitionConsumptionState.isErrorReported()) { - // String msg = "Replica: " + partitionConsumptionState.getReplicaId() - // + " is already errored. Skipping incoming record with topic-partition: {} and offset: {}"; - // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { - // LOGGER.info(msg, record.getTopicPartition(), record.getOffset()); - // } - // return false; - // } - // - // if (partitionConsumptionState.isEndOfPushReceived() && partitionConsumptionState.isBatchOnly()) { - // KafkaKey key = record.getKey(); - // KafkaMessageEnvelope value = record.getValue(); - // if (key.isControlMessage() - // && ControlMessageType.valueOf((ControlMessage) value.payloadUnion) == ControlMessageType.END_OF_SEGMENT) { - // // Still allow END_OF_SEGMENT control message - // return true; - // } - // // emit metric for unexpected messages - // if (emitMetrics.get()) { - // hostLevelIngestionStats.recordUnexpectedMessage(); - // } - // - // // Report such kind of message once per minute to reduce logging volume - // /* - // * TODO: right now, if we update a store to enable hybrid, {@link StoreIngestionTask} for the existing versions - // * won't know it since {@link #hybridStoreConfig} parameter is passed during construction. - // * - // * So far, to make hybrid store/incremental store work, customer needs to do a new push after enabling hybrid/ - // * incremental push feature of the store. - // */ - // String message = "The record was received after 'EOP', but the store: " + kafkaVersionTopic - // + " is neither hybrid nor incremental push enabled, so will skip it. Current records replica: {}"; - // if (!REDUNDANT_LOGGING_FILTER.isRedundantException(message)) { - // LOGGER.warn(message, partitionConsumptionState.getReplicaId()); - // } - // return false; - // } - // return true; - // } - protected boolean shouldPersistRecord( PubSubMessage record, PartitionConsumptionState partitionConsumptionState) { @@ -3919,99 +3593,6 @@ private int processKafkaDataMessage( return keyLen + valueLen; } - // /** - // * This method checks whether the given record needs to be checked schema availability. Only PUT and UPDATE message - // * needs to #checkValueSchemaAvail - // * @param record - // */ - // public void waitReadyToProcessRecord(PubSubMessage record) - // throws InterruptedException { - // KafkaMessageEnvelope kafkaValue = record.getValue(); - // if (record.getKey().isControlMessage() || kafkaValue == null) { - // return; - // } - // - // switch (MessageType.valueOf(kafkaValue)) { - // case PUT: - // Put put = (Put) kafkaValue.payloadUnion; - // waitReadyToProcessDataRecord(put.schemaId); - // try { - // deserializeValue(put.schemaId, put.putValue, record); - // } catch (Exception e) { - // PartitionConsumptionState pcs = - // partitionConsumptionStateMap.get(record.getTopicPartition().getPartitionNumber()); - // LeaderFollowerStateType state = pcs == null ? null : pcs.getLeaderFollowerState(); - // throw new VeniceException( - // "Failed to deserialize PUT for: " + record.getTopicPartition() + ", offset: " + record.getOffset() - // + ", schema id: " + put.schemaId + ", LF state: " + state, - // e); - // } - // break; - // case UPDATE: - // Update update = (Update) kafkaValue.payloadUnion; - // waitReadyToProcessDataRecord(update.schemaId); - // break; - // case DELETE: - // /* we don't need to check schema availability for DELETE */ - // break; - // default: - // throw new VeniceMessageException( - // ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); - // } - // } - // - // /** - // * Check whether the given schema id is available for current store. - // * The function will bypass the check if schema id is -1 (VPJ job is still using it before we finishes the - // integration with schema registry). - // * Right now, this function is maintaining a local cache for schema id of current store considering that the value - // schema is immutable; - // * If the schema id is not available, this function will polling until the schema appears or timeout: {@link - // #SCHEMA_POLLING_TIMEOUT_MS}; - // * - // * @param schemaId - // */ - // public void waitReadyToProcessDataRecord(int schemaId) throws InterruptedException { - // if (schemaId == -1) { - // // TODO: Once Venice Client (VeniceShellClient) finish the integration with schema registry, - // // we need to remove this check here. - // return; - // } - // - // if (schemaId == AvroProtocolDefinition.CHUNK.getCurrentProtocolVersion() - // || schemaId == AvroProtocolDefinition.CHUNKED_VALUE_MANIFEST.getCurrentProtocolVersion()) { - // StoreVersionState storeVersionState = waitVersionStateAvailable(kafkaVersionTopic); - // if (!storeVersionState.chunked) { - // throw new VeniceException( - // "Detected chunking in a store-version where chunking is NOT enabled. Will abort ingestion."); - // } - // return; - // } - // - // waitUntilValueSchemaAvailable(schemaId); - // } - // - // protected StoreVersionState waitVersionStateAvailable(String kafkaTopic) throws InterruptedException { - // long startTime = System.currentTimeMillis(); - // long elapsedTime; - // StoreVersionState state; - // for (;;) { - // state = storageEngine.getStoreVersionState(); - // elapsedTime = System.currentTimeMillis() - startTime; - // - // if (state != null) { - // return state; - // } - // - // if (elapsedTime > SCHEMA_POLLING_TIMEOUT_MS || !isRunning()) { - // LOGGER.warn("Version state is not available for {} after {}", kafkaTopic, elapsedTime); - // throw new VeniceException("Store version state is not available for " + kafkaTopic); - // } - // - // Thread.sleep(SCHEMA_POLLING_DELAY_MS); - // } - // } - void waitUntilValueSchemaAvailable(int schemaId) throws InterruptedException { // Considering value schema is immutable for an existing store, we can cache it locally if (availableSchemaIds.get(schemaId) != null) { @@ -4437,37 +4018,6 @@ protected void waitForAllMessageToBeProcessedFromTopicPartition( storeBufferService.drainBufferedRecordsFromTopicPartition(topicPartition); } - // protected abstract DelegateConsumerRecordResult delegateConsumerRecord( - // PubSubMessageProcessedResultWrapper consumerRecordWrapper, - // int partition, - // String kafkaUrl, - // int kafkaClusterId, - // long beforeProcessingPerRecordTimestampNs, - // long beforeProcessingBatchRecordsTimestampMs); - // - // /** - // * This enum represents all potential results after calling {@link - // #delegateConsumerRecord(PubSubMessageProcessedResultWrapper, int, String, int, long, long)}. - // */ - // protected enum DelegateConsumerRecordResult { - // /** - // * The consumer record has been produced to local version topic by leader. - // */ - // PRODUCED_TO_KAFKA, - // /** - // * The consumer record has been put into drainer queue; the following cases will result in putting to drainer - // directly: - // * 1. Online/Offline ingestion task - // * 2. Follower replicas - // * 3. Leader is consuming from local version topics - // */ - // QUEUED_TO_DRAINER, - // /** - // * The consumer record is skipped. e.g. remote VT's TS message during data recovery. - // */ - // SKIPPED_MESSAGE - // } - /** * The method measures the time between receiving the message from the local VT and when the message is committed in * the local db and ready to serve. diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 7af53a2f4c0..78998e2a278 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -452,16 +452,6 @@ private boolean shouldProcessRecord(PubSubMessage Date: Sun, 5 Jan 2025 20:57:58 -0800 Subject: [PATCH 30/32] =?UTF-8?q?Changed=20the=20StoreIngestionTask=20prox?= =?UTF-8?q?y=20methods=20to=20be=20package-private.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 19 +++----- .../LeaderFollowerStoreIngestionTask.java | 24 ++++------ .../kafka/consumer/StoreIngestionTask.java | 48 ++++++++----------- 3 files changed, 37 insertions(+), 54 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index d5c18049181..5c7535c935c 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -3,9 +3,7 @@ import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.LEADER; import static com.linkedin.venice.VeniceConstants.REWIND_TIME_DECIDED_BY_SERVER; -import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.davinci.client.DaVinciRecordTransformerConfig; -import com.linkedin.davinci.client.DaVinciRecordTransformerFunctionalInterface; import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.config.VeniceStoreVersionConfig; import com.linkedin.davinci.replication.merge.MergeConflictResolver; @@ -226,11 +224,6 @@ private byte[] prependReplicationMetadataBytesWithValueSchemaId(ByteBuffer metad return replicationMetadataBytesWithValueSchemaId; } - @Override - RmdSerDe getRmdSerDe() { - return rmdSerDe; - } - @Override protected Map calculateLeaderUpstreamOffsetWithTopicSwitch( PartitionConsumptionState partitionConsumptionState, @@ -545,10 +538,7 @@ protected long getLatestConsumedUpstreamOffsetForHybridOffsetLagMeasurement( } @Override - protected void updateLatestInMemoryLeaderConsumedRTOffset( - PartitionConsumptionState pcs, - String kafkaUrl, - long offset) { + void updateLatestInMemoryLeaderConsumedRTOffset(PartitionConsumptionState pcs, String kafkaUrl, long offset) { pcs.updateLeaderConsumedUpstreamRTOffset(kafkaUrl, offset); } @@ -757,7 +747,7 @@ Runnable buildRepairTask( } @Override - public int getRmdProtocolVersionId() { + int getRmdProtocolVersionId() { return rmdProtocolVersionId; } @@ -766,6 +756,11 @@ MergeConflictResolver getMergeConflictResolver() { return mergeConflictResolver; } + @Override + RmdSerDe getRmdSerDe() { + return rmdSerDe; + } + /** * This method does a few things for leader topic-partition subscription: * (1) Calculate Kafka URL to leader subscribe offset map. diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index affb7df8d19..a8e4d044542 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -1213,7 +1213,7 @@ private void checkAndUpdateDataRecoveryStatusOfHybridStore(PartitionConsumptionS * If buffer replay is disable, all replicas will stick to version topic, no one is going to produce any message. */ @Override - protected boolean shouldProduceToVersionTopic(PartitionConsumptionState partitionConsumptionState) { + boolean shouldProduceToVersionTopic(PartitionConsumptionState partitionConsumptionState) { if (!isLeader(partitionConsumptionState)) { return false; // Not leader } @@ -2322,10 +2322,7 @@ protected long getLatestConsumedUpstreamOffsetForHybridOffsetLagMeasurement( } @Override - protected void updateLatestInMemoryLeaderConsumedRTOffset( - PartitionConsumptionState pcs, - String ignoredKafkaUrl, - long offset) { + void updateLatestInMemoryLeaderConsumedRTOffset(PartitionConsumptionState pcs, String ignoredKafkaUrl, long offset) { pcs.updateLeaderConsumedUpstreamRTOffset(OffsetRecord.NON_AA_REPLICATION_UPSTREAM_OFFSET_MAP_KEY, offset); } @@ -2503,12 +2500,11 @@ protected void processControlMessageForViews( } } - protected Lazy> getVeniceWriter( - PartitionConsumptionState partitionConsumptionState) { + Lazy> getVeniceWriter(PartitionConsumptionState partitionConsumptionState) { return partitionConsumptionState.getVeniceWriterLazyRef(); } - protected void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState) { + void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState) { partitionConsumptionState.setVeniceWriterLazyRef(veniceWriterForRealTime); } @@ -2553,7 +2549,7 @@ CompletableFuture sendIngestionHeartbeatToRT(PubSubTopicPar } @Override - protected CompletableFuture sendIngestionHeartbeat( + CompletableFuture sendIngestionHeartbeat( PartitionConsumptionState partitionConsumptionState, PubSubTopicPartition topicPartition, PubSubProducerCallback callback, @@ -2761,27 +2757,27 @@ protected boolean hasViewWriters() { } @Override - public KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders() { + KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders() { return kafkaDataIntegrityValidatorForLeaders; } @Override - public StoreWriteComputeProcessor getStoreWriteComputeHandler() { + StoreWriteComputeProcessor getStoreWriteComputeHandler() { return storeWriteComputeHandler; } @Override - public AvroStoreDeserializerCache getStoreDeserializerCache() { + AvroStoreDeserializerCache getStoreDeserializerCache() { return storeDeserializerCache; } @Override - public Int2ObjectMap getKafkaClusterIdToUrlMap() { + Int2ObjectMap getKafkaClusterIdToUrlMap() { return kafkaClusterIdToUrlMap; } @Override - public boolean hasChangeCaptureView() { + boolean hasChangeCaptureView() { return hasChangeCaptureView; } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 3e0f1a7aaad..eae8be9a908 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -2808,7 +2808,7 @@ protected void processControlMessageForViews( // NoOp } - protected abstract boolean shouldProduceToVersionTopic(PartitionConsumptionState partitionConsumptionState); + abstract boolean shouldProduceToVersionTopic(PartitionConsumptionState partitionConsumptionState); protected boolean processTopicSwitch( ControlMessage controlMessage, @@ -3776,7 +3776,7 @@ public PubSubTopic getVersionTopic() { return versionTopic; } - public PubSubTopic getRealTimeTopic() { + PubSubTopic getRealTimeTopic() { return realTimeTopic; } @@ -4246,7 +4246,7 @@ Lazy getGracefulShutdownLatch() { return gracefulShutdownLatch; } - public StorageUtilizationManager getStorageUtilizationManager() { + StorageUtilizationManager getStorageUtilizationManager() { return storageUtilizationManager; } @@ -4255,30 +4255,22 @@ void putConsumerRecord(PubSubMessage recor storeBufferService.putConsumerRecord(record, this, null, partition, url, t); } - public SparseConcurrentList getAvailableSchemaIds() { - return availableSchemaIds; - } - - public SparseConcurrentList getDeserializedSchemaIds() { - return deserializedSchemaIds; - } - - public boolean isGlobalRtDivEnabled() { + boolean isGlobalRtDivEnabled() { return isGlobalRtDivEnabled; } - public Consumer getDivErrorMetricCallback() { + Consumer getDivErrorMetricCallback() { return divErrorMetricCallback; } - public abstract KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders(); + abstract KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders(); - protected abstract void updateLatestInMemoryLeaderConsumedRTOffset( + abstract void updateLatestInMemoryLeaderConsumedRTOffset( PartitionConsumptionState pcs, String ignoredKafkaUrl, long offset); - protected abstract CompletableFuture sendIngestionHeartbeat( + abstract CompletableFuture sendIngestionHeartbeat( PartitionConsumptionState partitionConsumptionState, PubSubTopicPartition topicPartition, PubSubProducerCallback callback, @@ -4288,7 +4280,7 @@ protected abstract CompletableFuture sendIngestionHeartbeat LeaderCompleteState leaderCompleteState, long originTimeStampMs); - public int getRmdProtocolVersionId() { + int getRmdProtocolVersionId() { throw new VeniceException("getRmdProtocolVersionId() should only be called in active active mode"); } @@ -4300,18 +4292,18 @@ RmdSerDe getRmdSerDe() { throw new VeniceException("getRmdSerDe() should only be called in active active mode"); } - public abstract Int2ObjectMap getKafkaClusterIdToUrlMap(); + abstract Int2ObjectMap getKafkaClusterIdToUrlMap(); - public abstract boolean hasChangeCaptureView(); + abstract boolean hasChangeCaptureView(); - public abstract StoreWriteComputeProcessor getStoreWriteComputeHandler(); + abstract StoreWriteComputeProcessor getStoreWriteComputeHandler(); - public abstract AvroStoreDeserializerCache getStoreDeserializerCache(); + abstract AvroStoreDeserializerCache getStoreDeserializerCache(); - protected abstract Lazy> getVeniceWriter( + abstract Lazy> getVeniceWriter( PartitionConsumptionState partitionConsumptionState); - protected abstract void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState); + abstract void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState); protected abstract boolean hasViewWriters(); @@ -4319,23 +4311,23 @@ void setWriteComputeFailureCode(int code) { this.writeComputeFailureCode = code; } - public ExecutorService getParallelProcessingThreadPool() { + ExecutorService getParallelProcessingThreadPool() { return parallelProcessingThreadPool; } - public boolean isDataRecovery() { + boolean isDataRecovery() { return isDataRecovery; } - public String getLocalKafkaServer() { + String getLocalKafkaServer() { return localKafkaServer; } - public int getLocalKafkaClusterId() { + int getLocalKafkaClusterId() { return localKafkaClusterId; } - public int getStoreVersionPartitionCount() { + int getStoreVersionPartitionCount() { return storeVersionPartitionCount; } From c0f68536cfa875874d9c3bb2bf45343dc1ac1b38 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Sun, 5 Jan 2025 21:19:58 -0800 Subject: [PATCH 31/32] =?UTF-8?q?Minor=20tweaks=20to=20the=20proxy=20metho?= =?UTF-8?q?ds=20to=20help=20with=20abstraction.=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../LeaderFollowerStoreIngestionTask.java | 21 ++++++++++++--- .../kafka/consumer/StoreIngestionTask.java | 21 ++++++++------- .../consumer/StorePartitionDataReceiver.java | 27 +++++++++---------- 3 files changed, 41 insertions(+), 28 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index a8e4d044542..63b0d9f4944 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -83,6 +83,7 @@ import java.util.function.LongPredicate; import java.util.function.Predicate; import java.util.function.Supplier; +import org.apache.avro.generic.GenericRecord; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -2762,8 +2763,20 @@ KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders() { } @Override - StoreWriteComputeProcessor getStoreWriteComputeHandler() { - return storeWriteComputeHandler; + byte[] applyWriteCompute( + GenericRecord currValue, + int writerValueSchemaId, + int readerValueSchemaId, + ByteBuffer writeComputeBytes, + int writerUpdateProtocolVersion, + int readerUpdateProtocolVersion) { + return storeWriteComputeHandler.applyWriteCompute( + currValue, + writerValueSchemaId, + readerValueSchemaId, + writeComputeBytes, + writerUpdateProtocolVersion, + readerUpdateProtocolVersion); } @Override @@ -2772,8 +2785,8 @@ AvroStoreDeserializerCache getStoreDeserializerCache() { } @Override - Int2ObjectMap getKafkaClusterIdToUrlMap() { - return kafkaClusterIdToUrlMap; + String getKafkaUrl(int kafkaClusterId) { + return kafkaClusterIdToUrlMap.get(kafkaClusterId); } @Override diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index eae8be9a908..0c045ee83ee 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -124,7 +124,6 @@ import com.linkedin.venice.writer.LeaderCompleteState; import com.linkedin.venice.writer.LeaderMetadataWrapper; import com.linkedin.venice.writer.VeniceWriter; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; import java.io.Closeable; import java.io.IOException; @@ -162,6 +161,7 @@ import java.util.function.Supplier; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -4246,8 +4246,12 @@ Lazy getGracefulShutdownLatch() { return gracefulShutdownLatch; } - StorageUtilizationManager getStorageUtilizationManager() { - return storageUtilizationManager; + void enforcePartitionQuota(int partition, long totalBytesRead) { + storageUtilizationManager.enforcePartitionQuota(partition, totalBytesRead); + } + + double getDiskQuotaUsage() { + return storageUtilizationManager.getDiskQuotaUsage(); } void putConsumerRecord(PubSubMessage record, int partition, String url, long t) @@ -4259,8 +4263,8 @@ boolean isGlobalRtDivEnabled() { return isGlobalRtDivEnabled; } - Consumer getDivErrorMetricCallback() { - return divErrorMetricCallback; + void handleDivErrorMetric(DuplicateDataException e) { + divErrorMetricCallback.accept(e); } abstract KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders(); @@ -4292,16 +4296,15 @@ RmdSerDe getRmdSerDe() { throw new VeniceException("getRmdSerDe() should only be called in active active mode"); } - abstract Int2ObjectMap getKafkaClusterIdToUrlMap(); + abstract String getKafkaUrl(int kafkaClusterId); abstract boolean hasChangeCaptureView(); - abstract StoreWriteComputeProcessor getStoreWriteComputeHandler(); + abstract byte[] applyWriteCompute(GenericRecord value, int ws, int rs, ByteBuffer bytes, int wp, int rp); abstract AvroStoreDeserializerCache getStoreDeserializerCache(); - abstract Lazy> getVeniceWriter( - PartitionConsumptionState partitionConsumptionState); + abstract Lazy> getVeniceWriter(PartitionConsumptionState pcs); abstract void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 78998e2a278..362516ccf1e 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -501,8 +501,7 @@ private void updateMetricsAndEnforceQuota( /** * Even if the records list is empty, we still need to check quota to potentially resume partition */ - final StorageUtilizationManager storageUtilizationManager = storeIngestionTask.getStorageUtilizationManager(); - storageUtilizationManager.enforcePartitionQuota(partition, totalBytesRead); + storeIngestionTask.enforcePartitionQuota(partition, totalBytesRead); if (storeIngestionTask.isMetricsEmissionEnabled()) { HostLevelIngestionStats hostLevelIngestionStats = storeIngestionTask.getHostLevelIngestionStats(); @@ -515,7 +514,7 @@ private void updateMetricsAndEnforceQuota( beforeProcessingBatchRecordsTimestampMs); } - hostLevelIngestionStats.recordStorageQuotaUsed(storageUtilizationManager.getDiskQuotaUsage()); + hostLevelIngestionStats.recordStorageQuotaUsed(storeIngestionTask.getDiskQuotaUsage()); } } @@ -575,7 +574,7 @@ public Iterable> validateAnd * override the DIV info for messages from RT; as a result, both leaders and followers will persisted duplicated * messages to disk, and potentially rewind a k/v pair to an old value. */ - storeIngestionTask.getDivErrorMetricCallback().accept(e); + storeIngestionTask.handleDivErrorMetric(e); LOGGER.debug( "Skipping a duplicate record from: {} offset: {} for replica: {}", record.getTopicPartition(), @@ -1503,14 +1502,13 @@ private PubSubMessageProcessedResult processMessage( updatedValueBytes = storeIngestionTask.getCompressor() .get() .compress( - storeIngestionTask.getStoreWriteComputeHandler() - .applyWriteCompute( - currValue, - update.schemaId, - readerValueSchemaId, - update.updateValue, - update.updateSchemaId, - readerUpdateProtocolVersion)); + storeIngestionTask.applyWriteCompute( + currValue, + update.schemaId, + readerValueSchemaId, + update.updateValue, + update.updateSchemaId, + readerUpdateProtocolVersion)); storeIngestionTask.getHostLevelIngestionStats() .recordWriteComputeUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(writeComputeStartTimeInNS)); } catch (Exception e) { @@ -1804,9 +1802,8 @@ private PubSubMessageProcessedResult processActiveActiveMessage( if (mergeConflictResult.isUpdateIgnored()) { storeIngestionTask.getHostLevelIngestionStats().recordUpdateIgnoredDCR(); // Record the last ignored offset - partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset( - storeIngestionTask.getKafkaClusterIdToUrlMap().get(kafkaClusterId), - sourceOffset); + partitionConsumptionState + .updateLatestIgnoredUpstreamRTOffset(storeIngestionTask.getKafkaUrl(kafkaClusterId), sourceOffset); return new PubSubMessageProcessedResult( new MergeConflictResultWrapper( mergeConflictResult, From 283b6702e5ef5d4114636a14a6d7b045f21f2311 Mon Sep 17 00:00:00 2001 From: Kai-Sern Lim Date: Fri, 7 Feb 2025 21:16:23 -0800 Subject: [PATCH 32/32] =?UTF-8?q?Reverted=20`IngestionBatchProcessor`=20an?= =?UTF-8?q?d=20`KeyLevelLocksManager`=20back=20to=20`StoreIngestionTask`.?= =?UTF-8?q?=20=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ActiveActiveStoreIngestionTask.java | 36 ++++++++++++- .../consumer/IngestionBatchProcessor.java | 6 +-- .../LeaderFollowerStoreIngestionTask.java | 21 ++++++++ .../kafka/consumer/StoreIngestionTask.java | 14 +++-- .../consumer/StorePartitionDataReceiver.java | 50 +++-------------- .../consumer/IngestionBatchProcessorTest.java | 53 +++++++++---------- .../consumer/StoreIngestionTaskTest.java | 12 ++--- 7 files changed, 102 insertions(+), 90 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index 5c7535c935c..7e7f0904feb 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -59,7 +59,9 @@ public class ActiveActiveStoreIngestionTask extends LeaderFollowerStoreIngestion private final int rmdProtocolVersionId; private final MergeConflictResolver mergeConflictResolver; private final RmdSerDe rmdSerDe; + private final Lazy keyLevelLocksManager; private final RemoteIngestionRepairService remoteIngestionRepairService; + private final Lazy ingestionBatchProcessorLazy; public ActiveActiveStoreIngestionTask( StorageService storageService, @@ -90,6 +92,14 @@ public ActiveActiveStoreIngestionTask( this.rmdProtocolVersionId = version.getRmdVersionId(); + int knownKafkaClusterNumber = serverConfig.getKafkaClusterIdToUrlMap().size(); + + int initialPoolSize = knownKafkaClusterNumber + 1; + this.keyLevelLocksManager = Lazy.of( + () -> new KeyLevelLocksManager( + getVersionTopic().getName(), + initialPoolSize, + getKeyLevelLockMaxPoolSizeBasedOnServerConfig(serverConfig, storeVersionPartitionCount))); StringAnnotatedStoreSchemaCache annotatedReadOnlySchemaRepository = new StringAnnotatedStoreSchemaCache(storeName, schemaRepository); @@ -105,6 +115,21 @@ public ActiveActiveStoreIngestionTask( isWriteComputationEnabled, getServerConfig().isComputeFastAvroEnabled()); this.remoteIngestionRepairService = builder.getRemoteIngestionRepairService(); + this.ingestionBatchProcessorLazy = Lazy.of(() -> { + if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { + LOGGER.info("AA/WC workload parallel processing is disabled for store version: {}", getKafkaVersionTopic()); + return null; + } + LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", getKafkaVersionTopic()); + return new IngestionBatchProcessor( + kafkaVersionTopic, + parallelProcessingThreadPool, + keyLevelLocksManager.get(), + isWriteComputationEnabled, + isActiveActiveReplicationEnabled(), + versionedIngestionStats, + getHostLevelIngestionStats()); + }); } public static int getKeyLevelLockMaxPoolSizeBasedOnServerConfig(VeniceServerConfig serverConfig, int partitionCount) { @@ -131,7 +156,6 @@ public static int getKeyLevelLockMaxPoolSizeBasedOnServerConfig(VeniceServerConf @Override protected void putInStorageEngine(int partition, byte[] keyBytes, Put put) { try { - // TODO: Honor BatchConflictResolutionPolicy and maybe persist RMD for batch push records. StorageOperationType storageOperationType = getStorageOperationType(partition, put.putValue, put.replicationMetadataPayload); @@ -224,6 +248,16 @@ private byte[] prependReplicationMetadataBytesWithValueSchemaId(ByteBuffer metad return replicationMetadataBytesWithValueSchemaId; } + @Override + KeyLevelLocksManager getKeyLevelLocksManager() { + return keyLevelLocksManager.get(); + } + + @Override + IngestionBatchProcessor getIngestionBatchProcessor() { + return ingestionBatchProcessorLazy.get(); + } + @Override protected Map calculateLeaderUpstreamOffsetWithTopicSwitch( PartitionConsumptionState partitionConsumptionState, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessor.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessor.java index f862560b79a..9186558188d 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessor.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessor.java @@ -48,7 +48,6 @@ PubSubMessageProcessedResult apply( private final KeyLevelLocksManager lockManager; private final boolean isWriteComputationEnabled; private final boolean isActiveActiveReplicationEnabled; - private final ProcessingFunction processingFunction; private final AggVersionedIngestionStats aggVersionedIngestionStats; private final HostLevelIngestionStats hostLevelIngestionStats; @@ -56,7 +55,6 @@ public IngestionBatchProcessor( String storeVersionName, ExecutorService batchProcessingThreadPool, KeyLevelLocksManager lockManager, - ProcessingFunction processingFunction, boolean isWriteComputationEnabled, boolean isActiveActiveReplicationEnabled, AggVersionedIngestionStats aggVersionedIngestionStats, @@ -64,7 +62,6 @@ public IngestionBatchProcessor( this.storeVersionName = storeVersionName; this.batchProcessingThreadPool = batchProcessingThreadPool; this.lockManager = lockManager; - this.processingFunction = processingFunction; this.isWriteComputationEnabled = isWriteComputationEnabled; this.isActiveActiveReplicationEnabled = isActiveActiveReplicationEnabled; this.aggVersionedIngestionStats = aggVersionedIngestionStats; @@ -130,7 +127,8 @@ public List ingestionBatchProcessingLazy; private final Version version; public LeaderFollowerStoreIngestionTask( @@ -315,6 +316,21 @@ public LeaderFollowerStoreIngestionTask( builder.getSchemaRepo(), getStoreName(), serverConfig.isComputeFastAvroEnabled()); + this.ingestionBatchProcessingLazy = Lazy.of(() -> { + if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { + LOGGER.info("AA/WC workload parallel processing is disabled for store version: {}", getKafkaVersionTopic()); + return null; + } + LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", getKafkaVersionTopic()); + return new IngestionBatchProcessor( + kafkaVersionTopic, + parallelProcessingThreadPool, + null, + isWriteComputationEnabled, + isActiveActiveReplicationEnabled(), + builder.getVersionedStorageIngestionStats(), + getHostLevelIngestionStats()); + }); } public static VeniceWriter constructVeniceWriter( @@ -347,6 +363,11 @@ public void closeVeniceWriters(boolean doFlush) { } } + @Override + IngestionBatchProcessor getIngestionBatchProcessor() { + return ingestionBatchProcessingLazy.get(); + } + @Override public Map getViewWriters() { return viewWriters; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 0c045ee83ee..063ba594b05 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -4310,12 +4310,14 @@ RmdSerDe getRmdSerDe() { protected abstract boolean hasViewWriters(); - void setWriteComputeFailureCode(int code) { - this.writeComputeFailureCode = code; + KeyLevelLocksManager getKeyLevelLocksManager() { + throw new VeniceException("getKeyLevelLocksManager() should only be called in active active mode"); } - ExecutorService getParallelProcessingThreadPool() { - return parallelProcessingThreadPool; + abstract IngestionBatchProcessor getIngestionBatchProcessor(); + + void setWriteComputeFailureCode(int code) { + this.writeComputeFailureCode = code; } boolean isDataRecovery() { @@ -4330,10 +4332,6 @@ int getLocalKafkaClusterId() { return localKafkaClusterId; } - int getStoreVersionPartitionCount() { - return storeVersionPartitionCount; - } - // For unit test purpose. void setVersionRole(PartitionReplicaIngestionContext.VersionRole versionRole) { this.versionRole = versionRole; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 362516ccf1e..a5cd684fedd 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -94,8 +94,6 @@ public class StorePartitionDataReceiver private final String kafkaUrl; private final String kafkaUrlForLogger; private final int kafkaClusterId; - private final Lazy keyLevelLocksManager; - private final Lazy ingestionBatchProcessorLazy; private long receivedRecordsCount; @@ -120,39 +118,6 @@ public StorePartitionDataReceiver( this.kafkaClusterId = kafkaClusterId; this.LOGGER = LogManager.getLogger(this.getClass().getSimpleName() + " [" + kafkaUrlForLogger + "]"); this.receivedRecordsCount = 0L; - this.keyLevelLocksManager = Lazy.of(() -> { - final int knownKafkaClusterNumber = storeIngestionTask.getServerConfig().getKafkaClusterIdToUrlMap().size(); - final int initialPoolSize = knownKafkaClusterNumber + 1; - return new KeyLevelLocksManager( - storeIngestionTask.getVersionTopic().getName(), - initialPoolSize, - ActiveActiveStoreIngestionTask.getKeyLevelLockMaxPoolSizeBasedOnServerConfig( - storeIngestionTask.getServerConfig(), - storeIngestionTask.getStoreVersionPartitionCount())); - }); - this.ingestionBatchProcessorLazy = Lazy.of(() -> { - final String kafkaVersionTopic = storeIngestionTask.getKafkaVersionTopic(); - if (!storeIngestionTask.getServerConfig().isAAWCWorkloadParallelProcessingEnabled()) { - LOGGER.info("AA/WC workload parallel processing is disabled for store version: {}", kafkaVersionTopic); - return null; - } - IngestionBatchProcessor.ProcessingFunction processingFunction = - (storeIngestionTask.isActiveActiveReplicationEnabled()) - ? this::processActiveActiveMessage - : this::processMessage; - KeyLevelLocksManager lockManager = - (storeIngestionTask.isActiveActiveReplicationEnabled()) ? keyLevelLocksManager.get() : null; - LOGGER.info("AA/WC workload parallel processing is enabled for store version: {}", kafkaVersionTopic); - return new IngestionBatchProcessor( - storeIngestionTask.getKafkaVersionTopic(), - storeIngestionTask.getParallelProcessingThreadPool(), - lockManager, - processingFunction, - storeIngestionTask.isTransientRecordBufferUsed(), - storeIngestionTask.isActiveActiveReplicationEnabled(), - storeIngestionTask.getVersionIngestionStats(), - storeIngestionTask.getHostLevelIngestionStats()); - }); } @Override @@ -308,7 +273,7 @@ public void produceToStoreBufferServiceOrKafkaInBatch( if (batches.isEmpty()) { return; } - IngestionBatchProcessor ingestionBatchProcessor = ingestionBatchProcessorLazy.get(); + IngestionBatchProcessor ingestionBatchProcessor = storeIngestionTask.getIngestionBatchProcessor(); if (ingestionBatchProcessor == null) { throw new VeniceException( "IngestionBatchProcessor object should present for store version: " @@ -329,7 +294,10 @@ public void produceToStoreBufferServiceOrKafkaInBatch( kafkaUrl, kafkaClusterId, beforeProcessingPerRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs); + beforeProcessingBatchRecordsTimestampMs, + (storeIngestionTask.isActiveActiveReplicationEnabled()) + ? this::processActiveActiveMessage + : this::processMessage); for (PubSubMessageProcessedResultWrapper processedRecord: processedResults) { totalBytesRead += handleSingleMessage( @@ -796,7 +764,7 @@ private DelegateConsumerRecordResult delegateConsumerRecordMaybeWithLock( * -> [fabric A thread]produce to VT */ final ByteArrayKey byteArrayKey = ByteArrayKey.wrap(consumerRecordWrapper.getMessage().getKey().getKey()); - ReentrantLock keyLevelLock = keyLevelLocksManager.get().acquireLockByKey(byteArrayKey); + ReentrantLock keyLevelLock = storeIngestionTask.getKeyLevelLocksManager().acquireLockByKey(byteArrayKey); keyLevelLock.lock(); try { return delegateConsumerRecord( @@ -808,7 +776,7 @@ private DelegateConsumerRecordResult delegateConsumerRecordMaybeWithLock( beforeProcessingBatchRecordsTimestampMs); } finally { keyLevelLock.unlock(); - keyLevelLocksManager.get().releaseLock(byteArrayKey); + storeIngestionTask.getKeyLevelLocksManager().releaseLock(byteArrayKey); } } } @@ -2550,8 +2518,4 @@ public String toString() { int getKafkaClusterId() { return this.kafkaClusterId; } - - IngestionBatchProcessor getIngestionBatchProcessor() { - return ingestionBatchProcessorLazy.get(); - } } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessorTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessorTest.java index 5c2ec5fc011..35b7aeb2ef2 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessorTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessorTest.java @@ -113,7 +113,6 @@ public void lockKeysTest() { "store_v1", mock(ExecutorService.class), mockKeyLevelLocksManager, - (ignored1, ignored2, ignored3, ignored4, ignored5, ignored6, ignored7) -> null, true, true, mock(AggVersionedIngestionStats.class), @@ -181,6 +180,19 @@ public void processTest() { "store_v1", Executors.newFixedThreadPool(1, new DaemonThreadFactory("test")), mockKeyLevelLocksManager, + true, + true, + mockAggVersionedIngestionStats, + mockHostLevelIngestionStats); + + List> result = batchProcessor.process( + Arrays.asList(rtMessage1, rtMessage2), + mock(PartitionConsumptionState.class), + 1, + "test_kafka", + 1, + 1, + 1, (consumerRecord, ignored2, ignored3, ignored4, ignored5, ignored6, ignored7) -> { if (Arrays.equals(consumerRecord.getKey().getKey(), "key1".getBytes())) { Put put = new Put(); @@ -194,20 +206,7 @@ public void processTest() { return new PubSubMessageProcessedResult(writeComputeResultWrapper); } return null; - }, - true, - true, - mockAggVersionedIngestionStats, - mockHostLevelIngestionStats); - - List> result = batchProcessor.process( - Arrays.asList(rtMessage1, rtMessage2), - mock(PartitionConsumptionState.class), - 1, - "test_kafka", - 1, - 1, - 1); + }); assertEquals(result.size(), 2); PubSubMessageProcessedResultWrapper resultForKey1 = result.get(0); @@ -228,17 +227,6 @@ public void processTest() { "store_v1", Executors.newFixedThreadPool(1, new DaemonThreadFactory("test")), mockKeyLevelLocksManager, - (consumerRecord, ignored2, ignored3, ignored4, ignored5, ignored6, ignored7) -> { - if (Arrays.equals(consumerRecord.getKey().getKey(), "key1".getBytes())) { - Put put = new Put(); - put.setPutValue(ByteBuffer.wrap("value1".getBytes())); - WriteComputeResultWrapper writeComputeResultWrapper = new WriteComputeResultWrapper(put, null, true); - return new PubSubMessageProcessedResult(writeComputeResultWrapper); - } else if (Arrays.equals(consumerRecord.getKey().getKey(), "key2".getBytes())) { - throw new VeniceException("Fake"); - } - return null; - }, true, true, mockAggVersionedIngestionStats, @@ -253,7 +241,18 @@ public void processTest() { "test_kafka", 1, 1, - 1)); + 1, + (consumerRecord, ignored2, ignored3, ignored4, ignored5, ignored6, ignored7) -> { + if (Arrays.equals(consumerRecord.getKey().getKey(), "key1".getBytes())) { + Put put = new Put(); + put.setPutValue(ByteBuffer.wrap("value1".getBytes())); + WriteComputeResultWrapper writeComputeResultWrapper = new WriteComputeResultWrapper(put, null, true); + return new PubSubMessageProcessedResult(writeComputeResultWrapper); + } else if (Arrays.equals(consumerRecord.getKey().getKey(), "key2".getBytes())) { + throw new VeniceException("Fake"); + } + return null; + })); assertTrue(exception.getMessage().contains("Failed to execute the batch processing")); verify(mockAggVersionedIngestionStats).recordBatchProcessingRequestError("store", 1); verify(mockHostLevelIngestionStats).recordBatchProcessingRequestError(); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java index 797e230a5cf..bab5818c564 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java @@ -3393,13 +3393,11 @@ public void testActiveActiveStoreIsReadyToServe(HybridConfig hybridConfig, NodeT null); if (hybridConfig.equals(HYBRID) && nodeType.equals(LEADER) && isAaWCParallelProcessingEnabled()) { - localConsumedDataReceiver = new StorePartitionDataReceiver( - storeIngestionTaskUnderTest, - fooTopicPartition, - inMemoryLocalKafkaBroker.getKafkaBootstrapServer(), - 1); - assertNotNull(localConsumedDataReceiver.getIngestionBatchProcessor()); - assertNotNull(localConsumedDataReceiver.getIngestionBatchProcessor().getLockManager()); + assertTrue(storeIngestionTaskUnderTest instanceof ActiveActiveStoreIngestionTask); + ActiveActiveStoreIngestionTask activeActiveStoreIngestionTask = + (ActiveActiveStoreIngestionTask) storeIngestionTaskUnderTest; + assertNotNull(activeActiveStoreIngestionTask.getIngestionBatchProcessor()); + assertNotNull(activeActiveStoreIngestionTask.getIngestionBatchProcessor().getLockManager()); } String rtTopicName = Utils.getRealTimeTopicName(mockStore);