From 7e9aa79a76cc908d449f333abcdea2d069e65e83 Mon Sep 17 00:00:00 2001 From: Jialin Liu Date: Thu, 31 Oct 2024 09:56:26 -0700 Subject: [PATCH 1/7] [server] Add store-aware partition-wise shared consumer assignment strategy (#1261) Add a new version of partition wise shared consumer assignment strategy. We have been seeing subscriptions to the same topic / store be assigned to the same consumer, and for a particular store push's view (can be inc push / full push) it can be competing with each other and becomes the long-tail partition and slow down the overall progress. Assuming the store/topic itself does not have data-skew, then we should try to assign these subscriptions to different consumers as even as possible. Especially for RT topics, backup / current / future version will share the same input volume, so we should not treat them differently within the same pool, but we can further optimize that in different level (Pool assignment strategy) This PR adds the new strategy so when a new topic partition is looking for assignment, it will compute and sort all the consumer's load based on general load and the store-specific load. It will assign the new topic partition to the least loaded consumer based on the computed load. --- .../kafka/consumer/KafkaConsumerService.java | 12 +- .../PartitionWiseKafkaConsumerService.java | 63 ++++++++- .../kafka/consumer/SharedKafkaConsumer.java | 10 +- ...warePartitionWiseKafkaConsumerService.java | 121 ++++++++++++++++++ .../consumer/KafkaConsumerServiceTest.java | 78 ++++++++++- ...erviceWithStoreAwarePartitionWiseTest.java | 7 + .../consumer/SharedKafkaConsumerTest.java | 2 +- 7 files changed, 278 insertions(+), 15 deletions(-) create mode 100644 clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreAwarePartitionWiseKafkaConsumerService.java create mode 100644 clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceWithStoreAwarePartitionWiseTest.java diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerService.java index f20e89d439..b7d76bb930 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerService.java @@ -173,7 +173,10 @@ protected KafkaConsumerService( } /** May be overridden to clean up state in sub-classes */ - void handleUnsubscription(SharedKafkaConsumer consumer, PubSubTopicPartition topicPartition) { + void handleUnsubscription( + SharedKafkaConsumer consumer, + PubSubTopic versionTopic, + PubSubTopicPartition topicPartition) { } private String getUniqueClientId(String kafkaUrl, int suffix) { @@ -563,7 +566,8 @@ private interface OffsetGetter { */ public enum ConsumerAssignmentStrategy { TOPIC_WISE_SHARED_CONSUMER_ASSIGNMENT_STRATEGY(TopicWiseKafkaConsumerService::new), - PARTITION_WISE_SHARED_CONSUMER_ASSIGNMENT_STRATEGY(PartitionWiseKafkaConsumerService::new); + PARTITION_WISE_SHARED_CONSUMER_ASSIGNMENT_STRATEGY(PartitionWiseKafkaConsumerService::new), + STORE_AWARE_PARTITION_WISE_SHARED_CONSUMER_ASSIGNMENT_STRATEGY(StoreAwarePartitionWiseKafkaConsumerService::new); final KCSConstructor constructor; @@ -576,4 +580,8 @@ public enum ConsumerAssignmentStrategy { public void setThreadFactory(RandomAccessDaemonThreadFactory threadFactory) { this.threadFactory = threadFactory; } + + IndexedMap getConsumerToConsumptionTask() { + return consumerToConsumptionTask; + } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PartitionWiseKafkaConsumerService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PartitionWiseKafkaConsumerService.java index 47f08942f9..bb315503c0 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PartitionWiseKafkaConsumerService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PartitionWiseKafkaConsumerService.java @@ -32,7 +32,7 @@ public class PartitionWiseKafkaConsumerService extends KafkaConsumerService { * have same real-time topics, we should avoid same real-time topic partition from different version topics sharing * the same consumer from consumer pool. */ - private final Map> rtTopicPartitionToConsumerMap = + protected final Map> rtTopicPartitionToConsumerMap = new VeniceConcurrentHashMap<>(); private final Logger LOGGER; @@ -58,6 +58,48 @@ public class PartitionWiseKafkaConsumerService extends KafkaConsumerService { final boolean isKafkaConsumerOffsetCollectionEnabled, final ReadOnlyStoreRepository metadataRepository, final boolean isUnregisterMetricForDeletedStoreEnabled) { + this( + poolType, + consumerFactory, + consumerProperties, + readCycleDelayMs, + numOfConsumersPerKafkaCluster, + ingestionThrottler, + kafkaClusterBasedRecordThrottler, + metricsRepository, + kafkaClusterAlias, + sharedConsumerNonExistingTopicCleanupDelayMS, + topicExistenceChecker, + liveConfigBasedKafkaThrottlingEnabled, + pubSubDeserializer, + time, + stats, + isKafkaConsumerOffsetCollectionEnabled, + metadataRepository, + isUnregisterMetricForDeletedStoreEnabled, + PartitionWiseKafkaConsumerService.class.toString()); + } + + PartitionWiseKafkaConsumerService( + final ConsumerPoolType poolType, + final PubSubConsumerAdapterFactory consumerFactory, + final Properties consumerProperties, + final long readCycleDelayMs, + final int numOfConsumersPerKafkaCluster, + final IngestionThrottler ingestionThrottler, + final KafkaClusterBasedRecordThrottler kafkaClusterBasedRecordThrottler, + final MetricsRepository metricsRepository, + final String kafkaClusterAlias, + final long sharedConsumerNonExistingTopicCleanupDelayMS, + final TopicExistenceChecker topicExistenceChecker, + final boolean liveConfigBasedKafkaThrottlingEnabled, + final PubSubMessageDeserializer pubSubDeserializer, + final Time time, + final AggKafkaConsumerServiceStats stats, + final boolean isKafkaConsumerOffsetCollectionEnabled, + final ReadOnlyStoreRepository metadataRepository, + final boolean isUnregisterMetricForDeletedStoreEnabled, + final String loggerNamePrefix) { super( poolType, consumerFactory, @@ -77,7 +119,7 @@ public class PartitionWiseKafkaConsumerService extends KafkaConsumerService { isKafkaConsumerOffsetCollectionEnabled, metadataRepository, isUnregisterMetricForDeletedStoreEnabled); - this.LOGGER = LogManager.getLogger(PartitionWiseKafkaConsumerService.class + " [" + kafkaUrlForLogger + "]"); + this.LOGGER = LogManager.getLogger(loggerNamePrefix + " [" + kafkaUrlForLogger + "]"); } @Override @@ -137,15 +179,18 @@ protected synchronized SharedKafkaConsumer pickConsumerForPartition( return consumer; } - private boolean alreadySubscribedRealtimeTopicPartition( + protected boolean alreadySubscribedRealtimeTopicPartition( SharedKafkaConsumer consumer, PubSubTopicPartition topicPartition) { - Set consumers = rtTopicPartitionToConsumerMap.get(topicPartition); + Set consumers = getRtTopicPartitionToConsumerMap().get(topicPartition); return consumers != null && consumers.contains(consumer); } @Override - void handleUnsubscription(SharedKafkaConsumer consumer, PubSubTopicPartition pubSubTopicPartition) { + void handleUnsubscription( + SharedKafkaConsumer consumer, + PubSubTopic versionTopic, + PubSubTopicPartition pubSubTopicPartition) { if (pubSubTopicPartition.getPubSubTopic().isRealTime()) { Set rtTopicConsumers = rtTopicPartitionToConsumerMap.get(pubSubTopicPartition); if (rtTopicConsumers != null) { @@ -153,4 +198,12 @@ void handleUnsubscription(SharedKafkaConsumer consumer, PubSubTopicPartition pub } } } + + Map> getRtTopicPartitionToConsumerMap() { + return rtTopicPartitionToConsumerMap; + } + + Logger getLOGGER() { + return LOGGER; + } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/SharedKafkaConsumer.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/SharedKafkaConsumer.java index f6a97bd461..735554a5e0 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/SharedKafkaConsumer.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/SharedKafkaConsumer.java @@ -109,7 +109,7 @@ public SharedKafkaConsumer( * Listeners may use this callback to clean up lingering state they may be holding about a consumer. */ interface UnsubscriptionListener { - void call(SharedKafkaConsumer consumer, PubSubTopicPartition pubSubTopicPartition); + void call(SharedKafkaConsumer consumer, PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition); } protected synchronized void updateCurrentAssignment(Set newAssignment) { @@ -155,8 +155,8 @@ synchronized void subscribe( public synchronized void unSubscribe(PubSubTopicPartition pubSubTopicPartition) { unSubscribeAction(() -> { this.delegate.unSubscribe(pubSubTopicPartition); - subscribedTopicPartitionToVersionTopic.remove(pubSubTopicPartition); - unsubscriptionListener.call(this, pubSubTopicPartition); + PubSubTopic versionTopic = subscribedTopicPartitionToVersionTopic.remove(pubSubTopicPartition); + unsubscriptionListener.call(this, versionTopic, pubSubTopicPartition); return Collections.singleton(pubSubTopicPartition); }); } @@ -166,8 +166,8 @@ public synchronized void batchUnsubscribe(Set pubSubTopicP unSubscribeAction(() -> { this.delegate.batchUnsubscribe(pubSubTopicPartitionSet); for (PubSubTopicPartition pubSubTopicPartition: pubSubTopicPartitionSet) { - subscribedTopicPartitionToVersionTopic.remove(pubSubTopicPartition); - unsubscriptionListener.call(this, pubSubTopicPartition); + PubSubTopic versionTopic = subscribedTopicPartitionToVersionTopic.remove(pubSubTopicPartition); + unsubscriptionListener.call(this, versionTopic, pubSubTopicPartition); } return pubSubTopicPartitionSet; }); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreAwarePartitionWiseKafkaConsumerService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreAwarePartitionWiseKafkaConsumerService.java new file mode 100644 index 0000000000..46100d8b58 --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreAwarePartitionWiseKafkaConsumerService.java @@ -0,0 +1,121 @@ +package com.linkedin.davinci.kafka.consumer; + +import com.linkedin.davinci.stats.AggKafkaConsumerServiceStats; +import com.linkedin.venice.meta.ReadOnlyStoreRepository; +import com.linkedin.venice.meta.Version; +import com.linkedin.venice.pubsub.PubSubConsumerAdapterFactory; +import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; +import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.utils.Time; +import io.tehuti.metrics.MetricsRepository; +import java.util.HashSet; +import java.util.Properties; + + +/** + * {@link StoreAwarePartitionWiseKafkaConsumerService} is used to allocate share consumer from consumer pool at partition + * granularity. One shared consumer may have multiple topics, and each topic may have multiple consumers. + * This is store-aware version of topic-wise shared consumer service. The topic partition assignment in this service has + * a heuristic that we should distribute the all the subscriptions related to a same store / version as even as possible. + * The load calculation for each consumer will be: + * Consumer assignment size + IMPOSSIBLE_MAX_PARTITION_COUNT_PER_CONSUMER * subscription count for the same store; + * and we will pick the least loaded consumer for a new topic partition request. If there is no eligible consumer, it + * will throw {@link IllegalStateException} + */ +public class StoreAwarePartitionWiseKafkaConsumerService extends PartitionWiseKafkaConsumerService { + // This constant makes sure the store subscription count will always be prioritized over consumer assignment count. + private static final int IMPOSSIBLE_MAX_PARTITION_COUNT_PER_CONSUMER = 10000; + + StoreAwarePartitionWiseKafkaConsumerService( + final ConsumerPoolType poolType, + final PubSubConsumerAdapterFactory consumerFactory, + final Properties consumerProperties, + final long readCycleDelayMs, + final int numOfConsumersPerKafkaCluster, + final IngestionThrottler ingestionThrottler, + final KafkaClusterBasedRecordThrottler kafkaClusterBasedRecordThrottler, + final MetricsRepository metricsRepository, + final String kafkaClusterAlias, + final long sharedConsumerNonExistingTopicCleanupDelayMS, + final TopicExistenceChecker topicExistenceChecker, + final boolean liveConfigBasedKafkaThrottlingEnabled, + final PubSubMessageDeserializer pubSubDeserializer, + final Time time, + final AggKafkaConsumerServiceStats stats, + final boolean isKafkaConsumerOffsetCollectionEnabled, + final ReadOnlyStoreRepository metadataRepository, + final boolean isUnregisterMetricForDeletedStoreEnabled) { + super( + poolType, + consumerFactory, + consumerProperties, + readCycleDelayMs, + numOfConsumersPerKafkaCluster, + ingestionThrottler, + kafkaClusterBasedRecordThrottler, + metricsRepository, + kafkaClusterAlias, + sharedConsumerNonExistingTopicCleanupDelayMS, + topicExistenceChecker, + liveConfigBasedKafkaThrottlingEnabled, + pubSubDeserializer, + time, + stats, + isKafkaConsumerOffsetCollectionEnabled, + metadataRepository, + isUnregisterMetricForDeletedStoreEnabled, + StoreAwarePartitionWiseKafkaConsumerService.class.toString()); + } + + @Override + protected synchronized SharedKafkaConsumer pickConsumerForPartition( + PubSubTopic versionTopic, + PubSubTopicPartition topicPartition) { + String storeName = versionTopic.getStoreName(); + long minLoad = Long.MAX_VALUE; + SharedKafkaConsumer minLoadConsumer = null; + for (SharedKafkaConsumer consumer: getConsumerToConsumptionTask().keySet()) { + int index = getConsumerToConsumptionTask().indexOf(consumer); + if (topicPartition.getPubSubTopic().isRealTime() + && alreadySubscribedRealtimeTopicPartition(consumer, topicPartition)) { + getLOGGER().info( + "Consumer id: {} has already subscribed the same real time topic-partition: {} and thus cannot be picked", + index, + topicPartition); + continue; + } + long overallLoad = getConsumerStoreLoad(consumer, storeName); + if (overallLoad < minLoad) { + minLoadConsumer = consumer; + minLoad = overallLoad; + } + } + if (minLoad == Long.MAX_VALUE) { + throw new IllegalStateException("Unable to find least loaded consumer entry."); + } + + // Update RT topic partition consumer map. + if (topicPartition.getPubSubTopic().isRealTime()) { + getRtTopicPartitionToConsumerMap().computeIfAbsent(topicPartition, key -> new HashSet<>()).add(minLoadConsumer); + } + + getLOGGER().info( + "Picked consumer id: {}, assignment size: {}, computed load: {} for topic partition: {}, version topic: {}", + getConsumerToConsumptionTask().indexOf(minLoadConsumer), + minLoadConsumer.getAssignmentSize(), + minLoad, + topicPartition, + versionTopic); + return minLoadConsumer; + } + + long getConsumerStoreLoad(SharedKafkaConsumer consumer, String storeName) { + long baseAssignmentCount = consumer.getAssignmentSize(); + long storeSubscriptionCount = consumer.getAssignment() + .stream() + .filter(x -> Version.parseStoreFromKafkaTopicName(x.getTopicName()).equals(storeName)) + .count(); + return storeSubscriptionCount * IMPOSSIBLE_MAX_PARTITION_COUNT_PER_CONSUMER + baseAssignmentCount; + } +} diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceTest.java index 79e3fa9f14..02b25ec091 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceTest.java @@ -8,6 +8,7 @@ import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -15,6 +16,8 @@ import static org.mockito.Mockito.when; import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; +import com.linkedin.davinci.utils.IndexedHashMap; +import com.linkedin.davinci.utils.IndexedMap; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.message.KafkaKey; import com.linkedin.venice.meta.ReadOnlyStoreRepository; @@ -33,10 +36,12 @@ import com.linkedin.venice.utils.SystemTime; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import com.linkedin.venice.utils.pools.LandFillObjectPool; import io.tehuti.metrics.MetricsRepository; import io.tehuti.metrics.Sensor; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -44,6 +49,7 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.logging.log4j.LogManager; import org.testng.Assert; import org.testng.annotations.Test; @@ -281,7 +287,7 @@ public void testTopicWiseGetConsumerForHybridMode() throws Exception { ConsumerPoolType.REGULAR_POOL, factory, properties, - 1000l, + 1000L, 2, mock(IngestionThrottler.class), mock(KafkaClusterBasedRecordThrottler.class), @@ -381,7 +387,7 @@ public void testPartitionWiseGetConsumer() { ConsumerPoolType.REGULAR_POOL, factory, properties, - 1000l, + 1000L, 2, mock(IngestionThrottler.class), mock(KafkaClusterBasedRecordThrottler.class), @@ -416,6 +422,74 @@ public void testPartitionWiseGetConsumer() { Assert.assertEquals(consumerForT1P3, consumerForT2P1); } + @Test + public void testStoreAwarePartitionWiseGetConsumer() { + String storeName1 = Utils.getUniqueString("test_consumer_service1"); + String topicForStoreName1 = Version.composeKafkaTopic(storeName1, 1); + PubSubTopic pubSubTopicForStoreName1 = pubSubTopicRepository.getTopic(topicForStoreName1); + + String storeName2 = Utils.getUniqueString("test_consumer_service2"); + String topicForStoreName2 = Version.composeKafkaTopic(storeName2, 1); + PubSubTopic pubSubTopicForStoreName2 = pubSubTopicRepository.getTopic(topicForStoreName2); + + String storeName3 = Utils.getUniqueString("test_consumer_service3"); + String topicForStoreName3 = Version.composeKafkaTopic(storeName3, 1); + PubSubTopic pubSubTopicForStoreName3 = pubSubTopicRepository.getTopic(topicForStoreName3); + + SharedKafkaConsumer consumer1 = mock(SharedKafkaConsumer.class); + SharedKafkaConsumer consumer2 = mock(SharedKafkaConsumer.class); + ConsumptionTask consumptionTask = mock(ConsumptionTask.class); + + StoreAwarePartitionWiseKafkaConsumerService consumerService = + mock(StoreAwarePartitionWiseKafkaConsumerService.class); + + // Prepare for the mock. + IndexedMap consumptionTaskIndexedMap = new IndexedHashMap<>(2); + consumptionTaskIndexedMap.put(consumer1, consumptionTask); + consumptionTaskIndexedMap.put(consumer2, consumptionTask); + when(consumerService.getConsumerToConsumptionTask()).thenReturn(consumptionTaskIndexedMap); + + Map> rtTopicPartitionToConsumerMap = + new VeniceConcurrentHashMap<>(); + when(consumerService.getRtTopicPartitionToConsumerMap()).thenReturn(rtTopicPartitionToConsumerMap); + when(consumerService.getLOGGER()) + .thenReturn(LogManager.getLogger(StoreAwarePartitionWiseKafkaConsumerService.class)); + doCallRealMethod().when(consumerService).pickConsumerForPartition(any(), any()); + doCallRealMethod().when(consumerService).getConsumerStoreLoad(any(), anyString()); + + when(consumer1.getAssignmentSize()).thenReturn(1); + when(consumer1.getAssignment()) + .thenReturn(Collections.singleton(new PubSubTopicPartitionImpl(pubSubTopicForStoreName1, 100))); + Set tpSet = new HashSet<>(); + tpSet.add(new PubSubTopicPartitionImpl(pubSubTopicForStoreName2, 100)); + tpSet.add(new PubSubTopicPartitionImpl(pubSubTopicForStoreName2, 101)); + when(consumer2.getAssignmentSize()).thenReturn(2); + when(consumer2.getAssignment()).thenReturn(tpSet); + Assert.assertEquals(consumerService.getConsumerStoreLoad(consumer1, storeName1), 10001); + Assert.assertEquals(consumerService.getConsumerStoreLoad(consumer1, storeName2), 1); + Assert.assertEquals(consumerService.getConsumerStoreLoad(consumer1, storeName3), 1); + + Assert.assertEquals(consumerService.getConsumerStoreLoad(consumer2, storeName2), 20002); + Assert.assertEquals(consumerService.getConsumerStoreLoad(consumer2, storeName1), 2); + Assert.assertEquals(consumerService.getConsumerStoreLoad(consumer2, storeName3), 2); + + Assert.assertEquals( + consumerService.pickConsumerForPartition( + pubSubTopicForStoreName1, + new PubSubTopicPartitionImpl(pubSubTopicForStoreName1, 0)), + consumer2); + Assert.assertEquals( + consumerService.pickConsumerForPartition( + pubSubTopicForStoreName2, + new PubSubTopicPartitionImpl(pubSubTopicForStoreName2, 0)), + consumer1); + Assert.assertEquals( + consumerService.pickConsumerForPartition( + pubSubTopicForStoreName3, + new PubSubTopicPartitionImpl(pubSubTopicForStoreName3, 0)), + consumer1); + } + @Test public void testGetMaxElapsedTimeMSSinceLastPollInConsumerPool() { // Mock the necessary components diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceWithStoreAwarePartitionWiseTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceWithStoreAwarePartitionWiseTest.java new file mode 100644 index 0000000000..857e1c0064 --- /dev/null +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceWithStoreAwarePartitionWiseTest.java @@ -0,0 +1,7 @@ +package com.linkedin.davinci.kafka.consumer; + +public class KafkaStoreIngestionServiceWithStoreAwarePartitionWiseTest extends KafkaStoreIngestionServiceTest { + protected KafkaConsumerService.ConsumerAssignmentStrategy getConsumerAssignmentStrategy() { + return KafkaConsumerService.ConsumerAssignmentStrategy.STORE_AWARE_PARTITION_WISE_SHARED_CONSUMER_ASSIGNMENT_STRATEGY; + } +} diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/SharedKafkaConsumerTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/SharedKafkaConsumerTest.java index 9c597fe7a9..fd0fd52d9e 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/SharedKafkaConsumerTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/SharedKafkaConsumerTest.java @@ -48,7 +48,7 @@ public void setUp() { public void testSubscriptionEmptyPoll() { PubSubTopic nonExistingTopic1 = pubSubTopicRepository.getTopic("nonExistingTopic1_v3"); - SharedKafkaConsumer sharedConsumer = new SharedKafkaConsumer(consumer, stats, () -> {}, (c, tp) -> {}); + SharedKafkaConsumer sharedConsumer = new SharedKafkaConsumer(consumer, stats, () -> {}, (c, vt, tp) -> {}); Set assignmentReturnedConsumer = new HashSet<>(); PubSubTopicPartition nonExistentPubSubTopicPartition = new PubSubTopicPartitionImpl(nonExistingTopic1, 1); From 03c83027c3a3f0b7e99b287c19c7e8d01871f89d Mon Sep 17 00:00:00 2001 From: "Xin(Adam) Chen" Date: Fri, 1 Nov 2024 17:31:33 -0700 Subject: [PATCH 2/7] [dvc][server] Support separate inc push topic with different pubsub entries (#1262) * [dvc][server] Support seperate inc push topic with different pubsub entries --- .../davinci/config/VeniceClusterConfig.java | 38 ++++++++++- .../consumer/AggKafkaConsumerService.java | 5 +- .../LeaderFollowerStoreIngestionTask.java | 18 +++-- .../kafka/consumer/StoreIngestionTask.java | 13 +++- .../consumer/StorePartitionDataReceiver.java | 5 ++ .../config/VeniceClusterConfigTest.java | 56 +++++++++++++++ .../consumer/AggKafkaConsumerServiceTest.java | 68 +++++++++++++++---- .../consumer/StoreIngestionTaskTest.java | 26 +++++++ .../java/com/linkedin/venice/utils/Utils.java | 15 +++- .../com/linkedin/venice/utils/UtilsTest.java | 9 +++ 10 files changed, 227 insertions(+), 26 deletions(-) create mode 100644 clients/da-vinci-client/src/test/java/com/linkedin/davinci/config/VeniceClusterConfigTest.java diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceClusterConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceClusterConfig.java index cd1ab58ff7..097936c17b 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceClusterConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceClusterConfig.java @@ -29,6 +29,7 @@ import com.linkedin.venice.meta.PersistenceType; import com.linkedin.venice.utils.KafkaSSLUtils; import com.linkedin.venice.utils.RegionUtils; +import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.VeniceProperties; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.Int2ObjectMaps; @@ -149,6 +150,21 @@ public VeniceClusterConfig(VeniceProperties clusterProps, Map tmpKafkaUrlResolution = new HashMap<>(); boolean foundBaseKafkaUrlInMappingIfItIsPopulated = kafkaClusterMap.isEmpty(); + /** + * The cluster ID, alias and kafka URL mappings are defined in the service config file + * so in order to support multiple cluster id mappings we pass them as separated entries + * for example, we can build a new cluster id with its alias and url + * + * + * + * + * + * + * + * For the separate incremental push topic feature, we duplicate entries with "_sep" suffix and different cluster id + * to support two RT topics (regular rt and incremental rt) with different cluster id. + */ + for (Map.Entry> kafkaCluster: kafkaClusterMap.entrySet()) { int clusterId = Integer.parseInt(kafkaCluster.getKey()); Map mappings = kafkaCluster.getValue(); @@ -207,11 +223,11 @@ public VeniceClusterConfig(VeniceProperties clusterProps, Map kafkaUrlResolution.getOrDefault(url, url); + ? Utils::resolveKafkaUrlForSepTopic + : url -> Utils.resolveKafkaUrlForSepTopic(kafkaUrlResolution.getOrDefault(url, url)); this.kafkaBootstrapServers = this.kafkaClusterUrlResolver.apply(baseKafkaBootstrapServers); if (this.kafkaBootstrapServers == null || this.kafkaBootstrapServers.isEmpty()) { throw new ConfigurationException("kafkaBootstrapServers can't be empty"); @@ -364,4 +380,20 @@ public VeniceProperties getClusterProperties() { public Map> getKafkaClusterMap() { return kafkaClusterMap; } + + /** + * For the separate incremental push topic feature, we need to resolve the cluster id to the original one for monitoring + * purposes as the incremental push topic essentially uses the same pubsub clusters as the regular push topic, though + * it appears to have a different cluster id + * @param clusterId + * @return + */ + public int getEquivalentKafkaClusterIdForSepTopic(int clusterId) { + String alias = kafkaClusterIdToAliasMap.get(clusterId); + if (alias == null || !alias.endsWith(Utils.SEPARATE_TOPIC_SUFFIX)) { + return clusterId; + } + String originalAlias = alias.substring(0, alias.length() - Utils.SEPARATE_TOPIC_SUFFIX.length()); + return kafkaClusterAliasToIdMap.getInt(originalAlias); + } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java index 9fbdbf87c2..628b93b510 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java @@ -264,6 +264,7 @@ protected static Runnable getStuckConsumerDetectionAndRepairRunnable( AbstractKafkaConsumerService getKafkaConsumerService(final String kafkaURL) { AbstractKafkaConsumerService consumerService = kafkaServerToConsumerServiceMap.get(kafkaURL); if (consumerService == null && kafkaClusterUrlResolver != null) { + // The resolver is needed to resolve a special format of kafka URL to the original kafka URL consumerService = kafkaServerToConsumerServiceMap.get(kafkaClusterUrlResolver.apply(kafkaURL)); } return consumerService; @@ -391,8 +392,8 @@ public ConsumedDataReceiver returnSet = new ArrayList<>(originalTopicSwitch.sourceKafkaServers.size()); for (CharSequence url: originalTopicSwitch.sourceKafkaServers) { + // For separate incremental topic URL, the original URL is not a valid URL, so need to resolve it. + // There's no issue for TS, as we do topic switch for both real-time and separate incremental topic. returnSet.add(kafkaClusterUrlResolver.apply(url.toString())); } originalTopicSwitch.sourceKafkaServers = returnSet; @@ -3423,12 +3425,16 @@ public void consumerBatchUnsubscribe(Set topicPartitionSet public abstract void consumerUnSubscribeAllTopics(PartitionConsumptionState partitionConsumptionState); public void consumerSubscribe(PubSubTopicPartition pubSubTopicPartition, long startOffset, String kafkaURL) { - final boolean consumeRemotely = !Objects.equals(kafkaURL, localKafkaServer); + // TODO: pass special format of kafka url as input here when subscribe to the separate incremental push topic + String resolvedKafkaURL = kafkaClusterUrlResolver != null ? kafkaClusterUrlResolver.apply(kafkaURL) : kafkaURL; + final boolean consumeRemotely = !Objects.equals(resolvedKafkaURL, localKafkaServer); // TODO: Move remote KafkaConsumerService creating operations into the aggKafkaConsumerService. aggKafkaConsumerService - .createKafkaConsumerService(createKafkaConsumerProperties(kafkaProps, kafkaURL, consumeRemotely)); + .createKafkaConsumerService(createKafkaConsumerProperties(kafkaProps, resolvedKafkaURL, consumeRemotely)); PartitionReplicaIngestionContext partitionReplicaIngestionContext = new PartitionReplicaIngestionContext(versionTopic, pubSubTopicPartition, versionRole, workloadType); + // localKafkaServer doesn't have suffix but kafkaURL may have suffix, + // and we don't want to pass the resolvedKafkaURL as it will be passed to data receiver for parsing cluster id aggKafkaConsumerService.subscribeConsumerFor(kafkaURL, this, partitionReplicaIngestionContext, startOffset); } @@ -4126,6 +4132,9 @@ public void updateOffsetMetadataAndSync(String topic, int partitionId) { * @return topic manager */ protected TopicManager getTopicManager(String sourceKafkaServer) { + if (kafkaClusterUrlResolver != null) { + sourceKafkaServer = kafkaClusterUrlResolver.apply(sourceKafkaServer); + } if (sourceKafkaServer.equals(localKafkaServer)) { // Use default kafka admin client (could be scala or java based) to get local topic manager return topicManagerRepository.getLocalTopicManager(); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java index 8e7a4a8e99..e5a59294b2 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StorePartitionDataReceiver.java @@ -135,4 +135,9 @@ public String toString() { return this.getClass().getSimpleName() + "{" + "VT=" + storeIngestionTask.getVersionTopic() + ", topicPartition=" + topicPartition + '}'; } + + // for testing purpose only + int getKafkaClusterId() { + return this.kafkaClusterId; + } } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/config/VeniceClusterConfigTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/config/VeniceClusterConfigTest.java new file mode 100644 index 0000000000..fa35bb5cd4 --- /dev/null +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/config/VeniceClusterConfigTest.java @@ -0,0 +1,56 @@ +package com.linkedin.davinci.config; + +import static com.linkedin.venice.ConfigKeys.CLUSTER_NAME; +import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; +import static com.linkedin.venice.ConfigKeys.ZOOKEEPER_ADDRESS; + +import com.linkedin.venice.utils.VeniceProperties; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +public class VeniceClusterConfigTest { + VeniceClusterConfig config; + static Map> KAFKA_CLUSTER_MAP; + static { + KAFKA_CLUSTER_MAP = new HashMap<>(); + for (int i = 0; i < 3; i++) { + Map entry = new HashMap<>(); + entry.put("name", "region_" + i); + entry.put("url", "localhost:" + i); + KAFKA_CLUSTER_MAP.put(String.valueOf(i), entry); + } + + // additional mapping for sep topics + for (int i = 3; i < 6; i++) { + int clusterId = i % 3; + Map entry = new HashMap<>(); + entry.put("name", "region_" + clusterId + "_sep"); + entry.put("url", "localhost:" + clusterId + "_sep"); + KAFKA_CLUSTER_MAP.put(String.valueOf(i), entry); + } + } + + @BeforeMethod + public void setUp() { + Properties props = new Properties(); + props.setProperty(CLUSTER_NAME, "test_cluster"); + props.setProperty(ZOOKEEPER_ADDRESS, "fake_zk_addr"); + props.setProperty(KAFKA_BOOTSTRAP_SERVERS, "fake_kafka_addr"); + config = new VeniceClusterConfig(new VeniceProperties(props), KAFKA_CLUSTER_MAP); + } + + @Test + public void testGetEquivalentKafkaClusterIdForSepTopic() { + Assert.assertEquals(config.getEquivalentKafkaClusterIdForSepTopic(0), 0); + Assert.assertEquals(config.getEquivalentKafkaClusterIdForSepTopic(1), 1); + Assert.assertEquals(config.getEquivalentKafkaClusterIdForSepTopic(2), 2); + Assert.assertEquals(config.getEquivalentKafkaClusterIdForSepTopic(3), 0); + Assert.assertEquals(config.getEquivalentKafkaClusterIdForSepTopic(4), 1); + Assert.assertEquals(config.getEquivalentKafkaClusterIdForSepTopic(5), 2); + } +} diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerServiceTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerServiceTest.java index 40ddd3bd17..1a3193238c 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerServiceTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerServiceTest.java @@ -1,6 +1,11 @@ package com.linkedin.davinci.kafka.consumer; +import static com.linkedin.davinci.kafka.consumer.KafkaConsumerService.ConsumerAssignmentStrategy.TOPIC_WISE_SHARED_CONSUMER_ASSIGNMENT_STRATEGY; +import static com.linkedin.davinci.kafka.consumer.KafkaConsumerServiceDelegator.ConsumerPoolStrategyType.DEFAULT; +import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -16,17 +21,24 @@ import com.linkedin.venice.pubsub.PubSubConsumerAdapterFactory; import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.pubsub.manager.TopicManagerContext.PubSubPropertiesSupplier; import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.VeniceProperties; import io.tehuti.metrics.MetricsRepository; +import io.tehuti.metrics.Sensor; +import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2IntMaps; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import java.util.HashMap; import java.util.Map; +import java.util.Properties; import java.util.function.Consumer; +import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -44,7 +56,8 @@ public class AggKafkaConsumerServiceTest { private ReadOnlyStoreRepository metadataRepository; private PubSubTopicRepository topicRepository; private AggKafkaConsumerService aggKafkaConsumerService; - private String pubSubUrl = "pubsub.venice.db"; + private String PUBSUB_URL = "pubsub.venice.db"; + private String PUBSUB_URL_SEP = "pubsub.venice.db_sep"; private PubSubTopic topic; private PubSubTopicPartition topicPartition; @@ -63,8 +76,20 @@ public void setUp() { killIngestionTaskRunnable = mock(Consumer.class); metadataRepository = mock(ReadOnlyStoreRepository.class); serverConfig = mock(VeniceServerConfig.class); - when(serverConfig.getKafkaClusterUrlToIdMap()).thenReturn(Object2IntMaps.EMPTY_MAP); - + Object2IntMap tmpKafkaClusterUrlToIdMap = new Object2IntOpenHashMap<>(); + tmpKafkaClusterUrlToIdMap.put(PUBSUB_URL, 0); + tmpKafkaClusterUrlToIdMap.put(PUBSUB_URL_SEP, 1); + when(serverConfig.getKafkaClusterUrlToIdMap()).thenReturn(Object2IntMaps.unmodifiable(tmpKafkaClusterUrlToIdMap)); + // note that this isn't the exact same as the one in VeniceClusterConfig, but it should be sufficient for most cases + when(serverConfig.getKafkaClusterUrlResolver()).thenReturn(Utils::resolveKafkaUrlForSepTopic); + when(serverConfig.getConsumerPoolStrategyType()).thenReturn(DEFAULT); + when(serverConfig.getConsumerPoolSizePerKafkaCluster()).thenReturn(5); + when(serverConfig.isUnregisterMetricForDeletedStoreEnabled()).thenReturn(Boolean.FALSE); + when(serverConfig.getSharedConsumerAssignmentStrategy()).thenReturn(TOPIC_WISE_SHARED_CONSUMER_ASSIGNMENT_STRATEGY); + Sensor dummySensor = mock(Sensor.class); + when(metricsRepository.sensor(anyString(), any())).thenReturn(dummySensor); + PubSubConsumerAdapter adapter = mock(PubSubConsumerAdapter.class); + when(consumerFactory.create(any(), anyBoolean(), any(), any())).thenReturn(adapter); aggKafkaConsumerService = new AggKafkaConsumerService( consumerFactory, pubSubPropertiesSupplier, @@ -82,47 +107,64 @@ public void setUp() { // test subscribeConsumerFor @Test public void testSubscribeConsumerFor() { + doReturn(new VeniceProperties(new Properties())).when(pubSubPropertiesSupplier).get(PUBSUB_URL); + AggKafkaConsumerService aggKafkaConsumerServiceSpy = spy(aggKafkaConsumerService); StoreIngestionTask storeIngestionTask = mock(StoreIngestionTask.class); TopicManager topicManager = mock(TopicManager.class); - doReturn(mock(AbstractKafkaConsumerService.class)).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(any()); + Properties props = new Properties(); + props.put(KAFKA_BOOTSTRAP_SERVERS, PUBSUB_URL); + aggKafkaConsumerService.createKafkaConsumerService(props); when(storeIngestionTask.getVersionTopic()).thenReturn(topic); - when(storeIngestionTask.getTopicManager(pubSubUrl)).thenReturn(topicManager); + when(storeIngestionTask.getTopicManager(PUBSUB_URL)).thenReturn(topicManager); PartitionReplicaIngestionContext partitionReplicaIngestionContext = new PartitionReplicaIngestionContext( topic, topicPartition, PartitionReplicaIngestionContext.VersionRole.CURRENT, PartitionReplicaIngestionContext.WorkloadType.NON_AA_OR_WRITE_COMPUTE); - aggKafkaConsumerServiceSpy - .subscribeConsumerFor(pubSubUrl, storeIngestionTask, partitionReplicaIngestionContext, -1); + StorePartitionDataReceiver dataReceiver = (StorePartitionDataReceiver) aggKafkaConsumerServiceSpy + .subscribeConsumerFor(PUBSUB_URL, storeIngestionTask, partitionReplicaIngestionContext, -1); + // regular pubsub url uses the default cluster id + Assert.assertEquals(dataReceiver.getKafkaClusterId(), 0); verify(topicManager).prefetchAndCacheLatestOffset(topicPartition); + + dataReceiver = (StorePartitionDataReceiver) aggKafkaConsumerServiceSpy + .subscribeConsumerFor(PUBSUB_URL_SEP, storeIngestionTask, partitionReplicaIngestionContext, -1); + // pubsub url for sep topic uses a different cluster id + Assert.assertEquals(dataReceiver.getKafkaClusterId(), 1); + + // Sep topic should share the same kafka consumer service + AbstractKafkaConsumerService kafkaConsumerService = aggKafkaConsumerService.getKafkaConsumerService(PUBSUB_URL); + AbstractKafkaConsumerService kafkaConsumerServiceForSep = + aggKafkaConsumerService.getKafkaConsumerService(PUBSUB_URL_SEP); + Assert.assertSame(kafkaConsumerService, kafkaConsumerServiceForSep); } @Test public void testGetOffsetLagBasedOnMetrics() { AggKafkaConsumerService aggKafkaConsumerServiceSpy = spy(aggKafkaConsumerService); - doReturn(null).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(pubSubUrl); - assertEquals(aggKafkaConsumerServiceSpy.getOffsetLagBasedOnMetrics(pubSubUrl, topic, topicPartition), -1); + doReturn(null).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(PUBSUB_URL); + assertEquals(aggKafkaConsumerServiceSpy.getOffsetLagBasedOnMetrics(PUBSUB_URL, topic, topicPartition), -1); AbstractKafkaConsumerService consumerService = mock(AbstractKafkaConsumerService.class); when(consumerService.getOffsetLagBasedOnMetrics(topic, topicPartition)).thenReturn(123L); doReturn(consumerService).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(any()); - assertEquals(aggKafkaConsumerServiceSpy.getOffsetLagBasedOnMetrics(pubSubUrl, topic, topicPartition), 123L); + assertEquals(aggKafkaConsumerServiceSpy.getOffsetLagBasedOnMetrics(PUBSUB_URL, topic, topicPartition), 123L); } @Test public void testGetLatestOffsetBasedOnMetrics() { AggKafkaConsumerService aggKafkaConsumerServiceSpy = spy(aggKafkaConsumerService); - doReturn(null).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(pubSubUrl); - assertEquals(aggKafkaConsumerServiceSpy.getLatestOffsetBasedOnMetrics(pubSubUrl, topic, topicPartition), -1); + doReturn(null).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(PUBSUB_URL); + assertEquals(aggKafkaConsumerServiceSpy.getLatestOffsetBasedOnMetrics(PUBSUB_URL, topic, topicPartition), -1); AbstractKafkaConsumerService consumerService = mock(AbstractKafkaConsumerService.class); when(consumerService.getLatestOffsetBasedOnMetrics(topic, topicPartition)).thenReturn(1234L); doReturn(consumerService).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(any()); - assertEquals(aggKafkaConsumerServiceSpy.getLatestOffsetBasedOnMetrics(pubSubUrl, topic, topicPartition), 1234L); + assertEquals(aggKafkaConsumerServiceSpy.getLatestOffsetBasedOnMetrics(PUBSUB_URL, topic, topicPartition), 1234L); } @Test 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 31e94689c5..4966daf8dc 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 @@ -4944,6 +4944,32 @@ public void testProcessConsumerActionsError() throws Exception { }, AA_OFF); } + @Test + public void testGetTopicManager() throws Exception { + String localKafkaBootstrapServer = inMemoryLocalKafkaBroker.getKafkaBootstrapServer(); + String remoteKafkaBootstrapServer = inMemoryRemoteKafkaBroker.getKafkaBootstrapServer(); + TopicManager mockTopicManagerRemoteKafka = mock(TopicManager.class); + doReturn(mockTopicManager).when(mockTopicManagerRepository) + .getTopicManager(inMemoryLocalKafkaBroker.getKafkaBootstrapServer()); + doReturn(mockTopicManagerRemoteKafka).when(mockTopicManagerRepository) + .getTopicManager(inMemoryRemoteKafkaBroker.getKafkaBootstrapServer()); + + runTest(Collections.singleton(PARTITION_FOO), () -> { + // local url returns the local manager + Assert.assertSame(mockTopicManager, storeIngestionTaskUnderTest.getTopicManager(localKafkaBootstrapServer)); + Assert.assertSame( + mockTopicManager, + storeIngestionTaskUnderTest.getTopicManager(localKafkaBootstrapServer + "_sep")); + // remote url returns the remote manager + Assert.assertSame( + mockTopicManagerRemoteKafka, + storeIngestionTaskUnderTest.getTopicManager(remoteKafkaBootstrapServer)); + Assert.assertSame( + mockTopicManagerRemoteKafka, + storeIngestionTaskUnderTest.getTopicManager(remoteKafkaBootstrapServer + "_sep")); + }, AA_OFF); + } + private VeniceStoreVersionConfig getDefaultMockVeniceStoreVersionConfig( Consumer storeVersionConfigOverride) { // mock the store config diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/utils/Utils.java b/internal/venice-common/src/main/java/com/linkedin/venice/utils/Utils.java index ef60f60528..b762eecc6c 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/utils/Utils.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/utils/Utils.java @@ -77,7 +77,7 @@ public class Utils { public static final String WILDCARD_MATCH_ANY = "*"; public static final String NEW_LINE_CHAR = System.lineSeparator(); public static final AtomicBoolean SUPPRESS_SYSTEM_EXIT = new AtomicBoolean(); - + public static final String SEPARATE_TOPIC_SUFFIX = "_sep"; public static final String FATAL_DATA_VALIDATION_ERROR = "fatal data validation problem"; /** @@ -923,4 +923,17 @@ public static String getReplicaId(PubSubTopic topic, int partition) { public static String escapeFilePathComponent(final String component) { return component.replaceAll("[^a-zA-Z0-9-_/\\.]", "_"); } + + /** + * Check whether the given kafka url has "_sep" or not. + * If it has, return the kafka url without "_sep". Otherwise, return the original kafka url. + * @param kafkaUrl + * @return + */ + public static String resolveKafkaUrlForSepTopic(String kafkaUrl) { + if (kafkaUrl != null && kafkaUrl.endsWith(SEPARATE_TOPIC_SUFFIX)) { + return kafkaUrl.substring(0, kafkaUrl.length() - SEPARATE_TOPIC_SUFFIX.length()); + } + return kafkaUrl; + } } diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/utils/UtilsTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/utils/UtilsTest.java index 54fecd1468..79f2fca4d5 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/utils/UtilsTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/utils/UtilsTest.java @@ -220,4 +220,13 @@ public void testParseCommaSeparatedStringToList() { Assert.assertEquals(list.get(1), "b"); Assert.assertEquals(list.get(2), "c"); } + + @Test + public void testResolveKafkaUrlForSepTopic() { + String originalKafkaUrl = "localhost:12345"; + String originalKafkaUrlForSep = "localhost:12345_sep"; + Assert.assertEquals(Utils.resolveKafkaUrlForSepTopic(""), ""); + Assert.assertEquals(Utils.resolveKafkaUrlForSepTopic(originalKafkaUrlForSep), originalKafkaUrl); + Assert.assertEquals(Utils.resolveKafkaUrlForSepTopic(originalKafkaUrl), originalKafkaUrl); + } } From 66bfa3635e790e6ea34623529b91a0ff88ba0909 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Sun, 3 Nov 2024 15:37:28 -0800 Subject: [PATCH 3/7] [dvc] Automatically delete DVRT's classHash in DaVinciClientTest (#1276) The classHash gets generated when DVRT is being used. This PR will automatically clean it up, so it doesn't get included when someone does git add . --- .../linkedin/venice/endToEnd/DaVinciClientTest.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java index e2e9a50e7b..6d413399ef 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java @@ -130,6 +130,7 @@ import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -173,6 +174,16 @@ public void cleanUp() { Utils.closeQuietlyWithErrorLogged(cluster); } + @BeforeMethod + @AfterClass + public void deleteClassHash() { + int storeVersion = 1; + File file = new File(String.format("./classHash-%d.txt", storeVersion)); + if (file.exists()) { + assertTrue(file.delete()); + } + } + @Test(timeOut = TEST_TIMEOUT) public void testConcurrentGetAndStart() throws Exception { String s1 = createStoreWithMetaSystemStoreAndPushStatusSystemStore(KEY_COUNT); From 9b6f027417d184597c6551153cd6e94254b3b71b Mon Sep 17 00:00:00 2001 From: Nisarg Thakkar Date: Mon, 4 Nov 2024 09:16:36 -0800 Subject: [PATCH 4/7] [controller] Make to_be_stopped_instances in aggregatedHealthStatus API optional (#1274) In Helix's aggregated stoppable check, the "to_be_stopped_instances" is optional and will be omitted in case there are no instances that have previously been approved. This commit makes our controller handle this. --- .../AggregatedHealthStatusRequest.java | 40 ++++++------- .../controllerapi/ControllerApiConstants.java | 2 - .../controllerapi/ControllerClient.java | 2 +- .../TestAggregatedHealthStatusRequest.java | 60 +++++++++++++++++++ .../server/TestAdminSparkServer.java | 27 +++++++-- .../controller/server/ControllerRoutes.java | 6 +- 6 files changed, 107 insertions(+), 30 deletions(-) create mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/TestAggregatedHealthStatusRequest.java diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/AggregatedHealthStatusRequest.java b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/AggregatedHealthStatusRequest.java index 2e82f8d596..0eff306ead 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/AggregatedHealthStatusRequest.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/AggregatedHealthStatusRequest.java @@ -2,27 +2,35 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Collections; import java.util.List; public class AggregatedHealthStatusRequest { - List instances; - List to_be_stopped_instances; - String cluster_id; + private final String cluster_id; + private final List instances; + private final List to_be_stopped_instances; @JsonCreator public AggregatedHealthStatusRequest( + @JsonProperty("cluster_id") String cluster_id, @JsonProperty("instances") List instances, - @JsonProperty("to_be_stopped_instances") List to_be_stopped_instances, - @JsonProperty("cluster_id") String cluster_id) { - this.instances = instances; - this.to_be_stopped_instances = to_be_stopped_instances; + @JsonProperty("to_be_stopped_instances") List to_be_stopped_instances) { + if (cluster_id == null) { + throw new IllegalArgumentException("'cluster_id' is required"); + } this.cluster_id = cluster_id; - } - @JsonProperty("cluster_id") - public void setClusterId(String cluster_id) { - this.cluster_id = cluster_id; + if (instances == null) { + throw new IllegalArgumentException("'instances' is required"); + } + this.instances = instances; + + if (to_be_stopped_instances == null) { + this.to_be_stopped_instances = Collections.emptyList(); + } else { + this.to_be_stopped_instances = to_be_stopped_instances; + } } @JsonProperty("cluster_id") @@ -35,18 +43,8 @@ public List getInstances() { return instances; } - @JsonProperty("instances") - public void setInstances(List instances) { - this.instances = instances; - } - @JsonProperty("to_be_stopped_instances") public List getToBeStoppedInstances() { return to_be_stopped_instances; } - - @JsonProperty("to_be_stopped_instances") - public void setToBeStoppedInstances(List to_be_stopped_instances) { - this.to_be_stopped_instances = to_be_stopped_instances; - } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerApiConstants.java b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerApiConstants.java index 09953dbe50..bc1bf148c9 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerApiConstants.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerApiConstants.java @@ -244,6 +244,4 @@ public class ControllerApiConstants { public static final String NEARLINE_PRODUCER_COMPRESSION_ENABLED = "nearline_producer_compression_enabled"; public static final String NEARLINE_PRODUCER_COUNT_PER_WRITER = "nearline_producer_count_per_writer"; - - public static final String AGGR_HEALTH_STATUS_URI = "/aggregatedHealthStatus"; } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerClient.java b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerClient.java index e122e8b372..fcd695c69d 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerClient.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerClient.java @@ -923,7 +923,7 @@ public StoppableNodeStatusResponse getAggregatedHealthStatus( List toBeStoppedInstances) throws JsonProcessingException { AggregatedHealthStatusRequest request = - new AggregatedHealthStatusRequest(instances, toBeStoppedInstances, clusterName); + new AggregatedHealthStatusRequest(clusterName, instances, toBeStoppedInstances); String requestString = OBJECT_MAPPER.writeValueAsString(request); return request( ControllerRoute.AGGREGATED_HEALTH_STATUS, diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/TestAggregatedHealthStatusRequest.java b/internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/TestAggregatedHealthStatusRequest.java new file mode 100644 index 0000000000..4620498b1e --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/TestAggregatedHealthStatusRequest.java @@ -0,0 +1,60 @@ +package com.linkedin.venice.controllerapi; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.exc.ValueInstantiationException; +import com.linkedin.venice.utils.ObjectMapperFactory; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class TestAggregatedHealthStatusRequest { + private static final ObjectMapper OBJECT_MAPPER = ObjectMapperFactory.getInstance(); + + @Test + public void testDeserializationWithAllFields() throws JsonProcessingException { + String json = + "{\"cluster_id\":\"cluster1\",\"instances\":[\"instance1\",\"instance2\"],\"to_be_stopped_instances\":[\"instance3\",\"instance4\"]}"; + AggregatedHealthStatusRequest request = OBJECT_MAPPER.readValue(json, AggregatedHealthStatusRequest.class); + + assertEquals(request.getClusterId(), "cluster1"); + assertEquals(request.getInstances().size(), 2); + assertEquals(request.getInstances().get(0), "instance1"); + assertEquals(request.getInstances().get(1), "instance2"); + assertEquals(request.getToBeStoppedInstances().size(), 2); + assertEquals(request.getToBeStoppedInstances().get(0), "instance3"); + assertEquals(request.getToBeStoppedInstances().get(1), "instance4"); + } + + @Test + public void testDeserializationWithMandatoryFields() throws JsonProcessingException { + String json = "{\"cluster_id\":\"cluster1\",\"instances\":[\"instance1\",\"instance2\"]}"; + AggregatedHealthStatusRequest request = OBJECT_MAPPER.readValue(json, AggregatedHealthStatusRequest.class); + + assertEquals(request.getClusterId(), "cluster1"); + assertEquals(request.getInstances().size(), 2); + assertEquals(request.getInstances().get(0), "instance1"); + assertEquals(request.getInstances().get(1), "instance2"); + assertNotNull(request.getToBeStoppedInstances()); + assertTrue(request.getToBeStoppedInstances().isEmpty()); + } + + @Test + public void testDeserializationWithMissingMandatoryFields() { + String json = "{\"instances\":[\"instance1\",\"instance2\"]}"; + ValueInstantiationException e = Assert.expectThrows( + ValueInstantiationException.class, + () -> OBJECT_MAPPER.readValue(json, AggregatedHealthStatusRequest.class)); + assertTrue(e.getMessage().contains("'cluster_id' is required"), e.getMessage()); + + String json2 = "{\"cluster_id\":\"cluster1\"}"; + ValueInstantiationException e2 = Assert.expectThrows( + ValueInstantiationException.class, + () -> OBJECT_MAPPER.readValue(json2, AggregatedHealthStatusRequest.class)); + assertTrue(e2.getMessage().contains("'instances' is required"), e2.getMessage()); + } +} diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkServer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkServer.java index a3779ebeb3..82fd3b9605 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkServer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkServer.java @@ -73,6 +73,7 @@ import org.apache.http.entity.StringEntity; import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; import org.apache.http.message.BasicNameValuePair; +import org.apache.http.nio.client.HttpAsyncClient; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -218,10 +219,30 @@ public void testAggregatedHealthStatusCall() throws IOException, ExecutionExcept httpClient.start(); int serverPort = venice.getChildRegions().get(0).getClusters().get(clusterName).getVeniceServers().get(0).getPort(); String server = Utils.getHelixNodeIdentifier(Utils.getHostName(), serverPort); + + // API call with all fields Map payloads = new HashMap<>(); payloads.put("cluster_id", clusterName); - payloads.put("instances", Arrays.asList(server)); + payloads.put("instances", Collections.singletonList(server)); payloads.put("to_be_stopped_instances", Collections.emptyList()); + + InstanceRemovableStatuses statuses = makeAggregatedHealthStatusCall(httpClient, payloads); + Assert.assertTrue(statuses.getNonStoppableInstancesWithReasons().containsKey(server)); + + // API call without optional to_be_stopped_instances + Map payloads2 = new HashMap<>(); + payloads2.put("cluster_id", clusterName); + payloads2.put("instances", Collections.singletonList(server)); + + InstanceRemovableStatuses statuses2 = makeAggregatedHealthStatusCall(httpClient, payloads2); + Assert.assertTrue(statuses2.getNonStoppableInstancesWithReasons().containsKey(server)); + + httpClient.close(); + } + + private InstanceRemovableStatuses makeAggregatedHealthStatusCall( + HttpAsyncClient httpClient, + Map payloads) throws IOException, ExecutionException, InterruptedException { StringEntity entity = new StringEntity(OBJECT_MAPPER.writeValueAsString(payloads), ContentType.APPLICATION_JSON); final HttpPost post = new HttpPost( @@ -232,9 +253,7 @@ public void testAggregatedHealthStatusCall() throws IOException, ExecutionExcept Assert.assertEquals(httpResponse.getStatusLine().getStatusCode(), 200); String responseString = IOUtils.toString(httpResponse.getEntity().getContent()); - InstanceRemovableStatuses statuses = OBJECT_MAPPER.readValue(responseString, InstanceRemovableStatuses.class); - Assert.assertTrue(statuses.getNonStoppableInstancesWithReasons().containsKey(server)); - httpClient.close(); + return OBJECT_MAPPER.readValue(responseString, InstanceRemovableStatuses.class); } @Test(timeOut = TEST_TIMEOUT) diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/ControllerRoutes.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/ControllerRoutes.java index c2580b5b11..629da787c3 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/ControllerRoutes.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/ControllerRoutes.java @@ -1,11 +1,11 @@ package com.linkedin.venice.controller.server; -import static com.linkedin.venice.controllerapi.ControllerApiConstants.AGGR_HEALTH_STATUS_URI; import static com.linkedin.venice.controllerapi.ControllerApiConstants.CLUSTER; import static com.linkedin.venice.controllerapi.ControllerApiConstants.KAFKA_TOPIC_LOG_COMPACTION_ENABLED; import static com.linkedin.venice.controllerapi.ControllerApiConstants.KAFKA_TOPIC_MIN_IN_SYNC_REPLICA; import static com.linkedin.venice.controllerapi.ControllerApiConstants.KAFKA_TOPIC_RETENTION_IN_MS; import static com.linkedin.venice.controllerapi.ControllerApiConstants.TOPIC; +import static com.linkedin.venice.controllerapi.ControllerRoute.AGGREGATED_HEALTH_STATUS; import static com.linkedin.venice.controllerapi.ControllerRoute.LEADER_CONTROLLER; import static com.linkedin.venice.controllerapi.ControllerRoute.LIST_CHILD_CLUSTERS; import static com.linkedin.venice.controllerapi.ControllerRoute.UPDATE_KAFKA_TOPIC_LOG_COMPACTION; @@ -211,7 +211,9 @@ public Route getAggregatedHealthStatus(Admin admin) { InstanceRemovableStatuses statuses = admin.getAggregatedHealthStatus(cluster, instanceList, toBeStoppedInstanceList, isSslEnabled()); if (statuses.getRedirectUrl() != null) { - response.redirect(statuses.getRedirectUrl() + AGGR_HEALTH_STATUS_URI, HttpStatus.SC_MOVED_TEMPORARILY); + response.redirect( + statuses.getRedirectUrl() + AGGREGATED_HEALTH_STATUS.getPath(), + HttpStatus.SC_MOVED_TEMPORARILY); return null; } else { responseObject.setNonStoppableInstancesWithReason(statuses.getNonStoppableInstancesWithReasons()); From 3f2a130ee96bba565125ed3f54425e8eae65371c Mon Sep 17 00:00:00 2001 From: Gaojie Liu Date: Mon, 4 Nov 2024 09:39:18 -0800 Subject: [PATCH 5/7] [common][samza-producer] Add a new option in stream job to leverage multiple producers (#1267) * [common][samza-producer] Add a new option in stream job to leverage multiple producers In a recent experiment, we noticed two issues: 1. KafkaProducer compression is quite time consuming for large records. 2. KafkaProducer is not scalable when using it in a multiple-thread env. Streaming job typically runs in a container to process the upstreaming topic partition, and they need to process these events sequentially even these events belong to different Venice partitions, and the single-threaded KafkaProducer is limiting the total throughput and cpu resources are mostly under utilized. To get round of this issue, in this PR, we add a capability to use multiple producers even in a single-threaded streaming application. And we don't want to disable compression as we would like to reduce pubsub usage and cross-colo replication bandwidth. Two new options in VeniceWriter: 1. venice.writer.producer.thread.count : default 1 2. venice.writer.producer.queue.size: default 5MB Stream job can configure it with a higher number of producer thread count to reduce the latency of `Producer#send` invocation, which is a blocking call, so that the total throughput will be improved. --- .../davinci/config/VeniceServerConfig.java | 5 +- .../kafka/consumer/StoreBufferService.java | 34 +- .../venice/samza/VeniceSystemProducer.java | 38 ++- .../samza/VeniceSystemProducerTest.java | 32 ++ .../com/linkedin/venice/message/KafkaKey.java | 10 +- .../pubsub/api/PubSubMessageHeader.java | 11 +- .../pubsub/api/PubSubMessageHeaders.java | 15 +- ...SubProducerAdapterConcurrentDelegator.java | 320 ++++++++++++++++++ .../linkedin/venice/utils/ProtocolUtils.java | 36 ++ .../MemoryBoundBlockingQueue.java | 9 +- .../linkedin/venice/writer/VeniceWriter.java | 7 + .../venice/writer/VeniceWriterFactory.java | 24 +- .../venice/writer/VeniceWriterOptions.java | 30 ++ ...roducerAdapterConcurrentDelegatorTest.java | 211 ++++++++++++ .../MemoryBoundBlockingQueueTest.java | 10 +- .../writer/VeniceWriterFactoryTest.java | 28 ++ .../writer/VeniceWriterOptionsTest.java | 4 + .../endToEnd/TestActiveActiveIngestion.java | 2 + 18 files changed, 767 insertions(+), 59 deletions(-) create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubProducerAdapterConcurrentDelegator.java create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/utils/ProtocolUtils.java rename {clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer => internal/venice-common/src/main/java/com/linkedin/venice/utils/collections}/MemoryBoundBlockingQueue.java (97%) create mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/pubsub/api/PubSubProducerAdapterConcurrentDelegatorTest.java rename {clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer => internal/venice-common/src/test/java/com/linkedin/venice/utils/collections}/MemoryBoundBlockingQueueTest.java (93%) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceServerConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceServerConfig.java index 12003eff7c..2a104e93bf 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceServerConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceServerConfig.java @@ -185,6 +185,7 @@ import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.VeniceProperties; +import com.linkedin.venice.utils.collections.MemoryBoundBlockingQueue; import com.linkedin.venice.utils.concurrent.BlockingQueueType; import io.netty.channel.WriteBufferWaterMark; import java.io.File; @@ -269,14 +270,14 @@ public class VeniceServerConfig extends VeniceClusterConfig { /** * Considering the consumer thread could put various sizes of messages into the shared queue, the internal - * {@link com.linkedin.davinci.kafka.consumer.MemoryBoundBlockingQueue} won't notify the waiting thread (consumer thread) + * {@link MemoryBoundBlockingQueue} won't notify the waiting thread (consumer thread) * right away when some message gets processed until the freed memory hit the follow config: {@link #storeWriterBufferNotifyDelta}. * The reason behind this design: * When the buffered queue is full, and the processing thread keeps processing small message, the bigger message won't * have chance to get queued into the buffer since the memory freed by the processed small message is not enough to * fit the bigger message. * - * With this delta config, {@link com.linkedin.davinci.kafka.consumer.MemoryBoundBlockingQueue} will guarantee some fairness + * With this delta config, {@link MemoryBoundBlockingQueue} will guarantee some fairness * among various sizes of messages when buffered queue is full. * * When tuning this config, we need to consider the following tradeoffs: diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreBufferService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreBufferService.java index 947f293e40..5e26b3a235 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreBufferService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreBufferService.java @@ -1,5 +1,6 @@ package com.linkedin.davinci.kafka.consumer; +import static com.linkedin.venice.utils.ProtocolUtils.getEstimateOfMessageEnvelopeSizeOnHeap; import static java.util.Collections.reverseOrder; import static java.util.Comparator.comparing; import static java.util.stream.Collectors.toList; @@ -10,13 +11,11 @@ import com.linkedin.venice.exceptions.VeniceChecksumException; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; -import com.linkedin.venice.kafka.protocol.Put; -import com.linkedin.venice.kafka.protocol.Update; -import com.linkedin.venice.kafka.protocol.enums.MessageType; import com.linkedin.venice.message.KafkaKey; import com.linkedin.venice.pubsub.api.PubSubMessage; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.utils.DaemonThreadFactory; +import com.linkedin.venice.utils.collections.MemoryBoundBlockingQueue; import io.tehuti.metrics.MetricsRepository; import java.util.ArrayList; import java.util.List; @@ -450,40 +449,15 @@ public int hashCode() { @Override public int getSize() { - // For FakePubSubMessage, the key and the value are null, return 0. + // For FakePubSubMessage, the key and the value are null. if (consumerRecord instanceof FakePubSubMessage) { - return 0; + return QUEUE_NODE_OVERHEAD_IN_BYTE; } - // N.B.: This is just an estimate. TODO: Consider if it is really useful, and whether to get rid of it. return this.consumerRecord.getKey().getEstimatedObjectSizeOnHeap() + getEstimateOfMessageEnvelopeSizeOnHeap(this.consumerRecord.getValue()) + QUEUE_NODE_OVERHEAD_IN_BYTE; } - private int getEstimateOfMessageEnvelopeSizeOnHeap(KafkaMessageEnvelope messageEnvelope) { - int kmeBaseOverhead = 100; // Super rough estimate. TODO: Measure with a more precise library and store statically - switch (MessageType.valueOf(messageEnvelope)) { - case PUT: - Put put = (Put) messageEnvelope.payloadUnion; - int size = put.putValue.capacity(); - if (put.replicationMetadataPayload != null - /** - * N.B.: When using the {@link org.apache.avro.io.OptimizedBinaryDecoder}, the {@link put.putValue} and the - * {@link put.replicationMetadataPayload} will be backed by the same underlying array. If that is the - * case, then we don't want to account for the capacity twice. - */ - && put.replicationMetadataPayload.array() != put.putValue.array()) { - size += put.replicationMetadataPayload.capacity(); - } - return size + kmeBaseOverhead; - case UPDATE: - Update update = (Update) messageEnvelope.payloadUnion; - return update.updateValue.capacity() + kmeBaseOverhead; - default: - return kmeBaseOverhead; - } - } - @Override public String toString() { return this.consumerRecord.toString(); diff --git a/clients/venice-samza/src/main/java/com/linkedin/venice/samza/VeniceSystemProducer.java b/clients/venice-samza/src/main/java/com/linkedin/venice/samza/VeniceSystemProducer.java index e0e835a212..2d8d89a85b 100644 --- a/clients/venice-samza/src/main/java/com/linkedin/venice/samza/VeniceSystemProducer.java +++ b/clients/venice-samza/src/main/java/com/linkedin/venice/samza/VeniceSystemProducer.java @@ -1,6 +1,7 @@ package com.linkedin.venice.samza; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; +import static com.linkedin.venice.pubsub.adapter.kafka.producer.ApacheKafkaProducerConfig.KAFKA_BUFFER_MEMORY; import static com.linkedin.venice.schema.AvroSchemaParseUtils.parseSchemaFromJSONLooseValidation; import static com.linkedin.venice.schema.AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation; @@ -371,10 +372,34 @@ public String getTopicName() { */ protected VeniceWriter getVeniceWriter(VersionCreationResponse store) { Properties veniceWriterProperties = new Properties(); + veniceWriterProperties.putAll(additionalWriterConfigs); veniceWriterProperties.put(KAFKA_BOOTSTRAP_SERVERS, store.getKafkaBootstrapServers()); return getVeniceWriter(store, veniceWriterProperties); } + protected static void extractConcurrentProducerConfig( + Properties veniceWriterProperties, + VeniceWriterOptions.Builder builder) { + String producerThreadCountProp = veniceWriterProperties.getProperty(VeniceWriter.PRODUCER_THREAD_COUNT); + if (producerThreadCountProp != null) { + int producerThreadCount = Integer.parseInt(producerThreadCountProp); + builder.setProducerThreadCount(producerThreadCount); + if (producerThreadCount > 1) { + /** + * Try to limit the producer buffer if the following config is not specified to reduce the total Kafka producer memory usage. + */ + String kafkaBufferConfig = veniceWriterProperties.getProperty(KAFKA_BUFFER_MEMORY); + if (kafkaBufferConfig == null) { + veniceWriterProperties.put(KAFKA_BUFFER_MEMORY, "8388608"); // 8MB + } + } + } + String producerQueueSizeProp = veniceWriterProperties.getProperty(VeniceWriter.PRODUCER_QUEUE_SIZE); + if (producerQueueSizeProp != null) { + builder.setProducerQueueSize(Integer.parseInt(producerQueueSizeProp)); + } + } + /** * Please don't remove this method since it is still being used by LinkedIn internally. */ @@ -394,13 +419,12 @@ protected VeniceWriter getVeniceWriter( partitionerProperties.putAll(store.getPartitionerParams()); VenicePartitioner venicePartitioner = PartitionUtils.getVenicePartitioner(store.getPartitionerClass(), new VeniceProperties(partitionerProperties)); - return constructVeniceWriter( - veniceWriterProperties, - new VeniceWriterOptions.Builder(store.getKafkaTopic()).setTime(time) - .setPartitioner(venicePartitioner) - .setPartitionCount(partitionCount) - .setChunkingEnabled(isChunkingEnabled) - .build()); + VeniceWriterOptions.Builder builder = new VeniceWriterOptions.Builder(store.getKafkaTopic()).setTime(time) + .setPartitioner(venicePartitioner) + .setPartitionCount(partitionCount) + .setChunkingEnabled(isChunkingEnabled); + extractConcurrentProducerConfig(veniceWriterProperties, builder); + return constructVeniceWriter(veniceWriterProperties, builder.build()); } // trickery for unit testing diff --git a/clients/venice-samza/src/test/java/com/linkedin/venice/samza/VeniceSystemProducerTest.java b/clients/venice-samza/src/test/java/com/linkedin/venice/samza/VeniceSystemProducerTest.java index 3496f54997..1af3cb0588 100644 --- a/clients/venice-samza/src/test/java/com/linkedin/venice/samza/VeniceSystemProducerTest.java +++ b/clients/venice-samza/src/test/java/com/linkedin/venice/samza/VeniceSystemProducerTest.java @@ -1,6 +1,7 @@ package com.linkedin.venice.samza; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; +import static com.linkedin.venice.pubsub.adapter.kafka.producer.ApacheKafkaProducerConfig.KAFKA_BUFFER_MEMORY; import static org.mockito.Mockito.*; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; @@ -269,4 +270,35 @@ public Version.PushType[] batchOrStreamReprocessing() { return new Version.PushType[] { Version.PushType.BATCH, Version.PushType.STREAM_REPROCESSING, Version.PushType.STREAM, Version.PushType.INCREMENTAL }; } + + @Test + public void testExtractConcurrentProducerConfig() { + Properties properties = new Properties(); + properties.put(VeniceWriter.PRODUCER_THREAD_COUNT, "2"); + properties.put(VeniceWriter.PRODUCER_QUEUE_SIZE, "102400000"); + + VeniceWriterOptions.Builder builder = new VeniceWriterOptions.Builder("test_rt"); + VeniceSystemProducer.extractConcurrentProducerConfig(properties, builder); + VeniceWriterOptions options = builder.build(); + assertEquals(options.getProducerThreadCount(), 2); + assertEquals(options.getProducerQueueSize(), 102400000); + assertEquals(properties.getProperty(KAFKA_BUFFER_MEMORY), "8388608"); + + /** + * if {@link KAFKA_BUFFER_MEMORY} is specified, {@link VeniceSystemProducer} shouldn't override. + */ + + properties = new Properties(); + properties.put(VeniceWriter.PRODUCER_THREAD_COUNT, "2"); + properties.put(VeniceWriter.PRODUCER_QUEUE_SIZE, "102400000"); + properties.put(KAFKA_BUFFER_MEMORY, "10240"); + + builder = new VeniceWriterOptions.Builder("test_rt"); + VeniceSystemProducer.extractConcurrentProducerConfig(properties, builder); + options = builder.build(); + assertEquals(options.getProducerThreadCount(), 2); + assertEquals(options.getProducerQueueSize(), 102400000); + assertEquals(properties.getProperty(KAFKA_BUFFER_MEMORY), "10240"); + + } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/message/KafkaKey.java b/internal/venice-common/src/main/java/com/linkedin/venice/message/KafkaKey.java index 9723e72a0b..163227f4d5 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/message/KafkaKey.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/message/KafkaKey.java @@ -73,14 +73,14 @@ public int getKeyLength() { return key == null ? 0 : key.length; } + public String toString() { + return getClass().getSimpleName() + "(" + (isControlMessage() ? "CONTROL_MESSAGE" : "PUT or DELETE") + ", " + + ByteUtils.toHexString(key) + ")"; + } + public int getEstimatedObjectSizeOnHeap() { // This constant is the estimated size of the enclosing object + the byte[]'s overhead. // TODO: Find a library that would allow us to precisely measure this and store it in a static constant. return getKeyLength() + 36; } - - public String toString() { - return getClass().getSimpleName() + "(" + (isControlMessage() ? "CONTROL_MESSAGE" : "PUT or DELETE") + ", " - + ByteUtils.toHexString(key) + ")"; - } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessageHeader.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessageHeader.java index 7bce130158..d4a604d617 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessageHeader.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessageHeader.java @@ -1,5 +1,6 @@ package com.linkedin.venice.pubsub.api; +import com.linkedin.venice.common.Measurable; import java.util.Arrays; import java.util.Objects; @@ -7,7 +8,7 @@ /** * A key-value pair that is associated with a message */ -public class PubSubMessageHeader { +public class PubSubMessageHeader implements Measurable { private final String key; private final byte[] value; @@ -41,4 +42,12 @@ public boolean equals(Object otherObj) { PubSubMessageHeader otherHeader = (PubSubMessageHeader) otherObj; return key.equals(otherHeader.key()) && Arrays.equals(value, otherHeader.value()); } + + /** + * TODO: the following estimation doesn't consider the overhead of the internal structure. + */ + @Override + public int getSize() { + return key.length() + value.length; + } } 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 e5f0283503..828f766948 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 @@ -1,5 +1,6 @@ package com.linkedin.venice.pubsub.api; +import com.linkedin.venice.common.Measurable; import java.util.ArrayList; import java.util.LinkedHashMap; import java.util.List; @@ -10,7 +11,7 @@ * Set of key-value pairs to tagged with messages produced to a topic. * In case of headers with the same key, only the most recently added headers value will be kept. */ -public class PubSubMessageHeaders { +public class PubSubMessageHeaders implements Measurable { /** * 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. @@ -44,4 +45,16 @@ public PubSubMessageHeaders remove(String key) { public List toList() { return new ArrayList<>(headers.values()); } + + /** + * TODO: the following estimation doesn't consider the overhead of the internal structure. + */ + @Override + public int getSize() { + int size = 0; + for (Map.Entry entry: headers.entrySet()) { + size += entry.getKey().length() + entry.getValue().getSize(); + } + return size; + } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubProducerAdapterConcurrentDelegator.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubProducerAdapterConcurrentDelegator.java new file mode 100644 index 0000000000..aa59261448 --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubProducerAdapterConcurrentDelegator.java @@ -0,0 +1,320 @@ +package com.linkedin.venice.pubsub.api; + +import com.linkedin.venice.common.Measurable; +import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; +import com.linkedin.venice.message.KafkaKey; +import com.linkedin.venice.utils.DaemonThreadFactory; +import com.linkedin.venice.utils.ProtocolUtils; +import com.linkedin.venice.utils.collections.MemoryBoundBlockingQueue; +import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; +import it.unimi.dsi.fastutil.objects.Object2DoubleMap; +import it.unimi.dsi.fastutil.objects.Object2DoubleMaps; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + + +/** + * This class will spin up multiple producer instances to speed up the producing. + * 1. Maintain a buffer queue per producer. + * 2. The writes to the same partition will be routed to the same queue all the time to guarantee ordering. + * 3. PubSubProducer in each thread will constantly be polling the corresponding queue and write to broker. + * + * Here are the main reasons we would like to leverage this new strategy in streaming processing job: + * 1. In the batch push job, each reducer only produces to one single Venice partition, so the multiple-thread-multiple-producer + * wouldn't help much. + * 2. In Venice-Server leader replica, it is natural to have multiple consumers (threads) producing to different + * Venice partitions as it is using partition-wise shared consumer assignment strategy, and multiple producers + * can help, and it will be covered by {@link PubSubProducerAdapterDelegator}. + * 3. Online writer is sharing a similar pattern as #2, but so far, we haven't heard any complains regarding online + * writer performance, so we will leave it out of scope for now. + * 4. Streaming job is typically running inside container and single-threaded to guarantee ordering, but the writes + * to Venice store can belong to several Venice partitions. We observed that KafkaProducer compression + * would introduce a lot of overheads for large records, which will slow down Producer#send and if Producer#send is slow, + * the streaming job throughput will be heavily affected. + * This strategy considers the following aspects: + * a. We still want to keep compression on producer end as the streaming app has enough cpu resources. + * b. We would like to utilize more cpu resources to improve the producing throughput. + * c. We would like to get round of the contention issue (KafkaProducer has several synchronized sections for sending + * messages to broker) when running in a multiple-threaded env. + * + * Here is how this strategy would fulfill #4. + * 1. Execute the Producer#send logic in a multi-thread env to better utilize cpu resources. + * 2. Each thread will have its own {@link PubSubProducerAdapter} to avoid contention. + * 3. The mapping between Venice partition and buffered producer is sticky, so the ordering is still guaranteed per + * Venice partition. + * + * + * The above analysis is correct in the high-level, but there are still some nuances, where this new strategy can perform + * better than the default one in VPJ reducer and Venice Server leader replicas: + * 1. If the record processing time is comparable with the producer#send latency, the new strategy will delegate the producing + * logic to a separate thread, which can potentially improve the overall throughput if there are enough cpu resources. + * 2. If the record processing time is minimal comparing with the producer#send latency, this new strategy won't help much. + * + * We will explore ^ after validating this new strategy in stream processing app. + * + */ +public class PubSubProducerAdapterConcurrentDelegator implements PubSubProducerAdapter { + private static final Logger LOGGER = LogManager.getLogger(PubSubProducerAdapterConcurrentDelegator.class); + private final String producingTopic; + private final ExecutorService producerExecutor; + private final List> blockingQueueList; + private final Map, ProducerQueueNode> lastNodePerQueue; + private final AtomicInteger selectedQueueId = new AtomicInteger(0); + private final Map> partitionQueueAssignment; + private final List producers; + private final List drainers; + + public PubSubProducerAdapterConcurrentDelegator( + String producingTopic, + int producerThreadCount, + int producerQueueSize, + Supplier producerAdapterSupplier) { + if (producerThreadCount <= 1) { + throw new VeniceException("'producerThreadCount' should be larger than 1, but got " + producerThreadCount); + } + if (producerQueueSize <= 2 * 1024 * 1024) { + throw new VeniceException("'producerQueueSize' should be larger than 2MB, but got " + producerQueueSize); + } + LOGGER.info( + "Initializing a 'PubSubProducerAdapterConcurrentDelegator' instance for topic: {} with thread" + + " count: {} and queue size: {}", + producingTopic, + producerThreadCount, + producerQueueSize); + List> tmpBlockingQueueList = new ArrayList<>(producerThreadCount); + this.producerExecutor = Executors + .newFixedThreadPool(producerThreadCount, new DaemonThreadFactory(producingTopic + "-concurrent-producer")); + this.partitionQueueAssignment = new VeniceConcurrentHashMap<>(); + this.lastNodePerQueue = new VeniceConcurrentHashMap<>(producerThreadCount); + this.producingTopic = producingTopic; + List tmpProducers = new ArrayList<>(producerThreadCount); + List tmpDrainers = new ArrayList<>(producerThreadCount); + for (int cur = 0; cur < producerThreadCount; ++cur) { + PubSubProducerAdapter producer = producerAdapterSupplier.get(); + tmpProducers.add(producer); + + MemoryBoundBlockingQueue blockingQueue = + new MemoryBoundBlockingQueue<>(producerQueueSize, 1 * 1024 * 1024l); + ProducerQueueDrainer drainer = new ProducerQueueDrainer(producingTopic, blockingQueue, cur, producer); + tmpBlockingQueueList.add(blockingQueue); + tmpDrainers.add(drainer); + this.producerExecutor.submit(drainer); + } + + this.blockingQueueList = Collections.unmodifiableList(tmpBlockingQueueList); + this.producers = Collections.unmodifiableList(tmpProducers); + this.drainers = Collections.unmodifiableList(tmpDrainers); + } + + public static class ProducerQueueNode implements Measurable { + // Rough estimation. + private static final int PRODUCER_QUEUE_NODE_OVERHEAD = 256; + + private final String topic; + private final int partition; + private final KafkaKey key; + private final KafkaMessageEnvelope value; + private final PubSubMessageHeaders headers; + private final PubSubProducerCallback callback; + private final CompletableFuture produceFuture; + + public ProducerQueueNode( + String topic, + int partition, + KafkaKey key, + KafkaMessageEnvelope value, + PubSubMessageHeaders headers, + PubSubProducerCallback callback, + CompletableFuture produceFuture) { + this.topic = topic; + this.partition = partition; + this.key = key; + this.value = value; + this.headers = headers; + this.callback = callback; + this.produceFuture = produceFuture; + } + + @Override + public int getSize() { + return topic.length() + 4 + key.getEstimatedObjectSizeOnHeap() + + ProtocolUtils.getEstimateOfMessageEnvelopeSizeOnHeap(value) + headers.getSize() + + PRODUCER_QUEUE_NODE_OVERHEAD; + } + } + + public static class ProducerQueueDrainer implements Runnable { + private static final Logger LOGGER = LogManager.getLogger(ProducerQueueDrainer.class); + private final AtomicBoolean isRunning = new AtomicBoolean(true); + private final String producingTopic; + private final BlockingQueue blockingQueue; + private final int drainerIndex; + private final PubSubProducerAdapter producer; + + public ProducerQueueDrainer( + String producingTopic, + BlockingQueue blockingQueue, + int drainerIndex, + PubSubProducerAdapter producer) { + this.producingTopic = producingTopic; + this.blockingQueue = blockingQueue; + this.drainerIndex = drainerIndex; + this.producer = producer; + } + + @Override + public void run() { + LOGGER + .info("Starting ProducerQueueDrainer Thread for drainer: {} to topic: {} ...", drainerIndex, producingTopic); + while (isRunning.get()) { + try { + final ProducerQueueNode node = blockingQueue.take(); + try { + producer.sendMessage(node.topic, node.partition, node.key, node.value, node.headers, node.callback) + .whenComplete((result, t) -> { + if (t != null) { + node.produceFuture.completeExceptionally(t); + } else { + node.produceFuture.complete(result); + } + }); + } catch (Exception ex) { + LOGGER.error( + "Got exception when producing record to topic: {}, partition: {} in drainer: {}", + node.topic, + node.partition, + drainerIndex, + ex); + // TODO: do we need to add retry here? + throw ex; + } + } catch (InterruptedException e) { + LOGGER.warn( + "Got interrupted when executing producer drainer to topic: {} with index: {}", + producingTopic, + drainerIndex); + } + } + isRunning.set(false); + LOGGER.info( + "Current ProducerQueueDrainer Thread for drainer: {} to topic: {} stopped", + drainerIndex, + producingTopic); + } + + public void stop() { + isRunning.set(false); + } + + } + + @Override + public int getNumberOfPartitions(String topic) { + return producers.get(0).getNumberOfPartitions(topic); + } + + public boolean hasAnyProducerStopped() { + for (ProducerQueueDrainer drainer: drainers) { + if (!drainer.isRunning.get()) { + return true; + } + } + return false; + } + + @Override + public CompletableFuture sendMessage( + String topic, + Integer partition, + KafkaKey key, + KafkaMessageEnvelope value, + PubSubMessageHeaders pubSubMessageHeaders, + PubSubProducerCallback pubSubProducerCallback) { + if (hasAnyProducerStopped()) { + throw new VeniceException("Some internal producer has already stopped running, pop-up the exception"); + } + /** + * Round-robin assignment. + */ + MemoryBoundBlockingQueue selectedQueue = partitionQueueAssignment.computeIfAbsent( + partition, + ignored -> blockingQueueList.get(selectedQueueId.getAndIncrement() % blockingQueueList.size())); + CompletableFuture resultFuture = new CompletableFuture<>(); + try { + ProducerQueueNode newNode = new ProducerQueueNode( + topic, + partition, + key, + value, + pubSubMessageHeaders, + pubSubProducerCallback, + resultFuture); + selectedQueue.put(newNode); + lastNodePerQueue.put(selectedQueue, newNode); + } catch (InterruptedException e) { + throw new VeniceException("Failed to produce to topic: " + topic, e); + } + + return resultFuture; + } + + @Override + public void flush() { + LOGGER.info("Start flushing a 'PubSubProducerAdapterConcurrentDelegator' instance for topic: {}", producingTopic); + /** + * Take the last message of each queue and make sure they will be produced before the producer flush. + */ + try { + for (ProducerQueueNode node: lastNodePerQueue.values()) { + node.produceFuture.get(); + } + } catch (Exception e) { + throw new VeniceException("Received exception when trying to flush the pending messages", e); + } + + producers.forEach(p -> p.flush()); + LOGGER + .info("Finished flushing a 'PubSubProducerAdapterConcurrentDelegator' instance for topic: {}", producingTopic); + } + + @Override + public void close(long closeTimeOutMs) { + LOGGER.info("Start closing a 'PubSubProducerAdapterConcurrentDelegator' instance for topic: {}", producingTopic); + drainers.forEach(d -> d.stop()); + producerExecutor.shutdownNow(); + try { + producerExecutor.awaitTermination(closeTimeOutMs, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOGGER.error( + "Received InterruptedException while shutting down the producer executor, will continue" + + " to close the producers", + e); + } + + producers.forEach(p -> p.close(closeTimeOutMs)); + LOGGER.info("Closed a 'PubSubProducerAdapterConcurrentDelegator' instance for topic: {}", producingTopic); + } + + @Override + public Object2DoubleMap getMeasurableProducerMetrics() { + return Object2DoubleMaps.emptyMap(); + } + + @Override + public String getBrokerAddress() { + return producers.get(0).getBrokerAddress(); + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/utils/ProtocolUtils.java b/internal/venice-common/src/main/java/com/linkedin/venice/utils/ProtocolUtils.java new file mode 100644 index 0000000000..3b8ccf9271 --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/utils/ProtocolUtils.java @@ -0,0 +1,36 @@ +package com.linkedin.venice.utils; + +import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; +import com.linkedin.venice.kafka.protocol.Put; +import com.linkedin.venice.kafka.protocol.Update; +import com.linkedin.venice.kafka.protocol.enums.MessageType; + + +public class ProtocolUtils { + /** + * Measure the heap usage of {@link KafkaMessageEnvelope}. + */ + public static int getEstimateOfMessageEnvelopeSizeOnHeap(KafkaMessageEnvelope messageEnvelope) { + int kmeBaseOverhead = 100; // Super rough estimate. TODO: Measure with a more precise library and store statically + switch (MessageType.valueOf(messageEnvelope)) { + case PUT: + Put put = (Put) messageEnvelope.payloadUnion; + int size = put.putValue.capacity(); + if (put.replicationMetadataPayload != null + /** + * N.B.: When using the {@link org.apache.avro.io.OptimizedBinaryDecoder}, the {@link put.putValue} and the + * {@link put.replicationMetadataPayload} will be backed by the same underlying array. If that is the + * case, then we don't want to account for the capacity twice. + */ + && put.replicationMetadataPayload.array() != put.putValue.array()) { + size += put.replicationMetadataPayload.capacity(); + } + return size + kmeBaseOverhead; + case UPDATE: + Update update = (Update) messageEnvelope.payloadUnion; + return update.updateValue.capacity() + kmeBaseOverhead; + default: + return kmeBaseOverhead; + } + } +} diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/MemoryBoundBlockingQueue.java b/internal/venice-common/src/main/java/com/linkedin/venice/utils/collections/MemoryBoundBlockingQueue.java similarity index 97% rename from clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/MemoryBoundBlockingQueue.java rename to internal/venice-common/src/main/java/com/linkedin/venice/utils/collections/MemoryBoundBlockingQueue.java index cb5f5720da..6e51a4d17d 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/MemoryBoundBlockingQueue.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/utils/collections/MemoryBoundBlockingQueue.java @@ -1,4 +1,4 @@ -package com.linkedin.davinci.kafka.consumer; +package com.linkedin.venice.utils.collections; import com.linkedin.venice.common.Measurable; import com.linkedin.venice.exceptions.VeniceException; @@ -51,6 +51,7 @@ public class MemoryBoundBlockingQueue implements BlockingQ * We can adjust this value later if necessary. */ public static final int LINKED_QUEUE_NODE_OVERHEAD_IN_BYTE = 48; + private final Queue queue; private final long memoryCapacityInByte; private final long notifyDeltaInByte; @@ -85,13 +86,13 @@ public long remainingMemoryCapacityInByte() { return remainingMemoryCapacityInByte.get(); } - private int getRecordSize(T record) { + private long getRecordSize(T record) { return record.getSize() + LINKED_QUEUE_NODE_OVERHEAD_IN_BYTE; } @Override public void put(T record) throws InterruptedException { - int recordSize = getRecordSize(record); + long recordSize = getRecordSize(record); if (recordSize > notifyDeltaInByte) { LOGGER.warn( "Record size of record: " + record + " is " + recordSize + ", which exceeds notifyDeltaInByte: " @@ -119,7 +120,7 @@ public T take() throws InterruptedException { while ((record = this.queue.poll()) == null) { notEmpty.await(); } - int recordSize = getRecordSize(record); + long recordSize = getRecordSize(record); currentFreedMemoryInBytes += recordSize; /** * It won't notify the blocked {@link #put(Measurable)} thread until the freed memory exceeds 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 28d28ffc4d..b88cdd8eee 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 @@ -139,6 +139,9 @@ public class VeniceWriter extends AbstractVeniceWriter { */ public static final String MAX_RECORD_SIZE_BYTES = VENICE_WRITER_CONFIG_PREFIX + "max.record.size.bytes"; + public static final String PRODUCER_THREAD_COUNT = VENICE_WRITER_CONFIG_PREFIX + "producer.thread.count"; + public static final String PRODUCER_QUEUE_SIZE = VENICE_WRITER_CONFIG_PREFIX + "producer.queue.size"; + // Config value defaults /** @@ -2104,4 +2107,8 @@ public int getMaxSizeForUserPayloadPerMessageInBytes() { public String getDestination() { return topicName + "@" + producerAdapter.getBrokerAddress(); } + + public PubSubProducerAdapter getProducerAdapter() { + return this.producerAdapter; + } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriterFactory.java b/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriterFactory.java index 9e411812ce..114184a3c2 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriterFactory.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriterFactory.java @@ -5,6 +5,7 @@ import com.linkedin.venice.pubsub.adapter.kafka.producer.ApacheKafkaProducerAdapterFactory; import com.linkedin.venice.pubsub.adapter.kafka.producer.ApacheKafkaProducerConfig; import com.linkedin.venice.pubsub.api.PubSubProducerAdapter; +import com.linkedin.venice.pubsub.api.PubSubProducerAdapterConcurrentDelegator; import com.linkedin.venice.pubsub.api.PubSubProducerAdapterDelegator; import com.linkedin.venice.stats.VeniceWriterStats; import com.linkedin.venice.utils.VeniceProperties; @@ -13,6 +14,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Properties; +import java.util.function.Supplier; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -63,19 +65,31 @@ public VeniceWriter createVeniceWriter(VeniceWriterOptions op ? venicePropertiesLazy.get() : venicePropertiesWithCompressionDisabledLazy.get(); + Supplier producerAdapterSupplier = + () -> producerAdapterFactory.create(props, options.getTopicName(), options.getBrokerAddress()); + + int producerThreadCnt = options.getProducerThreadCount(); + if (producerThreadCnt > 1) { + return new VeniceWriter<>( + options, + props, + new PubSubProducerAdapterConcurrentDelegator( + options.getTopicName(), + producerThreadCnt, + options.getProducerQueueSize(), + producerAdapterSupplier)); + } + int producerCnt = options.getProducerCount(); if (producerCnt > 1) { List producers = new ArrayList<>(producerCnt); for (int i = 0; i < producerCnt; ++i) { - producers.add(producerAdapterFactory.create(props, options.getTopicName(), options.getBrokerAddress())); + producers.add(producerAdapterSupplier.get()); } return new VeniceWriter<>(options, props, new PubSubProducerAdapterDelegator(producers)); } - return new VeniceWriter<>( - options, - props, - producerAdapterFactory.create(props, options.getTopicName(), options.getBrokerAddress())); + return new VeniceWriter<>(options, props, producerAdapterSupplier.get()); } // visible for testing diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriterOptions.java b/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriterOptions.java index 0c002b4949..6766067023 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriterOptions.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriterOptions.java @@ -31,6 +31,8 @@ public class VeniceWriterOptions { private final String brokerAddress; private final boolean producerCompressionEnabled; private final int producerCount; + private final int producerThreadCount; + private final int producerQueueSize; public String getBrokerAddress() { return brokerAddress; @@ -84,6 +86,14 @@ public int getProducerCount() { return producerCount; } + public int getProducerThreadCount() { + return producerThreadCount; + } + + public int getProducerQueueSize() { + return producerQueueSize; + } + private VeniceWriterOptions(Builder builder) { topicName = builder.topicName; keySerializer = builder.keySerializer; @@ -98,6 +108,8 @@ private VeniceWriterOptions(Builder builder) { brokerAddress = builder.brokerAddress; producerCompressionEnabled = builder.producerCompressionEnabled; producerCount = builder.producerCount; + producerThreadCount = builder.producerThreadCount; + producerQueueSize = builder.producerQueueSize; } @Override @@ -128,6 +140,12 @@ public String toString() { .append(", ") .append("producerCount:") .append(producerCount) + .append(", ") + .append("producerThreadCount:") + .append(producerThreadCount) + .append(", ") + .append("producerQueueSize:") + .append(producerQueueSize) .append("}") .toString(); } @@ -146,6 +164,8 @@ public static class Builder { private String brokerAddress = null; // default null private boolean producerCompressionEnabled = true; private int producerCount = 1; + private int producerThreadCount = 1; + private int producerQueueSize = 5 * 1024 * 1024; // 5MB by default private void addDefaults() { if (keySerializer == null) { @@ -240,5 +260,15 @@ public Builder setProducerCount(int producerCount) { this.producerCount = producerCount; return this; } + + public Builder setProducerThreadCount(int producerThreadCount) { + this.producerThreadCount = producerThreadCount; + return this; + } + + public Builder setProducerQueueSize(int producerQueueSize) { + this.producerQueueSize = producerQueueSize; + return this; + } } } diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/api/PubSubProducerAdapterConcurrentDelegatorTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/api/PubSubProducerAdapterConcurrentDelegatorTest.java new file mode 100644 index 0000000000..b9ca3f20df --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/api/PubSubProducerAdapterConcurrentDelegatorTest.java @@ -0,0 +1,211 @@ +package com.linkedin.venice.pubsub.api; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; + +import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; +import com.linkedin.venice.kafka.protocol.ProducerMetadata; +import com.linkedin.venice.kafka.protocol.Put; +import com.linkedin.venice.kafka.protocol.enums.MessageType; +import com.linkedin.venice.message.KafkaKey; +import com.linkedin.venice.utils.TestUtils; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import org.testng.annotations.Test; + + +public class PubSubProducerAdapterConcurrentDelegatorTest { + @Test + public void testConstructor() { + String producingTopic = "test_rt"; + assertThrows( + VeniceException.class, + () -> new PubSubProducerAdapterConcurrentDelegator( + producingTopic, + 1, + 10 * 1024 * 1024, + () -> mock(PubSubProducerAdapter.class))); + assertThrows( + VeniceException.class, + () -> new PubSubProducerAdapterConcurrentDelegator( + producingTopic, + 2, + 10 * 1024, + () -> mock(PubSubProducerAdapter.class))); + + int threadCnt = 3; + int queueSize = 10 * 1024 * 1024; + AtomicInteger producerInstanceCount = new AtomicInteger(0); + List producers = new ArrayList<>(); + PubSubProducerAdapterConcurrentDelegator delegator = + new PubSubProducerAdapterConcurrentDelegator(producingTopic, threadCnt, queueSize, () -> { + producerInstanceCount.incrementAndGet(); + PubSubProducerAdapter producer = mock(PubSubProducerAdapter.class); + producers.add(producer); + return producer; + }); + delegator.close(100); + assertEquals(producerInstanceCount.get(), threadCnt); + assertEquals(producers.size(), threadCnt); + producers.forEach(p -> verify(p).close(100)); + } + + @Test + public void testProducerQueueNode() { + String topic = "test_rt"; + KafkaKey key = new KafkaKey(MessageType.PUT, "test_key".getBytes()); + KafkaMessageEnvelope value = new KafkaMessageEnvelope( + MessageType.PUT.getValue(), + new ProducerMetadata(), + new Put(ByteBuffer.wrap("test_value".getBytes()), 1, -1, ByteBuffer.wrap(new byte[0])), + null); + + PubSubProducerAdapterConcurrentDelegator.ProducerQueueNode node = + new PubSubProducerAdapterConcurrentDelegator.ProducerQueueNode( + topic, + 1, + key, + value, + new PubSubMessageHeaders(), + null, + null); + assertTrue(node.getSize() > 0); + } + + @Test + public void testProducerQueueDrainer() throws InterruptedException { + String topic = "test_rt"; + KafkaKey key1 = mock(KafkaKey.class); + CompletableFuture future1 = mock(CompletableFuture.class); + KafkaKey key2 = mock(KafkaKey.class); + CompletableFuture future2 = mock(CompletableFuture.class); + + PubSubProducerAdapterConcurrentDelegator.ProducerQueueNode node1 = + new PubSubProducerAdapterConcurrentDelegator.ProducerQueueNode( + topic, + 1, + key1, + mock(KafkaMessageEnvelope.class), + new PubSubMessageHeaders(), + null, + future1); + + PubSubProducerAdapterConcurrentDelegator.ProducerQueueNode node2 = + new PubSubProducerAdapterConcurrentDelegator.ProducerQueueNode( + topic, + 1, + key2, + mock(KafkaMessageEnvelope.class), + new PubSubMessageHeaders(), + null, + future2); + + BlockingQueue queue = new LinkedBlockingQueue<>(); + queue.put(node1); + queue.put(node2); + + PubSubProducerAdapter producer = mock(PubSubProducerAdapter.class); + PubSubProduceResult mockSuccessfulResult = mock(PubSubProduceResult.class); + doReturn(CompletableFuture.completedFuture(mockSuccessfulResult)).when(producer) + .sendMessage(eq(topic), eq(1), eq(key1), any(), any(), any()); + Exception mockException = mock(Exception.class); + CompletableFuture exceptionFuture = new CompletableFuture<>(); + exceptionFuture.completeExceptionally(mockException); + doReturn(exceptionFuture).when(producer).sendMessage(eq(topic), eq(1), eq(key2), any(), any(), any()); + + PubSubProducerAdapterConcurrentDelegator.ProducerQueueDrainer drainer = + new PubSubProducerAdapterConcurrentDelegator.ProducerQueueDrainer(topic, queue, 1, producer); + + Thread drainerThread = new Thread(() -> drainer.run()); + drainerThread.start(); + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, () -> { + verify(producer).sendMessage(eq(topic), eq(1), eq(key1), any(), any(), any()); + verify(producer).sendMessage(eq(topic), eq(1), eq(key2), any(), any(), any()); + + verify(future1).complete(mockSuccessfulResult); + verify(future2).completeExceptionally(mockException); + + }); + drainerThread.interrupt(); + } + + @Test + public void testDelegator() { + PubSubProducerAdapter mockProducer1 = mock(PubSubProducerAdapter.class); + PubSubProducerAdapter mockProducer2 = mock(PubSubProducerAdapter.class); + + doReturn(CompletableFuture.completedFuture(mock(PubSubProduceResult.class))).when(mockProducer1) + .sendMessage(any(), any(), any(), any(), any(), any()); + doReturn(CompletableFuture.completedFuture(mock(PubSubProduceResult.class))).when(mockProducer2) + .sendMessage(any(), any(), any(), any(), any(), any()); + + List producers = Arrays.asList(mockProducer1, mockProducer2); + + int threadCnt = 2; + + AtomicInteger producerIdx = new AtomicInteger(0); + Supplier producerSupplier = () -> { + int currentProducerIdx = producerIdx.getAndIncrement(); + return producers.get(currentProducerIdx); + }; + String testTopicName = "test_rt"; + PubSubProducerAdapterConcurrentDelegator delegator = + new PubSubProducerAdapterConcurrentDelegator(testTopicName, threadCnt, 5 * 1024 * 1204, producerSupplier); + + delegator.getNumberOfPartitions(testTopicName); + verify(mockProducer1).getNumberOfPartitions(testTopicName); + + KafkaMessageEnvelope value = new KafkaMessageEnvelope( + MessageType.PUT.getValue(), + new ProducerMetadata(), + new Put(ByteBuffer.wrap("test_value".getBytes()), 1, -1, ByteBuffer.wrap(new byte[0])), + null); + + delegator.sendMessage(testTopicName, 1, mock(KafkaKey.class), value, mock(PubSubMessageHeaders.class), null); + delegator.sendMessage(testTopicName, 2, mock(KafkaKey.class), value, mock(PubSubMessageHeaders.class), null); + delegator.sendMessage(testTopicName, 3, mock(KafkaKey.class), value, mock(PubSubMessageHeaders.class), null); + delegator.sendMessage(testTopicName, 1, mock(KafkaKey.class), value, mock(PubSubMessageHeaders.class), null); + delegator.sendMessage(testTopicName, 2, mock(KafkaKey.class), value, mock(PubSubMessageHeaders.class), null); + + TestUtils.waitForNonDeterministicAssertion(3, TimeUnit.SECONDS, () -> { + verify(mockProducer1, times(2)).sendMessage(eq(testTopicName), eq(1), any(), any(), any(), any()); + verify(mockProducer1, times(1)).sendMessage(eq(testTopicName), eq(3), any(), any(), any(), any()); + verify(mockProducer2, times(2)).sendMessage(eq(testTopicName), eq(2), any(), any(), any(), any()); + }); + + delegator.flush(); + verify(mockProducer1).flush(); + verify(mockProducer2).flush(); + + delegator.close(1); + verify(mockProducer1).close(1); + verify(mockProducer2).close(1); + + assertThrows( + VeniceException.class, + () -> delegator + .sendMessage(testTopicName, 2, mock(KafkaKey.class), value, mock(PubSubMessageHeaders.class), null)); + assertTrue(delegator.hasAnyProducerStopped()); + + assertTrue(delegator.getMeasurableProducerMetrics().isEmpty()); + + delegator.getBrokerAddress(); + verify(mockProducer1).getBrokerAddress(); + } +} diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/MemoryBoundBlockingQueueTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/utils/collections/MemoryBoundBlockingQueueTest.java similarity index 93% rename from clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/MemoryBoundBlockingQueueTest.java rename to internal/venice-common/src/test/java/com/linkedin/venice/utils/collections/MemoryBoundBlockingQueueTest.java index 10d3b0c368..a390dbfdba 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/MemoryBoundBlockingQueueTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/utils/collections/MemoryBoundBlockingQueueTest.java @@ -1,4 +1,4 @@ -package com.linkedin.davinci.kafka.consumer; +package com.linkedin.venice.utils.collections; import com.linkedin.venice.common.Measurable; import com.linkedin.venice.utils.TestUtils; @@ -110,9 +110,11 @@ public void testThrottling() throws InterruptedException { } // This will trigger a notification, which will allow more puts queue.take(); - Thread.sleep(50); - Assert.assertTrue(t.isAlive()); - Assert.assertEquals(queue.size(), objectCntAtMost); + + TestUtils.waitForNonDeterministicAssertion(3, TimeUnit.SECONDS, () -> { + Assert.assertTrue(t.isAlive()); + Assert.assertEquals(queue.size(), objectCntAtMost); + }); } finally { TestUtils.shutdownThread(t); } diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/writer/VeniceWriterFactoryTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/writer/VeniceWriterFactoryTest.java index 7cc6152c9b..284e24ea86 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/writer/VeniceWriterFactoryTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/writer/VeniceWriterFactoryTest.java @@ -9,11 +9,14 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; import com.linkedin.venice.pubsub.PubSubProducerAdapterFactory; import com.linkedin.venice.pubsub.adapter.kafka.producer.ApacheKafkaProducerAdapterFactory; import com.linkedin.venice.pubsub.adapter.kafka.producer.ApacheKafkaProducerConfig; import com.linkedin.venice.pubsub.api.PubSubProducerAdapter; +import com.linkedin.venice.pubsub.api.PubSubProducerAdapterConcurrentDelegator; +import com.linkedin.venice.pubsub.api.PubSubProducerAdapterDelegator; import com.linkedin.venice.utils.VeniceProperties; import java.util.Properties; import org.mockito.ArgumentCaptor; @@ -78,6 +81,31 @@ public void testVeniceWriterFactoryWithProducerCompressionDisabled() { assertFalse(capturedProperties.getBoolean(ApacheKafkaProducerConfig.KAFKA_COMPRESSION_TYPE)); verify(producerFactoryMock, times(5)).create(any(), any(), any()); + assertTrue(veniceWriter.getProducerAdapter() instanceof PubSubProducerAdapterDelegator); + } + + // test concurrent delegator + try (VeniceWriter veniceWriter = veniceWriterFactory.createVeniceWriter( + new VeniceWriterOptions.Builder("store_v1").setBrokerAddress("kafka:9898") + .setPartitionCount(1) + .setProducerCompressionEnabled(false) + .setProducerThreadCount(3) + .build())) { + when(producerAdapterMock.getBrokerAddress()).thenReturn(brokerAddrCapture.getValue()); + assertNotNull(veniceWriter); + + String capturedBrokerAddr = veniceWriter.getDestination(); + assertNotNull(capturedBrokerAddr); + assertEquals(capturedBrokerAddr, "store_v1@kafka:9898"); + + assertEquals(veniceWriter.getMaxRecordSizeBytes(), VeniceWriter.UNLIMITED_MAX_RECORD_SIZE); + + VeniceProperties capturedProperties = propertiesCapture.getValue(); + assertNotNull(capturedProperties); + assertFalse(capturedProperties.getBoolean(ApacheKafkaProducerConfig.KAFKA_COMPRESSION_TYPE)); + + verify(producerFactoryMock, times(8)).create(any(), any(), any()); + assertTrue(veniceWriter.getProducerAdapter() instanceof PubSubProducerAdapterConcurrentDelegator); } } diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/writer/VeniceWriterOptionsTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/writer/VeniceWriterOptionsTest.java index dd4aae69ee..2c329d4d8f 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/writer/VeniceWriterOptionsTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/writer/VeniceWriterOptionsTest.java @@ -57,6 +57,8 @@ public void testVeniceWriterOptionsUsesUserSuppliedOptions() { .setBrokerAddress("kafka.broker.addr") .setProducerCompressionEnabled(false) .setProducerCount(10) + .setProducerThreadCount(10) + .setProducerQueueSize(1024 * 1024) .build(); assertNotNull(options); @@ -72,6 +74,8 @@ public void testVeniceWriterOptionsUsesUserSuppliedOptions() { assertEquals(options.getBrokerAddress(), "kafka.broker.addr"); assertFalse(options.isProducerCompressionEnabled()); assertEquals(options.getProducerCount(), 10); + assertEquals(options.getProducerThreadCount(), 10); + assertEquals(options.getProducerQueueSize(), 1024 * 1024); } @Test diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestion.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestion.java index 1c2cb688e5..a92865751a 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestion.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestion.java @@ -270,6 +270,8 @@ public void testKIFRepushActiveActiveStore(boolean isChunkingEnabled) throws Exc TestWriteUtils.runPushJob("Run push job", props); Map samzaConfig = getSamzaProducerConfig(childDatacenters, 0, storeName); + // Enable concurrent producer + samzaConfig.put(VeniceWriter.PRODUCER_THREAD_COUNT, "2"); VeniceSystemFactory factory = new VeniceSystemFactory(); // Use a unique key for DELETE with RMD validation int deleteWithRmdKeyIndex = 1000; From af000807a9e27a2bb40795601c188a5e28b7b91c Mon Sep 17 00:00:00 2001 From: Felix GV Date: Mon, 4 Nov 2024 13:23:32 -0500 Subject: [PATCH 6/7] [all] Changed Kafka from LI's 2.4.1.78 to Apache's 2.4.1 (#1000) Several code changes to make it work. Introduced a new PubSubSecurityProtocol to replace all usage of Kafka's SecurityProtocol enum, since that one has a different package name between the Apache and LinkedIn forks of Kafka. AK uses: org.apache.kafka.common.security.auth.SecurityProtocol While LI uses: org.apache.kafka.common.protocol.SecurityProtocol --- build.gradle | 16 ++-- .../davinci/config/VeniceClusterConfig.java | 20 ++--- .../consumer/KafkaStoreIngestionService.java | 6 +- .../KafkaStoreIngestionServiceTest.java | 4 +- clients/venice-admin-tool/build.gradle | 2 + .../pubsub/api/PubSubSecurityProtocol.java | 25 ++++++ .../linkedin/venice/utils/KafkaSSLUtils.java | 24 ++---- .../admin/ApacheKafkaAdminAdapterTest.java | 5 +- .../admin/ApacheKafkaAdminConfigTest.java | 14 +-- .../api/PubSubSecurityProtocolTest.java | 86 +++++++++++++++++++ .../venice/utils/KafkaSSLUtilsTest.java | 10 +-- internal/venice-test-common/build.gradle | 3 +- .../integration/utils/KafkaTestUtils.java | 4 +- .../utils/VeniceControllerWrapper.java | 4 +- .../utils/VeniceServerWrapper.java | 4 +- ...woLayerMultiRegionMultiClusterWrapper.java | 16 ++-- .../venice/utils/VeniceEnumValueTest.java | 9 +- .../VeniceControllerClusterConfig.java | 4 +- 18 files changed, 186 insertions(+), 70 deletions(-) create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubSecurityProtocol.java create mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/pubsub/api/PubSubSecurityProtocolTest.java diff --git a/build.gradle b/build.gradle index 9c02cfde4a..9f4fa4e3fa 100644 --- a/build.gradle +++ b/build.gradle @@ -40,8 +40,9 @@ if (project.hasProperty('overrideBuildEnvironment')) { def avroVersion = '1.10.2' def avroUtilVersion = '0.3.21' def grpcVersion = '1.49.2' -def kafkaGroup = 'com.linkedin.kafka' -def kafkaVersion = '2.4.1.78' +// N.B.: The build should also work when substituting Kafka from the Apache fork to LinkedIn's fork: +def kafkaGroup = 'org.apache.kafka' // 'com.linkedin.kafka' +def kafkaVersion = '2.4.1' // '2.4.1.65' def log4j2Version = '2.17.1' def pegasusVersion = '29.31.0' def protobufVersion = '3.21.7' @@ -52,14 +53,15 @@ def alpnAgentVersion = '2.0.10' def hadoopVersion = '2.10.2' def apacheSparkVersion = '3.3.3' def antlrVersion = '4.8' +def scala = '2.12' ext.libraries = [ alpnAgent: "org.mortbay.jetty.alpn:jetty-alpn-agent:${alpnAgentVersion}", antlr4: "org.antlr:antlr4:${antlrVersion}", antlr4Runtime: "org.antlr:antlr4-runtime:${antlrVersion}", - apacheSparkAvro: "org.apache.spark:spark-avro_2.12:${apacheSparkVersion}", - apacheSparkCore: "org.apache.spark:spark-core_2.12:${apacheSparkVersion}", - apacheSparkSql: "org.apache.spark:spark-sql_2.12:${apacheSparkVersion}", + apacheSparkAvro: "org.apache.spark:spark-avro_${scala}:${apacheSparkVersion}", + apacheSparkCore: "org.apache.spark:spark-core_${scala}:${apacheSparkVersion}", + apacheSparkSql: "org.apache.spark:spark-sql_${scala}:${apacheSparkVersion}", avro: "org.apache.avro:avro:${avroVersion}", avroCompiler: "org.apache.avro:avro-compiler:${avroVersion}", avroMapred: "org.apache.avro:avro-mapred:${avroVersion}", @@ -101,7 +103,7 @@ ext.libraries = [ jna: 'net.java.dev.jna:jna:4.5.1', jsr305: 'com.google.code.findbugs:jsr305:3.0.2', joptSimple: 'net.sf.jopt-simple:jopt-simple:3.2', - kafka: "${kafkaGroup}:kafka_2.12:${kafkaVersion}", + kafka: "${kafkaGroup}:kafka_${scala}:${kafkaVersion}", kafkaClients: "${kafkaGroup}:kafka-clients:${kafkaVersion}", kafkaClientsTest: "${kafkaGroup}:kafka-clients:${kafkaVersion}:test", log4j2api: "org.apache.logging.log4j:log4j-api:${log4j2Version}", @@ -194,6 +196,8 @@ subprojects { if (JavaVersion.current() >= JavaVersion.VERSION_1_9) { tasks.withType(JavaCompile) { + // Compiler arguments can be injected here... + // options.compilerArgs << '-Xlint:unchecked' options.release = 8 } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceClusterConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceClusterConfig.java index 097936c17b..d4e003ec65 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceClusterConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceClusterConfig.java @@ -27,6 +27,7 @@ import com.linkedin.venice.exceptions.ConfigurationException; import com.linkedin.venice.exceptions.UndefinedPropertyException; import com.linkedin.venice.meta.PersistenceType; +import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import com.linkedin.venice.utils.KafkaSSLUtils; import com.linkedin.venice.utils.RegionUtils; import com.linkedin.venice.utils.Utils; @@ -45,7 +46,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -90,8 +90,8 @@ public class VeniceClusterConfig { private final VeniceProperties clusterProperties; - private final SecurityProtocol kafkaSecurityProtocol; - private final Map kafkaBootstrapUrlToSecurityProtocol; + private final PubSubSecurityProtocol kafkaSecurityProtocol; + private final Map kafkaBootstrapUrlToSecurityProtocol; private final Optional sslConfig; public VeniceClusterConfig(VeniceProperties clusterProps, Map> kafkaClusterMap) @@ -136,17 +136,17 @@ public VeniceClusterConfig(VeniceProperties clusterProps, Map tmpKafkaClusterIdToUrlMap = new Int2ObjectOpenHashMap<>(); Object2IntMap tmpKafkaClusterUrlToIdMap = new Object2IntOpenHashMap<>(); Int2ObjectMap tmpKafkaClusterIdToAliasMap = new Int2ObjectOpenHashMap<>(); Object2IntMap tmpKafkaClusterAliasToIdMap = new Object2IntOpenHashMap<>(); - Map tmpKafkaBootstrapUrlToSecurityProtocol = new HashMap<>(); + Map tmpKafkaBootstrapUrlToSecurityProtocol = new HashMap<>(); Map tmpKafkaUrlResolution = new HashMap<>(); boolean foundBaseKafkaUrlInMappingIfItIsPopulated = kafkaClusterMap.isEmpty(); @@ -183,7 +183,7 @@ public VeniceClusterConfig(VeniceProperties clusterProps, Map sslConfig = serverConfig.getSslConfig(); if (!sslConfig.isPresent()) { @@ -1127,7 +1127,7 @@ private VeniceProperties getPubSubSSLPropertiesFromServerConfig(String kafkaBoot } properties.putAll(sslConfig.get().getKafkaSSLConfig()); } - properties.setProperty(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, securityProtocol.name); + properties.setProperty(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, securityProtocol.name()); return new VeniceProperties(properties); } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceTest.java index 37c664f45b..65ca79333e 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceTest.java @@ -44,6 +44,7 @@ import com.linkedin.venice.pubsub.PubSubProducerAdapterFactory; import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.schema.SchemaEntry; @@ -63,7 +64,6 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.function.Function; import org.apache.avro.Schema; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.BeforeClass; @@ -129,7 +129,7 @@ private void setupMockConfig() { doReturn(VeniceProperties.empty()).when(mockVeniceServerConfig).getKafkaConsumerConfigsForLocalConsumption(); doReturn(getConsumerAssignmentStrategy()).when(mockVeniceServerConfig).getSharedConsumerAssignmentStrategy(); doReturn(1).when(mockVeniceServerConfig).getConsumerPoolSizePerKafkaCluster(); - doReturn(SecurityProtocol.PLAINTEXT).when(mockVeniceServerConfig).getKafkaSecurityProtocol(dummyKafkaUrl); + doReturn(PubSubSecurityProtocol.PLAINTEXT).when(mockVeniceServerConfig).getKafkaSecurityProtocol(dummyKafkaUrl); doReturn(10).when(mockVeniceServerConfig).getKafkaMaxPollRecords(); doReturn(2).when(mockVeniceServerConfig).getTopicManagerMetadataFetcherConsumerPoolSize(); doReturn(2).when(mockVeniceServerConfig).getTopicManagerMetadataFetcherThreadPoolSize(); diff --git a/clients/venice-admin-tool/build.gradle b/clients/venice-admin-tool/build.gradle index 32de0e29f8..a6f46156a6 100644 --- a/clients/venice-admin-tool/build.gradle +++ b/clients/venice-admin-tool/build.gradle @@ -28,6 +28,8 @@ dependencies { exclude group: 'org.apache.helix' } implementation('org.apache.helix:metrics-common:1.4.1:jdk8') + implementation libraries.zstd + testImplementation project(':internal:venice-common').sourceSets.test.output } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubSecurityProtocol.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubSecurityProtocol.java new file mode 100644 index 0000000000..b09d63dabc --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubSecurityProtocol.java @@ -0,0 +1,25 @@ +package com.linkedin.venice.pubsub.api; + +import java.util.Locale; + + +/** + * This enum is equivalent to Kafka's SecurityProtocol enum. + * + * We need this abstraction because Kafka's enum is present in two different namespaces, which are different between + * LinkedIn's fork and the Apache fork. + */ +public enum PubSubSecurityProtocol { + /** Un-authenticated, non-encrypted channel */ + PLAINTEXT, + /** SSL channel */ + SSL, + /** SASL authenticated, non-encrypted channel */ + SASL_PLAINTEXT, + /** SASL authenticated, SSL channel */ + SASL_SSL; + + public static PubSubSecurityProtocol forName(String name) { + return PubSubSecurityProtocol.valueOf(name.toUpperCase(Locale.ROOT)); + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/utils/KafkaSSLUtils.java b/internal/venice-common/src/main/java/com/linkedin/venice/utils/KafkaSSLUtils.java index 9946842eed..19ed1a1a2c 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/utils/KafkaSSLUtils.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/utils/KafkaSSLUtils.java @@ -1,12 +1,12 @@ package com.linkedin.venice.utils; import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import java.util.Arrays; import java.util.List; import java.util.Properties; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.config.SslConfigs; -import org.apache.kafka.common.protocol.SecurityProtocol; public class KafkaSSLUtils { @@ -26,26 +26,20 @@ public class KafkaSSLUtils { SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG); - /** - * Right now, Venice only supports two Kafka protocols: - * {@link SecurityProtocol#PLAINTEXT} - * {@link SecurityProtocol#SSL} - * - * @param kafkaProtocol - * @return - */ public static boolean isKafkaProtocolValid(String kafkaProtocol) { - return kafkaProtocol.equals(SecurityProtocol.PLAINTEXT.name()) || kafkaProtocol.equals(SecurityProtocol.SSL.name()) - || kafkaProtocol.equals(SecurityProtocol.SASL_PLAINTEXT.name()) - || kafkaProtocol.equals(SecurityProtocol.SASL_SSL.name()); + return kafkaProtocol.equals(PubSubSecurityProtocol.PLAINTEXT.name()) + || kafkaProtocol.equals(PubSubSecurityProtocol.SSL.name()) + || kafkaProtocol.equals(PubSubSecurityProtocol.SASL_PLAINTEXT.name()) + || kafkaProtocol.equals(PubSubSecurityProtocol.SASL_SSL.name()); } public static boolean isKafkaSSLProtocol(String kafkaProtocol) { - return kafkaProtocol.equals(SecurityProtocol.SSL.name()) || kafkaProtocol.equals(SecurityProtocol.SASL_SSL.name()); + return kafkaProtocol.equals(PubSubSecurityProtocol.SSL.name()) + || kafkaProtocol.equals(PubSubSecurityProtocol.SASL_SSL.name()); } - public static boolean isKafkaSSLProtocol(SecurityProtocol kafkaProtocol) { - return kafkaProtocol == SecurityProtocol.SSL || kafkaProtocol == SecurityProtocol.SASL_SSL; + public static boolean isKafkaSSLProtocol(PubSubSecurityProtocol kafkaProtocol) { + return kafkaProtocol == PubSubSecurityProtocol.SSL || kafkaProtocol == PubSubSecurityProtocol.SASL_SSL; } /** diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapterTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapterTest.java index ecfea7c98c..00e2a9da70 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapterTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapterTest.java @@ -27,6 +27,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -192,7 +193,7 @@ public void testDeleteTopicValidTopicDeletion() throws Exception { DeleteTopicsResult deleteTopicsResultMock = mock(DeleteTopicsResult.class); KafkaFuture topicDeletionFutureMock = mock(KafkaFuture.class); - when(internalKafkaAdminClientMock.deleteTopics(any())).thenReturn(deleteTopicsResultMock); + when(internalKafkaAdminClientMock.deleteTopics(any(Collection.class))).thenReturn(deleteTopicsResultMock); when(deleteTopicsResultMock.all()).thenReturn(topicDeletionFutureMock); when(topicDeletionFutureMock.get(eq(1000L), eq(TimeUnit.MILLISECONDS))).thenReturn(null); @@ -208,7 +209,7 @@ public void testDeleteTopicThrowsException() throws Exception { DeleteTopicsResult deleteTopicsResultMock = mock(DeleteTopicsResult.class); KafkaFuture topicDeletionFutureMock = mock(KafkaFuture.class); - when(internalKafkaAdminClientMock.deleteTopics(any())).thenReturn(deleteTopicsResultMock); + when(internalKafkaAdminClientMock.deleteTopics(any(Collection.class))).thenReturn(deleteTopicsResultMock); when(deleteTopicsResultMock.all()).thenReturn(topicDeletionFutureMock); when(topicDeletionFutureMock.get(eq(1000L), eq(TimeUnit.MILLISECONDS))) diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminConfigTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminConfigTest.java index a0a197af02..1f2a998d7b 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminConfigTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminConfigTest.java @@ -2,13 +2,13 @@ import static org.testng.Assert.*; +import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import com.linkedin.venice.utils.VeniceProperties; import java.util.Properties; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.SslConfigs; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.testng.annotations.Test; @@ -20,23 +20,23 @@ public class ApacheKafkaAdminConfigTest { @Test public void testSetupSaslInKafkaAdminPlaintext() { - testSetupSaslInKafkaAdmin(SecurityProtocol.SASL_PLAINTEXT); + testSetupSaslInKafkaAdmin(PubSubSecurityProtocol.SASL_PLAINTEXT); } @Test public void testSetupSaslInKafkaAdminSSL() { - testSetupSaslInKafkaAdmin(SecurityProtocol.SASL_SSL); + testSetupSaslInKafkaAdmin(PubSubSecurityProtocol.SASL_SSL); } - private void testSetupSaslInKafkaAdmin(SecurityProtocol securityProtocol) { + private void testSetupSaslInKafkaAdmin(PubSubSecurityProtocol securityProtocol) { Properties properties = new Properties(); properties.put("cluster.name", "cluster"); properties.put("zookeeper.address", "localhost:2181"); properties.put("kafka.bootstrap.servers", "localhost:9092"); properties.put("kafka.sasl.jaas.config", SASL_JAAS_CONFIG); properties.put("kafka.sasl.mechanism", SASL_MECHANISM); - properties.put("kafka.security.protocol", securityProtocol.name); - if (securityProtocol.name.contains("SSL")) { + properties.put("kafka.security.protocol", securityProtocol.name()); + if (securityProtocol.name().contains("SSL")) { properties.put("ssl.truststore.location", "-"); properties.put("ssl.truststore.password", ""); properties.put("ssl.truststore.type", "JKS"); @@ -49,7 +49,7 @@ private void testSetupSaslInKafkaAdmin(SecurityProtocol securityProtocol) { Properties adminProperties = serverConfig.getAdminProperties(); assertEquals(SASL_JAAS_CONFIG, adminProperties.get("sasl.jaas.config")); assertEquals(SASL_MECHANISM, adminProperties.get("sasl.mechanism")); - assertEquals(securityProtocol.name, adminProperties.get("security.protocol")); + assertEquals(securityProtocol.name(), adminProperties.get("security.protocol")); } @Test diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/api/PubSubSecurityProtocolTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/api/PubSubSecurityProtocolTest.java new file mode 100644 index 0000000000..5edbd018e2 --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/api/PubSubSecurityProtocolTest.java @@ -0,0 +1,86 @@ +package com.linkedin.venice.pubsub.api; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + +import com.linkedin.venice.utils.ReflectUtils; +import java.util.HashSet; +import java.util.Set; +import org.testng.annotations.Test; + + +public class PubSubSecurityProtocolTest { + private static final Class SPECIFIC_ENUM_CLASS = getFQCN(); + + private static Class getFQCN() { + String simpleClassName = "SecurityProtocol"; + try { + String apacheKafkaFQCN = "org.apache.kafka.common.security.auth" + "." + simpleClassName; + Class apacheKafkaSecurityProtocol = ReflectUtils.loadClass(apacheKafkaFQCN); + if (apacheKafkaSecurityProtocol != null && apacheKafkaSecurityProtocol.isEnum()) { + return apacheKafkaSecurityProtocol; + } + } catch (Exception e) { + // Expected if not using Apache Kafka. + } + try { + String liKafkaFQCN = "org.apache.kafka.common.protocol" + "." + simpleClassName; + Class liKafkaSecurityProtocol = ReflectUtils.loadClass(liKafkaFQCN); + if (liKafkaSecurityProtocol != null && liKafkaSecurityProtocol.isEnum()) { + return liKafkaSecurityProtocol; + } + } catch (Exception e) { + // Expected if not using LI Kafka... + } + throw new IllegalStateException( + "Neither the Apache Kafka nor LinkedIn Kafka version of " + simpleClassName + " were found on the classpath!"); + } + + /** + * This is a proof of concept of how we could instantiate the specific enum, though as of now there is no need for it. + * + * @return an enum instance specific to the pub sub library present on the class path. + */ + private static Enum getPubSubSpecificEnum(String enumValueName) { + return Enum.valueOf(SPECIFIC_ENUM_CLASS, enumValueName); + } + + /** + * This test merely checks that our own PubSubSecurityProtocol contains only values which exist in Kafka's own enum. + * It can be run with the classpath containing either the Apache fork or LinkedIn's fork of Kafka, and succeed. + */ + @Test + public void testInstantiation() { + for (PubSubSecurityProtocol value: PubSubSecurityProtocol.values()) { + Enum specificEnum = getPubSubSpecificEnum(value.name()); + assertNotNull(specificEnum); + assertTrue(specificEnum.getClass().isEnum()); + assertEquals(specificEnum.getClass().getSimpleName(), "SecurityProtocol"); + Set expectedPackageNames = new HashSet<>(2); + expectedPackageNames.add("org.apache.kafka.common.security.auth"); + expectedPackageNames.add("org.apache.kafka.common.protocol"); + assertTrue(expectedPackageNames.contains(specificEnum.getClass().getPackage().getName())); + assertEquals(specificEnum.name(), value.name()); + } + } + + @Test + public void testForName() { + assertEquals(PubSubSecurityProtocol.forName("plaintext"), PubSubSecurityProtocol.PLAINTEXT); + assertEquals(PubSubSecurityProtocol.forName("PLAINTEXT"), PubSubSecurityProtocol.PLAINTEXT); + assertEquals(PubSubSecurityProtocol.forName("Plaintext"), PubSubSecurityProtocol.PLAINTEXT); + + assertEquals(PubSubSecurityProtocol.forName("ssl"), PubSubSecurityProtocol.SSL); + assertEquals(PubSubSecurityProtocol.forName("SSL"), PubSubSecurityProtocol.SSL); + assertEquals(PubSubSecurityProtocol.forName("Ssl"), PubSubSecurityProtocol.SSL); + + assertEquals(PubSubSecurityProtocol.forName("sasl_plaintext"), PubSubSecurityProtocol.SASL_PLAINTEXT); + assertEquals(PubSubSecurityProtocol.forName("SASL_PLAINTEXT"), PubSubSecurityProtocol.SASL_PLAINTEXT); + assertEquals(PubSubSecurityProtocol.forName("Sasl_Plaintext"), PubSubSecurityProtocol.SASL_PLAINTEXT); + + assertEquals(PubSubSecurityProtocol.forName("sasl_ssl"), PubSubSecurityProtocol.SASL_SSL); + assertEquals(PubSubSecurityProtocol.forName("SASL_SSL"), PubSubSecurityProtocol.SASL_SSL); + assertEquals(PubSubSecurityProtocol.forName("Sasl_Ssl"), PubSubSecurityProtocol.SASL_SSL); + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/utils/KafkaSSLUtilsTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/utils/KafkaSSLUtilsTest.java index 0a36349c14..ef67ef05ae 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/utils/KafkaSSLUtilsTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/utils/KafkaSSLUtilsTest.java @@ -2,7 +2,7 @@ import static org.testng.Assert.*; -import org.apache.kafka.common.protocol.SecurityProtocol; +import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import org.testng.annotations.Test; @@ -25,9 +25,9 @@ public void testIsKafkaSSLProtocol() { @Test public void testTestIsKafkaSSLProtocol() { - assertTrue(KafkaSSLUtils.isKafkaSSLProtocol(SecurityProtocol.SSL)); - assertFalse(KafkaSSLUtils.isKafkaSSLProtocol(SecurityProtocol.PLAINTEXT)); - assertTrue(KafkaSSLUtils.isKafkaSSLProtocol(SecurityProtocol.SASL_SSL)); - assertFalse(KafkaSSLUtils.isKafkaSSLProtocol(SecurityProtocol.SASL_PLAINTEXT)); + assertTrue(KafkaSSLUtils.isKafkaSSLProtocol(PubSubSecurityProtocol.SSL)); + assertFalse(KafkaSSLUtils.isKafkaSSLProtocol(PubSubSecurityProtocol.PLAINTEXT)); + assertTrue(KafkaSSLUtils.isKafkaSSLProtocol(PubSubSecurityProtocol.SASL_SSL)); + assertFalse(KafkaSSLUtils.isKafkaSSLProtocol(PubSubSecurityProtocol.SASL_PLAINTEXT)); } } diff --git a/internal/venice-test-common/build.gradle b/internal/venice-test-common/build.gradle index 12942f12a6..6b0b6b792d 100644 --- a/internal/venice-test-common/build.gradle +++ b/internal/venice-test-common/build.gradle @@ -36,7 +36,6 @@ configurations { } } implementation { - exclude group: 'org.apache.kafka' exclude group: 'org.mortbay.jetty', module: 'servlet-api' } integrationTestImplementation.extendsFrom testImplementation @@ -109,6 +108,7 @@ dependencies { implementation libraries.samzaApi implementation libraries.spark implementation libraries.testng + implementation libraries.zstd implementation (libraries.mapreduceClientJobClient) { exclude group: 'org.apache.avro' @@ -119,6 +119,7 @@ dependencies { testImplementation project(':internal:venice-common').sourceSets.test.output testImplementation libraries.log4j2core testImplementation libraries.log4j2api + testImplementation libraries.kafkaClients jmhAnnotationProcessor 'org.openjdk.jmh:jmh-generator-annprocess:' + jmh.jmhVersion.get() jmhImplementation project(path: ':internal:venice-test-common', configuration: 'integrationTestUtils') diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/KafkaTestUtils.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/KafkaTestUtils.java index aa0d6d7730..b65d2b1f9a 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/KafkaTestUtils.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/KafkaTestUtils.java @@ -1,12 +1,12 @@ package com.linkedin.venice.integration.utils; +import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import com.linkedin.venice.utils.SslUtils; import com.linkedin.venice.utils.SslUtils.VeniceTlsConfiguration; import java.util.Properties; import kafka.server.KafkaConfig; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.config.SslConfigs; -import org.apache.kafka.common.protocol.SecurityProtocol; /** @@ -43,7 +43,7 @@ public static Properties getLocalCommonKafkaSSLConfig(VeniceTlsConfiguration tls public static Properties getLocalKafkaClientSSLConfig() { Properties properties = new Properties(); - properties.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name()); + properties.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, PubSubSecurityProtocol.SSL.name()); properties.putAll(getLocalCommonKafkaSSLConfig(SslUtils.getTlsConfiguration())); return properties; } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceControllerWrapper.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceControllerWrapper.java index a13c8ba6eb..78e51df38b 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceControllerWrapper.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceControllerWrapper.java @@ -65,6 +65,7 @@ import com.linkedin.venice.meta.PersistenceType; import com.linkedin.venice.pubsub.PubSubClientsFactory; import com.linkedin.venice.pubsub.adapter.kafka.admin.ApacheKafkaAdminAdapter; +import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import com.linkedin.venice.servicediscovery.ServiceDiscoveryAnnouncer; import com.linkedin.venice.stats.TehutiUtils; import com.linkedin.venice.utils.PropertyBuilder; @@ -82,7 +83,6 @@ import java.util.Optional; import java.util.stream.Collectors; import org.apache.commons.lang.StringUtils; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -228,7 +228,7 @@ static StatefulServiceProvider generateService(VeniceCo } if (options.isSslToKafka()) { - builder.put(KAFKA_SECURITY_PROTOCOL, SecurityProtocol.SSL.name); + builder.put(KAFKA_SECURITY_PROTOCOL, PubSubSecurityProtocol.SSL.name()); builder.put(KafkaTestUtils.getLocalCommonKafkaSSLConfig(SslUtils.getTlsConfiguration())); } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceServerWrapper.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceServerWrapper.java index 82c7393b4f..669abfa9fe 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceServerWrapper.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceServerWrapper.java @@ -42,6 +42,7 @@ import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.helix.AllowlistAccessor; import com.linkedin.venice.helix.ZkAllowlistAccessor; +import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import com.linkedin.venice.security.SSLFactory; import com.linkedin.venice.server.VeniceServer; import com.linkedin.venice.server.VeniceServerContext; @@ -72,7 +73,6 @@ import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; import org.apache.commons.io.FileUtils; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -256,7 +256,7 @@ static StatefulServiceProvider generateService( .put(SERVER_LEADER_COMPLETE_STATE_CHECK_IN_FOLLOWER_VALID_INTERVAL_MS, 5000) .put(SERVER_RESUBSCRIPTION_TRIGGERED_BY_VERSION_INGESTION_CONTEXT_CHANGE_ENABLED, true); if (sslToKafka) { - serverPropsBuilder.put(KAFKA_SECURITY_PROTOCOL, SecurityProtocol.SSL.name); + serverPropsBuilder.put(KAFKA_SECURITY_PROTOCOL, PubSubSecurityProtocol.SSL.name()); serverPropsBuilder.put(KafkaTestUtils.getLocalCommonKafkaSSLConfig(SslUtils.getTlsConfiguration())); } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceTwoLayerMultiRegionMultiClusterWrapper.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceTwoLayerMultiRegionMultiClusterWrapper.java index 5065c3d52e..3c9e760c5c 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceTwoLayerMultiRegionMultiClusterWrapper.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceTwoLayerMultiRegionMultiClusterWrapper.java @@ -18,6 +18,7 @@ import static com.linkedin.venice.integration.utils.VeniceClusterWrapperConstants.DEFAULT_PARENT_DATA_CENTER_REGION_NAME; import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; import java.io.File; @@ -33,7 +34,6 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -257,8 +257,8 @@ private static Map> addKafkaClusterIDMappingToServer List regionNames, List kafkaBrokers) { if (serverProperties.isPresent()) { - SecurityProtocol baseSecurityProtocol = SecurityProtocol - .valueOf(serverProperties.get().getProperty(KAFKA_SECURITY_PROTOCOL, SecurityProtocol.PLAINTEXT.name)); + PubSubSecurityProtocol baseSecurityProtocol = PubSubSecurityProtocol.valueOf( + serverProperties.get().getProperty(KAFKA_SECURITY_PROTOCOL, PubSubSecurityProtocol.PLAINTEXT.name())); Map> kafkaClusterMap = new HashMap<>(); Map mapping; @@ -266,21 +266,21 @@ private static Map> addKafkaClusterIDMappingToServer mapping = new HashMap<>(); int clusterId = i - 1; mapping.put(KAFKA_CLUSTER_MAP_KEY_NAME, regionNames.get(clusterId)); - SecurityProtocol securityProtocol = baseSecurityProtocol; + PubSubSecurityProtocol securityProtocol = baseSecurityProtocol; if (clusterId > 0) { // Testing mixed security on any 2-layer setup with 2 or more DCs. - securityProtocol = SecurityProtocol.SSL; + securityProtocol = PubSubSecurityProtocol.SSL; } - mapping.put(KAFKA_CLUSTER_MAP_SECURITY_PROTOCOL, securityProtocol.name); + mapping.put(KAFKA_CLUSTER_MAP_SECURITY_PROTOCOL, securityProtocol.name()); // N.B. the first Kafka broker in the list is the parent, which we're excluding from the mapping, so this // is why the index here is offset by 1 compared to the cluster ID. PubSubBrokerWrapper pubSubBrokerWrapper = kafkaBrokers.get(i); - String kafkaAddress = securityProtocol == SecurityProtocol.SSL + String kafkaAddress = securityProtocol == PubSubSecurityProtocol.SSL ? pubSubBrokerWrapper.getSSLAddress() : pubSubBrokerWrapper.getAddress(); mapping.put(KAFKA_CLUSTER_MAP_KEY_URL, kafkaAddress); - String otherKafkaAddress = securityProtocol == SecurityProtocol.PLAINTEXT + String otherKafkaAddress = securityProtocol == PubSubSecurityProtocol.PLAINTEXT ? pubSubBrokerWrapper.getSSLAddress() : pubSubBrokerWrapper.getAddress(); mapping.put(KAFKA_CLUSTER_MAP_KEY_OTHER_URLS, otherKafkaAddress); diff --git a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValueTest.java b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValueTest.java index 05a0861ac6..fedfec47a8 100644 --- a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValueTest.java +++ b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValueTest.java @@ -60,7 +60,9 @@ public void test() { Function valueOfFunction = value -> { try { - return (T) valueOfMethod.invoke(null, value); + @SuppressWarnings("unchecked") + T valueOfReturn = (T) valueOfMethod.invoke(null, value); + return valueOfReturn; } catch (Exception e) { if (e.getClass() == InvocationTargetException.class && e.getCause() instanceof VeniceException) { // Those are expected for invalid values, so we bubble them up. @@ -95,7 +97,8 @@ public void test() { // Check that no other enum values exist besides those that are expected Method valuesFunction = getPublicStaticFunction(this.enumClass, VALUES_METHOD_NAME, new Class[0]); try { - T[] types = (T[]) valuesFunction.invoke(null, new Class[0]); + @SuppressWarnings("unchecked") + T[] types = (T[]) valuesFunction.invoke(null, new Object[0]); for (T type: types) { assertTrue( expectedMapping.containsKey(type.getValue()), @@ -106,7 +109,7 @@ public void test() { } } - private static Method getPublicStaticFunction(Class klass, String functionName, Class... params) { + private static Method getPublicStaticFunction(Class klass, String functionName, Class... params) { try { Method function = klass.getDeclaredMethod(functionName, params); assertTrue( diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java index 93fb6e139a..7867440bf3 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java @@ -189,6 +189,7 @@ import com.linkedin.venice.meta.RoutingStrategy; import com.linkedin.venice.pubsub.PubSubAdminAdapterFactory; import com.linkedin.venice.pubsub.PubSubClientsFactory; +import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import com.linkedin.venice.pushmonitor.LeakedPushStatusCleanUpService; import com.linkedin.venice.pushmonitor.PushMonitorType; import com.linkedin.venice.status.BatchJobHeartbeatConfigs; @@ -212,7 +213,6 @@ import java.util.stream.Collectors; import org.apache.helix.cloud.constants.CloudProvider; import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy; -import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -603,7 +603,7 @@ public VeniceControllerClusterConfig(VeniceProperties props) { this.sslKafkaBootStrapServers = sslToKafka ? props.getString(SSL_KAFKA_BOOTSTRAP_SERVERS) : null; this.helixSendMessageTimeoutMilliseconds = props.getInt(HELIX_SEND_MESSAGE_TIMEOUT_MS, 10000); - this.kafkaSecurityProtocol = props.getString(KAFKA_SECURITY_PROTOCOL, SecurityProtocol.PLAINTEXT.name()); + this.kafkaSecurityProtocol = props.getString(KAFKA_SECURITY_PROTOCOL, PubSubSecurityProtocol.PLAINTEXT.name()); if (!KafkaSSLUtils.isKafkaProtocolValid(kafkaSecurityProtocol)) { throw new ConfigurationException("Invalid kafka security protocol: " + kafkaSecurityProtocol); } From e4fac90937472e2d2ff6a1c70088fcd366e545f2 Mon Sep 17 00:00:00 2001 From: Zac Policzer Date: Mon, 4 Nov 2024 13:45:20 -0800 Subject: [PATCH 7/7] [common][consistency][specs] Add in diff comparison utility functions and leap frog specs (#1277) This code has been committed in internal repos previously, and has been part of unsubmitted PR's in the past, but now we're finally getting around to using it in main OS. So committing this part of the code as we advance on that project. --- .../consistency/DiffValidationUtils.java | 119 +++++ .../consistency/DiffValidationUtilsTest.java | 155 ++++++ specs/TLA+/LeapFrog/MCleapfrog.cfg | 14 + specs/TLA+/LeapFrog/MCleapfrog.tla | 17 + specs/TLA+/LeapFrog/leapfrog.tla | 441 ++++++++++++++++++ 5 files changed, 746 insertions(+) create mode 100644 internal/venice-client-common/src/main/java/com/linkedin/venice/utils/consistency/DiffValidationUtils.java create mode 100644 internal/venice-client-common/src/test/java/com/linkedin/venice/utils/consistency/DiffValidationUtilsTest.java create mode 100644 specs/TLA+/LeapFrog/MCleapfrog.cfg create mode 100644 specs/TLA+/LeapFrog/MCleapfrog.tla create mode 100644 specs/TLA+/LeapFrog/leapfrog.tla diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/consistency/DiffValidationUtils.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/consistency/DiffValidationUtils.java new file mode 100644 index 0000000000..893323bebc --- /dev/null +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/consistency/DiffValidationUtils.java @@ -0,0 +1,119 @@ +package com.linkedin.venice.utils.consistency; + +import java.util.List; + + +public final class DiffValidationUtils { + private DiffValidationUtils() { + // not called + } + + /** + * This method determines if a given record in a snapshot has been lost. It determines this by looking at the records + * ReplicationMetadata and comparing it to the highwatermark of the colo which did not have this record. If the colo + * which does not have this record has advanced beyond the offset vector held by the individual record, then we know that + * this record has been passed by. + * + * This function should not be used on any record which holds a tombstone (as the colo which is farther ahead will have + * purged out a record which currently holds a tombstone). + * @param existingValueOffsetVector + * @param nonExistentValuePartitionOffsetWatermark + * @return True if the record seems to be missing. + */ + static public boolean isRecordMissing( + List existingValueOffsetVector, + List nonExistentValuePartitionOffsetWatermark) { + if (!hasOffsetAdvanced(existingValueOffsetVector, nonExistentValuePartitionOffsetWatermark)) { + return false; + } + return true; + } + + /** + * This method determines that if given the values for two equal keys in a snapshot of venice data, have the records diverged. + * It does this, by determining if each colo has received all data which is pertinent to coming to the current resolution + * held by each snapshot. In other words, does each colo have all the same information to come to the same conclusion, + * and do these conclusions diverge? + * + * This method DOES NOT catch divergence for 'missing' records. Missing records should be determined with { + * @link #isRecordMissing(List, List)} + * + * @param firstValueChecksum A checksum or hash that represents the value of the first record in this comparison. + * The firstValueChecksum should correspond to the secondValueChecksum with the same key. + * @param secondValueChecksum A checksum or hash that represents the value of the second record in this comparison. + * The secondValueChecksum should correspond to the firstValueChecksum with the same key. + * @param firstValuePartitionOffsetWatermark A list of offsets which give the highwatermark of remote consumption for the + * snapshot of the partition which held the firstValueChecksum. The list + * should be ordered with offsets that correspond to colo id's exactly as it's + * presented in the ReplicationMetadata of venice records. + * @param secondValuePartitionOffsetWatermark A list of offsets which give the highwatermark of remote consumption for the + * snapshot of the partition which held the secondValueChecksum. The list + * should be ordered with offsets that correspond to colo id's exactly as it's + * presented in the ReplicationMetadata of venice records. + * @param firstValueOffsetVector A list of offsets pulled from the ReplicationMetadata of the first record + * @param secondValueOffsetVector A list of offsets pulled from the ReplicationMetadata of the second record + * @return True if the data seems to have diverged + */ + static public boolean doRecordsDiverge( + String firstValueChecksum, + String secondValueChecksum, + List firstValuePartitionOffsetWatermark, + List secondValuePartitionOffsetWatermark, + List firstValueOffsetVector, + List secondValueOffsetVector) { + + // Sanity check, it's possible metadata wasn't emitted, so give this a pass if any of these are blank + if (firstValuePartitionOffsetWatermark.isEmpty() || secondValuePartitionOffsetWatermark.isEmpty() + || firstValueOffsetVector.isEmpty() || secondValueOffsetVector.isEmpty()) { + return false; + } + + // If the values are the same, then these records do not diverge + if (firstValueChecksum.equals(secondValueChecksum)) { + return false; + } + + // First, we need to determine if enough information has been broad casted. That is, the PartitionOffsetWatermarks + // for both values need to be greater then all component vector parts of the individual value offsets. E.g.: + // all entries in secondValuePartitionOffsetWatermark must be greater then all entries in firstValueOffsetVector and + // all the entries in the firstValuePartitionOffsetWatermark must be greater then the secondValueOffsetVector. + // we need not compare secondValueOffsetVector to secondValuePartitionOffsetWatermark + if (!hasOffsetAdvanced(firstValueOffsetVector, secondValuePartitionOffsetWatermark)) { + return false; + } + + if (!hasOffsetAdvanced(secondValueOffsetVector, firstValuePartitionOffsetWatermark)) { + return false; + } + + // At this time we know the following + // 1) the values are different + // 2) Both colos have received enough information to have seen all the pertinent events that caused + // the rows to converge in this way. + // These records should have converged, but have not. Therefore, they have diverged! + + return true; + } + + /** + * Checks to see if an offset vector has advanced completely beyond some base offset vector or not. + * + * @param baseOffset The vector to compare against. + * @param advancedOffset The vector has should be advanced along. + * @return True if the advancedOffset vector has grown beyond the baseOffset + */ + static public boolean hasOffsetAdvanced(List baseOffset, List advancedOffset) { + if (baseOffset.size() > advancedOffset.size()) { + // the baseoffset has more entries then the advanced one, meaning that it's seen entries from more colos + // meaning that it's automatically further along then the second argument. We break early to avoid any + // array out of bounds exception + return false; + } + for (int i = 0; i < baseOffset.size(); i++) { + if (advancedOffset.get(i) < baseOffset.get(i)) { + return false; + } + } + return true; + } +} diff --git a/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/consistency/DiffValidationUtilsTest.java b/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/consistency/DiffValidationUtilsTest.java new file mode 100644 index 0000000000..8d2afd0eff --- /dev/null +++ b/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/consistency/DiffValidationUtilsTest.java @@ -0,0 +1,155 @@ +package com.linkedin.venice.utils.consistency; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class DiffValidationUtilsTest { + @Test + public void testDoRecordsDiverge() { + List firstValueOffsetRecord = new ArrayList<>(); + List secondValueOffsetRecord = new ArrayList<>(); + List firstPartitionHighWaterMark = new ArrayList<>(); + List secondPartitionHighWatermark = new ArrayList<>(); + + // metadata isn't populated in both colo's + Assert.assertFalse( + DiffValidationUtils.doRecordsDiverge( + "foo", + "bar", + Collections.EMPTY_LIST, + Collections.EMPTY_LIST, + Collections.EMPTY_LIST, + Collections.EMPTY_LIST)); + + // metadata isn't populated in first colo + Collections.addAll(secondPartitionHighWatermark, 10L, 20L, 1500L); + Collections.addAll(secondValueOffsetRecord, 3L, 0L); + Assert.assertFalse( + DiffValidationUtils.doRecordsDiverge( + "foo", + "bar", + Collections.EMPTY_LIST, + secondPartitionHighWatermark, + Collections.EMPTY_LIST, + secondValueOffsetRecord)); + + // metadata isn't populated in second colo + Collections.addAll(firstPartitionHighWaterMark, 10L, 20L, 1500L); + Collections.addAll(firstValueOffsetRecord, 3L, 0L); + Assert.assertFalse( + DiffValidationUtils.doRecordsDiverge( + "foo", + "bar", + firstPartitionHighWaterMark, + Collections.EMPTY_LIST, + firstValueOffsetRecord, + Collections.EMPTY_LIST)); + + // values are the same + Assert.assertFalse( + DiffValidationUtils.doRecordsDiverge( + "foo", + "foo", + firstPartitionHighWaterMark, + secondPartitionHighWatermark, + firstValueOffsetRecord, + secondValueOffsetRecord)); + + // Clean up + firstPartitionHighWaterMark.clear(); + firstValueOffsetRecord.clear(); + secondPartitionHighWatermark.clear(); + firstPartitionHighWaterMark.clear(); + + // first colo is ahead completely + Collections.addAll(firstPartitionHighWaterMark, 20L, 40L, 1600L); + Collections.addAll(firstValueOffsetRecord, 20L, 40L); + Collections.addAll(secondPartitionHighWatermark, 10L, 20L, 1500L); + Collections.addAll(secondValueOffsetRecord, 3L, 0L); + Assert.assertFalse( + DiffValidationUtils.doRecordsDiverge( + "foo", + "bar", + firstPartitionHighWaterMark, + secondPartitionHighWatermark, + firstValueOffsetRecord, + secondValueOffsetRecord)); + firstPartitionHighWaterMark.clear(); + firstValueOffsetRecord.clear(); + secondPartitionHighWatermark.clear(); + secondValueOffsetRecord.clear(); + + // second colo is ahead completely + Collections.addAll(firstPartitionHighWaterMark, 10L, 20L, 1500L); + Collections.addAll(firstValueOffsetRecord, 3L, 0L); + Collections.addAll(secondPartitionHighWatermark, 20L, 40L, 1600L); + Collections.addAll(secondValueOffsetRecord, 20L, 39L); + Assert.assertFalse( + DiffValidationUtils.doRecordsDiverge( + "foo", + "bar", + firstPartitionHighWaterMark, + secondPartitionHighWatermark, + firstValueOffsetRecord, + secondValueOffsetRecord)); + firstPartitionHighWaterMark.clear(); + firstValueOffsetRecord.clear(); + secondPartitionHighWatermark.clear(); + secondValueOffsetRecord.clear(); + + // fist colo has a lagging colo + Collections.addAll(firstPartitionHighWaterMark, 10L, 20L, 1500L); + Collections.addAll(firstValueOffsetRecord, 3L, 0L); + Collections.addAll(secondPartitionHighWatermark, 10L, 40L, 1500L); + Collections.addAll(secondValueOffsetRecord, 10L, 25L); + Assert.assertFalse( + DiffValidationUtils.doRecordsDiverge( + "foo", + "bar", + firstPartitionHighWaterMark, + secondPartitionHighWatermark, + firstValueOffsetRecord, + secondValueOffsetRecord)); + firstPartitionHighWaterMark.clear(); + firstValueOffsetRecord.clear(); + secondPartitionHighWatermark.clear(); + secondValueOffsetRecord.clear(); + + // second colo has a lagging colo + Collections.addAll(firstPartitionHighWaterMark, 10L, 40L, 1500L); + Collections.addAll(firstValueOffsetRecord, 3L, 25L); + Collections.addAll(secondPartitionHighWatermark, 10L, 20L, 1500L); + Collections.addAll(secondValueOffsetRecord, 10L, 19L); + Assert.assertFalse( + DiffValidationUtils.doRecordsDiverge( + "foo", + "bar", + firstPartitionHighWaterMark, + secondPartitionHighWatermark, + firstValueOffsetRecord, + secondValueOffsetRecord)); + firstPartitionHighWaterMark.clear(); + firstValueOffsetRecord.clear(); + secondPartitionHighWatermark.clear(); + secondValueOffsetRecord.clear(); + + // records diverge + Collections.addAll(firstPartitionHighWaterMark, 10L, 40L, 1505L); + Collections.addAll(firstValueOffsetRecord, 3L, 25L); + Collections.addAll(secondPartitionHighWatermark, 10L, 40L, 1500L); + Collections.addAll(secondValueOffsetRecord, 10L, 19L); + Assert.assertTrue( + DiffValidationUtils.doRecordsDiverge( + "foo", + "bar", + firstPartitionHighWaterMark, + secondPartitionHighWatermark, + firstValueOffsetRecord, + secondValueOffsetRecord)); + + } +} diff --git a/specs/TLA+/LeapFrog/MCleapfrog.cfg b/specs/TLA+/LeapFrog/MCleapfrog.cfg new file mode 100644 index 0000000000..4b3e23dc41 --- /dev/null +++ b/specs/TLA+/LeapFrog/MCleapfrog.cfg @@ -0,0 +1,14 @@ +SPECIFICATION Spec +CONSTRAINTS + TerminateComparison + NoSuccessiveControlMessages +CONSTANTS + COLOS = {"lor1", "ltx1"} + KEYS = {"key1", "key2"} + VALUES = {"squirrel", "elephant"} + MAX_WRITES = 3 + +INVARIANT MaxDiameter +PROPERTIES + Safe + Live \ No newline at end of file diff --git a/specs/TLA+/LeapFrog/MCleapfrog.tla b/specs/TLA+/LeapFrog/MCleapfrog.tla new file mode 100644 index 0000000000..f2cba428b4 --- /dev/null +++ b/specs/TLA+/LeapFrog/MCleapfrog.tla @@ -0,0 +1,17 @@ +----------- MODULE MCleapfrog ---- +EXTENDS TLC, leapfrog + +(* Do not explore states that have more writes then whats been configured *) +TerminateComparison == + \/ WRITES <= MAX_WRITES + +(* Do not explore states where we're just running comparisons infinitely *) +NoSuccessiveControlMessages == + /\ \A i,j \in 1..Len(WALs.coloA): + (j = i + 1 /\ WALs.coloA[i].key = "controlKey") => WALs.coloA[j].key # WALs.coloA[i].key + /\ \A i,j \in 1..Len(WALs.coloB): + (j = i + 1 /\ WALs.coloB[i].key = "controlKey") => WALs.coloB[j].key # WALs.coloB[i].key + +(* INVARIANT meant to police state explosion (possible bug) *) +MaxDiameter == TLCGet("level") < 50 +==== \ No newline at end of file diff --git a/specs/TLA+/LeapFrog/leapfrog.tla b/specs/TLA+/LeapFrog/leapfrog.tla new file mode 100644 index 0000000000..2c4ed88b62 --- /dev/null +++ b/specs/TLA+/LeapFrog/leapfrog.tla @@ -0,0 +1,441 @@ +----------------------------- MODULE leapfrog ------------------------------- +(***************************************************************************) +(* LeapFrog is an algorithm that is meant to detect data divergence of a *) +(* pair of active replicas for a datastore. It is able to take advantage *) +(* of the efficiencies of batch processing of incremental changes in *) +(* either a streaming environment, or, a map reduce environment. It treats *) +(* the upstream datastore (for the most part) as a black box. It does not *) +(* make assumptions about any conflict resolution strategy or consensus *) +(* protocols which might be employed, therefore, it does not make *) +(* determinations on the correctness of those strategies. This algorithm *) +(* is only capable of making judgements on the consistency of COMMITTED *) +(* data of replicas. *) +(* *) +(* COMMITED data is defined as data which has undergone conflict *) +(* resolution/concensus and been written to a write ahead log (WAL) *) +(* *) +(* We also assume that active replicas of the store leverage a WAL where *) +(* committed data is recorded. Events written to this WAL have a *) +(* coordinate system where events in a WAL can be causally compared (given *) +(* two events, we can determine which event got committed before the *) +(* other) and we can trace where these writes originated from. As an *) +(* example, given an entry in a WAL by looking at the entries metadata we *) +(* can determine the order in which it was committed to the attached *) +(* replica relative to other commits, and we can determine it's ordering *) +(* relative to events committed from it's original source replica. *) +(***************************************************************************) + +EXTENDS Naturals, Sequences + +(***************************************************************************) +(* We first declare the following inputs to represent our replicating *) +(* environment: *) +(***************************************************************************) +(***************************************************************************) +(* COLOS: A set of active replicas that are applying updates *) +(***************************************************************************) +(***************************************************************************) +(* KEYS: The keyspace for updates being applied to active replicas *) +(***************************************************************************) +(***************************************************************************) +(* VALUES: The set of values to write for those keys *) +(***************************************************************************) +(***************************************************************************) +(* MAX_WRITES: An upper bound on the number of events we write (just to *) +(* keep runtime of the model checker within reason) *) +(***************************************************************************) + +CONSTANTS COLOS, KEYS, VALUES, MAX_WRITES + +(***************************************************************************) +(* Global Variables *) +(***************************************************************************) +(* A pair of collections to represent the WAL's for the active replicas we *) +(* mean to compare. Each collection is an ordered list of updates to a *) +(* given replica of the upstream store. *) +(* This variable is a function with a domain of coloA and coloB meant to *) +(* represent the two replicas we're comparing. *) +VARIABLE WALs + +(* A set of monotonically increasing counters for each event committed by *) +(* each replica. In the Venice implementation, this corresponds to the *) +(* incrementing offsets given to messages in the kafka real time topic. In *) +(* MysQL it would correspond to binlog GTID's *) +VARIABLE coloCounters + +(* A boolean not used for implementation, but used in this specification. *) +(* This is meant do denote that in a given run, data was not symmetrically *) +(* delivered to a replica, therefore, the checker MUST flag this. If it's *) +(* false, the checker MUST NOT flag this. *) +VARIABLE dataDiverges + +(* A boolean meant to represent the flag the consistency checker will mark *) +(* once it detects that data has diverged. *) +VARIABLE divergenceDetected + +(* A pair of key value stores meant to simulate the replicas we mean to *) +(* compare. Each update delivered will merge the record and record *) +(* metadata and add an event to it's respective WAL. *) +VARIABLE keyValueStores + +(* In the implementation of this algorithm, we perform a stepwise *) +(* computation of data in a WAL at a given time. To get equivalent *) +(* behavior, we'll increment a token per destination replica in order to *) +(* represent a snapshot in time of the pair of WALs. We'll advance this *) +(* token in one of the two comparingReplicas with each check. *) +VARIABLE leapFrogIndexes + +(* As leap frog scans WAL entries, it maintains a pair of vectors where *) +(* each component part of the vector is the high water mark of a given WAL *) +(* coordinate for a given active replica that has been thus far scanned. *) +(* watermark of a source replica write. *) +VARIABLE leapFrogHighWaterMarks + +(* A pair of maps keep the last update in a given WAL to each key as *) +(* of the position of the last leapFrogIndexes. Functionally, it's a way *) +(* to track intermediate states for each key in a replica. *) +VARIABLE leapFrogLatestUpdates + +(* The combined key space including control keys (KEYS \union controlKey) *) +VARIABLE KEY_SPACE + +(* A simple counter of submitted writes meant to track termination in TLC *) +VARIABLE WRITES + +(* This spec assumes that we are comparing two replicas at a time. *) +comparingReplicas == {"coloA", "coloB"} + +(* These are special keys/values that we leverage to deal with the edge *) +(* case of a dropped write being the last write to one of a pair of *) +(* replicas. It's assumed that there will always be another write in most *) +(* production use cases, but in order to account for all scenarios in TLC *) +(* we must account for this by writing a new event to each replica as a *) +(* kind of 'capstone' write when initiating a comparison. In a production *) +(* environment it is likely not needed to impelemnt this as part of the *) +(* the consistency checker. Automatic replica heartbeats which exist in *) +(* most replicating sytems can fill the same role and catch such cases as *) +(* a completely broken replicating pipeline. *) +controlKey == "controlKey" +controlValue == "controlValue" + +(* This is a special value that is meant to be temporary. It is *) +(* committed, and then overriden and is meant to simulate a system where *) +(* there is an eventually consistent model in play (data may temporarily *) +(* diverge but eventually will become consistent). Usually this means *) +(* that we'll see asymmetric WAL entries (one commit in one colo, and two *) +(* commits in another). Such an event should not be flagged as an *) +(* an inconsistency as it's part of normal operation. *) +temporaryValue == "temporaryRecord" + +(* Default value placeholder for initialization *) +null == "null" + +(* Enumeration of variables for TLC state tracking *) +vars == + <> + +(* Variables which only change when we run leapfrog comparison *) +leapfrogVars == <> +---- +(***************************************************************************) +(* Properties: *) +(***************************************************************************) +(* Live: If data diverges, then we should detect it eventually and not *) +(* flip the state back. *) +Live == + dataDiverges ~> []divergenceDetected + +(* Safe: we should never detect divergence when there is none at any state *) +Safe == + [](divergenceDetected => dataDiverges) +---- +(**************************************************************************) +(* KV Store Helper functions *) +(**************************************************************************) +(* These Helpers describe a key value store with the following behaviors: *) +(* *) +(* Updates/Inserts to a key are accompanied by metadata which designates *) +(* a source replica and a monotonically increasing counter with values *) +(* that are not reused across writes from a given replica. When a key is *) +(* updated, metadata for that update is merged with the existing record *) +(* maintaining the highest counter value from each active replica that *) +(* has applied an update on this record. *) +(* *) +(* This merge behavior is described with MergeRowReplicaCounters which *) +(* returns the union of DOMAIN of two functions and keeps the greater of *) +(* each RANGE value *) +(* Ex: if r1=[lor1 |-> 20, lva1 |-> 10] *) +(* r2=[lor1 |-> 9, lva1 |-> 15, ltx1 |-> 5] *) +(* *) +(* Then return [lor1 |-> 20, lva1 |-> 15, ltx1 |-> 5] *) +(**************************************************************************) +MergeRowReplicaCounters(r1, r2) == + LET D1 == DOMAIN r1 + D2 == DOMAIN r2 IN + [k \in (D1 \cup D2) |-> + IF k \in D1 THEN + IF k \in D2 + THEN IF r1[k] > r2[k] THEN r1[k] ELSE r2[k] + ELSE r1[k] + ELSE r2[k] + ] + +(**************************************************************************) +(* Merge together two KV records based on the semantic that a new update *) +(* should have it's value overwrite from the previous one, and then we *) +(* merge the metadata of the new value with the old one with *) +(**************************************************************************) +MergeKVRecords(r1, r2) == + [value |-> r2.value, replicaCounters |-> + MergeRowReplicaCounters(r1.replicaCounters, r2.replicaCounters)] + +(**************************************************************************) +(* Build a record which conforms to the structure we expect when parsing *) +(* WAL events *) +(**************************************************************************) +BuildWALRecord(newKey, newValue, newreplicaCounters) == + [key |-> newKey, value |-> newValue, replicaCounters |-> newreplicaCounters] + +(***************************************************************************) +(* Write a replicated event to each replica. This is used to simulate an *) +(* update getting applied to an active replica and then replicating to *) +(* the two active replicas we are comparing *) +(***************************************************************************) +SymmetricWrite(sourceColo, newKey, newValue) == + /\ keyValueStores' = [ keyValueStores EXCEPT + !.coloA[newKey].value = newValue, + !.coloA[newKey].replicaCounters[sourceColo] = coloCounters[sourceColo], + !.coloB[newKey].value = newValue, + !.coloB[newKey].replicaCounters[sourceColo] = coloCounters[sourceColo]] + /\ WALs' = [WALs EXCEPT + !.coloA = Append(WALs.coloA, [key |-> newKey, + value |-> keyValueStores.coloA[newKey].value', + replicaCounters |-> keyValueStores.coloA[newKey].replicaCounters']), + !.coloB = Append(WALs.coloB, [key |-> newKey, + value |-> keyValueStores.coloB[newKey].value', + replicaCounters |-> keyValueStores.coloB[newKey].replicaCounters'])] + /\ coloCounters' = [coloCounters EXCEPT + ![sourceColo] = coloCounters[sourceColo] + 1] + /\ UNCHANGED dataDiverges + +(***************************************************************************) +(* This is where things go wrong. This write is meant to simulate a bug *) +(* in the consistency model of this store and results in divergent data. *) +(* We model this as a situation where a write is applied to only one *) +(* but it does not replicate. *) +(***************************************************************************) +AsymmetricWrite(sourceColo, newKey, newValue) == + /\ keyValueStores' = [ keyValueStores EXCEPT + !.coloA[newKey].value = "EVILPOISONPILLVALUE", + !.coloA[newKey].replicaCounters[sourceColo] = coloCounters[sourceColo]] + /\ WALs' = [WALs EXCEPT + !.coloA = Append(WALs.coloA, [key |-> newKey, + value |-> keyValueStores.coloA[newKey].value', + replicaCounters |-> keyValueStores.coloA[newKey].replicaCounters'])] + /\ coloCounters' = [coloCounters EXCEPT + ![sourceColo] = coloCounters[sourceColo] + 1] + /\ dataDiverges'=TRUE + +(**************************************************************************) +(* This function applies two conflicting writes to the same key to two *) +(* replicas, and then immediately overrides one of the keys in one of the *) +(* repolicas with the write from the other replica. The effect of this *) +(* is that data converges for the two replicas, but, there is an *) +(* asymmetric history of writes for the two replicas recorded in the two *) +(* WAL's. This scenario is typical of data systems which utilize async *) +(* replication and eventual consistency. Leap Frog should observe these *) +(* differences, but recognize that they don't mean that the replicas *) +(* reached a state of permanent inconsistency. Rather, it treats this as *) +(* normal operation, and does not flag it. *) +(**************************************************************************) +EventuallyConsistentWrite(sourceColo, conflictingColo, newKey, newValue) == + /\ keyValueStores' = [ keyValueStores EXCEPT + !.coloA[newKey].value = newValue, + !.coloA[newKey].replicaCounters[sourceColo] = coloCounters[sourceColo], + !.coloA[newKey].replicaCounters[conflictingColo] = coloCounters[conflictingColo], + !.coloB[newKey].value = newValue, + !.coloB[newKey].replicaCounters[sourceColo] = coloCounters[sourceColo]] + /\ WALs' = [WALs EXCEPT + !.coloA = WALs.coloA \o + < coloCounters[conflictingColo]])) + ,BuildWALRecord(newKey, newValue, keyValueStores.coloA[newKey].replicaCounters')>>, + !.coloB = WALs.coloB \o + <>] + /\ coloCounters' = [coloCounters EXCEPT + ![sourceColo] = coloCounters[sourceColo] + 1] + /\ UNCHANGED dataDiverges + +(***************************************************************************) +(* Write an event to each replica that increments the counter from each *) +(* source replica. A production equivalent would be a set of heartbeats *) +(* from all active source replicas getting written to the pair of replicas *) +(* that we're comparing *) +(***************************************************************************) +SymmetricWriteWithFullVector(newKey, newValue) == + /\ keyValueStores' = [ keyValueStores EXCEPT + !.coloA[newKey].value = newValue, + !.coloA[newKey].replicaCounters = coloCounters, + !.coloB[newKey].value = newValue, + !.coloB[newKey].replicaCounters = coloCounters] + /\ WALs' = [WALs EXCEPT + !.coloA = Append(WALs.coloA, [key |-> newKey, + value |-> keyValueStores.coloA[newKey].value', + replicaCounters |-> keyValueStores.coloA[newKey].replicaCounters']), + !.coloB = Append(WALs.coloB, [key |-> newKey, + value |-> keyValueStores.coloB[newKey].value', + replicaCounters |-> keyValueStores.coloB[newKey].replicaCounters'])] + /\ coloCounters' = [k \in COLOS |-> coloCounters[k] + 1] + /\ UNCHANGED dataDiverges + +---- +(**************************************************************************) +(* LeapFrog functions *) +(**************************************************************************) +(* These functions describe the core steps of the leap frog algorithm *) +(**************************************************************************) + +(**************************************************************************) +(* Based on the leap frog state variables, determine if divergence is *) +(* detected. Records diverge iff ALL of the following criteria are met: *) +(**************************************************************************) +(* 1. Records share a key. Records which don't share a key are not *) +(* comparable. *) +(* 2. Values are not the same. If values are the same for a key, then *) +(* they don't diverge. An implementation should define value as the *) +(* user data as well as any metadata which may will effect system *) +(* behavior. Things like timestamps and TTL or whatever metadata *) +(* which might effect future rounds of conflict resolution). *) +(* 3. If the high watermark of observed offsets for all writers to the *) +(* first replica is greater then the offsets recorded for all writers *) +(* have touched this record in the second replica. Another way to put *) +(* it, the first replica has seen at least all events which have *) +(* touched this record in the second replica. *) +(* 3. If the high watermark of observed offsets for all writers to the *) +(* second repolica is greater then the offsets recorded for all all *) +(* writers that have touched this record in the first repolica. *) +(* Another way to put it, the second repolica has seen at least all *) +(* events which have touched this record in the first replica. *) +(**************************************************************************) +DoRecordsDiverge(keyA, keyB, valueA, valueB, HWMreplicaCountersA, + HWMreplicaCountersB, keyreplicaCountersA, keyreplicaCountersB) == + /\ keyA = keyB + /\ valueA # valueB + /\ \A x \in (DOMAIN HWMreplicaCountersA): + HWMreplicaCountersA[x] <= keyreplicaCountersB[x] + /\ \A x \in (DOMAIN HWMreplicaCountersA): + HWMreplicaCountersB[x] <= keyreplicaCountersA[x] + + +(**************************************************************************) +(* At a given step, update all states. This includes adjusting the *) +(* observed high water marks for each WAL at the current WAL position as *) +(* well as the state of those keys at that position. We also increment *) +(* the index at which we've advanced in a given WAL based on which *) +(* high water mark has advanced completely ahead of the other. That is *) +(* if one high watermark for a given replica is ahead for all components *) +(* then we advance the token of the one that is behind. Otherwise we *) +(* advance the other. We don't advance any index beyond the length of *) +(* either WAL *) +(**************************************************************************) +UpdateLeapFrogStates == + /\ leapFrogHighWaterMarks' = [leapFrogHighWaterMarks EXCEPT + !.coloA = MergeRowReplicaCounters(leapFrogHighWaterMarks.coloA, + WALs.coloA[leapFrogIndexes.coloA].replicaCounters), + !.coloB = MergeRowReplicaCounters(leapFrogHighWaterMarks.coloB, + WALs.coloB[leapFrogIndexes.coloB].replicaCounters)] + /\ leapFrogLatestUpdates' = [leapFrogLatestUpdates EXCEPT + !.coloA[WALs.coloA[leapFrogIndexes.coloA].key] = + [value |-> WALs.coloA[leapFrogIndexes.coloA].value, + replicaCounters |-> WALs.coloA[leapFrogIndexes.coloA].replicaCounters], + !.coloB[WALs.coloB[leapFrogIndexes.coloB].key] = + [value |-> WALs.coloB[leapFrogIndexes.coloB].value, + replicaCounters |-> WALs.coloB[leapFrogIndexes.coloB].replicaCounters]] + /\ IF \A n \in DOMAIN leapFrogHighWaterMarks.coloA': + leapFrogHighWaterMarks.coloA[n]' > leapFrogHighWaterMarks.coloB[n]' + THEN leapFrogIndexes' = + [leapFrogIndexes EXCEPT !.coloB = + IF leapFrogIndexes.coloB >= Len(WALs.coloB) + THEN leapFrogIndexes.coloB + ELSE leapFrogIndexes.coloB + 1] + ELSE leapFrogIndexes' = + [leapFrogIndexes EXCEPT !.coloA = + IF leapFrogIndexes.coloA >= Len(WALs.coloA) + THEN leapFrogIndexes.coloA + ELSE leapFrogIndexes.coloA + 1] + + +(**************************************************************************) +(* Optional for production environments, delivers a capstone write to all *) +(* replicas *) +(**************************************************************************) +DeliverControlWrite == + SymmetricWriteWithFullVector(controlKey, controlValue) + +(**************************************************************************) +(* Run the comparison for a given step. Here we deliver our control *) +(* write to all replicas, update our states, and then check if we have *) +(* detected divergence and flag it. *) +(**************************************************************************) +LeapFrogCompare == + /\ DeliverControlWrite + /\ divergenceDetected' = + \E n \in KEY_SPACE : DoRecordsDiverge(n, n, + leapFrogLatestUpdates.coloA[n].value, + leapFrogLatestUpdates.coloB[n].value, + leapFrogHighWaterMarks.coloA, + leapFrogHighWaterMarks.coloB, + leapFrogLatestUpdates.coloA[n].replicaCounters, + leapFrogLatestUpdates.coloB[n].replicaCounters) + /\ UpdateLeapFrogStates + /\ UNCHANGED <> + +---- +(**************************************************************************) +(* Control Functions *) +(**************************************************************************) + +(* Selects a colo from the set of colos to compete with a given colo *) +SelectCompeteingColo(colo) == + CHOOSE otherColo \in COLOS \ {colo} : TRUE + +(**************************************************************************) +(* Write a new random record to a random colo and randomly choose it to *) +(* to be a symmetric write, an asymmetric write (bug) or an eventually *) +(* consistent write *) +(**************************************************************************) +DeliverWrite == + /\ WRITES' = WRITES + 1 + /\ \E <> \in COLOS \X KEYS \X VALUES : + /\ \/ SymmetricWrite(n, k, v) + \/ AsymmetricWrite(n, k, v) + \/ EventuallyConsistentWrite(n, SelectCompeteingColo(n), k, v) + /\ UNCHANGED KEY_SPACE + /\ UNCHANGED leapfrogVars + +Next == + \/ DeliverWrite + \/ LeapFrogCompare + +Init == + /\ WRITES = 0 + /\ KEY_SPACE = KEYS \union {controlKey} + /\ coloCounters = [i \in COLOS |-> 1] + /\ dataDiverges = FALSE + /\ divergenceDetected = FALSE + /\ WALs = [i \in comparingReplicas |-> <<[key |-> controlKey, + value |-> controlValue, replicaCounters |-> [k \in COLOS |-> 0]]>>] + /\ leapFrogHighWaterMarks = [i \in comparingReplicas |-> [k \in COLOS |-> 0]] + /\ leapFrogIndexes = [i \in comparingReplicas |-> 1] + /\ leapFrogLatestUpdates = [colo \in comparingReplicas |-> [ j \in KEY_SPACE |-> [value |-> + null, replicaCounters |-> [k \in COLOS |-> 0]]] ] + /\ keyValueStores = [colo \in comparingReplicas |-> [ j \in KEY_SPACE |-> [value |-> + null, replicaCounters |-> [k \in COLOS |-> 0]]] ] + +Spec == Init /\ [][Next]_vars /\ WF_vars(LeapFrogCompare) + +==== \ No newline at end of file