From 459d9e092b87e7c18f970e05a091c91a980dd9f9 Mon Sep 17 00:00:00 2001 From: Felix GV Date: Wed, 10 Jan 2024 13:01:37 -0500 Subject: [PATCH] [server][dvc] Lag measurement safeguard and logging improvements (#802) [server][dvc] Lag measurement safeguard and logging improvements It seems that sometimes the offset lag used in lag measurement can be negative, which results in the lag being considered acceptable and the replica becoming ready to serve (prematurely). It seems that negative lag is a somewhat normal phenomenon since the formula is basically max_offset - consumed_offset, and the former is cached, so if stale it could be smaller than the latter (which of course doesn't make sense, but is an artifact of the caching). This is a benign case. However, there is also a case where the max_offset retrieval can fail, in which case a negative error code would be returned. This also leads to negative lag, but it is a mistake. This commit adds safeguards to prevent prematurely declaring ready-to-serve. There are a bunch of code paths interacting with offset retrieval, which makes this commit a bit big. In order to improve maintainability, the tricky bits are centralized as much as possible into a new SIT::measureLagWithCallToPubSub function. Miscellaneous: - Added a get_partition_latest_offset_with_retry_error metric. - Deleted the SLOPPY_OFFSET_CATCHUP_THRESHOLD in SIT, which is not necessary and is, well, sloppy. - Introduced a new EmptyPubSubMessageHeaders which is immutable (the previous EMPTY_MSG_HEADERS in VeniceWriter was mutable). - Fixed a NPE in AbstractVeniceStatsReporter. - Fixed a concurrency issue about closing the non-threadsafe Kafka consumer in PartitionOffsetFetcher. - Fixed the sensor name of KafkaConsumerServiceStats's offset present/absent metrics. Added a test for this as well. - Tweaked the API in AbstractVeniceStats so that it is less error-prone. The registerSensor function with sensorFullName parameter is now private, to avoid misuse. - Rewrote the TopicPartitionsOffsetsTracker so that it is a bit more efficient, not stale, and uses less code. - Deleted two functions from LFSIT which were used only to specify default values to some parameters in a deeper function. Kept only measureRTOffsetLagForSingleRegion, which is the first function of the call chain. There are also various log tweaks to make logs less voluminous: - Deleted two log lines from SIT::getDefaultReadyToServeChecker - "Reopen partition {}_{} for reading after ready-to-serve." - "Partition {} synced offset" - Passed shouldLogLag false in SIT::updateOffsetLagInMetadata. - TopicPartitionsOffsetsTracker doesn't periodically log a bunch of lines anymore. Test changes: - Overhauled DataProviders in StoreIngestionTaskTest. Rather than using a bunch of booleans which are hard reason about in test logs, we now use enums, which makes it easy to read which test permutations passed or failed. In one case, this allows squashing 2 booleans (of which 1 of 4 permutations is invalid) into a single 3 items enum, thus reducing the test permutations count. - Fixed testOffsetSyncBeforeGracefulShutDown flakiness in StoreIngestionTaskTest. - SITTest::testGetAndUpdateLeaderCompletedState which was mutating the static instance of VW.EMPTY_MSG_HEADERS, thus contaminating subsequent tests (when running in the same JVM, as is the case in IntelliJ, but not in Gradle). - Added some unit tests for parent stats. --- .../linkedin/davinci/ingestion/LagType.java | 15 + .../ActiveActiveStoreIngestionTask.java | 35 +- .../consumer/CachedPubSubMetadataGetter.java | 2 + .../LeaderFollowerStoreIngestionTask.java | 183 ++--- .../kafka/consumer/StoreIngestionTask.java | 97 +-- .../davinci/stats/IngestionStatsReporter.java | 16 +- .../stats/KafkaConsumerServiceStats.java | 11 +- .../davinci/store/AbstractStorageEngine.java | 1 - .../store/rocksdb/RocksDBSstFileWriter.java | 7 +- .../consumer/StoreIngestionTaskTest.java | 666 ++++++++++-------- .../stats/KafkaConsumerServiceStatsTest.java | 92 +++ .../venice/stats/AbstractVeniceStats.java | 13 +- .../com/linkedin/venice/stats/LambdaStat.java | 4 +- .../venice/stats/AbstractVeniceStatsTest.java | 73 +- .../InstrumentedPartitionOffsetFetcher.java | 16 +- .../PartitionOffsetFetcherImpl.java | 4 +- .../PartitionOffsetFetcherStats.java | 11 +- .../KafkaPartitionTopicOffsetMetrics.java | 50 ++ .../kafka/TopicPartitionsOffsetsTracker.java | 194 ++--- .../consumer/ApacheKafkaConsumerAdapter.java | 2 +- .../pubsub/api/EmptyPubSubMessageHeaders.java | 34 + .../venice/pubsub/api/PubSubMessage.java | 5 +- .../pubsub/api/PubSubMessageHeaders.java | 8 +- .../linkedin/venice/writer/VeniceWriter.java | 9 +- .../TopicPartitionsOffsetsTrackerTest.java | 39 +- 25 files changed, 945 insertions(+), 642 deletions(-) create mode 100644 clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/LagType.java create mode 100644 clients/da-vinci-client/src/test/java/com/linkedin/davinci/stats/KafkaConsumerServiceStatsTest.java create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/KafkaPartitionTopicOffsetMetrics.java create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/EmptyPubSubMessageHeaders.java diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/LagType.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/LagType.java new file mode 100644 index 00000000000..9881cb93db9 --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/LagType.java @@ -0,0 +1,15 @@ +package com.linkedin.davinci.ingestion; + +public enum LagType { + OFFSET_LAG("Offset"), TIME_LAG("Time"); + + private final String prettyString; + + LagType(String prettyString) { + this.prettyString = prettyString; + } + + public String prettyString() { + return this.prettyString; + } +} 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 682d9d42628..5b5b716a2f6 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 @@ -9,6 +9,7 @@ import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.davinci.client.DaVinciRecordTransformer; import com.linkedin.davinci.config.VeniceStoreVersionConfig; +import com.linkedin.davinci.ingestion.LagType; import com.linkedin.davinci.replication.RmdWithValueSchemaId; import com.linkedin.davinci.replication.merge.MergeConflictResolver; import com.linkedin.davinci.replication.merge.MergeConflictResolverFactory; @@ -1307,9 +1308,15 @@ protected boolean checkAndLogIfLagIsAcceptableForHybridStore( long offsetLag, long offsetThreshold, boolean shouldLogLag, - boolean isOffsetBasedLag, + LagType lagType, long latestConsumedProducerTimestamp) { - boolean isLagAcceptable = offsetLag <= offsetThreshold; + boolean isLagAcceptable = super.checkAndLogIfLagIsAcceptableForHybridStore( + pcs, + offsetLag, + offsetThreshold, + false, // We'll take care of logging at the end of this function + lagType, + latestConsumedProducerTimestamp); if (isLagAcceptable && isHybridFollower(pcs)) { if (!getServerConfig().isLeaderCompleteStateCheckInFollowerEnabled()) { @@ -1331,20 +1338,17 @@ protected boolean checkAndLogIfLagIsAcceptableForHybridStore( if (shouldLogLag) { String lagLogFooter; if (isHybridFollower(pcs)) { - lagLogFooter = ". Leader Complete State: {" + pcs.getLeaderCompleteState().toString() - + "}, Last update In Ms: {" + pcs.getLastLeaderCompleteStateUpdateInMs() + "}."; + lagLogFooter = "Leader Complete State: {" + pcs.getLeaderCompleteState().toString() + "}, Last update In Ms: {" + + pcs.getLastLeaderCompleteStateUpdateInMs() + "}."; } else { lagLogFooter = ""; } - LOGGER.info( - "{} [{} lag] partition {} is {}lagging. {}Lag: [{}] {} Threshold [{}]{}", - isOffsetBasedLag ? "Offset" : "Time", - consumerTaskId, + logLag( + lagType, pcs.getPartition(), - (isLagAcceptable ? "not " : ""), - (isOffsetBasedLag ? "" : "The latest producer timestamp is " + latestConsumedProducerTimestamp + ". "), + isLagAcceptable, + latestConsumedProducerTimestamp, offsetLag, - (isLagAcceptable ? "<" : ">"), offsetThreshold, lagLogFooter); } @@ -1399,10 +1403,13 @@ public long getRegionHybridOffsetLag(int regionId) { } // Fall back to calculate offset lag in the old way - return (cachedPubSubMetadataGetter - .getOffset(getTopicManager(kafkaSourceAddress), currentLeaderTopic, pcs.getUserPartition()) - 1) - - pcs.getLeaderConsumedUpstreamRTOffset(kafkaSourceAddress); + return measureLagWithCallToPubSub( + kafkaSourceAddress, + currentLeaderTopic, + pcs.getUserPartition(), + pcs.getLeaderConsumedUpstreamRTOffset(kafkaSourceAddress)); }) + .filter(VALID_LAG) .sum(); return minZeroLag(offsetLag); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/CachedPubSubMetadataGetter.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/CachedPubSubMetadataGetter.java index 6495d9fc828..63420004ab3 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/CachedPubSubMetadataGetter.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/CachedPubSubMetadataGetter.java @@ -42,6 +42,8 @@ class CachedPubSubMetadataGetter { * @return Users of this method should be aware that Kafka will actually * return the next available offset rather the latest used offset. Therefore, * the value will be 1 offset greater than what's expected. + * + * TODO: Refactor this using PubSubTopicPartition */ long getOffset(TopicManager topicManager, PubSubTopic pubSubTopic, int partitionId) { final String sourcePubSubServer = topicManager.getPubSubBootstrapServers(); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index a4c3112e92e..21a8dde37ab 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -1,5 +1,6 @@ package com.linkedin.davinci.kafka.consumer; +import static com.linkedin.davinci.ingestion.LagType.OFFSET_LAG; import static com.linkedin.davinci.kafka.consumer.ConsumerActionType.LEADER_TO_STANDBY; import static com.linkedin.davinci.kafka.consumer.ConsumerActionType.STANDBY_TO_LEADER; import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.IN_TRANSITION_FROM_STANDBY_TO_LEADER; @@ -19,6 +20,7 @@ import com.linkedin.davinci.client.DaVinciRecordTransformer; import com.linkedin.davinci.config.VeniceStoreVersionConfig; import com.linkedin.davinci.helix.LeaderFollowerPartitionStateModel; +import com.linkedin.davinci.ingestion.LagType; import com.linkedin.davinci.schema.merge.CollectionTimestampMergeRecordHelper; import com.linkedin.davinci.schema.merge.MergeRecordHelper; import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; @@ -98,8 +100,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; -import java.util.function.BiFunction; import java.util.function.BooleanSupplier; +import java.util.function.LongPredicate; import java.util.function.Predicate; import java.util.function.Supplier; import org.apache.avro.generic.GenericRecord; @@ -716,7 +718,7 @@ private boolean canSwitchToLeaderTopic(PartitionConsumptionState pcs) { */ private boolean isLocalVersionTopicPartitionFullyConsumed(PartitionConsumptionState pcs) { long localVTOff = pcs.getLatestProcessedLocalVersionTopicOffset(); - long localVTEndOffset = getKafkaTopicPartitionEndOffSet(localKafkaServer, versionTopic, pcs.getPartition()); + long localVTEndOffset = getTopicPartitionEndOffSet(localKafkaServer, versionTopic, pcs.getPartition()); if (localVTEndOffset == StatsErrorCode.LAG_MEASUREMENT_FAILURE.code) { return false; } @@ -1067,16 +1069,16 @@ private void checkAndUpdateDataRecoveryStatusOfHybridStore(PartitionConsumptionS } } if (!isDataRecoveryCompleted) { - long dataRecoverySourceVTEndOffset = cachedPubSubMetadataGetter.getOffset( - topicManagerRepository.getTopicManager(nativeReplicationSourceVersionTopicKafkaURL), - versionTopic, - partitionConsumptionState.getPartition()); - - // If the last few records in source VT is old then we can also complete data recovery if the leader idles and we + // If the last few records in source VT is old then we can also complete data recovery if the leader idles, and we // have reached/passed the end offset of the VT (around the time when ingestion is started for that partition). - long localVTOffsetProgress = partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset(); - // -2 because getOffset returns the next available offset (-1) and we are skipping the remote TS message (-1). - boolean isAtEndOfSourceVT = dataRecoverySourceVTEndOffset - 2 <= localVTOffsetProgress; + // We subtract 1 because we are skipping the remote TS message. + long localVTOffsetProgress = partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset() - 1; + long dataRecoveryLag = measureLagWithCallToPubSub( + nativeReplicationSourceVersionTopicKafkaURL, + versionTopic, + partitionConsumptionState.getPartition(), + localVTOffsetProgress); + boolean isAtEndOfSourceVT = dataRecoveryLag <= 0; long lastTimestamp = getLastConsumedMessageTimestamp(partitionConsumptionState.getPartition()); if (isAtEndOfSourceVT && LatencyUtils.getElapsedTimeInMs(lastTimestamp) > newLeaderInactiveTime) { LOGGER.info( @@ -1605,8 +1607,11 @@ protected long measureHybridOffsetLag(PartitionConsumptionState partitionConsump // Followers and Davinci clients, use local VT to compute hybrid lag. if (isDaVinciClient || partitionConsumptionState.getLeaderFollowerState().equals(STANDBY)) { - return cachedPubSubMetadataGetter.getOffset(getTopicManager(localKafkaServer), versionTopic, partition) - - partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset(); + return measureLagWithCallToPubSub( + localKafkaServer, + versionTopic, + partition, + partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset()); } // leaderTopic is the real-time topic now @@ -1622,17 +1627,6 @@ protected long measureHybridOffsetLag(PartitionConsumptionState partitionConsump } } - protected long measureRTOffsetLagForSingleRegion( - String sourceRealTimeTopicKafkaURL, - PartitionConsumptionState partitionConsumptionState, - boolean shouldLogLag) { - return getLatestLeaderPersistedOffsetAndHybridTopicOffset( - sourceRealTimeTopicKafkaURL, - partitionConsumptionState.getOffsetRecord().getLeaderTopic(pubSubTopicRepository), - partitionConsumptionState, - shouldLogLag); - } - protected long measureRTOffsetLagForMultiRegions( Set sourceRealTimeTopicKafkaURLs, PartitionConsumptionState partitionConsumptionState, @@ -1673,8 +1667,12 @@ protected void reportIfCatchUpVersionTopicOffset(PartitionConsumptionState pcs) int partition = pcs.getPartition(); if (pcs.isEndOfPushReceived() && !pcs.isLatchReleased()) { - if (cachedPubSubMetadataGetter.getOffset(getTopicManager(localKafkaServer), versionTopic, partition) - 1 <= pcs - .getLatestProcessedLocalVersionTopicOffset()) { + long lag = measureLagWithCallToPubSub( + localKafkaServer, + versionTopic, + partition, + pcs.getLatestProcessedLocalVersionTopicOffset()); + if (lag <= 0) { statusReportAdapter.reportCatchUpVersionTopicOffsetLag(pcs); /** @@ -1919,26 +1917,45 @@ protected boolean checkAndLogIfLagIsAcceptableForHybridStore( long offsetLag, long offsetThreshold, boolean shouldLogLag, - boolean isOffsetBasedLag, + LagType lagType, long latestConsumedProducerTimestamp) { boolean isLagAcceptable = offsetLag <= offsetThreshold; if (shouldLogLag) { - LOGGER.info( - "{} [{} lag] partition {} is {}lagging. {}Lag: [{}] {} Threshold [{}]", - isOffsetBasedLag ? "Offset" : "Time", - consumerTaskId, + logLag( + lagType, pcs.getPartition(), - (isLagAcceptable ? "not " : ""), - (isOffsetBasedLag ? "" : "The latest producer timestamp is " + latestConsumedProducerTimestamp + ". "), + isLagAcceptable, + latestConsumedProducerTimestamp, offsetLag, - (isLagAcceptable ? "<" : ">"), - offsetThreshold); + offsetThreshold, + ""); } return isLagAcceptable; } + protected void logLag( + LagType lagType, + int partition, + boolean isLagAcceptable, + long latestConsumedProducerTimestamp, + long lag, + long threshold, + String lagLogFooter) { + LOGGER.info( + "{} [{} lag] partition {} is {}. {}Lag: [{}] {} Threshold [{}]. {}", + this.consumerTaskId, + lagType.prettyString(), + partition, + (isLagAcceptable ? "not lagging" : "lagging"), + (lagType == OFFSET_LAG ? "" : "The latest producer timestamp is " + latestConsumedProducerTimestamp + ". "), + lag, + (isLagAcceptable ? "<" : ">"), + threshold, + lagLogFooter); + } + /** * 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.

@@ -2518,36 +2535,34 @@ public long getBatchReplicationLag() { return 0; } - long replicationLag = partitionConsumptionStateMap.values() - .stream() - .filter(BATCH_REPLICATION_LAG_FILTER) - // the lag is (latest VT offset in remote kafka - latest VT offset in local kafka) - .mapToLong((pcs) -> { + long replicationLag = + partitionConsumptionStateMap.values().stream().filter(BATCH_REPLICATION_LAG_FILTER).mapToLong((pcs) -> { PubSubTopic currentLeaderTopic = pcs.getOffsetRecord().getLeaderTopic(pubSubTopicRepository); if (currentLeaderTopic == null) { currentLeaderTopic = versionTopic; } String sourceKafkaURL = getSourceKafkaUrlForOffsetLagMeasurement(pcs); - // Consumer might not existed after the consumption state is created, but before attaching the corresponding + // Consumer might not exist after the consumption state is created, but before attaching the corresponding // consumer. long offsetLagOptional = getPartitionOffsetLag(sourceKafkaURL, currentLeaderTopic, pcs.getUserPartition()); if (offsetLagOptional >= 0) { return offsetLagOptional; } - // Fall back to use the old way - return (cachedPubSubMetadataGetter.getOffset( - getTopicManager(nativeReplicationSourceVersionTopicKafkaURL), + // Fall back to use the old way (latest VT offset in remote kafka - latest VT offset in local kafka) + long localOffset = cachedPubSubMetadataGetter + .getOffset(getTopicManager(localKafkaServer), currentLeaderTopic, pcs.getPartition()) - 1; + return measureLagWithCallToPubSub( + nativeReplicationSourceVersionTopicKafkaURL, currentLeaderTopic, - pcs.getPartition()) - 1) - - (cachedPubSubMetadataGetter - .getOffset(getTopicManager(localKafkaServer), currentLeaderTopic, pcs.getPartition()) - 1); - }) - .sum(); + pcs.getPartition(), + localOffset); + }).filter(VALID_LAG).sum(); return minZeroLag(replicationLag); } + protected static final LongPredicate VALID_LAG = value -> value < Long.MAX_VALUE; public static final Predicate LEADER_OFFSET_LAG_FILTER = pcs -> pcs.getLeaderFollowerState().equals(LEADER); private static final Predicate BATCH_LEADER_OFFSET_LAG_FILTER = @@ -2596,11 +2611,14 @@ private long getLeaderOffsetLag(Predicate par if (currentLeaderTopic.isRealTime()) { return this.measureHybridOffsetLag(pcs, false); } else { - return (cachedPubSubMetadataGetter - .getOffset(getTopicManager(kafkaSourceAddress), currentLeaderTopic, pcs.getPartition()) - 1) - - pcs.getLatestProcessedLocalVersionTopicOffset(); + return measureLagWithCallToPubSub( + kafkaSourceAddress, + currentLeaderTopic, + pcs.getPartition(), + pcs.getLatestProcessedLocalVersionTopicOffset()); } }) + .filter(VALID_LAG) .sum(); return minZeroLag(offsetLag); @@ -2665,10 +2683,13 @@ private long getFollowerOffsetLag(Predicate p return offsetLagOptional; } // Fall back to calculate offset lag in the old way - return (cachedPubSubMetadataGetter - .getOffset(getTopicManager(localKafkaServer), versionTopic, pcs.getPartition()) - 1) - - pcs.getLatestProcessedLocalVersionTopicOffset(); + return measureLagWithCallToPubSub( + localKafkaServer, + versionTopic, + pcs.getPartition(), + pcs.getLatestProcessedLocalVersionTopicOffset()); }) + .filter(VALID_LAG) .sum(); return minZeroLag(offsetLag); @@ -3203,34 +3224,19 @@ 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 * latest leader persisted offset for different data replication policy. * @return the lag (lastOffsetInRealTimeTopic - latestPersistedLeaderOffset) */ - protected long getLatestLeaderPersistedOffsetAndHybridTopicOffset( - String sourceRealTimeTopicKafkaURL, - PubSubTopic leaderTopic, - PartitionConsumptionState pcs, - boolean shouldLog) { - return getLatestLeaderOffsetAndHybridTopicOffset( - sourceRealTimeTopicKafkaURL, - leaderTopic, - pcs, - this::getLatestPersistedUpstreamOffsetForHybridOffsetLagMeasurement, - shouldLog); - } - - private boolean isIngestingSystemStore() { - return VeniceSystemStoreUtils.isSystemStore(storeName); - } - - private long getLatestLeaderOffsetAndHybridTopicOffset( + protected long measureRTOffsetLagForSingleRegion( String sourceRealTimeTopicKafkaURL, - PubSubTopic leaderTopic, PartitionConsumptionState pcs, - BiFunction getLeaderLatestOffset, boolean shouldLog) { int partition = pcs.getPartition(); long latestLeaderOffset; @@ -3254,7 +3260,9 @@ private long getLatestLeaderOffsetAndHybridTopicOffset( for (int i = 0; i < subPartitions.size(); i++) { subPartitionConsumptionState = partitionConsumptionStateMap.get(subPartitions.getInt(i)); if (subPartitionConsumptionState != null && subPartitionConsumptionState.getOffsetRecord() != null) { - upstreamOffset = getLeaderLatestOffset.apply(subPartitionConsumptionState, sourceRealTimeTopicKafkaURL); + upstreamOffset = getLatestPersistedUpstreamOffsetForHybridOffsetLagMeasurement( + subPartitionConsumptionState, + sourceRealTimeTopicKafkaURL); if (upstreamOffset > latestLeaderOffset) { latestLeaderOffset = upstreamOffset; } @@ -3262,16 +3270,25 @@ private long getLatestLeaderOffsetAndHybridTopicOffset( } } else { partitionToGetLatestOffsetFor = partition; - latestLeaderOffset = getLeaderLatestOffset.apply(pcs, sourceRealTimeTopicKafkaURL); + latestLeaderOffset = + getLatestPersistedUpstreamOffsetForHybridOffsetLagMeasurement(pcs, sourceRealTimeTopicKafkaURL); } - long offsetFromConsumer = - getPartitionLatestOffset(sourceRealTimeTopicKafkaURL, leaderTopic, partitionToGetLatestOffsetFor); + PubSubTopic leaderTopic = pcs.getOffsetRecord().getLeaderTopic(pubSubTopicRepository); - long lastOffsetInRealTimeTopic = offsetFromConsumer >= 0 - ? offsetFromConsumer - : cachedPubSubMetadataGetter - .getOffset(getTopicManager(sourceRealTimeTopicKafkaURL), leaderTopic, partitionToGetLatestOffsetFor); + long lastOffsetInRealTimeTopic = + getTopicPartitionEndOffSet(sourceRealTimeTopicKafkaURL, leaderTopic, partitionToGetLatestOffsetFor); + + if (lastOffsetInRealTimeTopic < 0) { + if (!REDUNDANT_LOGGING_FILTER.isRedundantException("Got a negative lastOffsetInRealTimeTopic")) { + LOGGER.warn( + "Unexpected! Got a negative lastOffsetInRealTimeTopic ({})! Will return Long.MAX_VALUE ({}) as the lag.", + lastOffsetInRealTimeTopic, + Long.MAX_VALUE, + new VeniceException("Exception not thrown, just for logging purposes.")); + } + return Long.MAX_VALUE; + } if (latestLeaderOffset == -1) { // If leader hasn't consumed anything yet we should use the value of 0 to calculate the exact offset lag. 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 c5c16673dde..36e514a03bc 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 @@ -1,5 +1,7 @@ package com.linkedin.davinci.kafka.consumer; +import static com.linkedin.davinci.ingestion.LagType.OFFSET_LAG; +import static com.linkedin.davinci.ingestion.LagType.TIME_LAG; import static com.linkedin.davinci.kafka.consumer.ConsumerActionType.RESET_OFFSET; import static com.linkedin.davinci.kafka.consumer.ConsumerActionType.SUBSCRIBE; import static com.linkedin.davinci.kafka.consumer.ConsumerActionType.UNSUBSCRIBE; @@ -15,6 +17,7 @@ import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.config.VeniceStoreVersionConfig; import com.linkedin.davinci.helix.LeaderFollowerPartitionStateModel; +import com.linkedin.davinci.ingestion.LagType; import com.linkedin.davinci.listener.response.AdminResponse; import com.linkedin.davinci.notifier.VeniceNotifier; import com.linkedin.davinci.stats.AggVersionedDIVStats; @@ -80,7 +83,6 @@ import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; import com.linkedin.venice.serializer.AvroGenericDeserializer; import com.linkedin.venice.serializer.FastSerializerDeserializerFactory; -import com.linkedin.venice.stats.StatsErrorCode; import com.linkedin.venice.system.store.MetaStoreWriter; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.ComplementSet; @@ -152,7 +154,6 @@ public abstract class StoreIngestionTask implements Runnable, Closeable { private static final int CONSUMER_ACTION_QUEUE_INIT_CAPACITY = 11; protected static final long KILL_WAIT_TIME_MS = 5000L; private static final int MAX_KILL_CHECKING_ATTEMPTS = 10; - private static final int SLOPPY_OFFSET_CATCHUP_THRESHOLD = 100; protected static final RedundantExceptionFilter REDUNDANT_LOGGING_FILTER = RedundantExceptionFilter.getRedundantExceptionFilter(); @@ -770,7 +771,7 @@ protected abstract boolean checkAndLogIfLagIsAcceptableForHybridStore( long offsetLag, long offsetThreshold, boolean shouldLogLag, - boolean isOffsetBasedLag, + LagType lagType, long latestConsumedProducerTimestamp); /** @@ -804,16 +805,12 @@ protected boolean isReadyToServe(PartitionConsumptionState partitionConsumptionS boolean isLagAcceptable = false; if (!hybridStoreConfig.isPresent()) { - /** - * In theory, it will be 1 offset ahead of the current offset since getOffset returns the next available offset. - * Currently, we make it a sloppy in case Kafka topic have duplicate messages. - * TODO: find a better solution - */ - final long versionTopicPartitionOffset = - cachedPubSubMetadataGetter.getOffset(getTopicManager(localKafkaServer), versionTopic, partitionId); - isLagAcceptable = - versionTopicPartitionOffset <= partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset() - + SLOPPY_OFFSET_CATCHUP_THRESHOLD; + long lag = measureLagWithCallToPubSub( + localKafkaServer, + versionTopic, + partitionId, + partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset()); + isLagAcceptable = lag <= 0; } else { try { // Looks like none of the short-circuitry fired, so we need to measure lag! @@ -833,7 +830,7 @@ protected boolean isReadyToServe(PartitionConsumptionState partitionConsumptionS measureHybridOffsetLag(partitionConsumptionState, shouldLogLag), offsetThreshold, shouldLogLag, - true, + OFFSET_LAG, 0); } @@ -857,7 +854,7 @@ protected boolean isReadyToServe(PartitionConsumptionState partitionConsumptionS LatencyUtils.getElapsedTimeInMs(latestConsumedProducerTimestamp), producerTimeLagThresholdInMS, shouldLogLag, - false, + TIME_LAG, latestConsumedProducerTimestamp); /** * If time lag is not acceptable but the producer timestamp of the last message of RT is smaller or equal than @@ -1931,9 +1928,16 @@ private void reportStoreVersionTopicOffsetRewindMetrics(PartitionConsumptionStat if (offset == OffsetRecord.LOWEST_OFFSET) { return; } + /** + * N.B.: We do not want to use {@link #getTopicPartitionEndOffSet(String, PubSubTopic, int)} because it can return + * a cached value which will result in a false positive in the below check. + */ + long endOffset = aggKafkaConsumerService.getLatestOffsetFor( + localKafkaServer, + versionTopic, + new PubSubTopicPartitionImpl(versionTopic, pcs.getPartition())); // Proceed if persisted OffsetRecord exists and has meaningful content. - long endOffset = getKafkaTopicPartitionEndOffSet(localKafkaServer, versionTopic, pcs.getPartition()); - if (endOffset != StatsErrorCode.LAG_MEASUREMENT_FAILURE.code && offset > endOffset) { + if (endOffset >= 0 && offset > endOffset) { // report offset rewind. LOGGER.warn( "Offset rewind for version topic: {}, partition: {}, persisted record offset: {}, Kafka topic partition end-offset: {}", @@ -1946,21 +1950,19 @@ private void reportStoreVersionTopicOffsetRewindMetrics(PartitionConsumptionStat } /** - * @return the end offset in kafka for the topic partition in SIT. + * @return the end offset in kafka for the topic partition in SIT, or a negative value if it failed to get it. + * + * N.B.: The returned end offset is the last successfully replicated message plus one. If the partition has never been + * written to, the end offset is 0. */ - protected long getKafkaTopicPartitionEndOffSet(String kafkaUrl, PubSubTopic pubSubTopic, int partition) { - long offsetFromConsumer = getPartitionLatestOffset(kafkaUrl, pubSubTopic, partition); + protected long getTopicPartitionEndOffSet(String kafkaUrl, PubSubTopic pubSubTopic, int partition) { + long offsetFromConsumer = aggKafkaConsumerService + .getLatestOffsetFor(kafkaUrl, versionTopic, new PubSubTopicPartitionImpl(pubSubTopic, partition)); if (offsetFromConsumer >= 0) { return offsetFromConsumer; } - /** - * The returned end offset is the last successfully replicated message plus one. If the partition has never been - * written to, the end offset is 0. - * @see CachedPubSubMetadataGetter#getOffset(TopicManager, String, int) - * TODO: Refactor this using PubSubTopicPartition. - */ - return cachedPubSubMetadataGetter.getOffset(getTopicManager(kafkaUrl), versionTopic, partition); + return cachedPubSubMetadataGetter.getOffset(getTopicManager(kafkaUrl), pubSubTopic, partition); } protected long getPartitionOffsetLag(String kafkaSourceAddress, PubSubTopic topic, int partition) { @@ -1968,11 +1970,6 @@ protected long getPartitionOffsetLag(String kafkaSourceAddress, PubSubTopic topi .getOffsetLagFor(kafkaSourceAddress, versionTopic, new PubSubTopicPartitionImpl(topic, partition)); } - protected long getPartitionLatestOffset(String kafkaSourceAddress, PubSubTopic topic, int partition) { - return aggKafkaConsumerService - .getLatestOffsetFor(kafkaSourceAddress, versionTopic, new PubSubTopicPartitionImpl(topic, partition)); - } - protected abstract void checkLongRunningTaskState() throws InterruptedException; protected abstract void processConsumerAction(ConsumerAction message, Store store) throws InterruptedException; @@ -2333,7 +2330,7 @@ private void syncOffset(String topic, PartitionConsumptionState pcs) { private void updateOffsetLagInMetadata(PartitionConsumptionState ps) { // Measure and save real-time offset lag. - long offsetLag = measureHybridOffsetLag(ps, true); + long offsetLag = measureHybridOffsetLag(ps, false); ps.getOffsetRecord().setOffsetLag(offsetLag); } @@ -2372,6 +2369,30 @@ public void recordChecksumVerificationFailure() { public abstract long getHybridLeaderOffsetLag(); + /** + * @param pubSubServerName Pub Sub deployment to interrogate + * @param topic topic to measure + * @param partition for which to measure lag + * @return the lag, or {@value Long#MAX_VALUE} if it failed to measure it + * + * N.B.: Note that the returned lag can be negative since the end offset used in the calculation is cached. + */ + protected long measureLagWithCallToPubSub( + String pubSubServerName, + PubSubTopic topic, + int partition, + long currentOffset) { + if (currentOffset < 0) { + return Long.MAX_VALUE; + } + TopicManager tm = getTopicManager(pubSubServerName); + long endOffset = cachedPubSubMetadataGetter.getOffset(tm, topic, partition) - 1; + if (endOffset < 0) { + return Long.MAX_VALUE; + } + return endOffset - currentOffset; + } + /** * Measure the offset lag between follower and leader */ @@ -3488,18 +3509,10 @@ private ReadyToServeCheck getDefaultReadyToServeChecker() { if (storageEngineReloadedFromRepo == null) { LOGGER.warn("Storage engine {} was removed before reopening", kafkaVersionTopic); } else { - LOGGER.info("Reopen partition {}_{} for reading after ready-to-serve.", kafkaVersionTopic, partition); storageEngineReloadedFromRepo.preparePartitionForReading(partition); } } - if (partitionConsumptionState.isCompletionReported()) { - // Completion has been reported so extraDisjunctionCondition must be true to enter here. - LOGGER.info( - "{} Partition {} synced offset: {}", - consumerTaskId, - partition, - partitionConsumptionState.getLatestProcessedLocalVersionTopicOffset()); - } else { + if (!partitionConsumptionState.isCompletionReported()) { reportCompleted(partitionConsumptionState); warmupSchemaCache(store); } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStatsReporter.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStatsReporter.java index 92c3be7e4a2..07827f7fac9 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStatsReporter.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStatsReporter.java @@ -39,6 +39,8 @@ import io.tehuti.metrics.MetricsRepository; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import java.util.function.DoubleSupplier; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; /** @@ -47,6 +49,8 @@ * collection/visualization system. */ public class IngestionStatsReporter extends AbstractVeniceStatsReporter { + private static final Logger LOGGER = LogManager.getLogger(IngestionStatsReporter.class); + public IngestionStatsReporter(MetricsRepository metricsRepository, String storeName) { super(metricsRepository, storeName); } @@ -159,8 +163,18 @@ protected void registerConditionalStats() { new IngestionStatsGauge(this, () -> getStats().getNearlineLocalBrokerToReadyToServeLatencyMax(), 0)); } + if (getStats() == null) { + LOGGER.warn("Failed to fully registerConditionalStats because getStats() returns null for: {}", storeName); + return; + } else if (getStats().getIngestionTask() == null) { + LOGGER.warn( + "Failed to fully registerConditionalStats because getStats().getIngestionTask() returns null for: {}", + storeName); + return; + } + // Do not need to check store name here as per user system store is not in active/active mode. - if (null != getStats() && getStats().getIngestionTask().isActiveActiveReplicationEnabled()) { + if (getStats().getIngestionTask().isActiveActiveReplicationEnabled()) { registerSensor(UPDATE_IGNORED_DCR, new IngestionStatsGauge(this, () -> getStats().getUpdateIgnoredRate(), 0)); registerSensor(TOTAL_DCR, new IngestionStatsGauge(this, () -> getStats().getTotalDCRRate(), 0)); registerSensor( diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/KafkaConsumerServiceStats.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/KafkaConsumerServiceStats.java index bea76365b60..86e7ac87001 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/KafkaConsumerServiceStats.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/KafkaConsumerServiceStats.java @@ -72,18 +72,15 @@ public KafkaConsumerServiceStats( Sensor getOffsetLagSensor = registerSensor("getOffsetLag", new OccurrenceRate()); Sensor[] offsetLagParent = new Sensor[] { getOffsetLagSensor }; - this.getOffsetLagIsAbsentSensor = - registerSensor("getOffsetLagIsAbsent", null, offsetLagParent, new OccurrenceRate()); - this.getOffsetLagIsPresentSensor = - registerSensor("getOffsetLagIsPresent", null, offsetLagParent, new OccurrenceRate()); + this.getOffsetLagIsAbsentSensor = registerSensor("getOffsetLagIsAbsent", offsetLagParent, new OccurrenceRate()); + this.getOffsetLagIsPresentSensor = registerSensor("getOffsetLagIsPresent", offsetLagParent, new OccurrenceRate()); Sensor getLatestOffsetSensor = registerSensor("getLatestOffset", new OccurrenceRate()); Sensor[] latestOffsetParent = new Sensor[] { getLatestOffsetSensor }; this.getLatestOffsetIsAbsentSensor = - registerSensor("getLatestOffsetIsAbsent", null, latestOffsetParent, new OccurrenceRate()); + registerSensor("getLatestOffsetIsAbsent", latestOffsetParent, new OccurrenceRate()); this.getLatestOffsetIsPresentSensor = - registerSensor("getLatestOffsetIsPresent", null, latestOffsetParent, new OccurrenceRate()); - + registerSensor("getLatestOffsetIsPresent", latestOffsetParent, new OccurrenceRate()); } public void recordPollRequestLatency(double latency) { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStorageEngine.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStorageEngine.java index c065ecd7383..16b57792cc7 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStorageEngine.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/AbstractStorageEngine.java @@ -190,7 +190,6 @@ public synchronized void adjustStoragePartition(StoragePartitionConfig partition int partitionId = partitionConfig.getPartitionId(); AbstractStoragePartition partition = getPartitionOrThrow(partitionId); if (partition.verifyConfig(partitionConfig)) { - LOGGER.info("No adjustment needed for store name: {}, partition id: {}", getStoreName(), partitionId); return; } // Need to re-open storage partition according to the provided partition config diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/rocksdb/RocksDBSstFileWriter.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/rocksdb/RocksDBSstFileWriter.java index 4f9149a2ede..7ebba532a28 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/rocksdb/RocksDBSstFileWriter.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/store/rocksdb/RocksDBSstFileWriter.java @@ -476,8 +476,11 @@ public void ingestSSTFiles(RocksDB rocksDB, List columnFamil rocksDB.ingestExternalFile(columnFamilyHandle, sstFilePaths, ingestOptions); LOGGER.info( - "Finished ingestion to store: " + storeName + ", partition id: " + partitionId + " from files: " - + sstFilePaths); + "Finished {} ingestion to store: {}, partition id: {} from files: {}", + isRMD ? "RMD" : "data", + storeName, + partitionId, + sstFilePaths); } catch (RocksDBException e) { throw new VeniceException("Received exception during RocksDB#ingestExternalFile", e); } 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 b7f8493ba2f..97c4f2fd058 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 @@ -1,8 +1,15 @@ package com.linkedin.davinci.kafka.consumer; import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.IN_TRANSITION_FROM_STANDBY_TO_LEADER; -import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.LEADER; import static com.linkedin.davinci.kafka.consumer.LeaderFollowerStateType.STANDBY; +import static com.linkedin.davinci.kafka.consumer.StoreIngestionTaskTest.AAConfig.AA_OFF; +import static com.linkedin.davinci.kafka.consumer.StoreIngestionTaskTest.AAConfig.AA_ON; +import static com.linkedin.davinci.kafka.consumer.StoreIngestionTaskTest.HybridConfig.HYBRID; +import static com.linkedin.davinci.kafka.consumer.StoreIngestionTaskTest.LeaderCompleteCheck.LEADER_COMPLETE_CHECK_ON; +import static com.linkedin.davinci.kafka.consumer.StoreIngestionTaskTest.NodeType.DA_VINCI; +import static com.linkedin.davinci.kafka.consumer.StoreIngestionTaskTest.NodeType.FOLLOWER; +import static com.linkedin.davinci.kafka.consumer.StoreIngestionTaskTest.NodeType.LEADER; +import static com.linkedin.davinci.kafka.consumer.StoreIngestionTaskTest.SortedInput.SORTED; import static com.linkedin.venice.ConfigKeys.CLUSTER_NAME; import static com.linkedin.venice.ConfigKeys.FREEZE_INGESTION_IF_READY_TO_SERVE_OR_LOCAL_DATA_EXISTS; import static com.linkedin.venice.ConfigKeys.HYBRID_QUOTA_ENFORCEMENT_ENABLED; @@ -31,7 +38,6 @@ import static com.linkedin.venice.writer.LeaderCompleteState.LEADER_COMPLETE_STATE_UNKNOWN; import static com.linkedin.venice.writer.LeaderCompleteState.LEADER_NOT_COMPLETED; import static com.linkedin.venice.writer.VeniceWriter.DEFAULT_LEADER_METADATA_WRAPPER; -import static com.linkedin.venice.writer.VeniceWriter.EMPTY_MSG_HEADERS; import static com.linkedin.venice.writer.VeniceWriter.generateHeartbeatMessage; import static com.linkedin.venice.writer.VeniceWriter.getHeartbeatKME; import static org.mockito.ArgumentMatchers.anyBoolean; @@ -72,6 +78,7 @@ import com.linkedin.davinci.config.VeniceStoreVersionConfig; import com.linkedin.davinci.helix.LeaderFollowerIngestionProgressNotifier; import com.linkedin.davinci.helix.LeaderFollowerPartitionStateModel; +import com.linkedin.davinci.ingestion.LagType; import com.linkedin.davinci.notifier.LogNotifier; import com.linkedin.davinci.notifier.PartitionPushStatusNotifier; import com.linkedin.davinci.notifier.VeniceNotifier; @@ -238,6 +245,7 @@ import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -247,6 +255,46 @@ */ @Test(singleThreaded = true) public abstract class StoreIngestionTaskTest { + enum NodeType { + LEADER, FOLLOWER, DA_VINCI + } + + enum HybridConfig { + HYBRID, BATCH_ONLY + } + + enum AAConfig { + AA_ON, AA_OFF + } + + enum SortedInput { + SORTED, UNSORTED + } + + enum LeaderCompleteCheck { + LEADER_COMPLETE_CHECK_ON, LEADER_COMPLETE_CHECK_OFF + } + + @DataProvider + public static Object[][] aaConfigProvider() { + return DataProviderUtils.allPermutationGenerator(AAConfig.values()); + } + + @DataProvider + public static Object[][] nodeTypeAndAAConfigProvider() { + return DataProviderUtils.allPermutationGenerator(NodeType.values(), AAConfig.values()); + } + + @DataProvider + public static Object[][] hybridConfigAndNodeTypeProvider() { + return DataProviderUtils.allPermutationGenerator(HybridConfig.values(), NodeType.values()); + } + + @DataProvider + public static Object[][] sortedInputAndAAConfigProvider() { + return DataProviderUtils.allPermutationGenerator(SortedInput.values(), AAConfig.values()); + } + private static final Logger LOGGER = LogManager.getLogger(StoreIngestionTaskTest.class); private static final long READ_CYCLE_DELAY_MS = 5; @@ -558,16 +606,15 @@ private long getOffset(Future produceResultFuture) return produceResultFuture.get().getOffset(); } - private void runTest(Set partitions, Runnable assertions, boolean isActiveActiveReplicationEnabled) - throws Exception { - runTest(partitions, () -> {}, assertions, isActiveActiveReplicationEnabled); + private void runTest(Set partitions, Runnable assertions, AAConfig aaConfig) throws Exception { + runTest(partitions, () -> {}, assertions, aaConfig); } private void runTest( Set partitions, Runnable beforeStartingConsumption, Runnable assertions, - boolean isActiveActiveReplicationEnabled) throws Exception { + AAConfig aaConfig) throws Exception { runTest( new RandomPollStrategy(), partitions, @@ -576,7 +623,7 @@ private void runTest( this.hybridStoreConfig, false, Optional.empty(), - isActiveActiveReplicationEnabled, + aaConfig, 1, Collections.emptyMap(), storeVersionConfigOverride -> {}); @@ -586,7 +633,7 @@ private void runTest( Set partitions, Runnable beforeStartingConsumption, Runnable assertions, - boolean isActiveActiveReplicationEnabled, + AAConfig aaConfig, Consumer storeVersionConfigOverride) throws Exception { runTest( new RandomPollStrategy(), @@ -596,7 +643,7 @@ private void runTest( this.hybridStoreConfig, false, Optional.empty(), - isActiveActiveReplicationEnabled, + aaConfig, 1, Collections.emptyMap(), storeVersionConfigOverride); @@ -607,7 +654,7 @@ private void runTest( Set partitions, Runnable beforeStartingConsumption, Runnable assertions, - boolean isActiveActiveReplicationEnabled) throws Exception { + AAConfig aaConfig) throws Exception { runTest( pollStrategy, partitions, @@ -616,7 +663,7 @@ private void runTest( this.hybridStoreConfig, false, Optional.empty(), - isActiveActiveReplicationEnabled, + aaConfig, 1, Collections.emptyMap(), storeVersionConfigOverride -> {}); @@ -630,7 +677,7 @@ private void runTest( Optional hybridStoreConfig, boolean incrementalPushEnabled, Optional diskUsageForTest, - boolean isActiveActiveReplicationEnabled, + AAConfig aaConfig, int amplificationFactor, Map extraServerProperties) throws Exception { runTest( @@ -641,7 +688,7 @@ private void runTest( hybridStoreConfig, incrementalPushEnabled, diskUsageForTest, - isActiveActiveReplicationEnabled, + aaConfig, amplificationFactor, extraServerProperties, storeVersionConfigOverride -> {}); @@ -659,7 +706,7 @@ private void runTest( * @param hybridStoreConfig, the config for hybrid store * @param incrementalPushEnabled, the flag to turn on incremental push for SIT * @param diskUsageForTest, optionally field to mock the disk usage for the test - * @param isActiveActiveReplicationEnabled, the flag to turn on ActiveActiveReplication for SIT + * @param aaConfig, the flag to turn on ActiveActiveReplication for SIT * @param amplificationFactor, the amplificationFactor * @param extraServerProperties, the extra config for server * @param storeVersionConfigOverride, the override for store version config @@ -673,7 +720,7 @@ private void runTest( Optional hybridStoreConfig, boolean incrementalPushEnabled, Optional diskUsageForTest, - boolean isActiveActiveReplicationEnabled, + AAConfig aaConfig, int amplificationFactor, Map extraServerProperties, Consumer storeVersionConfigOverride) throws Exception { @@ -690,7 +737,7 @@ private void runTest( hybridStoreConfig, incrementalPushEnabled, false, - isActiveActiveReplicationEnabled, + aaConfig, storeVersionConfigOverride); Store mockStore = storeAndVersionConfigs.store; Version version = storeAndVersionConfigs.version; @@ -746,14 +793,14 @@ private MockStoreVersionConfigs setupStoreAndVersionMocks( Optional hybridStoreConfig, boolean incrementalPushEnabled, boolean isNativeReplicationEnabled, - boolean isActiveActiveReplicationEnabled) { + AAConfig aaConfig) { return setupStoreAndVersionMocks( partitionCount, partitionerConfig, hybridStoreConfig, incrementalPushEnabled, isNativeReplicationEnabled, - isActiveActiveReplicationEnabled, + aaConfig, storeVersionConfigOverride -> {}); } @@ -763,7 +810,7 @@ private MockStoreVersionConfigs setupStoreAndVersionMocks( Optional hybridStoreConfig, boolean incrementalPushEnabled, boolean isNativeReplicationEnabled, - boolean isActiveActiveReplicationEnabled, + AAConfig aaConfig, Consumer storeVersionConfigOverride) { boolean isHybrid = hybridStoreConfig.isPresent(); HybridStoreConfig hybridSoreConfigValue = null; @@ -805,8 +852,8 @@ private MockStoreVersionConfigs setupStoreAndVersionMocks( doReturn(-1).when(mockStore).getCurrentVersion(); doReturn(1).when(mockStore).getBootstrapToOnlineTimeoutInHours(); - version.setActiveActiveReplicationEnabled(isActiveActiveReplicationEnabled); - doReturn(isActiveActiveReplicationEnabled).when(mockStore).isActiveActiveReplicationEnabled(); + version.setActiveActiveReplicationEnabled(aaConfig == AA_ON); + doReturn(aaConfig == AA_ON).when(mockStore).isActiveActiveReplicationEnabled(); version.setRmdVersionId(REPLICATION_METADATA_VERSION_ID); doReturn(Optional.of(version)).when(mockStore).getVersion(anyInt()); @@ -1126,7 +1173,7 @@ public void testMissingZstdDictionary() throws Exception { .getMessage() .contains("compression Dictionary should not be empty if CompressionStrategy is ZSTD_WITH_DICT")); }); - }, true); + }, AA_ON); } /** @@ -1135,8 +1182,8 @@ public void testMissingZstdDictionary() throws Exception { * 2. A VeniceMessage with DELETE requests leads to invoking of AbstractStorageEngine#delete. * 3. A VeniceMessage with a Kafka offset that was already processed is ignored. */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testVeniceMessagesProcessing(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testVeniceMessagesProcessing(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); PubSubProduceResult putMetadata = (PubSubProduceResult) localVeniceWriter .put(putKeyFoo, putValue, EXISTING_SCHEMA_ID, PUT_KEY_FOO_TIMESTAMP, null) @@ -1157,7 +1204,7 @@ public void testVeniceMessagesProcessing(boolean isActiveActiveReplicationEnable runTest(pollStrategy, Utils.setOf(PARTITION_FOO), () -> {}, () -> { // Verify it retrieves the offset from the OffSet Manager verify(mockStorageMetadataService, timeout(TEST_TIMEOUT_MS)).getLastOffset(topic, PARTITION_FOO); - verifyPutAndDelete(1, isActiveActiveReplicationEnabled, true); + verifyPutAndDelete(1, aaConfig, true); // Verify it commits the offset to Offset Manager OffsetRecord expectedOffsetRecordForDeleteMessage = getOffsetRecord(deleteMetadata.getOffset()); verify(mockStorageMetadataService, timeout(TEST_TIMEOUT_MS)) @@ -1165,14 +1212,14 @@ public void testVeniceMessagesProcessing(boolean isActiveActiveReplicationEnable verify(mockVersionedStorageIngestionStats, timeout(TEST_TIMEOUT_MS).atLeast(3)) .recordConsumedRecordEndToEndProcessingLatency(any(), eq(1), anyDouble(), anyLong()); - }, isActiveActiveReplicationEnabled); + }, aaConfig); // verify the shared consumer should be detached when the ingestion task is closed. verify(aggKafkaConsumerService).unsubscribeAll(pubSubTopic); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testAmplificationFactor(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testAmplificationFactor(AAConfig aaConfig) throws Exception { final int amplificationFactor = 2; inMemoryLocalKafkaBroker .createTopic(Version.composeRealTimeTopic(storeNameWithoutVersionInfo), PARTITION_COUNT / amplificationFactor); @@ -1220,7 +1267,7 @@ public void testAmplificationFactor(boolean isActiveActiveReplicationEnabled) th rtWriter.put(putKeyFoo, putValue, EXISTING_SCHEMA_ID, PUT_KEY_FOO_TIMESTAMP, null).get(); rtWriter.delete(deleteKeyFoo, DELETE_KEY_FOO_TIMESTAMP, null).get(); - verifyPutAndDelete(amplificationFactor, isActiveActiveReplicationEnabled, false); + verifyPutAndDelete(amplificationFactor, aaConfig, false); } catch (Exception e) { e.printStackTrace(); } @@ -1228,14 +1275,13 @@ public void testAmplificationFactor(boolean isActiveActiveReplicationEnabled) th Optional.of(hybridStoreConfig), false, Optional.empty(), - isActiveActiveReplicationEnabled, + aaConfig, amplificationFactor, Collections.singletonMap(SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS, 3L)); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testMissingMessagesForTopicWithLogCompactionEnabled(boolean isActiveActiveReplicationEnabled) - throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testMissingMessagesForTopicWithLogCompactionEnabled(AAConfig aaConfig) throws Exception { // enable log compaction when(mockTopicManager.isTopicCompactionEnabled(pubSubTopic)).thenReturn(true); @@ -1279,12 +1325,11 @@ public void testMissingMessagesForTopicWithLogCompactionEnabled(boolean isActive OffsetRecord expectedOffsetRecordForLastMessage = getOffsetRecord(putMetadata4.getOffset()); verify(mockStorageMetadataService, timeout(TEST_TIMEOUT_MS)) .put(topic, PARTITION_FOO, expectedOffsetRecordForLastMessage); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testVeniceMessagesProcessingWithExistingSchemaId(boolean isActiveActiveReplicationEnabled) - throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testVeniceMessagesProcessingWithExistingSchemaId(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); long fooLastOffset = getOffset(localVeniceWriter.put(putKeyFoo, putValue, EXISTING_SCHEMA_ID)); @@ -1302,16 +1347,15 @@ public void testVeniceMessagesProcessingWithExistingSchemaId(boolean isActiveAct // Verify it commits the offset to Offset Manager OffsetRecord expected = getOffsetRecord(fooLastOffset); verify(mockStorageMetadataService, timeout(TEST_TIMEOUT_MS)).put(topic, PARTITION_FOO, expected); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } /** * Test the situation where records arrive faster than the schemas. * In this case, Venice would keep polling schemaRepo until schemas arrive. */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testVeniceMessagesProcessingWithTemporarilyNotAvailableSchemaId(boolean isActiveActiveReplicationEnabled) - throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testVeniceMessagesProcessingWithTemporarilyNotAvailableSchemaId(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.put(putKeyFoo, putValue, NON_EXISTING_SCHEMA_ID); long existingSchemaOffset = getOffset(localVeniceWriter.put(putKeyFoo, putValue, EXISTING_SCHEMA_ID)); @@ -1339,15 +1383,14 @@ public void testVeniceMessagesProcessingWithTemporarilyNotAvailableSchemaId(bool OffsetRecord expected = getOffsetRecord(existingSchemaOffset); verify(mockStorageMetadataService, timeout(TEST_TIMEOUT_MS)).put(topic, PARTITION_FOO, expected); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } /** * Test the situation where records' schemas never arrive. In the case, the StoreIngestionTask will keep being blocked. */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testVeniceMessagesProcessingWithNonExistingSchemaId(boolean isActiveActiveReplicationEnabled) - throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testVeniceMessagesProcessingWithNonExistingSchemaId(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.put(putKeyFoo, putValue, NON_EXISTING_SCHEMA_ID); localVeniceWriter.put(putKeyFoo, putValue, EXISTING_SCHEMA_ID); @@ -1369,11 +1412,11 @@ public void testVeniceMessagesProcessingWithNonExistingSchemaId(boolean isActive // Only two records(start_of_segment, start_of_push) offset were able to be recorded before // 'NON_EXISTING_SCHEMA_ID' blocks #putConsumerRecord verify(mockStorageMetadataService, atMost(2)).put(eq(topic), eq(PARTITION_FOO), any(OffsetRecord.class)); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testReportStartWhenRestarting(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testReportStartWhenRestarting(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); final long STARTING_OFFSET = 2; runTest(Utils.setOf(PARTITION_FOO, PARTITION_BAR), () -> { @@ -1381,16 +1424,24 @@ public void testReportStartWhenRestarting(boolean isActiveActiveReplicationEnabl }, () -> { // Verify STARTED is NOT reported when offset is 0 verify(mockLogNotifier, never()).started(topic, PARTITION_BAR); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testNotifier(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testNotifier(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); long fooLastOffset = getOffset(localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID)); long barLastOffset = getOffset(localVeniceWriter.put(putKeyBar, putValue, SCHEMA_ID)); localVeniceWriter.broadcastEndOfPush(new HashMap<>()); localVeniceWriter.broadcastEndOfPush(new HashMap<>()); + doReturn(fooLastOffset + 1).when(mockTopicManager) + .getPartitionLatestOffsetAndRetry( + argThat(argument -> argument.getPartitionNumber() == PARTITION_FOO), + anyInt()); + doReturn(barLastOffset + 1).when(mockTopicManager) + .getPartitionLatestOffsetAndRetry( + argThat(argument -> argument.getPartitionNumber() == PARTITION_BAR), + anyInt()); runTest(Utils.setOf(PARTITION_FOO, PARTITION_BAR), () -> { /** @@ -1425,11 +1476,11 @@ public void testNotifier(boolean isActiveActiveReplicationEnabled) throws Except verify(mockPartitionStatusNotifier, atLeastOnce()).started(topic, PARTITION_BAR); verify(mockPartitionStatusNotifier, atLeastOnce()).endOfPushReceived(topic, PARTITION_FOO, fooLastOffset); verify(mockPartitionStatusNotifier, atLeastOnce()).endOfPushReceived(topic, PARTITION_BAR, barLastOffset); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testReadyToServePartition(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testReadyToServePartition(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.broadcastEndOfPush(new HashMap<>()); @@ -1448,12 +1499,11 @@ public void testReadyToServePartition(boolean isActiveActiveReplicationEnabled) 10, TimeUnit.SECONDS, () -> verify(mockAbstractStorageEngine, atLeastOnce()).preparePartitionForReading(PARTITION_FOO)); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testReadyToServePartitionValidateIngestionSuccess(boolean isActiveActiveReplicationEnabled) - throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testReadyToServePartitionValidateIngestionSuccess(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.broadcastEndOfPush(new HashMap<>()); Store mockStore = mock(Store.class); @@ -1468,11 +1518,11 @@ public void testReadyToServePartitionValidateIngestionSuccess(boolean isActiveAc runTest(Utils.setOf(PARTITION_FOO), () -> { verify(mockAbstractStorageEngine, never()).preparePartitionForReading(PARTITION_FOO); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testReadyToServePartitionWriteOnly(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testReadyToServePartitionWriteOnly(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.broadcastEndOfPush(new HashMap<>()); Store mockStore = mock(Store.class); @@ -1489,11 +1539,11 @@ public void testReadyToServePartitionWriteOnly(boolean isActiveActiveReplication runTest(Utils.setOf(PARTITION_FOO), () -> { verify(mockAbstractStorageEngine, never()).preparePartitionForReading(PARTITION_FOO); verify(mockAbstractStorageEngine, never()).preparePartitionForReading(PARTITION_BAR); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testResetPartition(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testResetPartition(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID).get(); @@ -1506,11 +1556,11 @@ public void testResetPartition(boolean isActiveActiveReplicationEnabled) throws verify(mockStorageMetadataService, timeout(TEST_TIMEOUT_MS)).clearOffset(topic, PARTITION_FOO); verify(mockAbstractStorageEngine, timeout(TEST_TIMEOUT_MS).times(2)) .put(PARTITION_FOO, putKeyFoo, ByteBuffer.wrap(ValueRecord.create(SCHEMA_ID, putValue).serialize())); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testResetPartitionAfterUnsubscription(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testResetPartitionAfterUnsubscription(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID).get(); @@ -1527,7 +1577,7 @@ public void testResetPartitionAfterUnsubscription(boolean isActiveActiveReplicat // StoreIngestionTask won't invoke consumer.resetOffset() if it already unsubscribe from that topic/partition verify(mockLocalKafkaConsumer, timeout(TEST_TIMEOUT_MS).times(0)).resetOffset(fooTopicPartition); verify(mockStorageMetadataService, timeout(TEST_TIMEOUT_MS)).clearOffset(topic, PARTITION_FOO); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } /** @@ -1535,8 +1585,8 @@ public void testResetPartitionAfterUnsubscription(boolean isActiveActiveReplicat * * The {@link VeniceNotifier} should see the completion and error reported for the appropriate partitions. */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testDetectionOfMissingRecord(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testDetectionOfMissingRecord(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); long fooLastOffset = getOffset(localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID)); long barOffsetToSkip = getOffset(localVeniceWriter.put(putKeyBar, putValue, SCHEMA_ID)); @@ -1561,15 +1611,15 @@ public void testDetectionOfMissingRecord(boolean isActiveActiveReplicationEnable verify(mockLogNotifier, atLeastOnce()).started(topic, PARTITION_FOO); verify(mockLogNotifier, atLeastOnce()).started(topic, PARTITION_BAR); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } /** * In this test, partition FOO will complete normally, but partition BAR will contain a duplicate record. The * {@link VeniceNotifier} should see the completion for both partitions. */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testSkippingOfDuplicateRecord(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testSkippingOfDuplicateRecord(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); long fooLastOffset = getOffset(localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID)); long barOffsetToDupe = getOffset(localVeniceWriter.put(putKeyBar, putValue, SCHEMA_ID)); @@ -1593,11 +1643,11 @@ public void testSkippingOfDuplicateRecord(boolean isActiveActiveReplicationEnabl verify(mockLogNotifier, atLeastOnce()).started(topic, PARTITION_FOO); verify(mockLogNotifier, atLeastOnce()).started(topic, PARTITION_BAR); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testThrottling(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testThrottling(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID); localVeniceWriter.delete(deleteKeyFoo, null); @@ -1606,7 +1656,7 @@ public void testThrottling(boolean isActiveActiveReplicationEnabled) throws Exce // START_OF_SEGMENT, START_OF_PUSH, PUT, DELETE verify(mockRecordsThrottler, timeout(TEST_TIMEOUT_MS).times(4)).maybeThrottle(1); verify(mockBandwidthThrottler, timeout(TEST_TIMEOUT_MS).times(4)).maybeThrottle(anyDouble()); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } /** @@ -1614,8 +1664,8 @@ public void testThrottling(boolean isActiveActiveReplicationEnabled) throws Exce * message in {@link #PARTITION_FOO} will receive a bad message type, whereas the message in {@link #PARTITION_BAR} * will receive a bad control message type. */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testBadMessageTypesFailFast(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testBadMessageTypesFailFast(AAConfig aaConfig) throws Exception { int badMessageTypeId = 99; // Venice got 99 problems, but a bad message type ain't one. // Dear future maintainer, @@ -1680,7 +1730,7 @@ public void testBadMessageTypesFailFast(boolean isActiveActiveReplicationEnabled runTest(Utils.setOf(PARTITION_FOO, PARTITION_BAR), () -> { verify(kafkaConsumerServiceStats, timeout(TEST_TIMEOUT_MS).atLeastOnce()).recordPollError(); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } /** @@ -1688,8 +1738,8 @@ public void testBadMessageTypesFailFast(boolean isActiveActiveReplicationEnabled * including a corrupt message followed by a good one. We expect the Notifier to not report any errors after the * EOP. */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testCorruptMessagesDoNotFailFastAfterEOP(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testCorruptMessagesDoNotFailFastAfterEOP(AAConfig aaConfig) throws Exception { VeniceWriter veniceWriterForDataDuringPush = getVeniceWriter(new MockInMemoryProducerAdapter(inMemoryLocalKafkaBroker)); VeniceWriter veniceWriterForDataAfterPush = getCorruptedVeniceWriter(putValueToCorrupt, inMemoryLocalKafkaBroker); @@ -1734,7 +1784,7 @@ public void testCorruptMessagesDoNotFailFastAfterEOP(boolean isActiveActiveRepli && args[3] instanceof CorruptDataException); } - }, isActiveActiveReplicationEnabled); + }, aaConfig); } catch (VerifyError e) { StringBuilder msg = new StringBuilder(); ClassLoader cl = ClassLoader.getSystemClassLoader(); @@ -1758,8 +1808,8 @@ public void testCorruptMessagesDoNotFailFastAfterEOP(boolean isActiveActiveRepli * including a corrupt message followed by a missing message and a good one. * We expect the Notifier to not report any errors after the EOP. */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testDIVErrorMessagesNotFailFastAfterEOP(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testDIVErrorMessagesNotFailFastAfterEOP(AAConfig aaConfig) throws Exception { VeniceWriter veniceWriterCorrupted = getCorruptedVeniceWriter(putValueToCorrupt, inMemoryLocalKafkaBroker); // do a batch push @@ -1793,7 +1843,7 @@ public void testDIVErrorMessagesNotFailFastAfterEOP(boolean isActiveActiveReplic args[0].equals(topic) && args[1].equals(PARTITION_FOO) && ((String) args[2]).length() > 0 && args[3] instanceof FatalDataValidationException); } - }, isActiveActiveReplicationEnabled); + }, aaConfig); } /** @@ -1804,8 +1854,8 @@ public void testDIVErrorMessagesNotFailFastAfterEOP(boolean isActiveActiveReplic * should ensure that if this test is ever made flaky again, it will be detected right away. The skipFailedInvocations * annotation parameter makes the test skip any invocation after the first failure. */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class, invocationCount = 100, skipFailedInvocations = true) - public void testCorruptMessagesFailFast(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider", invocationCount = 100, skipFailedInvocations = true) + public void testCorruptMessagesFailFast(AAConfig aaConfig) throws Exception { VeniceWriter veniceWriterForData = getCorruptedVeniceWriter(putValueToCorrupt, inMemoryLocalKafkaBroker); localVeniceWriter.broadcastStartOfPush(new HashMap<>()); @@ -1834,11 +1884,11 @@ public void testCorruptMessagesFailFast(boolean isActiveActiveReplicationEnabled * this test is to detect this edge case. */ verify(mockLogNotifier, never()).completed(eq(topic), eq(PARTITION_BAR), anyLong()); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testSubscribeCompletedPartition(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testSubscribeCompletedPartition(AAConfig aaConfig) throws Exception { final int offset = 100; localVeniceWriter.broadcastStartOfPush(new HashMap<>()); runTest( @@ -1848,11 +1898,11 @@ public void testSubscribeCompletedPartition(boolean isActiveActiveReplicationEna () -> { verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS)).completed(topic, PARTITION_FOO, offset, "STANDBY"); }, - isActiveActiveReplicationEnabled); + aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testSubscribeCompletedPartitionUnsubscribe(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testSubscribeCompletedPartitionUnsubscribe(AAConfig aaConfig) throws Exception { final int offset = 100; final long LONG_TEST_TIMEOUT = 2 * TEST_TIMEOUT_MS; localVeniceWriter.broadcastStartOfPush(new HashMap<>()); @@ -1874,12 +1924,11 @@ public void testSubscribeCompletedPartitionUnsubscribe(boolean isActiveActiveRep verify(mockLocalKafkaConsumer, timeout(LONG_TEST_TIMEOUT)) .batchUnsubscribe(Collections.singleton(fooTopicPartition)); verify(mockLocalKafkaConsumer, never()).unSubscribe(barTopicPartition); - }, this.hybridStoreConfig, false, Optional.empty(), isActiveActiveReplicationEnabled, 1, extraServerProperties); + }, this.hybridStoreConfig, false, Optional.empty(), aaConfig, 1, extraServerProperties); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testCompleteCalledWhenUnsubscribeAfterBatchPushDisabled(boolean isActiveActiveReplicationEnabled) - throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testCompleteCalledWhenUnsubscribeAfterBatchPushDisabled(AAConfig aaConfig) throws Exception { final int offset = 10; localVeniceWriter.broadcastStartOfPush(new HashMap<>()); @@ -1893,11 +1942,11 @@ public void testCompleteCalledWhenUnsubscribeAfterBatchPushDisabled(boolean isAc doReturn(getOffsetRecord(offset, true)).when(mockStorageMetadataService).getLastOffset(topic, PARTITION_FOO); }, () -> verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS)).completed(topic, PARTITION_FOO, offset, "STANDBY"), - isActiveActiveReplicationEnabled); + aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testUnsubscribeConsumption(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testUnsubscribeConsumption(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID); @@ -1906,11 +1955,11 @@ public void testUnsubscribeConsumption(boolean isActiveActiveReplicationEnabled) // Start of push has already been consumed. Stop consumption storeIngestionTaskUnderTest.unSubscribePartition(fooTopicPartition); verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS)).stopped(anyString(), anyInt(), anyLong()); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testKillConsumption(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testKillConsumption(AAConfig aaConfig) throws Exception { final Thread writingThread = new Thread(() -> { while (true) { localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID); @@ -1947,14 +1996,14 @@ public void testKillConsumption(boolean isActiveActiveReplicationEnabled) throws TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS, () -> storeIngestionTaskUnderTest.isRunning() == false); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } finally { TestUtils.shutdownThread(writingThread); } } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testKillActionPriority(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testKillActionPriority(AAConfig aaConfig) throws Exception { runTest(Utils.setOf(PARTITION_FOO), () -> { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID); @@ -1981,7 +2030,7 @@ public void testKillActionPriority(boolean isActiveActiveReplicationEnabled) thr TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS, () -> storeIngestionTaskUnderTest.isRunning() == false); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } private byte[] getNumberedKey(int number) { @@ -1992,17 +2041,16 @@ private byte[] getNumberedValue(int number) { return ByteBuffer.allocate(putValue.length + Integer.BYTES).put(putValue).putInt(number).array(); } - @Test(dataProvider = "Two-True-and-False", dataProviderClass = DataProviderUtils.class) - public void testDataValidationCheckPointing(boolean sortedInput, boolean isActiveActiveReplicationEnabled) - throws Exception { + @Test(dataProvider = "sortedInputAndAAConfigProvider") + public void testDataValidationCheckPointing(SortedInput sortedInput, AAConfig aaConfig) throws Exception { final Map maxOffsetPerPartition = new HashMap<>(); final Map, ByteArray> pushedRecords = new HashMap<>(); final int totalNumberOfMessages = 1000; final int totalNumberOfConsumptionRestarts = 10; final long LONG_TEST_TIMEOUT = 2 * TEST_TIMEOUT_MS; - setStoreVersionStateSupplier(sortedInput); - localVeniceWriter.broadcastStartOfPush(sortedInput, new HashMap<>()); + setStoreVersionStateSupplier(sortedInput == SORTED); + localVeniceWriter.broadcastStartOfPush(sortedInput == SORTED, new HashMap<>()); for (int i = 0; i < totalNumberOfMessages; i++) { byte[] key = getNumberedKey(i); byte[] value = getNumberedValue(i); @@ -2078,7 +2126,7 @@ public void testDataValidationCheckPointing(boolean sortedInput, boolean isActiv verify(mockLocalKafkaConsumer, timeout(LONG_TEST_TIMEOUT).atLeast(totalNumberOfConsumptionRestarts)) .unSubscribe(eq(pubSubTopicPartition)); - if (sortedInput) { + if (sortedInput == SORTED) { // Check database mode switches from deferred-write to transactional after EOP control message StoragePartitionConfig deferredWritePartitionConfig = new StoragePartitionConfig(topic, partition); deferredWritePartitionConfig.setDeferredWrite(true); @@ -2111,11 +2159,11 @@ public void testDataValidationCheckPointing(boolean sortedInput, boolean isActiv PartitionConsumptionState pcs = storeIngestionTaskUnderTest.getPartitionConsumptionState(partition); Assert.assertTrue(pcs.getLatestProcessedUpstreamRTOffsetMap().isEmpty()); }); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testKillAfterPartitionIsCompleted(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testKillAfterPartitionIsCompleted(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); long fooLastOffset = getOffset(localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID)); localVeniceWriter.broadcastEndOfPush(new HashMap<>()); @@ -2127,11 +2175,11 @@ public void testKillAfterPartitionIsCompleted(boolean isActiveActiveReplicationE storeIngestionTaskUnderTest.kill(); verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS).atLeastOnce()) .endOfPushReceived(topic, PARTITION_FOO, fooLastOffset); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testNeverReportProgressBeforeStart(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testNeverReportProgressBeforeStart(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); // Read one message for each poll. runTest(new RandomPollStrategy(1), Utils.setOf(PARTITION_FOO), () -> {}, () -> { @@ -2141,11 +2189,11 @@ public void testNeverReportProgressBeforeStart(boolean isActiveActiveReplication // of messages in bytes, since control message is being counted as 0 bytes (no data persisted in disk), // then no progress will be reported during start, but only for processed messages. verify(mockLogNotifier, after(TEST_TIMEOUT_MS).never()).progress(any(), anyInt(), anyInt()); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testOffsetPersistent(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testOffsetPersistent(AAConfig aaConfig) throws Exception { // Do not persist every message. List offsets = new ArrayList<>(); for (int i = 0; i < PARTITION_COUNT; i++) { @@ -2174,7 +2222,7 @@ public void testOffsetPersistent(boolean isActiveActiveReplicationEnabled) throw BufferReplayPolicy.REWIND_FROM_EOP)), false, Optional.empty(), - isActiveActiveReplicationEnabled, + aaConfig, 1, Collections.emptyMap()); } finally { @@ -2183,8 +2231,8 @@ public void testOffsetPersistent(boolean isActiveActiveReplicationEnabled) throw } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testVeniceMessagesProcessingWithSortedInput(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testVeniceMessagesProcessingWithSortedInput(AAConfig aaConfig) throws Exception { setStoreVersionStateSupplier(true); localVeniceWriter.broadcastStartOfPush(true, new HashMap<>()); PubSubProduceResult putMetadata = @@ -2196,7 +2244,7 @@ public void testVeniceMessagesProcessingWithSortedInput(boolean isActiveActiveRe // Verify it retrieves the offset from the Offset Manager verify(mockStorageMetadataService, timeout(TEST_TIMEOUT_MS)).getLastOffset(topic, PARTITION_FOO); - verifyPutAndDelete(1, isActiveActiveReplicationEnabled, true); + verifyPutAndDelete(1, aaConfig, true); // Verify it commits the offset to Offset Manager after receiving EOP control message OffsetRecord expectedOffsetRecordForDeleteMessage = getOffsetRecord(deleteMetadata.getOffset() + 1, true); @@ -2214,12 +2262,11 @@ public void testVeniceMessagesProcessingWithSortedInput(boolean isActiveActiveRe .beginBatchWrite(eq(deferredWritePartitionConfig), any(), eq(Optional.empty())); StoragePartitionConfig transactionalPartitionConfig = new StoragePartitionConfig(topic, PARTITION_FOO); verify(mockAbstractStorageEngine, times(1)).endBatchWrite(transactionalPartitionConfig); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testVeniceMessagesProcessingWithSortedInputVerifyChecksum(boolean isActiveActiveReplicationEnabled) - throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testVeniceMessagesProcessingWithSortedInputVerifyChecksum(AAConfig aaConfig) throws Exception { databaseChecksumVerificationEnabled = true; doReturn(false).when(rocksDBServerConfig).isRocksDBPlainTableFormatEnabled(); setStoreVersionStateSupplier(true); @@ -2251,11 +2298,11 @@ public void testVeniceMessagesProcessingWithSortedInputVerifyChecksum(boolean is Optional> checksumSupplier = checksumCaptor.getValue(); Assert.assertTrue(checksumSupplier.isPresent()); Assert.assertTrue(Arrays.equals(checksumSupplier.get().get(), checksum.getCheckSum())); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testDelayedTransitionToOnlineInHybridMode(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testDelayedTransitionToOnlineInHybridMode(AAConfig aaConfig) throws Exception { final long MESSAGES_BEFORE_EOP = 100; final long MESSAGES_AFTER_EOP = 100; mockStorageMetadataService = new InMemoryStorageMetadataService(); @@ -2266,16 +2313,34 @@ public void testDelayedTransitionToOnlineInHybridMode(boolean isActiveActiveRepl HybridStoreConfigImpl.DEFAULT_HYBRID_TIME_LAG_THRESHOLD, DataReplicationPolicy.NON_AGGREGATE, BufferReplayPolicy.REWIND_FROM_EOP)); + long[] messageCountPerPartition = new long[PARTITION_COUNT]; + + when(mockTopicManager.getPartitionLatestOffsetAndRetry(any(), anyInt())).thenAnswer(invocation -> { + PubSubTopicPartition pt = invocation.getArgument(0); + return messageCountPerPartition[pt.getPartitionNumber()]; + }); + runTest(ALL_PARTITIONS, () -> { localVeniceWriter.broadcastStartOfPush(Collections.emptyMap()); + for (int partition: ALL_PARTITIONS) { + // Taking into account both the initial SOS and the SOP + messageCountPerPartition[partition] += 2; + } for (int i = 0; i < MESSAGES_BEFORE_EOP; i++) { try { - localVeniceWriter.put(getNumberedKey(i), getNumberedValue(i), SCHEMA_ID).get(); + CompletableFuture future = + localVeniceWriter.put(getNumberedKey(i), getNumberedValue(i), SCHEMA_ID); + PubSubProduceResult result = future.get(); + int partition = result.getPartition(); + messageCountPerPartition[partition]++; } catch (InterruptedException | ExecutionException e) { throw new VeniceException(e); } } localVeniceWriter.broadcastEndOfPush(Collections.emptyMap()); + for (int partition: ALL_PARTITIONS) { + messageCountPerPartition[partition]++; + } }, () -> { verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS).atLeast(ALL_PARTITIONS.size())).started(eq(topic), anyInt()); @@ -2286,10 +2351,17 @@ public void testDelayedTransitionToOnlineInHybridMode(boolean isActiveActiveRepl Version.composeRealTimeTopic(storeNameWithoutVersionInfo), System.currentTimeMillis() - TimeUnit.SECONDS.toMillis(10), Collections.emptyMap()); + for (int partition: ALL_PARTITIONS) { + messageCountPerPartition[partition]++; + } for (int i = 0; i < MESSAGES_AFTER_EOP; i++) { try { - localVeniceWriter.put(getNumberedKey(i), getNumberedValue(i), SCHEMA_ID).get(); + CompletableFuture future = + localVeniceWriter.put(getNumberedKey(i), getNumberedValue(i), SCHEMA_ID); + PubSubProduceResult result = future.get(); + int partition = result.getPartition(); + messageCountPerPartition[partition]++; } catch (InterruptedException | ExecutionException e) { throw new VeniceException(e); } @@ -2310,12 +2382,13 @@ public void testDelayedTransitionToOnlineInHybridMode(boolean isActiveActiveRepl true, LeaderCompleteState.getLeaderCompleteState(true), System.currentTimeMillis()); + messageCountPerPartition[partition]++; } verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS).atLeast(ALL_PARTITIONS.size())) .completed(anyString(), anyInt(), anyLong(), anyString()); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } /** @@ -2324,8 +2397,8 @@ public void testDelayedTransitionToOnlineInHybridMode(boolean isActiveActiveRepl * the record, it will receive a disk full error. This test checks for that disk full error on the Notifier object. * @throws Exception */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testStoreIngestionTaskRespectsDiskUsage(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testStoreIngestionTaskRespectsDiskUsage(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.put(putKeyFoo, putValue, EXISTING_SCHEMA_ID); localVeniceWriter.broadcastEndOfPush(new HashMap<>()); @@ -2358,13 +2431,13 @@ public void testStoreIngestionTaskRespectsDiskUsage(boolean isActiveActiveReplic Optional.empty(), false, Optional.of(diskFullUsage), - isActiveActiveReplicationEnabled, + aaConfig, 1, Collections.emptyMap()); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class, invocationCount = 10) - public void testIncrementalPush(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testIncrementalPush(AAConfig aaConfig) throws Exception { setStoreVersionStateSupplier(true); localVeniceWriter.broadcastStartOfPush(true, new HashMap<>()); long fooOffset = getOffset(localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID)); @@ -2401,17 +2474,11 @@ public void testIncrementalPush(boolean isActiveActiveReplicationEnabled) throws verify(mockLogNotifier, atLeastOnce()) .endOfIncrementalPushReceived(topic, PARTITION_FOO, fooNewOffset, version); }); - }, - Optional.of(hybridStoreConfig), - true, - Optional.empty(), - isActiveActiveReplicationEnabled, - 1, - Collections.emptyMap()); + }, Optional.of(hybridStoreConfig), true, Optional.empty(), aaConfig, 1, Collections.emptyMap()); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testSchemaCacheWarming(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testSchemaCacheWarming(AAConfig aaConfig) throws Exception { setStoreVersionStateSupplier(true); localVeniceWriter.broadcastStartOfPush(true, new HashMap<>()); long fooOffset = getOffset(localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID)); @@ -2440,13 +2507,13 @@ public void testSchemaCacheWarming(boolean isActiveActiveReplicationEnabled) thr Optional.empty(), false, Optional.empty(), - isActiveActiveReplicationEnabled, + aaConfig, 1, Collections.singletonMap(SERVER_NUM_SCHEMA_FAST_CLASS_WARMUP, 1)); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testReportErrorWithEmptyPcsMap(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testReportErrorWithEmptyPcsMap(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); localVeniceWriter.put(putKeyFoo, putValue, EXISTING_SCHEMA_ID); // Dummy exception to put ingestion task into ERROR state @@ -2455,11 +2522,11 @@ public void testReportErrorWithEmptyPcsMap(boolean isActiveActiveReplicationEnab runTest(Utils.setOf(PARTITION_FOO), () -> { verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS)).error(eq(topic), eq(PARTITION_FOO), anyString(), any()); - }, isActiveActiveReplicationEnabled); + }, aaConfig); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class, timeOut = TEST_TIMEOUT_MS * 5) - public void testPartitionExceptionIsolation(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testPartitionExceptionIsolation(AAConfig aaConfig) throws Exception { localVeniceWriter.broadcastStartOfPush(new HashMap<>()); getOffset(localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID)); long barLastOffset = getOffset(localVeniceWriter.put(putKeyBar, putValue, SCHEMA_ID)); @@ -2482,7 +2549,7 @@ public void testPartitionExceptionIsolation(boolean isActiveActiveReplicationEna storeIngestionTaskUnderTest.getPartitionIngestionExceptionList().get(PARTITION_FOO), "Exception for the errored partition should be cleared after unsubscription"); assertEquals(storeIngestionTaskUnderTest.getFailedPartitions().size(), 1, "Only one partition should be failed"); - }, isActiveActiveReplicationEnabled); + }, aaConfig); for (int i = 0; i < 10000; ++i) { storeIngestionTaskUnderTest .setIngestionException(0, new VeniceException("new fake looooooooooooooooong exception")); @@ -2517,16 +2584,13 @@ private VeniceServerConfig buildVeniceServerConfig(Map extraProp return new VeniceServerConfig(propertyBuilder.build(), kafkaClusterMap); } - private void verifyPutAndDelete( - int amplificationFactor, - boolean isActiveActiveReplicationEnabled, - boolean recordsInBatchPush) { + private void verifyPutAndDelete(int amplificationFactor, AAConfig aaConfig, boolean recordsInBatchPush) { VenicePartitioner partitioner = getVenicePartitioner(amplificationFactor); int targetPartitionPutKeyFoo = partitioner.getPartitionId(putKeyFoo, PARTITION_COUNT); int targetPartitionDeleteKeyFoo = partitioner.getPartitionId(deleteKeyFoo, PARTITION_COUNT); // Batch push records for Active/Active do not persist replication metadata. - if (isActiveActiveReplicationEnabled && !recordsInBatchPush) { + if (aaConfig == AA_ON && !recordsInBatchPush) { // Verify StorageEngine#putWithReplicationMetadata is invoked only once and with appropriate key & value. verify(mockAbstractStorageEngine, timeout(100000)).putWithReplicationMetadata( targetPartitionPutKeyFoo, @@ -2590,7 +2654,7 @@ public void testRecordsCanBeThrottledPerRegion() throws ExecutionException, Inte Optional.of(hybridStoreConfig), false, false, - true); + AA_ON); Store mockStore = storeAndVersionConfigs.store; Version version = storeAndVersionConfigs.version; VeniceStoreVersionConfig storeConfig = storeAndVersionConfigs.storeVersionConfig; @@ -2705,13 +2769,8 @@ public void testRecordsCanBeThrottledPerRegion() throws ExecutionException, Inte "Remote consumer should not poll for new records but return previously cached records"); } - @Test(dataProvider = "Three-True-and-False", dataProviderClass = DataProviderUtils.class) - public void testIsReadyToServe(boolean isDaVinciClient, boolean isLeader, boolean isActiveActiveReplicationEnabled) { - if (isDaVinciClient && isLeader) { - // DaVinci client can't be leader - return; - } - + @Test(dataProvider = "nodeTypeAndAAConfigProvider") + public void testIsReadyToServe(NodeType nodeType, AAConfig aaConfig) { int partitionCount = 2; int amplificationFactor = 1; @@ -2733,7 +2792,7 @@ public void testIsReadyToServe(boolean isDaVinciClient, boolean isLeader, boolea Optional.of(hybridStoreConfig), false, true, - isActiveActiveReplicationEnabled); + aaConfig); Store mockStore = storeAndVersionConfigs.store; Version version = storeAndVersionConfigs.version; VeniceStoreVersionConfig storeConfig = storeAndVersionConfigs.storeVersionConfig; @@ -2749,7 +2808,7 @@ public void testIsReadyToServe(boolean isDaVinciClient, boolean isLeader, boolea Optional.empty(), 1, extraServerProperties, - false).setIsDaVinciClient(isDaVinciClient).setAggKafkaConsumerService(aggKafkaConsumerService).build(); + false).setIsDaVinciClient(nodeType == DA_VINCI).setAggKafkaConsumerService(aggKafkaConsumerService).build(); TopicManager mockTopicManagerRemoteKafka = mock(TopicManager.class); @@ -2846,9 +2905,9 @@ public void testIsReadyToServe(boolean isDaVinciClient, boolean isLeader, boolea doReturn(0).when(mockPcsBufferReplayStartedLagCaughtUp).getPartition(); doReturn(0).when(mockPcsBufferReplayStartedLagCaughtUp).getUserPartition(); storeIngestionTaskUnderTest.setPartitionConsumptionState(0, mockPcsBufferReplayStartedLagCaughtUp); - if (isLeader) { + if (nodeType == NodeType.LEADER) { // case 5a: leader replica => partition is ready to serve - doReturn(LEADER).when(mockPcsBufferReplayStartedLagCaughtUp).getLeaderFollowerState(); + doReturn(LeaderFollowerStateType.LEADER).when(mockPcsBufferReplayStartedLagCaughtUp).getLeaderFollowerState(); assertTrue(storeIngestionTaskUnderTest.isReadyToServe(mockPcsBufferReplayStartedLagCaughtUp)); } else { // case 5b: standby replica and !LEADER_COMPLETED => partition is not ready to serve @@ -2856,7 +2915,7 @@ public void testIsReadyToServe(boolean isDaVinciClient, boolean isLeader, boolea doReturn(LEADER_NOT_COMPLETED).when(mockPcsBufferReplayStartedLagCaughtUp).getLeaderCompleteState(); assertEquals( storeIngestionTaskUnderTest.isReadyToServe(mockPcsBufferReplayStartedLagCaughtUp), - !isActiveActiveReplicationEnabled); + aaConfig == AA_OFF); // case 5c: standby replica and LEADER_COMPLETED => partition is ready to serve doReturn(LEADER_COMPLETED).when(mockPcsBufferReplayStartedLagCaughtUp).getLeaderCompleteState(); doCallRealMethod().when(mockPcsBufferReplayStartedLagCaughtUp).isLeaderCompleted(); @@ -2880,9 +2939,9 @@ public void testIsReadyToServe(boolean isDaVinciClient, boolean isLeader, boolea doReturn(5L).when(mockTopicManager).getPartitionLatestOffsetAndRetry(any(), anyInt()); doReturn(150L).when(mockTopicManagerRemoteKafka).getPartitionLatestOffsetAndRetry(any(), anyInt()); doReturn(150L).when(aggKafkaConsumerService).getLatestOffsetFor(anyString(), any(), any()); - if (isLeader) { + if (nodeType == NodeType.LEADER) { // case 6a: leader replica => partition is not ready to serve - doReturn(LEADER).when(mockPcsBufferReplayStartedRemoteLagging).getLeaderFollowerState(); + doReturn(LeaderFollowerStateType.LEADER).when(mockPcsBufferReplayStartedRemoteLagging).getLeaderFollowerState(); assertFalse(storeIngestionTaskUnderTest.isReadyToServe(mockPcsBufferReplayStartedRemoteLagging)); } else { // case 6b: standby replica and !LEADER_COMPLETED => partition is not ready to serve @@ -2890,7 +2949,7 @@ public void testIsReadyToServe(boolean isDaVinciClient, boolean isLeader, boolea doReturn(LEADER_NOT_COMPLETED).when(mockPcsBufferReplayStartedRemoteLagging).getLeaderCompleteState(); assertEquals( storeIngestionTaskUnderTest.isReadyToServe(mockPcsBufferReplayStartedRemoteLagging), - !isActiveActiveReplicationEnabled); + aaConfig == AA_OFF); // case 6c: standby replica and LEADER_COMPLETED => partition is ready to serve doReturn(LEADER_COMPLETED).when(mockPcsBufferReplayStartedRemoteLagging).getLeaderCompleteState(); doCallRealMethod().when(mockPcsBufferReplayStartedRemoteLagging).isLeaderCompleted(); @@ -2916,9 +2975,9 @@ public void testIsReadyToServe(boolean isDaVinciClient, boolean isLeader, boolea .getProducerTimestampOfLastDataRecord(any(), anyInt()); doReturn(System.currentTimeMillis()).when(mockTopicManagerRemoteKafka) .getProducerTimestampOfLastDataRecord(any(), anyInt()); - if (isLeader) { + if (nodeType == NodeType.LEADER) { // case 7a: leader replica => partition is not ready to serve - doReturn(LEADER).when(mockPcsOffsetLagCaughtUpTimestampLagging).getLeaderFollowerState(); + doReturn(LeaderFollowerStateType.LEADER).when(mockPcsOffsetLagCaughtUpTimestampLagging).getLeaderFollowerState(); assertFalse(storeIngestionTaskUnderTest.isReadyToServe(mockPcsOffsetLagCaughtUpTimestampLagging)); } else { // case 7b: standby replica and !LEADER_COMPLETED => partition is not ready to serve @@ -2926,7 +2985,7 @@ public void testIsReadyToServe(boolean isDaVinciClient, boolean isLeader, boolea doReturn(LEADER_NOT_COMPLETED).when(mockPcsOffsetLagCaughtUpTimestampLagging).getLeaderCompleteState(); assertEquals( storeIngestionTaskUnderTest.isReadyToServe(mockPcsOffsetLagCaughtUpTimestampLagging), - !isActiveActiveReplicationEnabled); + aaConfig == AA_OFF); // case 7c: standby replica and LEADER_COMPLETED => partition is ready to serve doReturn(LEADER_COMPLETED).when(mockPcsOffsetLagCaughtUpTimestampLagging).getLeaderCompleteState(); doCallRealMethod().when(mockPcsOffsetLagCaughtUpTimestampLagging).isLeaderCompleted(); @@ -2937,13 +2996,8 @@ public void testIsReadyToServe(boolean isDaVinciClient, boolean isLeader, boolea } } - @Test(dataProvider = "Three-True-and-False", dataProviderClass = DataProviderUtils.class) - public void testActiveActiveStoreIsReadyToServe(boolean isHybrid, boolean isDaVinciClient, boolean isLeader) { - if (isDaVinciClient && isLeader) { - // DaVinci client can't be leader - return; - } - + @Test(dataProvider = "hybridConfigAndNodeTypeProvider") + public void testActiveActiveStoreIsReadyToServe(HybridConfig hybridConfig, NodeType nodeType) { int partitionCount = 2; int amplificationFactor = 1; VenicePartitioner partitioner = getVenicePartitioner(1); @@ -2951,7 +3005,7 @@ public void testActiveActiveStoreIsReadyToServe(boolean isHybrid, boolean isDaVi partitionerConfig.setPartitionerClass(partitioner.getClass().getName()); partitionerConfig.setAmplificationFactor(amplificationFactor); HybridStoreConfig hybridStoreConfig = null; - if (isHybrid) { + if (hybridConfig == HYBRID) { hybridStoreConfig = new HybridStoreConfigImpl( 100, 100, @@ -2966,7 +3020,7 @@ public void testActiveActiveStoreIsReadyToServe(boolean isHybrid, boolean isDaVi Optional.ofNullable(hybridStoreConfig), false, true, - true); + AA_ON); Store mockStore = storeAndVersionConfigs.store; Version version = storeAndVersionConfigs.version; VeniceStoreVersionConfig storeConfig = storeAndVersionConfigs.storeVersionConfig; @@ -2977,7 +3031,7 @@ public void testActiveActiveStoreIsReadyToServe(boolean isHybrid, boolean isDaVi Optional.empty(), 1, new HashMap<>(), - false).setIsDaVinciClient(isDaVinciClient).setAggKafkaConsumerService(aggKafkaConsumerService).build(); + false).setIsDaVinciClient(nodeType == DA_VINCI).setAggKafkaConsumerService(aggKafkaConsumerService).build(); TopicManager mockTopicManagerRemoteKafka = mock(TopicManager.class); doReturn(mockTopicManager).when(mockTopicManagerRepository) @@ -3026,7 +3080,7 @@ public void testActiveActiveStoreIsReadyToServe(boolean isHybrid, boolean isDaVi doReturn(true).when(mockPcsMultipleSourceKafkaServers).isEndOfPushReceived(); doReturn(false).when(mockPcsMultipleSourceKafkaServers).isComplete(); doReturn(true).when(mockPcsMultipleSourceKafkaServers).isWaitingForReplicationLag(); - doReturn(isHybrid).when(mockPcsMultipleSourceKafkaServers).isHybrid(); + doReturn(hybridConfig == HYBRID).when(mockPcsMultipleSourceKafkaServers).isHybrid(); doReturn(topicSwitchWithMultipleSourceKafkaServersWrapper).when(mockPcsMultipleSourceKafkaServers).getTopicSwitch(); doReturn(mockOffsetRecord).when(mockPcsMultipleSourceKafkaServers).getOffsetRecord(); doReturn(5L).when(mockTopicManager).getPartitionLatestOffsetAndRetry(any(), anyInt()); @@ -3034,8 +3088,9 @@ public void testActiveActiveStoreIsReadyToServe(boolean isHybrid, boolean isDaVi doReturn(150L).when(aggKafkaConsumerService).getLatestOffsetFor(anyString(), any(), any()); doReturn(0).when(mockPcsMultipleSourceKafkaServers).getPartition(); doReturn(0).when(mockPcsMultipleSourceKafkaServers).getUserPartition(); - if (isLeader) { - doReturn(LEADER).when(mockPcsMultipleSourceKafkaServers).getLeaderFollowerState(); + doReturn(5L).when(mockPcsMultipleSourceKafkaServers).getLatestProcessedLocalVersionTopicOffset(); + if (nodeType == NodeType.LEADER) { + doReturn(LeaderFollowerStateType.LEADER).when(mockPcsMultipleSourceKafkaServers).getLeaderFollowerState(); } else { doReturn(STANDBY).when(mockPcsMultipleSourceKafkaServers).getLeaderFollowerState(); doReturn(LEADER_COMPLETED).when(mockPcsMultipleSourceKafkaServers).getLeaderCompleteState(); @@ -3045,25 +3100,34 @@ public void testActiveActiveStoreIsReadyToServe(boolean isHybrid, boolean isDaVi .getLastLeaderCompleteStateUpdateInMs(); } storeIngestionTaskUnderTest.setPartitionConsumptionState(0, mockPcsMultipleSourceKafkaServers); - if (isHybrid && isLeader) { + if (hybridConfig == HYBRID && nodeType == NodeType.LEADER) { assertFalse(storeIngestionTaskUnderTest.isReadyToServe(mockPcsMultipleSourceKafkaServers)); } else { assertTrue(storeIngestionTaskUnderTest.isReadyToServe(mockPcsMultipleSourceKafkaServers)); } } - @Test(dataProvider = "Five-True-and-False", dataProviderClass = DataProviderUtils.class) - public void testCheckAndLogIfLagIsAcceptableForHybridStore( - boolean isOffsetBasedLag, - boolean isDaVinciClient, - boolean isLeader, - boolean isActiveActiveReplicationEnabled, - boolean leaderCompleteStateCheckEnabled) { - if (isDaVinciClient && isLeader) { - // DaVinci client can't be leader - return; - } + @DataProvider + public static Object[][] testCheckAndLogIfLagIsAcceptableForHybridStoreProvider() { + return DataProviderUtils.allPermutationGenerator( + LagType.values(), + new NodeType[] { DA_VINCI, FOLLOWER }, + AAConfig.values(), + LeaderCompleteCheck.values()); + } + /** + * @param lagType N.B. this only affects cosmetic logging details at the level where we mock it + * @param nodeType Can be either DVC or follower + * @param aaConfig AA on/off + * @param leaderCompleteCheck Whether followers/DVC should wait for the leader to be complete + */ + @Test(dataProvider = "testCheckAndLogIfLagIsAcceptableForHybridStoreProvider") + public void testCheckAndLogIfLagIsAcceptableForHybridStore( + LagType lagType, + NodeType nodeType, + AAConfig aaConfig, + LeaderCompleteCheck leaderCompleteCheck) { int partitionCount = 2; int amplificationFactor = 1; VenicePartitioner partitioner = getVenicePartitioner(1); @@ -3084,7 +3148,7 @@ public void testCheckAndLogIfLagIsAcceptableForHybridStore( Optional.of(hybridStoreConfig), false, true, - isActiveActiveReplicationEnabled); + aaConfig); Store mockStore = storeAndVersionConfigs.store; Version version = storeAndVersionConfigs.version; VeniceStoreVersionConfig storeConfig = storeAndVersionConfigs.storeVersionConfig; @@ -3092,7 +3156,8 @@ public void testCheckAndLogIfLagIsAcceptableForHybridStore( Map serverProperties = new HashMap<>(); serverProperties.put(SERVER_INGESTION_HEARTBEAT_INTERVAL_MS, 5000L); serverProperties.put(SERVER_LEADER_COMPLETE_STATE_CHECK_IN_FOLLOWER_VALID_INTERVAL_MS, 5000L); - serverProperties.put(SERVER_LEADER_COMPLETE_STATE_CHECK_IN_FOLLOWER_ENABLED, leaderCompleteStateCheckEnabled); + serverProperties + .put(SERVER_LEADER_COMPLETE_STATE_CHECK_IN_FOLLOWER_ENABLED, leaderCompleteCheck == LEADER_COMPLETE_CHECK_ON); StoreIngestionTaskFactory ingestionTaskFactory = getIngestionTaskFactoryBuilder( new RandomPollStrategy(), @@ -3100,7 +3165,7 @@ public void testCheckAndLogIfLagIsAcceptableForHybridStore( Optional.empty(), 1, serverProperties, - false).setIsDaVinciClient(isDaVinciClient).setAggKafkaConsumerService(aggKafkaConsumerService).build(); + false).setIsDaVinciClient(nodeType == DA_VINCI).setAggKafkaConsumerService(aggKafkaConsumerService).build(); TopicManager mockTopicManagerRemoteKafka = mock(TopicManager.class); doReturn(mockTopicManager).when(mockTopicManagerRepository) @@ -3129,15 +3194,15 @@ public void testCheckAndLogIfLagIsAcceptableForHybridStore( // Case 1: offsetLag > offsetThreshold and instance is leader long offsetLag = 100; long offsetThreshold = 50; - if (!isDaVinciClient) { - doReturn(LEADER).when(mockPartitionConsumptionState).getLeaderFollowerState(); + if (nodeType != DA_VINCI) { + doReturn(LeaderFollowerStateType.LEADER).when(mockPartitionConsumptionState).getLeaderFollowerState(); assertFalse( storeIngestionTaskUnderTest.checkAndLogIfLagIsAcceptableForHybridStore( mockPartitionConsumptionState, offsetLag, offsetThreshold, - false, - isOffsetBasedLag, + true, + lagType, 0)); } @@ -3149,21 +3214,21 @@ public void testCheckAndLogIfLagIsAcceptableForHybridStore( offsetLag, offsetThreshold, false, - isOffsetBasedLag, + lagType, 0)); // Case 3: offsetLag <= offsetThreshold and instance is not a standby or DaVinciClient offsetLag = 50; offsetThreshold = 100; - if (!isDaVinciClient) { - doReturn(LEADER).when(mockPartitionConsumptionState).getLeaderFollowerState(); + if (nodeType != DA_VINCI) { + doReturn(LeaderFollowerStateType.LEADER).when(mockPartitionConsumptionState).getLeaderFollowerState(); assertTrue( storeIngestionTaskUnderTest.checkAndLogIfLagIsAcceptableForHybridStore( mockPartitionConsumptionState, offsetLag, offsetThreshold, false, - isOffsetBasedLag, + lagType, 0)); } @@ -3177,9 +3242,9 @@ public void testCheckAndLogIfLagIsAcceptableForHybridStore( offsetLag, offsetThreshold, false, - isOffsetBasedLag, + lagType, 0), - !(leaderCompleteStateCheckEnabled && isActiveActiveReplicationEnabled)); + !(leaderCompleteCheck == LEADER_COMPLETE_CHECK_ON && aaConfig == AA_ON)); // Case 5: offsetLag <= offsetThreshold and instance is a standby or DaVinciClient // and first heart beat SOS has been received and leaderCompleteState is unknown @@ -3191,7 +3256,7 @@ public void testCheckAndLogIfLagIsAcceptableForHybridStore( offsetLag, offsetThreshold, false, - isOffsetBasedLag, + lagType, 0)); // Case 6: offsetLag <= offsetThreshold and instance is a standby or DaVinciClient @@ -3207,7 +3272,7 @@ public void testCheckAndLogIfLagIsAcceptableForHybridStore( offsetLag, offsetThreshold, false, - isOffsetBasedLag, + lagType, 0)); // Case 7: offsetLag <= offsetThreshold and instance is a standby or DaVinciClient @@ -3221,9 +3286,9 @@ public void testCheckAndLogIfLagIsAcceptableForHybridStore( offsetLag, offsetThreshold, false, - isOffsetBasedLag, + lagType, 0), - !(leaderCompleteStateCheckEnabled && isActiveActiveReplicationEnabled)); + !(leaderCompleteCheck == LEADER_COMPLETE_CHECK_ON && aaConfig == AA_ON)); // Case 8: offsetLag <= offsetThreshold and instance is a standby or DaVinciClient // and first heart beat SOS has been received and leaderCompleteState is LEADER_NOT_COMPLETED @@ -3235,15 +3300,23 @@ public void testCheckAndLogIfLagIsAcceptableForHybridStore( offsetLag, offsetThreshold, false, - isOffsetBasedLag, + lagType, 0), - !(leaderCompleteStateCheckEnabled && isActiveActiveReplicationEnabled)); + !(leaderCompleteCheck == LEADER_COMPLETE_CHECK_ON && aaConfig == AA_ON)); } - @Test(dataProvider = "Three-True-and-False", dataProviderClass = DataProviderUtils.class) + @DataProvider + public static Object[][] testGetAndUpdateLeaderCompletedStateProvider() { + return DataProviderUtils.allPermutationGenerator( + HybridConfig.values(), + new NodeType[] { DA_VINCI, FOLLOWER }, + DataProviderUtils.BOOLEAN); + } + + @Test(dataProvider = "testGetAndUpdateLeaderCompletedStateProvider") public void testGetAndUpdateLeaderCompletedState( - boolean isHybrid, - boolean isDaVinciClient, + HybridConfig hybridConfig, + NodeType nodeType, boolean leaderCompletedHeaderFound) { int partitionCount = 2; @@ -3254,7 +3327,7 @@ public void testGetAndUpdateLeaderCompletedState( partitionerConfig.setAmplificationFactor(amplificationFactor); HybridStoreConfig hybridStoreConfig = null; - if (isHybrid) { + if (hybridConfig == HYBRID) { hybridStoreConfig = new HybridStoreConfigImpl(100, 100, 100, DataReplicationPolicy.AGGREGATE, BufferReplayPolicy.REWIND_FROM_EOP); } @@ -3264,7 +3337,7 @@ public void testGetAndUpdateLeaderCompletedState( Optional.ofNullable(hybridStoreConfig), false, true, - true); + AA_ON); Store mockStore = storeAndVersionConfigs.store; Version version = storeAndVersionConfigs.version; VeniceStoreVersionConfig storeConfig = storeAndVersionConfigs.storeVersionConfig; @@ -3275,7 +3348,7 @@ public void testGetAndUpdateLeaderCompletedState( Optional.empty(), 1, new HashMap<>(), - false).setIsDaVinciClient(isDaVinciClient).setAggKafkaConsumerService(aggKafkaConsumerService).build(); + false).setIsDaVinciClient(nodeType == DA_VINCI).setAggKafkaConsumerService(aggKafkaConsumerService).build(); Properties kafkaProps = new Properties(); kafkaProps.put(KAFKA_BOOTSTRAP_SERVERS, inMemoryLocalKafkaBroker.getKafkaBootstrapServer()); @@ -3301,7 +3374,7 @@ public void testGetAndUpdateLeaderCompletedState( KafkaMessageEnvelope kafkaMessageEnvelope = getHeartbeatKME(producerTimestamp, mockLeaderMetadataWrapper, generateHeartbeatMessage(CheckSumType.NONE), "0"); - PubSubMessageHeaders pubSubMessageHeaders = EMPTY_MSG_HEADERS; + PubSubMessageHeaders pubSubMessageHeaders = new PubSubMessageHeaders(); if (leaderCompletedHeaderFound) { pubSubMessageHeaders.add(VeniceWriter.getLeaderCompleteStateHeader(LEADER_COMPLETED)); } @@ -3321,8 +3394,8 @@ public void testGetAndUpdateLeaderCompletedState( KafkaMessageEnvelope kafkaValue = pubSubMessage.getValue(); ControlMessage controlMessage = (ControlMessage) kafkaValue.payloadUnion; - if (!isDaVinciClient) { - partitionConsumptionState.setLeaderFollowerState(LEADER); + if (nodeType != DA_VINCI) { + partitionConsumptionState.setLeaderFollowerState(LeaderFollowerStateType.LEADER); ingestionTask.getAndUpdateLeaderCompletedState( kafkaKey, kafkaValue, @@ -3342,7 +3415,7 @@ public void testGetAndUpdateLeaderCompletedState( pubSubMessage.getPubSubMessageHeaders(), partitionConsumptionState); if (leaderCompletedHeaderFound) { - if (isHybrid) { + if (hybridConfig == HYBRID) { assertEquals(partitionConsumptionState.getLeaderCompleteState(), LEADER_COMPLETED); assertEquals(partitionConsumptionState.getLastLeaderCompleteStateUpdateInMs(), producerTimestamp); } else { @@ -3353,11 +3426,16 @@ public void testGetAndUpdateLeaderCompletedState( assertEquals(partitionConsumptionState.getLeaderCompleteState(), LEADER_COMPLETE_STATE_UNKNOWN); assertEquals(partitionConsumptionState.getLastLeaderCompleteStateUpdateInMs(), 0L); } - assertEquals(partitionConsumptionState.isFirstHeartBeatSOSReceived(), isHybrid); + assertEquals(partitionConsumptionState.isFirstHeartBeatSOSReceived(), hybridConfig == HYBRID); + } + + @DataProvider + public static Object[][] testProcessTopicSwitchProvider() { + return new Object[][] { { LEADER }, { DA_VINCI } }; } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testProcessTopicSwitch(boolean isDaVinciClient) { + @Test(dataProvider = "testProcessTopicSwitchProvider") + public void testProcessTopicSwitch(NodeType nodeType) { int amplificationFactor = 1; VenicePartitioner partitioner = getVenicePartitioner(amplificationFactor); PartitionerConfig partitionerConfig = new PartitionerConfigImpl(); @@ -3366,7 +3444,7 @@ public void testProcessTopicSwitch(boolean isDaVinciClient) { HybridStoreConfig hybridStoreConfig = new HybridStoreConfigImpl(100, 100, 100, DataReplicationPolicy.AGGREGATE, BufferReplayPolicy.REWIND_FROM_EOP); MockStoreVersionConfigs storeAndVersionConfigs = - setupStoreAndVersionMocks(2, partitionerConfig, Optional.of(hybridStoreConfig), false, true, false); + setupStoreAndVersionMocks(2, partitionerConfig, Optional.of(hybridStoreConfig), false, true, AA_OFF); Store mockStore = storeAndVersionConfigs.store; Version version = storeAndVersionConfigs.version; VeniceStoreVersionConfig storeConfig = storeAndVersionConfigs.storeVersionConfig; @@ -3378,7 +3456,7 @@ public void testProcessTopicSwitch(boolean isDaVinciClient) { Optional.empty(), amplificationFactor, new HashMap<>(), - false).setIsDaVinciClient(isDaVinciClient).build(); + false).setIsDaVinciClient(nodeType == DA_VINCI).build(); int leaderSubPartition = PartitionUtils.getLeaderSubPartition(PARTITION_FOO, amplificationFactor); Properties kafkaProps = new Properties(); kafkaProps.put(KAFKA_BOOTSTRAP_SERVERS, inMemoryLocalKafkaBroker.getKafkaBootstrapServer()); @@ -3413,12 +3491,12 @@ public void testProcessTopicSwitch(boolean isDaVinciClient) { storeIngestionTaskUnderTest.getStatusReportAdapter().initializePartitionReportStatus(PARTITION_FOO); storeIngestionTaskUnderTest.processTopicSwitch(controlMessage, PARTITION_FOO, 10, mockPcs); - verify(mockTopicManagerRemoteKafka, isDaVinciClient ? never() : times(1)) + verify(mockTopicManagerRemoteKafka, nodeType == DA_VINCI ? never() : times(1)) .getPartitionOffsetByTime(any(), anyLong()); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testUpdateConsumedUpstreamRTOffsetMapDuringRTSubscription(boolean activeActive) { + @Test(dataProvider = "aaConfigProvider") + public void testUpdateConsumedUpstreamRTOffsetMapDuringRTSubscription(AAConfig aaConfig) { String storeName = Utils.getUniqueString("store"); String versionTopic = Version.composeKafkaTopic(storeName, 1); @@ -3434,7 +3512,7 @@ public void testUpdateConsumedUpstreamRTOffsetMapDuringRTSubscription(boolean ac doReturn(mockStore).when(mockReadOnlyStoreRepository).getStoreOrThrow(eq(storeName)); doReturn(false).when(mockStore).isHybridStoreDiskQuotaEnabled(); doReturn(Optional.of(mockVersion)).when(mockStore).getVersion(1); - doReturn(activeActive).when(mockVersion).isActiveActiveReplicationEnabled(); + doReturn(aaConfig == AA_ON).when(mockVersion).isActiveActiveReplicationEnabled(); Properties mockKafkaConsumerProperties = mock(Properties.class); doReturn("localhost").when(mockKafkaConsumerProperties).getProperty(eq(KAFKA_BOOTSTRAP_SERVERS)); @@ -3485,19 +3563,19 @@ public void testUpdateConsumedUpstreamRTOffsetMapDuringRTSubscription(boolean ac // Test whether consumedUpstreamRTOffsetMap is updated when leader subscribes to RT after state transition ingestionTask.startConsumingAsLeaderInTransitionFromStandby(mockPcs); verify(mockPcs, times(1)).updateLeaderConsumedUpstreamRTOffset( - eq(activeActive ? "localhost" : OffsetRecord.NON_AA_REPLICATION_UPSTREAM_OFFSET_MAP_KEY), + eq(aaConfig == AA_ON ? "localhost" : OffsetRecord.NON_AA_REPLICATION_UPSTREAM_OFFSET_MAP_KEY), eq(1000L)); PubSubTopic rtTopic = pubSubTopicRepository.getTopic("test_rt"); Supplier mockPcsSupplier = () -> { PartitionConsumptionState mock = mock(PartitionConsumptionState.class); - doReturn(LEADER).when(mock).getLeaderFollowerState(); + doReturn(LeaderFollowerStateType.LEADER).when(mock).getLeaderFollowerState(); doReturn(topicSwitchWrapper).when(mock).getTopicSwitch(); OffsetRecord mockOR = mock(OffsetRecord.class); doReturn(rtTopic).when(mockOR).getLeaderTopic(any()); System.out.println(mockOR.getLeaderTopic(null)); doReturn(1000L).when(mockOR).getUpstreamOffset(anyString()); - if (activeActive) { + if (aaConfig == AA_ON) { doReturn(1000L).when(mock).getLatestProcessedUpstreamRTOffsetWithNoDefault(anyString()); } else { doReturn(1000L).when(mock).getLatestProcessedUpstreamRTOffset(anyString()); @@ -3511,13 +3589,13 @@ public void testUpdateConsumedUpstreamRTOffsetMapDuringRTSubscription(boolean ac // Test whether consumedUpstreamRTOffsetMap is updated when leader subscribes to RT after executing TS ingestionTask.leaderExecuteTopicSwitch(mockPcs, topicSwitch, newSourceTopic); verify(mockPcs, times(1)).updateLeaderConsumedUpstreamRTOffset( - eq(activeActive ? "localhost" : OffsetRecord.NON_AA_REPLICATION_UPSTREAM_OFFSET_MAP_KEY), + eq(aaConfig == AA_ON ? "localhost" : OffsetRecord.NON_AA_REPLICATION_UPSTREAM_OFFSET_MAP_KEY), eq(1000L)); // Test alternative branch of the code Supplier mockPcsSupplier2 = () -> { PartitionConsumptionState mock = mockPcsSupplier.get(); - if (activeActive) { + if (aaConfig == AA_ON) { doReturn(-1L).when(mock).getLatestProcessedUpstreamRTOffsetWithNoDefault(anyString()); } else { doReturn(-1L).when(mock).getLatestProcessedUpstreamRTOffset(anyString()); @@ -3610,9 +3688,8 @@ public void testLeaderShouldSubscribeToCorrectVTOffset() { .consumerSubscribe(any(), eq(remoteVersionTopicOffset), anyString()); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testWrappedInterruptExceptionDuringGracefulShutdown(boolean isActiveActiveReplicationEnabled) - throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testWrappedInterruptExceptionDuringGracefulShutdown(AAConfig aaConfig) throws Exception { hybridStoreConfig = Optional.of( new HybridStoreConfigImpl( 10, @@ -3628,7 +3705,7 @@ public void testWrappedInterruptExceptionDuringGracefulShutdown(boolean isActive verify(mockLogNotifier, timeout(TEST_TIMEOUT_MS)).restarted(eq(topic), eq(PARTITION_FOO), anyLong()); storeIngestionTaskUnderTest.close(); verify(aggKafkaConsumerService, timeout(TEST_TIMEOUT_MS)).unsubscribeConsumerFor(eq(pubSubTopic), any()); - }, isActiveActiveReplicationEnabled); + }, aaConfig); Assert.assertEquals(mockNotifierError.size(), 0); } @@ -3639,11 +3716,9 @@ public void testWrappedInterruptExceptionDuringGracefulShutdown(boolean isActive * 1. offsetRecord and pcs has the same state at the beginning * 2. pcs consumes 2 records and offsetRecords doesn't sync up with pcs due to high sync interval. * 3. enforce to gracefully shutdown and validate offsetRecord has been synced up with pcs once. - * @param isActiveActiveReplicationEnabled - * @throws Exception */ - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testOffsetSyncBeforeGracefulShutDown(boolean isActiveActiveReplicationEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testOffsetSyncBeforeGracefulShutDown(AAConfig aaConfig) throws Exception { // prepare to send 2 messages localVeniceWriter.put(putKeyFoo, putValue, SCHEMA_ID); localVeniceWriter.put(putKeyFoo2, putValue, SCHEMA_ID); @@ -3663,12 +3738,26 @@ public void testOffsetSyncBeforeGracefulShutDown(boolean isActiveActiveReplicati // Verify offsetRecord hasn't been synced yet PartitionConsumptionState pcs = storeIngestionTaskUnderTest.getPartitionConsumptionState(PARTITION_FOO); - OffsetRecord offsetRecord = pcs.getOffsetRecord(); - Assert.assertEquals(pcs.getLatestProcessedLocalVersionTopicOffset(), 0L); - Assert.assertEquals(offsetRecord.getLocalVersionTopicOffset(), 0L); + if (pcs == null) { + LOGGER.info( + "pcs for PARTITION_FOO is null, which is an indication that it was never synced before, so we carry on."); + } else { + // If the pcs is non-null, then we perform additional checks to ensure that it was not synced + Assert.assertEquals( + pcs.getLatestProcessedLocalVersionTopicOffset(), + 0L, + "pcs.getLatestProcessedLocalVersionTopicOffset() for PARTITION_FOO is expected to be zero!"); + OffsetRecord offsetRecord = pcs.getOffsetRecord(); + assertNotNull(offsetRecord); + Assert.assertEquals(offsetRecord.getLocalVersionTopicOffset(), 0L); + } // verify 2 messages were processed verify(mockStoreIngestionStats, timeout(TEST_TIMEOUT_MS).times(2)).recordTotalRecordsConsumed(); + pcs = storeIngestionTaskUnderTest.getPartitionConsumptionState(PARTITION_FOO); // We re-fetch in case it was null + assertNotNull(pcs, "pcs for PARTITION_FOO is null!"); + OffsetRecord offsetRecord = pcs.getOffsetRecord(); + assertNotNull(offsetRecord); Assert.assertEquals(pcs.getLatestProcessedLocalVersionTopicOffset(), 2L); // PCS updated Assert.assertEquals(offsetRecord.getLocalVersionTopicOffset(), 0L); // offsetRecord hasn't been updated yet @@ -3678,15 +3767,15 @@ public void testOffsetSyncBeforeGracefulShutDown(boolean isActiveActiveReplicati verify(mockStorageMetadataService, timeout(TEST_TIMEOUT_MS).times(1)).put(eq(topic), eq(PARTITION_FOO), any()); Assert.assertEquals(offsetRecord.getLocalVersionTopicOffset(), 2L); - }, isActiveActiveReplicationEnabled, configOverride -> { + }, aaConfig, configOverride -> { // set very high threshold so offsetRecord isn't be synced during regular consumption doReturn(100_000L).when(configOverride).getDatabaseSyncBytesIntervalForTransactionalMode(); }); Assert.assertEquals(mockNotifierError.size(), 0); } - @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testProduceToStoreBufferService(boolean activeActiveEnabled) throws Exception { + @Test(dataProvider = "aaConfigProvider") + public void testProduceToStoreBufferService(AAConfig aaConfig) throws Exception { byte[] keyBytes = new byte[1]; KafkaKey kafkaKey = new KafkaKey(MessageType.PUT, keyBytes); KafkaMessageEnvelope kafkaMessageEnvelope = new KafkaMessageEnvelope(); @@ -3755,7 +3844,7 @@ public void testProduceToStoreBufferService(boolean activeActiveEnabled) throws ArgumentMatchers.doubleThat(argument -> argument >= 0 && argument < 1000), ArgumentMatchers.longThat( argument -> argument > currentTimeMs - errorMargin && argument < currentTimeMs + errorMargin)); - }, activeActiveEnabled); + }, aaConfig); } private void verifyStats( @@ -3780,7 +3869,7 @@ public void testShouldPersistRecord() throws Exception { runTest(Collections.singleton(PARTITION_FOO), () -> { assertFalse(storeIngestionTaskUnderTest.shouldPersistRecord(pubSubMessage, null)); - }, false); + }, AA_OFF); Map serverProperties = new HashMap<>(); serverProperties.put(FREEZE_INGESTION_IF_READY_TO_SERVE_OR_LOCAL_DATA_EXISTS, true); @@ -3796,7 +3885,7 @@ public void testShouldPersistRecord() throws Exception { runTest(new RandomPollStrategy(), Collections.singleton(PARTITION_FOO), () -> {}, () -> { PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateSupplier.get(); assertFalse(storeIngestionTaskUnderTest.shouldPersistRecord(pubSubMessage, partitionConsumptionState)); - }, this.hybridStoreConfig, false, Optional.empty(), false, 1, serverProperties); + }, this.hybridStoreConfig, false, Optional.empty(), AA_OFF, 1, serverProperties); runTest(Collections.singleton(PARTITION_FOO), () -> { PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateSupplier.get(); @@ -3805,9 +3894,9 @@ public void testShouldPersistRecord() throws Exception { when(offsetRecord.getLeaderTopic(any())).thenReturn(wrongTopic); when(partitionConsumptionState.getOffsetRecord()).thenReturn(offsetRecord); - when(partitionConsumptionState.getLeaderFollowerState()).thenReturn(LEADER); + when(partitionConsumptionState.getLeaderFollowerState()).thenReturn(LeaderFollowerStateType.LEADER); assertFalse(storeIngestionTaskUnderTest.shouldPersistRecord(pubSubMessage, partitionConsumptionState)); - }, false); + }, AA_OFF); runTest(Collections.singleton(PARTITION_FOO), () -> { PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateSupplier.get(); @@ -3818,7 +3907,7 @@ public void testShouldPersistRecord() throws Exception { when(partitionConsumptionState.getLeaderFollowerState()).thenReturn(STANDBY); assertFalse(storeIngestionTaskUnderTest.shouldPersistRecord(pubSubMessage2, partitionConsumptionState)); - }, false); + }, AA_OFF); } @Test @@ -3832,7 +3921,7 @@ public void testIngestionTaskForNonCurrentVersionShouldFailWhenEncounteringMemor verify(mockAbstractStorageEngine, timeout(1000)).put(eq(PARTITION_FOO), any(), (ByteBuffer) any()); verify(mockLogNotifier, timeout(1000)).error(any(), eq(PARTITION_FOO), any(), isA(VeniceException.class)); verify(runnableForKillNonCurrentVersion, never()).run(); - }, false); + }, AA_OFF); } @Test @@ -3853,7 +3942,7 @@ public void testIngestionTaskForCurrentVersionShouldTryToKillOngoingPushWhenEnco verify(mockAbstractStorageEngine, timeout(1000)).reopenStoragePartition(PARTITION_FOO); verify(mockLogNotifier, timeout(1000)).completed(anyString(), eq(PARTITION_FOO), anyLong(), anyString()); verify(runnableForKillNonCurrentVersion, times(1)).run(); - }, false); + }, AA_OFF); } @Test @@ -3864,7 +3953,7 @@ public void testShouldProduceToVersionTopic() throws Exception { when(partitionConsumptionState.getLeaderFollowerState()).thenReturn(STANDBY); assertFalse(lfsit.shouldProduceToVersionTopic(partitionConsumptionState)); - when(partitionConsumptionState.getLeaderFollowerState()).thenReturn(LEADER); + when(partitionConsumptionState.getLeaderFollowerState()).thenReturn(LeaderFollowerStateType.LEADER); OffsetRecord offsetRecord = mock(OffsetRecord.class); when(offsetRecord.getLeaderTopic(any())).thenReturn(pubSubTopic); when(partitionConsumptionState.getOffsetRecord()).thenReturn(offsetRecord); @@ -3876,7 +3965,7 @@ public void testShouldProduceToVersionTopic() throws Exception { when(offsetRecord.getLeaderTopic(any())).thenReturn(pubSubTopic); when(partitionConsumptionState.consumeRemotely()).thenReturn(true); assertTrue(lfsit.shouldProduceToVersionTopic(partitionConsumptionState)); - }, false); + }, AA_OFF); } @Test @@ -3945,12 +4034,21 @@ public void testCheckIngestionTaskActiveness() { Assert.assertTrue(storeIngestionTask.maybeSetIngestionTaskActiveState(false)); } - @Test(dataProvider = "Four-True-and-False", dataProviderClass = DataProviderUtils.class) + @DataProvider + public static Object[][] testMaybeSendIngestionHeartbeatProvider() { + return DataProviderUtils.allPermutationGenerator( + AAConfig.values(), + DataProviderUtils.BOOLEAN, + new NodeType[] { FOLLOWER, NodeType.LEADER }, + HybridConfig.values()); + } + + @Test(dataProvider = "testMaybeSendIngestionHeartbeatProvider") public void testMaybeSendIngestionHeartbeat( - boolean isActiveActive, + AAConfig aaConfig, boolean isRealTimeTopic, - boolean isLeader, - boolean isHybridStore) { + NodeType nodeType, + HybridConfig hybridConfig) { String storeName = Utils.getUniqueString("store"); Store mockStore = mock(Store.class); String versionTopic = Version.composeKafkaTopic(storeName, 1); @@ -3960,7 +4058,7 @@ public void testMaybeSendIngestionHeartbeat( doReturn(1).when(mockVersion).getPartitionCount(); doReturn(VersionStatus.STARTED).when(mockVersion).getStatus(); doReturn(true).when(mockVersion).isUseVersionLevelHybridConfig(); - if (isHybridStore) { + if (hybridConfig == HYBRID) { HybridStoreConfig mockHybridConfig = mock(HybridStoreConfig.class); doReturn(mockHybridConfig).when(mockVersion).getHybridStoreConfig(); } else { @@ -3972,7 +4070,7 @@ public void testMaybeSendIngestionHeartbeat( doReturn(mockStore).when(mockReadOnlyStoreRepository).getStoreOrThrow(eq(storeName)); doReturn(false).when(mockStore).isHybridStoreDiskQuotaEnabled(); doReturn(Optional.of(mockVersion)).when(mockStore).getVersion(1); - doReturn(isActiveActive).when(mockVersion).isActiveActiveReplicationEnabled(); + doReturn(aaConfig == AA_ON).when(mockVersion).isActiveActiveReplicationEnabled(); VeniceServerConfig mockVeniceServerConfig = mock(VeniceServerConfig.class); VeniceProperties mockVeniceProperties = mock(VeniceProperties.class); doReturn(true).when(mockVeniceProperties).isEmpty(); @@ -3983,7 +4081,7 @@ public void testMaybeSendIngestionHeartbeat( PartitionConsumptionState pcs = mock(PartitionConsumptionState.class); OffsetRecord offsetRecord = mock(OffsetRecord.class); doReturn(offsetRecord).when(pcs).getOffsetRecord(); - doReturn(isLeader ? LEADER : STANDBY).when(pcs).getLeaderFollowerState(); + doReturn(nodeType == NodeType.LEADER ? LeaderFollowerStateType.LEADER : STANDBY).when(pcs).getLeaderFollowerState(); PubSubTopic pubsubTopic = mock(PubSubTopic.class); doReturn(pubsubTopic).when(offsetRecord).getLeaderTopic(any()); doReturn(isRealTimeTopic).when(pubsubTopic).isRealTime(); @@ -4018,7 +4116,7 @@ public void testMaybeSendIngestionHeartbeat( ingestionTask.maybeSendIngestionHeartbeat(); // Second invocation should be skipped since it shouldn't be time for another heartbeat yet. ingestionTask.maybeSendIngestionHeartbeat(); - if (isHybridStore && isRealTimeTopic && isLeader) { + if (hybridConfig == HYBRID && isRealTimeTopic && nodeType == NodeType.LEADER) { verify(veniceWriter, times(1)).sendHeartbeat(any(), any(), any(), anyBoolean(), any(), anyLong()); } else { verify(veniceWriter, never()).sendHeartbeat(any(), any(), any(), anyBoolean(), any(), anyLong()); @@ -4064,10 +4162,10 @@ public void testMaybeSendIngestionHeartbeatWithHBSuccessOrFailure() throws Inter OffsetRecord offsetRecord = mock(OffsetRecord.class); PartitionConsumptionState pcs0 = mock(PartitionConsumptionState.class); doReturn(offsetRecord).when(pcs0).getOffsetRecord(); - doReturn(LEADER).when(pcs0).getLeaderFollowerState(); + doReturn(LeaderFollowerStateType.LEADER).when(pcs0).getLeaderFollowerState(); PartitionConsumptionState pcs1 = mock(PartitionConsumptionState.class); doReturn(offsetRecord).when(pcs1).getOffsetRecord(); - doReturn(LEADER).when(pcs1).getLeaderFollowerState(); + doReturn(LeaderFollowerStateType.LEADER).when(pcs1).getLeaderFollowerState(); doReturn(1).when(pcs1).getUserPartition(); PubSubTopic pubsubTopic = mock(PubSubTopic.class); doReturn(pubsubTopic).when(offsetRecord).getLeaderTopic(any()); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/stats/KafkaConsumerServiceStatsTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/stats/KafkaConsumerServiceStatsTest.java new file mode 100644 index 00000000000..9930eee1e70 --- /dev/null +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/stats/KafkaConsumerServiceStatsTest.java @@ -0,0 +1,92 @@ +package com.linkedin.davinci.stats; + +import static com.linkedin.venice.stats.AbstractVeniceStats.getSensorFullName; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; + +import io.tehuti.Metric; +import io.tehuti.metrics.MetricsRepository; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.testng.annotations.Test; + + +public class KafkaConsumerServiceStatsTest { + private static final Logger LOGGER = LogManager.getLogger(KafkaConsumerServiceStatsTest.class); + + @Test + public void testParentStats() { + MetricsRepository metricsRepository = new MetricsRepository(); + String nameWithKafkaClusterAlias = "moon-east-1"; + String getLatestOffsetIsPresentName = + getSensorFullName(nameWithKafkaClusterAlias, "getLatestOffsetIsPresent.OccurrenceRate"); + String getLatestOffsetIsAbsentName = + getSensorFullName(nameWithKafkaClusterAlias, "getLatestOffsetIsAbsent.OccurrenceRate"); + String getLatestOffsetName = getSensorFullName(nameWithKafkaClusterAlias, "getLatestOffset.OccurrenceRate"); + String getOffsetLagIsPresentName = + getSensorFullName(nameWithKafkaClusterAlias, "getOffsetLagIsPresent.OccurrenceRate"); + String getOffsetLagIsAbsentName = + getSensorFullName(nameWithKafkaClusterAlias, "getOffsetLagIsAbsent.OccurrenceRate"); + String getOffsetLagName = getSensorFullName(nameWithKafkaClusterAlias, "getOffsetLag.OccurrenceRate"); + Metric getLatestOffsetIsPresent = metricsRepository.getMetric(getLatestOffsetIsPresentName); + Metric getLatestOffsetIsAbsent = metricsRepository.getMetric(getLatestOffsetIsAbsentName); + Metric getLatestOffset = metricsRepository.getMetric(getLatestOffsetName); + Metric getOffsetLagIsPresent = metricsRepository.getMetric(getOffsetLagIsPresentName); + Metric getOffsetLagIsAbsent = metricsRepository.getMetric(getOffsetLagIsAbsentName); + Metric getOffsetLag = metricsRepository.getMetric(getOffsetLagName); + Metric[] allMetrics = new Metric[] { getLatestOffsetIsPresent, getLatestOffsetIsAbsent, getLatestOffset, + getOffsetLagIsPresent, getOffsetLagIsAbsent, getOffsetLag }; + + // Verify initial state + for (Metric metric: allMetrics) { + assertNull(metric); + } + + // Create objects under test + KafkaConsumerServiceStats stats = + new KafkaConsumerServiceStats(metricsRepository, nameWithKafkaClusterAlias, () -> 1); + LOGGER.info(metricsRepository.metrics().keySet().toString().replace(", ", ",\n")); + getLatestOffsetIsPresent = metricsRepository.getMetric(getLatestOffsetIsPresentName); + getLatestOffsetIsAbsent = metricsRepository.getMetric(getLatestOffsetIsAbsentName); + getLatestOffset = metricsRepository.getMetric(getLatestOffsetName); + getOffsetLagIsPresent = metricsRepository.getMetric(getOffsetLagIsPresentName); + getOffsetLagIsAbsent = metricsRepository.getMetric(getOffsetLagIsAbsentName); + getOffsetLag = metricsRepository.getMetric(getOffsetLagName); + allMetrics = new Metric[] { getLatestOffsetIsPresent, getLatestOffsetIsAbsent, getLatestOffset, + getOffsetLagIsPresent, getOffsetLagIsAbsent, getOffsetLag }; + + // Verify that metrics exist but are empty + for (Metric metric: allMetrics) { + assertNotNull(metric); + assertEquals(metric.value(), 0.0); + } + + // Record into one of two children stats + stats.recordLatestOffsetIsPresent(); + stats.recordOffsetLagIsPresent(); + + // Verify that the parent is incremented but not the other child + assertTrue(getLatestOffsetIsPresent.value() > 0.0); + assertEquals(getLatestOffsetIsAbsent.value(), 0.0); + assertTrue(getLatestOffset.value() > 0.0); + assertTrue(getOffsetLagIsPresent.value() > 0.0); + assertEquals(getOffsetLagIsAbsent.value(), 0.0); + assertTrue(getOffsetLag.value() > 0.0); + + // Record into the second child + stats.recordLatestOffsetIsAbsent(); + stats.recordOffsetLagIsAbsent(); + + // Verify that the parent is larger than both children + assertTrue(getLatestOffsetIsPresent.value() > 0.0); + assertTrue(getLatestOffsetIsAbsent.value() > 0.0); + assertTrue(getLatestOffset.value() > getLatestOffsetIsPresent.value()); + assertTrue(getLatestOffset.value() > getLatestOffsetIsAbsent.value()); + assertTrue(getOffsetLagIsPresent.value() > 0.0); + assertTrue(getOffsetLagIsAbsent.value() > 0.0); + assertTrue(getOffsetLag.value() > getLatestOffsetIsPresent.value()); + assertTrue(getOffsetLag.value() > getLatestOffsetIsAbsent.value()); + } +} diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/AbstractVeniceStats.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/AbstractVeniceStats.java index 5b66c65b6b4..42be952df19 100644 --- a/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/AbstractVeniceStats.java +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/AbstractVeniceStats.java @@ -63,12 +63,13 @@ protected Sensor registerSensor(String sensorName, Sensor[] parents, MeasurableS return registerSensor(getSensorFullName(getName(), sensorName), null, parents, stats); } + /** + * N.B.: This function is private because it requires the full sensor name, which should be generated from + * {@link #getSensorFullName(String)}, and is therefore less user-friendly for developers of subclasses. + * The other functions which call this one require just the partial sensor name, which is less error-prone. + */ @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") - protected Sensor registerSensor( - String sensorFullName, - MetricConfig config, - Sensor[] parents, - MeasurableStat... stats) { + private Sensor registerSensor(String sensorFullName, MetricConfig config, Sensor[] parents, MeasurableStat... stats) { return sensors.computeIfAbsent(sensorFullName, key -> { /** * The sensors concurrentmap will not prevent other objects working on the same metrics repository to execute @@ -155,7 +156,7 @@ protected String getSensorFullName(String sensorName) { return getSensorFullName(getName(), sensorName); } - protected String getSensorFullName(String resourceName, String sensorName) { + public static String getSensorFullName(String resourceName, String sensorName) { if (resourceName.charAt(0) != '.') { resourceName = "." + resourceName; } diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/LambdaStat.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/LambdaStat.java index b29e5f77682..bafe2e4d850 100644 --- a/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/LambdaStat.java +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/LambdaStat.java @@ -1,13 +1,15 @@ package com.linkedin.venice.stats; import io.tehuti.metrics.Measurable; +import io.tehuti.metrics.MeasurableStat; +import io.tehuti.metrics.Sensor; /** * @deprecated, use {@link Gauge} instead. * * The reason to deprecate {@link LambdaStat} is that {@link Gauge} is a better name when appending the class name - * as the suffix of metric name here: {@link AbstractVeniceStats#registerSensor}. + * as the suffix of metric name here: {@link AbstractVeniceStats#registerSensor(String, Sensor[], MeasurableStat...)}. */ @Deprecated public class LambdaStat extends Gauge { diff --git a/internal/venice-client-common/src/test/java/com/linkedin/venice/stats/AbstractVeniceStatsTest.java b/internal/venice-client-common/src/test/java/com/linkedin/venice/stats/AbstractVeniceStatsTest.java index c687762e925..11dc74a83c2 100644 --- a/internal/venice-client-common/src/test/java/com/linkedin/venice/stats/AbstractVeniceStatsTest.java +++ b/internal/venice-client-common/src/test/java/com/linkedin/venice/stats/AbstractVeniceStatsTest.java @@ -1,6 +1,12 @@ package com.linkedin.venice.stats; -import static org.testng.AssertJUnit.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; import com.linkedin.venice.client.stats.BasicClientStats; import com.linkedin.venice.client.stats.ClientStats; @@ -9,13 +15,16 @@ import io.tehuti.Metric; import io.tehuti.metrics.MeasurableStat; import io.tehuti.metrics.MetricConfig; +import io.tehuti.metrics.MetricsReporter; import io.tehuti.metrics.MetricsRepository; import io.tehuti.metrics.Sensor; -import java.util.Optional; +import io.tehuti.metrics.stats.Count; +import io.tehuti.metrics.stats.OccurrenceRate; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; @@ -29,7 +38,7 @@ public StatsTestImpl(MetricsRepository metricsRepository, String name) { /** * This test creates the same metric via many objects using multiple threads. - * Without the synchronization in {@link AbstractVeniceStats#registerSensor(String, Optional, MetricConfig, Sensor[], MeasurableStat...)} + * Without the synchronization in {@link AbstractVeniceStats#registerSensor(String, Sensor[], MeasurableStat...)} * this test fails consistently. With the synchronization added the test passes. * @throws InterruptedException */ @@ -107,4 +116,62 @@ public void testMetricPrefix() { clientStats.recordRequestRetryCount(); } + @Test + public void testParentStats() { + MetricsRepository metricsRepository = new MetricsRepository(); + MetricsReporter reporter = mock(MetricsReporter.class); + metricsRepository.addReporter(reporter); + AbstractVeniceStats avs = new AbstractVeniceStats(metricsRepository, "AVS"); + Count parentCount = new Count(), childCount1 = new Count(), childCount2 = new Count(); + OccurrenceRate parentOccurrenceRate = new OccurrenceRate(), childOccurrenceRate1 = new OccurrenceRate(), + childOccurrenceRate2 = new OccurrenceRate(); + long now = System.currentTimeMillis(); + MetricConfig metricConfig = new MetricConfig(); + + // Test initial state + assertEquals(childCount1.measure(metricConfig, now), 0.0); + assertEquals(childCount2.measure(metricConfig, now), 0.0); + assertEquals(parentCount.measure(metricConfig, now), 0.0); + assertEquals(childOccurrenceRate1.measure(metricConfig, now), 0.0); + assertEquals(childOccurrenceRate2.measure(metricConfig, now), 0.0); + assertEquals(parentOccurrenceRate.measure(metricConfig, now), 0.0); + Mockito.verify(reporter).init(argThat(argument -> argument.size() == 0)); + Mockito.verify(reporter, never()).addMetric(any()); + + // Register metrics + Sensor parentSensor = avs.registerSensor("parent", parentCount, parentOccurrenceRate); + Sensor[] parentSensorArray = new Sensor[] { parentSensor }; + Sensor childSensor1 = avs.registerSensor("child1", parentSensorArray, childCount1, childOccurrenceRate1); + Sensor childSensor2 = avs.registerSensor("child2", parentSensorArray, childCount2, childOccurrenceRate2); + + // Test reporter + Mockito.verify(reporter).init(argThat(argument -> argument.size() == 0)); + Mockito.verify(reporter, times(6)).addMetric(any()); + + // Test that recording propagates from child to parent + childSensor1.record(1); + assertEquals(childCount1.measure(metricConfig, now), 1.0); + assertEquals(childCount2.measure(metricConfig, now), 0.0); + assertEquals(parentCount.measure(metricConfig, now), 1.0); + assertTrue(childOccurrenceRate1.measure(metricConfig, now) > 0.0); + assertEquals(childOccurrenceRate2.measure(metricConfig, now), 0.0); + assertTrue(parentOccurrenceRate.measure(metricConfig, now) > 0.0); + + childSensor2.record(1); + assertEquals(childCount1.measure(metricConfig, now), 1.0); + assertEquals(childCount2.measure(metricConfig, now), 1.0); + assertEquals(parentCount.measure(metricConfig, now), 2.0); + assertTrue(childOccurrenceRate1.measure(metricConfig, now) > 0.0); + assertTrue(childOccurrenceRate2.measure(metricConfig, now) > 0.0); + assertTrue(parentOccurrenceRate.measure(metricConfig, now) > 0.0); + + // Test that recording does not propagate from parent to child + parentSensor.record(1); + assertEquals(childCount1.measure(metricConfig, now), 1.0); + assertEquals(childCount2.measure(metricConfig, now), 1.0); + assertEquals(parentCount.measure(metricConfig, now), 3.0); + assertTrue(childOccurrenceRate1.measure(metricConfig, now) > 0.0); + assertTrue(childOccurrenceRate2.measure(metricConfig, now) > 0.0); + assertTrue(parentOccurrenceRate.measure(metricConfig, now) > 0.0); + } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/InstrumentedPartitionOffsetFetcher.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/InstrumentedPartitionOffsetFetcher.java index 78c757f71f7..084d9ceed96 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/InstrumentedPartitionOffsetFetcher.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/InstrumentedPartitionOffsetFetcher.java @@ -3,6 +3,7 @@ import com.linkedin.venice.pubsub.PubSubTopicPartitionInfo; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; import it.unimi.dsi.fastutil.ints.Int2LongMap; @@ -41,11 +42,16 @@ public Int2LongMap getTopicLatestOffsets(PubSubTopic topic) { @Override public long getPartitionLatestOffsetAndRetry(PubSubTopicPartition topicPartition, int retries) { final long startTimeMs = time.getMilliseconds(); - long res = partitionOffsetFetcher.getPartitionLatestOffsetAndRetry(topicPartition, retries); - stats.recordLatency( - PartitionOffsetFetcherStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_PARTITION_LATEST_OFFSET_WITH_RETRY, - Utils.calculateDurationMs(time, startTimeMs)); - return res; + try { + long value = partitionOffsetFetcher.getPartitionLatestOffsetAndRetry(topicPartition, retries); + stats.recordLatency( + PartitionOffsetFetcherStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_PARTITION_LATEST_OFFSET_WITH_RETRY, + Utils.calculateDurationMs(time, startTimeMs)); + return value; + } catch (PubSubTopicDoesNotExistException e) { + stats.recordGetLatestOffsetError(); + throw e; + } } @Override diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherImpl.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherImpl.java index 0d51857e8b8..d288b7e3be5 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherImpl.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherImpl.java @@ -506,7 +506,9 @@ public void close() { IOUtils.closeQuietly(kafkaAdminWrapper.get(), logger::error); } if (pubSubConsumer.isPresent()) { - IOUtils.closeQuietly(pubSubConsumer.get(), logger::error); + try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { + IOUtils.closeQuietly(pubSubConsumer.get(), logger::error); + } } } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherStats.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherStats.java index 4f666c0eef4..851638b4e8f 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherStats.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherStats.java @@ -9,7 +9,7 @@ import io.tehuti.metrics.stats.Min; import io.tehuti.metrics.stats.OccurrenceRate; import java.util.Collections; -import java.util.HashMap; +import java.util.EnumMap; import java.util.Map; @@ -21,11 +21,12 @@ public enum OCCURRENCE_LATENCY_SENSOR_TYPE { } private final Map sensorsByTypes; + private Sensor getPartitionLatestOffsetError; public PartitionOffsetFetcherStats(MetricsRepository metricsRepository, String name) { super(metricsRepository, name); Map tmpRateSensorsByTypes = - new HashMap<>(OCCURRENCE_LATENCY_SENSOR_TYPE.values().length); + new EnumMap<>(OCCURRENCE_LATENCY_SENSOR_TYPE.class); for (OCCURRENCE_LATENCY_SENSOR_TYPE sensorType: OCCURRENCE_LATENCY_SENSOR_TYPE.values()) { final String sensorName = sensorType.name().toLowerCase(); tmpRateSensorsByTypes.put( @@ -40,9 +41,15 @@ public PartitionOffsetFetcherStats(MetricsRepository metricsRepository, String n } this.sensorsByTypes = Collections.unmodifiableMap(tmpRateSensorsByTypes); + this.getPartitionLatestOffsetError = + registerSensorIfAbsent("get_partition_latest_offset_with_retry_error", new OccurrenceRate()); } public void recordLatency(OCCURRENCE_LATENCY_SENSOR_TYPE sensor_type, long requestLatencyMs) { sensorsByTypes.get(sensor_type).record(requestLatencyMs); } + + public void recordGetLatestOffsetError() { + this.getPartitionLatestOffsetError.record(); + } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/KafkaPartitionTopicOffsetMetrics.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/KafkaPartitionTopicOffsetMetrics.java new file mode 100644 index 00000000000..5695ae54af9 --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/KafkaPartitionTopicOffsetMetrics.java @@ -0,0 +1,50 @@ +package com.linkedin.venice.pubsub.adapter.kafka; + +import static com.linkedin.venice.pubsub.adapter.kafka.TopicPartitionsOffsetsTracker.INVALID; + +import org.apache.kafka.common.Metric; + + +/** + * A wrapper for a Kafka {@link Metric} object representing a topic-partition's lag. We also carry the + * current offset so that we can calculate the end offset. + */ +class KafkaPartitionTopicOffsetMetrics { + private long currentOffset = INVALID; + private Metric lagMetric = null; + + void setCurrentOffset(long currentOffset) { + this.currentOffset = currentOffset; + } + + void setLagMetric(Metric lagMetric) { + this.lagMetric = lagMetric; + } + + boolean isLagMetricMissing() { + return this.lagMetric == null; + } + + long getLag() { + if (this.lagMetric != null) { + Object metricValue = this.lagMetric.metricValue(); + if (metricValue instanceof Double) { + // Double is the way all metrics are internally represented in Kafka, but since we are dealing with lag, we + // want an integral type, so we cast it. + return ((Double) metricValue).longValue(); + } + } + return INVALID; + } + + long getEndOffset() { + if (this.currentOffset == INVALID) { + return INVALID; + } + long lag = getLag(); + if (lag == INVALID) { + return INVALID; + } + return this.currentOffset + lag; + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/TopicPartitionsOffsetsTracker.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/TopicPartitionsOffsetsTracker.java index 43a7da24a8c..4910062e1f1 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/TopicPartitionsOffsetsTracker.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/TopicPartitionsOffsetsTracker.java @@ -1,61 +1,49 @@ package com.linkedin.venice.pubsub.adapter.kafka; -import com.linkedin.venice.utils.LatencyUtils; +import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; -import java.time.Duration; -import java.time.Instant; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.StringJoiner; -import javax.annotation.Nonnull; -import org.apache.commons.lang.Validate; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; /** * This class tracks consumed topic partitions' offsets */ public class TopicPartitionsOffsetsTracker { - public enum ResultType { - VALID_OFFSET_LAG, NO_OFFSET_LAG, INVALID_OFFSET_LAG - } - - private static final Logger LOGGER = LogManager.getLogger(TopicPartitionsOffsetsTracker.class); - private static final Duration DEFAULT_OFFSETS_UPDATE_INTERVAL = Duration.ofSeconds(30); - private static final Duration DEFAULT_MIN_LOG_INTERVAL = Duration.ofMinutes(3); - private static final ResultType[] RESULT_TYPE_VALUES = ResultType.values(); + static final long INVALID = -1; + private static final long DEFAULT_OFFSETS_UPDATE_INTERVAL_MS = 30 * Time.MS_PER_SECOND; - private final Map topicPartitionCurrentOffset; - private final Map topicPartitionEndOffset; - private Instant lastMetricsCollectedTime; - private final Duration offsetsUpdateInterval; - private final StatsAccumulator statsAccumulator; + private final Map topicPartitionOffsetInfo; + private volatile long lastMetricsCollectedTime; + private final long offsetsUpdateIntervalMs; public TopicPartitionsOffsetsTracker() { - this(DEFAULT_OFFSETS_UPDATE_INTERVAL); + this(validateInterval(DEFAULT_OFFSETS_UPDATE_INTERVAL_MS)); } - public TopicPartitionsOffsetsTracker(Duration offsetsUpdateInterval) { - this(offsetsUpdateInterval, DEFAULT_MIN_LOG_INTERVAL); + /** + * N.B. Package-private, for tests only. This constructor skips the {@link #validateInterval(long)} check. + */ + TopicPartitionsOffsetsTracker(long offsetsUpdateIntervalMs) { + this.offsetsUpdateIntervalMs = offsetsUpdateIntervalMs; + this.lastMetricsCollectedTime = 0; + // N.B. This map can be accessed both by poll (one caller at a time) and by metrics (arbitrary concurrency) + // so it needs to be threadsafe. + this.topicPartitionOffsetInfo = new VeniceConcurrentHashMap<>(); } - public TopicPartitionsOffsetsTracker(@Nonnull Duration offsetsUpdateInterval, Duration minLogInterval) { - Validate.notNull(offsetsUpdateInterval); - this.offsetsUpdateInterval = offsetsUpdateInterval; - this.lastMetricsCollectedTime = null; - // N.B. These maps can be accessed both by poll (one caller at a time) and by metrics (arbitrary concurrency) - // so they need to be threadsafe. - this.topicPartitionCurrentOffset = new VeniceConcurrentHashMap<>(); - this.topicPartitionEndOffset = new VeniceConcurrentHashMap<>(); - this.statsAccumulator = new StatsAccumulator(minLogInterval); + private static long validateInterval(long interval) { + if (interval > 0) { + return interval; + } + throw new IllegalArgumentException("The interval must be above zero."); } /** @@ -64,70 +52,64 @@ public TopicPartitionsOffsetsTracker(@Nonnull Duration offsetsUpdateInterval, Du * to be the largest offset amongst the records passed in. * * @param records consumed records + * @param kafkaConsumer from which to extract metrics */ public void updateEndAndCurrentOffsets( ConsumerRecords records, - Map metrics) { - if (lastMetricsCollectedTime != null && LatencyUtils - .getElapsedTimeInMs(lastMetricsCollectedTime.toEpochMilli()) < offsetsUpdateInterval.toMillis()) { - return; // Not yet - } - lastMetricsCollectedTime = Instant.now(); + Consumer kafkaConsumer) { + + boolean lagMetricMissing = false; // Update current offset cache for all topics partition. List> listOfRecordsForOnePartition; ConsumerRecord lastConsumerRecordOfPartition; + KafkaPartitionTopicOffsetMetrics offsetInfo; for (TopicPartition tp: records.partitions()) { listOfRecordsForOnePartition = records.records(tp); lastConsumerRecordOfPartition = listOfRecordsForOnePartition.get(listOfRecordsForOnePartition.size() - 1); - topicPartitionCurrentOffset.put(tp, (double) lastConsumerRecordOfPartition.offset()); + offsetInfo = topicPartitionOffsetInfo.computeIfAbsent(tp, k -> new KafkaPartitionTopicOffsetMetrics()); + offsetInfo.setCurrentOffset(lastConsumerRecordOfPartition.offset()); + if (offsetInfo.isLagMetricMissing()) { + lagMetricMissing = true; + } } + long currentTime = System.currentTimeMillis(); + long elapsedTime = currentTime - lastMetricsCollectedTime; + if (elapsedTime < offsetsUpdateIntervalMs && !lagMetricMissing) { + return; // Not yet + } + + lastMetricsCollectedTime = currentTime; MetricName metricName; Metric metric; TopicPartition tp; - Double currOffset; - for (Map.Entry entry: metrics.entrySet()) { + for (Map.Entry entry: kafkaConsumer.metrics().entrySet()) { metricName = entry.getKey(); metric = entry.getValue(); - if (isMetricEntryRecordsLag(metricName, metric)) { + if (Objects.equals(metricName.name(), "records-lag")) { tp = new TopicPartition(metricName.tags().get("topic"), Integer.parseInt(metricName.tags().get("partition"))); - currOffset = topicPartitionCurrentOffset.get(tp); - if (currOffset != null) { - topicPartitionEndOffset.put(tp, currOffset + ((Double) metric.metricValue())); - } + offsetInfo = topicPartitionOffsetInfo.computeIfAbsent(tp, k -> new KafkaPartitionTopicOffsetMetrics()); + offsetInfo.setLagMetric(metric); } } } - private boolean isMetricEntryRecordsLag(MetricName metricName, Metric metric) { - try { - return Objects.equals(metricName.name(), "records-lag") && (metric.metricValue() instanceof Double); - } catch (Exception e) { - LOGGER.warn( - "Caught exception: {} when attempting to get consumer metrics. Incomplete metrics might be returned.", - e.getMessage()); - return false; - } - } - /** * Remove tracked offsets state of a topic partition. * * @param topicPartition */ public void removeTrackedOffsets(TopicPartition topicPartition) { - topicPartitionCurrentOffset.remove(topicPartition); - topicPartitionEndOffset.remove(topicPartition); + topicPartitionOffsetInfo.remove(topicPartition); } /** * Clear all tracked offsets state */ public void clearAllOffsetState() { - topicPartitionCurrentOffset.clear(); - topicPartitionEndOffset.clear(); + topicPartitionOffsetInfo.clear(); } /** @@ -137,8 +119,8 @@ public void clearAllOffsetState() { * @return end offset of a topic partition if there is any. */ public long getEndOffset(String topic, int partition) { - Double endOffset = topicPartitionEndOffset.get(new TopicPartition(topic, partition)); - return endOffset == null ? -1 : endOffset.longValue(); + KafkaPartitionTopicOffsetMetrics offsetInfo = topicPartitionOffsetInfo.get(new TopicPartition(topic, partition)); + return offsetInfo == null ? INVALID : offsetInfo.getEndOffset(); } /** @@ -148,85 +130,7 @@ public long getEndOffset(String topic, int partition) { * @return end offset of a topic partition if there is any. */ public long getOffsetLag(String topic, int partition) { - TopicPartition topicPartition = new TopicPartition(topic, partition); - final Double endOffset = topicPartitionEndOffset.get(topicPartition); - if (endOffset == null) { - statsAccumulator.recordResult(ResultType.NO_OFFSET_LAG); - return -1; - } - final Double currOffset = topicPartitionCurrentOffset.get(topicPartition); - if (currOffset == null) { - statsAccumulator.recordResult(ResultType.NO_OFFSET_LAG); - return -1; - } - final long offsetLag = endOffset.longValue() - currOffset.longValue(); - if (offsetLag < 0) { // Invalid offset lag - statsAccumulator.recordResult(ResultType.INVALID_OFFSET_LAG); - return -1; - } - statsAccumulator.recordResult(ResultType.VALID_OFFSET_LAG); - statsAccumulator.maybeLogAccumulatedStats(LOGGER); - return offsetLag; - } - - /** - * Package private for testing purpose - */ - public Map getResultsStats() { - return statsAccumulator.getResultsStats(); - } - - /** - * This class keeps track of results stats and can be used to log the accumulated results stats once in a while - */ - private static class StatsAccumulator { - private final Map resultsStats; - private final Duration minLogInterval; - private Instant lastLoggedTime; - - private StatsAccumulator(Duration minLogInterval) { - // N.B. This map can be accessed by many functions in the ingestion path and by metrics. It is not 100% - // clear whether concurrent access to this map is possible, but just to be on the safe side, we are going - // to make it a threadsafe map. - this.resultsStats = new VeniceConcurrentHashMap<>(RESULT_TYPE_VALUES.length); - this.minLogInterval = minLogInterval; - this.lastLoggedTime = Instant.now(); - } - - private void recordResult(ResultType resultType) { - resultsStats.compute(resultType, (k, v) -> v == null ? 1 : v + 1); - } - - /** - * @param LOGGER the {@link Logger} instance which is used to log the accumulated stats - */ - private void maybeLogAccumulatedStats(Logger LOGGER) { - if (resultsStats.isEmpty()) { - return; - } - final Instant now = Instant.now(); - final Duration timeSinceLastTimeLogged = Duration.between(lastLoggedTime, now); - if (timeSinceLastTimeLogged.toMillis() >= minLogInterval.toMillis()) { - LOGGER.info( - String.format( - "In the last %d second(s), results states are: %s", - timeSinceLastTimeLogged.getSeconds(), - resultsStatsToString())); - lastLoggedTime = now; - resultsStats.clear(); - } - } - - private String resultsStatsToString() { - StringJoiner sj = new StringJoiner(", "); - for (ResultType resultType: RESULT_TYPE_VALUES) { - sj.add(resultType + " count: " + resultsStats.getOrDefault(resultType, 0)); - } - return sj.toString(); - } - - Map getResultsStats() { - return Collections.unmodifiableMap(resultsStats); - } + KafkaPartitionTopicOffsetMetrics offsetInfo = topicPartitionOffsetInfo.get(new TopicPartition(topic, partition)); + return offsetInfo == null ? INVALID : offsetInfo.getLag(); } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerAdapter.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerAdapter.java index 212c9b3956d..41023428e57 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerAdapter.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerAdapter.java @@ -275,7 +275,7 @@ public Map toList() { + return Collections.emptyList(); + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessage.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessage.java index b02df46c56d..f7ac19388e6 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessage.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessage.java @@ -1,8 +1,5 @@ package com.linkedin.venice.pubsub.api; -import static com.linkedin.venice.writer.VeniceWriter.EMPTY_MSG_HEADERS; - - public interface PubSubMessage { /** * @return the key part of this message @@ -48,6 +45,6 @@ default int getPartition() { boolean isEndOfBootstrap(); default PubSubMessageHeaders getPubSubMessageHeaders() { - return EMPTY_MSG_HEADERS; + return EmptyPubSubMessageHeaders.SINGLETON; } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessageHeaders.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessageHeaders.java index 38197cec466..e5f02835031 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessageHeaders.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessageHeaders.java @@ -11,10 +11,10 @@ * In case of headers with the same key, only the most recently added headers value will be kept. */ public class PubSubMessageHeaders { - - // Kafka allows duplicate keys in the headers but some pubsub systems may not - // allow it. Hence, it would be good to enforce uniqueness of keys in headers - // from the beginning. + /** + * N.B.: Kafka allows duplicate keys in the headers but some pubsub systems may not + * allow it. Hence, we will enforce uniqueness of keys in headers from the beginning. + */ private final Map headers = new LinkedHashMap<>(); public static final String VENICE_TRANSPORT_PROTOCOL_HEADER = "vtp"; diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriter.java b/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriter.java index 5bf4a983154..ded4f3fed7e 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriter.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriter.java @@ -36,6 +36,7 @@ import com.linkedin.venice.message.KafkaKey; import com.linkedin.venice.meta.Version; import com.linkedin.venice.partitioner.VenicePartitioner; +import com.linkedin.venice.pubsub.api.EmptyPubSubMessageHeaders; import com.linkedin.venice.pubsub.api.PubSubMessageHeader; import com.linkedin.venice.pubsub.api.PubSubMessageHeaders; import com.linkedin.venice.pubsub.api.PubSubProduceResult; @@ -210,8 +211,6 @@ public class VeniceWriter extends AbstractVeniceWriter { public static final LeaderMetadataWrapper DEFAULT_LEADER_METADATA_WRAPPER = new LeaderMetadataWrapper(DEFAULT_UPSTREAM_OFFSET, DEFAULT_UPSTREAM_KAFKA_CLUSTER_ID); - public static final PubSubMessageHeaders EMPTY_MSG_HEADERS = new PubSubMessageHeaders(); - // Immutable state private final PubSubMessageHeaders protocolSchemaHeaders; @@ -343,7 +342,7 @@ public VeniceWriter( this.threadPoolExecutor.allowCoreThreadTimeOut(true); // allow core threads to timeout this.protocolSchemaHeaders = overrideProtocolSchema == null - ? EMPTY_MSG_HEADERS + ? EmptyPubSubMessageHeaders.SINGLETON : new PubSubMessageHeaders() .add(VENICE_TRANSPORT_PROTOCOL_HEADER, overrideProtocolSchema.toString().getBytes(StandardCharsets.UTF_8)); try { @@ -1357,7 +1356,7 @@ private PubSubMessageHeaders getHeaders(ProducerMetadata producerMetadata) { } /** - * {@link PubSubMessageHeaders#VENICE_TRANSPORT_PROTOCOL_HEADER} or {@link VeniceWriter#EMPTY_MSG_HEADERS} is used for + * {@link PubSubMessageHeaders#VENICE_TRANSPORT_PROTOCOL_HEADER} or {@link EmptyPubSubMessageHeaders} is used for * all messages to a partition based on {@link VeniceWriter} param overrideProtocolSchema and whether it's a first message. * {@link PubSubMessageHeaders#VENICE_LEADER_COMPLETION_STATE_HEADER} is added to the above headers for HB SOS message. * @@ -1378,7 +1377,7 @@ private PubSubMessageHeaders getHeaders( PubSubMessageHeaders pubSubMessageHeaders = producerMetadata.getSegmentNumber() == 0 && producerMetadata.getMessageSequenceNumber() == 0 ? protocolSchemaHeaders - : EMPTY_MSG_HEADERS; + : EmptyPubSubMessageHeaders.SINGLETON; if (addLeaderCompleteState) { // copy protocolSchemaHeaders locally and add extra header for leaderCompleteState diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/TopicPartitionsOffsetsTrackerTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/TopicPartitionsOffsetsTrackerTest.java index 7f2400a71af..12d219bb0f5 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/TopicPartitionsOffsetsTrackerTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/TopicPartitionsOffsetsTrackerTest.java @@ -1,13 +1,14 @@ package com.linkedin.venice.pubsub.adapter.kafka; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import java.time.Duration; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.Metric; @@ -19,7 +20,7 @@ public class TopicPartitionsOffsetsTrackerTest { - private static final Duration OFFSETS_UPDATE_INTERVAL = Duration.ZERO; // No interval + private static final long OFFSETS_UPDATE_INTERVAL = 0; // No interval private static final String TOPIC_1 = "topic_1"; private static final String TOPIC_2 = "topic_2"; private static final int PARTITION_ID = 2; @@ -34,16 +35,9 @@ public void initTopicPartitionsOffsetsTracker() { public void testNoUpdateWithRecords() { Assert.assertEquals(topicPartitionsOffsetsTracker.getEndOffset(TOPIC_1, PARTITION_ID), -1); Assert.assertEquals(topicPartitionsOffsetsTracker.getEndOffset(TOPIC_2, PARTITION_ID), -1); - Assert.assertTrue(topicPartitionsOffsetsTracker.getResultsStats().isEmpty()); Assert.assertEquals(topicPartitionsOffsetsTracker.getOffsetLag(TOPIC_1, PARTITION_ID), -1); Assert.assertEquals(topicPartitionsOffsetsTracker.getOffsetLag(TOPIC_2, PARTITION_ID), -1); - Assert.assertEquals(topicPartitionsOffsetsTracker.getResultsStats().size(), 1); - Assert.assertEquals( - topicPartitionsOffsetsTracker.getResultsStats() - .get(TopicPartitionsOffsetsTracker.ResultType.NO_OFFSET_LAG) - .intValue(), - 2); } @Test @@ -82,7 +76,7 @@ public void testUpdateWithRecords() { metricsTags.put("partition", String.valueOf(PARTITION_ID)); MetricName metricName = new MetricName("records-lag", "", "", metricsTags); Metric metricValue = mock(Metric.class); - when(metricValue.metricValue()).thenReturn((double) firstPartitionLag); + when(metricValue.metricValue()).thenReturn(Double.valueOf(firstPartitionLag)); mockMetrics.put(metricName, metricValue); // Set up partition record lag for the second topic partition @@ -91,26 +85,15 @@ public void testUpdateWithRecords() { metricsTags.put("partition", String.valueOf(PARTITION_ID)); metricName = new MetricName("records-lag", "", "", metricsTags); metricValue = mock(Metric.class); - when(metricValue.metricValue()).thenReturn((double) secondPartitionLag); + when(metricValue.metricValue()).thenReturn(Double.valueOf(secondPartitionLag)); mockMetrics.put(metricName, metricValue); + Consumer kafkaConsumer = mock(Consumer.class); + doReturn(mockMetrics).when(kafkaConsumer).metrics(); - topicPartitionsOffsetsTracker.updateEndAndCurrentOffsets(mockRecords, mockMetrics); + topicPartitionsOffsetsTracker.updateEndAndCurrentOffsets(mockRecords, kafkaConsumer); Assert.assertEquals(topicPartitionsOffsetsTracker.getOffsetLag(TOPIC_1, PARTITION_ID), firstPartitionLag); - Assert.assertEquals(topicPartitionsOffsetsTracker.getResultsStats().size(), 1); - Assert.assertEquals( - topicPartitionsOffsetsTracker.getResultsStats() - .get(TopicPartitionsOffsetsTracker.ResultType.VALID_OFFSET_LAG) - .intValue(), - 1); - Assert.assertEquals(topicPartitionsOffsetsTracker.getOffsetLag(TOPIC_2, PARTITION_ID), secondPartitionLag); - Assert.assertEquals(topicPartitionsOffsetsTracker.getResultsStats().size(), 1); - Assert.assertEquals( - topicPartitionsOffsetsTracker.getResultsStats() - .get(TopicPartitionsOffsetsTracker.ResultType.VALID_OFFSET_LAG) - .intValue(), - 2); // End offset == current offset + lag Assert.assertEquals( @@ -129,11 +112,5 @@ public void testUpdateWithRecords() { topicPartitionsOffsetsTracker.clearAllOffsetState(); Assert.assertEquals(topicPartitionsOffsetsTracker.getEndOffset(TOPIC_1, PARTITION_ID), -1); Assert.assertEquals(topicPartitionsOffsetsTracker.getEndOffset(TOPIC_2, PARTITION_ID), -1); - Assert.assertEquals(topicPartitionsOffsetsTracker.getResultsStats().size(), 1); - Assert.assertEquals( - topicPartitionsOffsetsTracker.getResultsStats() - .get(TopicPartitionsOffsetsTracker.ResultType.VALID_OFFSET_LAG) - .intValue(), - 2); } }