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 adb5dd9032..7e7f0904fe 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,36 +2,23 @@ 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; 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.MergeConflictResult; 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.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.davinci.utils.ByteArrayKey; 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; @@ -43,23 +30,13 @@ 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.storage.protocol.ChunkedValueManifest; 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 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; -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; @@ -67,11 +44,7 @@ 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; -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; @@ -82,25 +55,14 @@ */ 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, StoreIngestionTaskFactory.Builder builder, @@ -130,7 +92,6 @@ public ActiveActiveStoreIngestionTask( this.rmdProtocolVersionId = version.getRmdVersionId(); - this.aggVersionedIngestionStats = versionedIngestionStats; int knownKafkaClusterNumber = serverConfig.getKafkaClusterIdToUrlMap().size(); int initialPoolSize = knownKafkaClusterNumber + 1; @@ -164,10 +125,9 @@ public ActiveActiveStoreIngestionTask( kafkaVersionTopic, parallelProcessingThreadPool, keyLevelLocksManager.get(), - this::processActiveActiveMessage, isWriteComputationEnabled, isActiveActiveReplicationEnabled(), - aggVersionedIngestionStats, + versionedIngestionStats, getHostLevelIngestionStats()); }); } @@ -193,59 +153,9 @@ 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 { - // TODO: Honor BatchConflictResolutionPolicy and maybe persist RMD for batch push records. StorageOperationType storageOperationType = getStorageOperationType(partition, put.putValue, put.replicationMetadataPayload); @@ -338,580 +248,14 @@ 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 - */ - 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; - } - - 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(); - } - @Override - protected IngestionBatchProcessor getIngestionBatchProcessor() { - return ingestionBatchProcessorLazy.get(); - } - - 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( - 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(); - }); - } - - private 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! - 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 - */ - 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( - 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 - */ - 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) { - 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); - } + KeyLevelLocksManager getKeyLevelLocksManager() { + return keyLevelLocksManager.get(); } @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()); - } + IngestionBatchProcessor getIngestionBatchProcessor() { + return ingestionBatchProcessorLazy.get(); } @Override @@ -1228,10 +572,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); } @@ -1439,58 +780,19 @@ Runnable buildRepairTask( }; } + @Override int getRmdProtocolVersionId() { return rmdProtocolVersionId; } - 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 + MergeConflictResolver getMergeConflictResolver() { + return mergeConflictResolver; } - 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); + @Override + RmdSerDe getRmdSerDe() { + return rmdSerDe; } /** 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 f862560b79..9186558188 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 getViewWriters() { + return viewWriters; + } + @Override protected void closeVeniceViewWriters() { if (!viewWriters.isEmpty()) { @@ -402,11 +380,6 @@ protected void closeVeniceViewWriters() { } } - @Override - protected IngestionBatchProcessor getIngestionBatchProcessor() { - return ingestionBatchProcessingLazy.get(); - } - @Override public synchronized void promoteToLeader( PubSubTopicPartition topicPartition, @@ -711,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. */ @@ -1261,7 +1234,8 @@ private void checkAndUpdateDataRecoveryStatusOfHybridStore(PartitionConsumptionS * * If buffer replay is disable, all replicas will stick to version topic, no one is going to produce any message. */ - protected boolean shouldProduceToVersionTopic(PartitionConsumptionState partitionConsumptionState) { + @Override + boolean shouldProduceToVersionTopic(PartitionConsumptionState partitionConsumptionState) { if (!isLeader(partitionConsumptionState)) { return false; // Not leader } @@ -1269,7 +1243,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); } @@ -1695,31 +1669,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(); @@ -1865,94 +1814,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 @@ -2070,23 +1931,6 @@ protected final void recordAssembledRecordSizeRatio(double ratio, long currentTi } } - private 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)); @@ -2154,96 +1998,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. - */ - 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. - */ - private 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, @@ -2280,440 +2034,6 @@ protected void recordHeartbeatReceived( } } - @Override - protected 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 StoreIngestionTask#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. */ @@ -2798,41 +2118,6 @@ protected void waitForAllMessageToBeProcessedFromTopicPartition( } } - /** - * 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 == 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 = @@ -3058,10 +2343,8 @@ protected long getLatestConsumedUpstreamOffsetForHybridOffsetLagMeasurement( return pcs.getLeaderConsumedUpstreamRTOffset(OffsetRecord.NON_AA_REPLICATION_UPSTREAM_OFFSET_MAP_KEY); } - protected void updateLatestInMemoryLeaderConsumedRTOffset( - PartitionConsumptionState pcs, - String ignoredKafkaUrl, - long offset) { + @Override + void updateLatestInMemoryLeaderConsumedRTOffset(PartitionConsumptionState pcs, String ignoredKafkaUrl, long offset) { pcs.updateLeaderConsumedUpstreamRTOffset(OffsetRecord.NON_AA_REPLICATION_UPSTREAM_OFFSET_MAP_KEY, offset); } @@ -3138,448 +2421,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. - */ - 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); - } - - 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 = 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); - } - } - - 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 - */ - 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( - 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. * @@ -3617,10 +2458,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 @@ -3685,26 +2522,12 @@ protected void processControlMessageForViews( } } - 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); + Lazy> getVeniceWriter(PartitionConsumptionState partitionConsumptionState) { + return partitionConsumptionState.getVeniceWriterLazyRef(); } - protected Lazy> getVeniceWriter( - PartitionConsumptionState partitionConsumptionState) { - return partitionConsumptionState.getVeniceWriterLazyRef(); + void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState) { + partitionConsumptionState.setVeniceWriterLazyRef(veniceWriterForRealTime); } // test method @@ -3747,25 +2570,8 @@ 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 CompletableFuture sendIngestionHeartbeat( + @Override + CompletableFuture sendIngestionHeartbeat( PartitionConsumptionState partitionConsumptionState, PubSubTopicPartition topicPartition, PubSubProducerCallback callback, @@ -3942,35 +2748,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 @@ -3997,33 +2774,44 @@ 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 + KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders() { + return kafkaDataIntegrityValidatorForLeaders; + } + + @Override + byte[] applyWriteCompute( + GenericRecord currValue, + int writerValueSchemaId, + int readerValueSchemaId, + ByteBuffer writeComputeBytes, + int writerUpdateProtocolVersion, + int readerUpdateProtocolVersion) { + return storeWriteComputeHandler.applyWriteCompute( + currValue, + writerValueSchemaId, + readerValueSchemaId, + writeComputeBytes, + writerUpdateProtocolVersion, + readerUpdateProtocolVersion); + } + + @Override + AvroStoreDeserializerCache getStoreDeserializerCache() { + return storeDeserializerCache; + } + + @Override + String getKafkaUrl(int kafkaClusterId) { + return kafkaClusterIdToUrlMap.get(kafkaClusterId); + } + + @Override + boolean hasChangeCaptureView() { + return hasChangeCaptureView; } } 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 87150f5e98..cc83fd1128 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 e1620eb5cc..063ba594b0 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; @@ -42,7 +44,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.utils.ByteArrayKey; +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; @@ -72,7 +74,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; @@ -88,12 +89,15 @@ 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.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; 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; @@ -114,10 +118,12 @@ 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; +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; @@ -129,10 +135,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 +154,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; @@ -158,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; @@ -174,7 +178,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 @@ -573,8 +577,6 @@ void setPurgeTransientRecordBuffer(boolean purgeTransientRecordBuffer) { this.purgeTransientRecordBuffer = purgeTransientRecordBuffer; } - protected abstract IngestionBatchProcessor getIngestionBatchProcessor(); - public AbstractStorageEngine getStorageEngine() { return storageEngine; } @@ -1134,269 +1136,6 @@ protected void produceToStoreBufferService( } } - protected abstract Iterable> validateAndFilterOutDuplicateMessagesFromLeaderTopic( - Iterable> records, - String kafkaUrl, - PubSubTopicPartition topicPartition); - - private 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 ((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()); - } - } - - protected 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; @@ -1547,9 +1286,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(); @@ -1918,6 +1657,8 @@ public void closeVeniceWriters(boolean doFlush) { protected void closeVeniceViewWriters() { } + public abstract Map getViewWriters(); + /** * Consumes the kafka actions messages in the queue. */ @@ -2433,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) { @@ -2913,7 +2598,7 @@ private void syncEndOfPushTimestampToMetadataService(long endOfPushTimestamp) { }); } - private void processStartOfPush( + void processStartOfPush( KafkaMessageEnvelope startOfPushKME, ControlMessage controlMessage, int partition, @@ -3123,6 +2808,8 @@ protected void processControlMessageForViews( // NoOp } + abstract boolean shouldProduceToVersionTopic(PartitionConsumptionState partitionConsumptionState); + protected boolean processTopicSwitch( ControlMessage controlMessage, int partition, @@ -3164,7 +2851,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: @@ -3174,7 +2861,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: @@ -3906,97 +3593,7 @@ 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 - */ - 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 { - 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); - } - } - - 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; @@ -4041,17 +3638,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; } @@ -4180,6 +3776,10 @@ public PubSubTopic getVersionTopic() { return versionTopic; } + PubSubTopic getRealTimeTopic() { + return realTimeTopic; + } + public boolean isMetricsEmissionEnabled() { return emitMetrics.get(); } @@ -4418,35 +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. @@ -4675,6 +4246,92 @@ Lazy getGracefulShutdownLatch() { return gracefulShutdownLatch; } + 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) + throws InterruptedException { + storeBufferService.putConsumerRecord(record, this, null, partition, url, t); + } + + boolean isGlobalRtDivEnabled() { + return isGlobalRtDivEnabled; + } + + void handleDivErrorMetric(DuplicateDataException e) { + divErrorMetricCallback.accept(e); + } + + abstract KafkaDataIntegrityValidator getKafkaDataIntegrityValidatorForLeaders(); + + abstract void updateLatestInMemoryLeaderConsumedRTOffset( + PartitionConsumptionState pcs, + String ignoredKafkaUrl, + long offset); + + abstract CompletableFuture sendIngestionHeartbeat( + PartitionConsumptionState partitionConsumptionState, + PubSubTopicPartition topicPartition, + PubSubProducerCallback callback, + LeaderMetadataWrapper leaderMetadataWrapper, + boolean shouldLog, + boolean addLeaderCompleteState, + LeaderCompleteState leaderCompleteState, + long originTimeStampMs); + + 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"); + } + + abstract String getKafkaUrl(int kafkaClusterId); + + abstract boolean hasChangeCaptureView(); + + abstract byte[] applyWriteCompute(GenericRecord value, int ws, int rs, ByteBuffer bytes, int wp, int rp); + + abstract AvroStoreDeserializerCache getStoreDeserializerCache(); + + abstract Lazy> getVeniceWriter(PartitionConsumptionState pcs); + + abstract void setRealTimeVeniceWriterRef(PartitionConsumptionState partitionConsumptionState); + + protected abstract boolean hasViewWriters(); + + KeyLevelLocksManager getKeyLevelLocksManager() { + throw new VeniceException("getKeyLevelLocksManager() should only be called in active active mode"); + } + + abstract IngestionBatchProcessor getIngestionBatchProcessor(); + + void setWriteComputeFailureCode(int code) { + this.writeComputeFailureCode = code; + } + + boolean isDataRecovery() { + return isDataRecovery; + } + + String getLocalKafkaServer() { + return localKafkaServer; + } + + 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 e5a59294b2..a5cd684fed 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,31 +1,111 @@ 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.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.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.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; +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; +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; +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; +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; 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.RawBytesStoreDeserializerCache; +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; +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; +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; +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.Optional; +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.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 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, @@ -75,12 +155,2310 @@ 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 = validateAndFilterOutDuplicateMessagesFromLeaderTopic(records, kafkaUrl, topicPartition); + + if (shouldProduceInBatch(records)) { + 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 (!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); + } + + 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 (!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 = 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, + (storeIngestionTask.isActiveActiveReplicationEnabled()) + ? this::processActiveActiveMessage + : this::processMessage); + + for (PubSubMessageProcessedResultWrapper processedRecord: processedResults) { + totalBytesRead += handleSingleMessage( + processedRecord, + topicPartition, + partitionConsumptionState, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs, + metricsEnabled, + elapsedTimeForPuttingIntoQueue); + } + } finally { + ingestionBatchProcessor.unlockKeys(keyLockMap); + } + } + + updateMetricsAndEnforceQuota( + totalBytesRead, + topicPartition.getPartitionNumber(), + elapsedTimeForPuttingIntoQueue, + beforeProcessingBatchRecordsTimestampMs); + } + + private boolean shouldProduceInBatch(Iterable> records) { + return (storeIngestionTask.isActiveActiveReplicationEnabled() || storeIngestionTask.isTransientRecordBufferUsed()) + && storeIngestionTask.getServerConfig().isAAWCWorkloadParallelProcessingEnabled() + && IngestionBatchProcessor.isAllMessagesFromRTTopic(records); + } + + /** + * 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. + */ + private boolean shouldProcessRecord(PubSubMessage 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. + */ + 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, + ValueHolder elapsedTimeForPuttingIntoQueue, + long beforeProcessingBatchRecordsTimestampMs) { + /** + * Even if the records list is empty, we still need to check quota to potentially resume partition + */ + storeIngestionTask.enforcePartitionQuota(partition, totalBytesRead); + + if (storeIngestionTask.isMetricsEmissionEnabled()) { + HostLevelIngestionStats hostLevelIngestionStats = storeIngestionTask.getHostLevelIngestionStats(); + if (totalBytesRead > 0) { + hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); + } + if (elapsedTimeForPuttingIntoQueue.getValue() > 0) { + hostLevelIngestionStats.recordConsumerRecordsQueuePutLatency( + elapsedTimeForPuttingIntoQueue.getValue(), + beforeProcessingBatchRecordsTimestampMs); + } + + hostLevelIngestionStats.recordStorageQuotaUsed(storeIngestionTask.getDiskQuotaUsage()); + } + } + + 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.handleDivErrorMetric(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 + * @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); + } + } + + 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. + DelegateConsumerRecordResult delegateConsumerRecordResult = delegateConsumerRecordMaybeWithLock( + 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(); + } + + /** + * 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().acquireLockByKey(byteArrayKey); + keyLevelLock.lock(); + try { + return delegateConsumerRecord( + consumerRecordWrapper, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); + } finally { + keyLevelLock.unlock(); + storeIngestionTask.getKeyLevelLocksManager().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 #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. + * + * 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; + 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 + validateRecordBeforeProducingToLocalKafka(consumerRecord, partitionConsumptionState, kafkaUrl, kafkaClusterId); + + if (consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { + 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. + */ + 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, + () -> 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, + () -> 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. + */ + 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); + 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. + 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(); + } + } + + /** + * Checks before producing local version topic. + * + * 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 = 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); + 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( + storeIngestionTask.getIngestionTaskName() + " : Invalid/Unrecognized operation type submitted: " + + kafkaValue.messageType); + } + } + + 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 = 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 = 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.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( + 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 = 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 (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 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( + () -> 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 = 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; + + storeIngestionTask.getVersionIngestionStats() + .recordTotalDCR(storeIngestionTask.getStoreName(), storeIngestionTask.getVersionNumber()); + + Lazy oldValueByteBufferProvider = unwrapByteBufferFromOldValueProvider(oldValueProvider); + + long beforeDCRTimestampInNs = System.nanoTime(); + switch (msgType) { + case PUT: + mergeConflictResult = storeIngestionTask.getMergeConflictResolver() + .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 = storeIngestionTask.getMergeConflictResolver() + .delete( + oldValueByteBufferProvider, + rmdWithValueSchemaID, + writeTimestamp, + sourceOffset, + kafkaClusterId, + kafkaClusterId); + storeIngestionTask.getHostLevelIngestionStats() + .recordIngestionActiveActiveDeleteLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); + break; + + case UPDATE: + mergeConflictResult = storeIngestionTask.getMergeConflictResolver() + .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.getKafkaUrl(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 = storeIngestionTask.getRmdSerDe() + .serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); + + if (updatedValueBytes == null) { + storeIngestionTask.getHostLevelIngestionStats().recordTombstoneCreatedDCR(); + storeIngestionTask.getVersionIngestionStats() + .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. + *

+ * 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.getVersionIngestionStats() + .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, + VeniceWriter.APP_DEFAULT_LOGICAL_TS, + new DeleteMetadata(valueSchemaId, storeIngestionTask.getRmdProtocolVersionId(), 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 = storeIngestionTask.getRmdProtocolVersionId(); + 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); + } + } + + 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, + 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); + } + } + } + + 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); + 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, + long upstreamOffset, + long currentTimeMs) { + if (kafkaClusterId >= 0) { + storeIngestionTask.getVersionIngestionStats() + .recordRegionHybridConsumption( + storeIngestionTask.getStoreName(), + storeIngestionTask.getVersionNumber(), + kafkaClusterId, + producedRecordSize, + upstreamOffset, + currentTimeMs); + storeIngestionTask.getHostLevelIngestionStats() + .recordTotalRegionHybridBytesConsumed(kafkaClusterId, producedRecordSize, currentTimeMs); + } + } + + /** + * 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 = 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); + } + + 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); + } + + /** + * 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.getVersionIngestionStats() + .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.getVersionIngestionStats() + .recordTimestampRegressionDCRError( + storeIngestionTask.getStoreName(), + storeIngestionTask.getVersionNumber()); + LOGGER.error( + "Timestamp found to have gone backwards!! Invalid replication metadata result: {}", + mergeConflictResult.getRmdRecord()); + } + } + } + + 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); + } + + /** + * 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; + } + + /** + * 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. + * @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); + } + } + + /** + * 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 945615055f..ccdda58c21 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; @@ -12,6 +11,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,30 +130,35 @@ 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); - verify(ingestionTask, times(1)).produceToLocalKafka( + storePartitionDataReceiver.processMessageAndMaybeProduceToKafka(resultWrapper, pcs, 0, "dummyUrl", 0, 0L, 0L); + verify(storePartitionDataReceiver, times(1)).produceToLocalKafka( any(), any(), leaderProducedRecordContextArgumentCaptor.capture(), @@ -172,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; @@ -187,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); @@ -321,17 +328,18 @@ 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)); 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); @@ -400,11 +408,11 @@ 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, - ingestionTask.getProduceToTopicFunction( + storePartitionDataReceiver.getProduceToTopicFunction( partitionConsumptionState, updatedKeyBytes, updatedValueBytes, @@ -520,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); @@ -557,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); @@ -593,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); @@ -602,11 +614,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()); 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 5c2ec5fc01..35b7aeb2ef 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/LeaderFollowerStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTaskTest.java index c3b4e17d43..2a5ad46db1 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; @@ -206,6 +207,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 +249,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); @@ -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)); @@ -332,8 +335,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); 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 bc157da75f..bab5818c56 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 @@ -3705,16 +3705,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); @@ -3725,7 +3728,7 @@ public void testGetAndUpdateLeaderCompletedState(HybridConfig hybridConfig, Node if (nodeType != DA_VINCI) { partitionConsumptionState.setLeaderFollowerState(LeaderFollowerStateType.LEADER); - ingestionTask.getAndUpdateLeaderCompletedState( + storePartitionDataReceiver.getAndUpdateLeaderCompletedState( kafkaKey, kafkaValue, controlMessage, @@ -3736,7 +3739,7 @@ public void testGetAndUpdateLeaderCompletedState(HybridConfig hybridConfig, Node } partitionConsumptionState.setLeaderFollowerState(STANDBY); - ingestionTask.getAndUpdateLeaderCompletedState( + storePartitionDataReceiver.getAndUpdateLeaderCompletedState( kafkaKey, kafkaValue, controlMessage,