From 27c6cc40ef131c6f1d6802a50e46ead4ea906898 Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Wed, 13 Nov 2024 15:39:43 -0800 Subject: [PATCH 1/2] fix: ensure real-time topic partition count matches hybrid version partition count This fix addresses an issue where the real-time topic partition count did not align with the hybrid version partition count, causing errors during hybrid store operations. The issue occurred in the following scenario: 1. Create a store with 1 partition. 2. Perform a batch push, creating a batch version with 1 partition. 3. Update the store to 3 partitions and convert it to a hybrid store. 4. Start real-time writes using push type STREAM. 5. Perform a full push to create a hybrid version with 3 partitions. This push fails because, after the topic switch, real-time consumers cannot find partitions 2 and 3 due to the real-time topic having only 1 partition. Root Cause: - In step 4, if the real-time topic did not exist, it was created with a partition count derived from the largest existing version (batch version with 1 partition), leading to a mismatch. Solution: - STREAM push type is now disallowed if there is no online hybrid version. - If an online hybrid version exists, it ensures the real-time topic partition count matches the hybrid version partition count. - The `requestTopicForPushing` method no longer creates a real-time topic if it does not already exist. Move real-time topic creation logic in addVersion enable participant message store and revert HB interval Fix tests in 1430 Fix test Fix tests Fix cc tests revert log4j2 Disable topic creation in RT topic switcher Fix flakies Do not call getRealTimeTopic fix tests --- .../LeaderFollowerStoreIngestionTask.java | 1 + .../kafka/consumer/StoreIngestionTask.java | 17 +- .../RequestTopicForPushRequest.java | 187 +++++ .../venice/pubsub/PubSubConstants.java | 1 + .../venice/pubsub/manager/TopicManager.java | 11 + .../pubsub/manager/TopicMetadataFetcher.java | 2 +- .../java/com/linkedin/venice/utils/Utils.java | 6 + .../RequestTopicForPushRequestTest.java | 114 +++ ...stomizedViewOfflinePushRepositoryTest.java | 9 +- .../helix/TestHelixLiveInstanceMonitor.java | 6 +- .../admin/ApacheKafkaAdminConfigTest.java | 2 +- .../manager/TopicMetadataFetcherTest.java | 3 +- .../PushStatusStoreReaderTest.java | 2 +- .../venice/utils/KafkaSSLUtilsTest.java | 3 +- .../com/linkedin/venice/utils/UtilsTest.java | 33 + .../consumer/TestChangelogConsumer.java | 1 - .../AbstractTestVeniceHelixAdmin.java | 24 +- .../TestDeleteStoreDeletesRealtimeTopic.java | 52 +- .../controller/TestInstanceRemovable.java | 23 +- ...VeniceHelixAdminWithSharedEnvironment.java | 16 +- .../VeniceParentHelixAdminTest.java | 25 +- .../TestAdminSparkServerWithMultiServers.java | 4 +- .../server/TestAdminSparkWithMocks.java | 34 +- .../BlobP2PTransferAmongServersTest.java | 2 +- ...TestActiveActiveReplicationForIncPush.java | 135 ++++ .../com/linkedin/venice/router/TestRead.java | 1 + .../venice/server/VeniceServerTest.java | 48 +- .../com/linkedin/venice/controller/Admin.java | 22 +- .../VeniceControllerClusterConfig.java | 2 +- .../venice/controller/VeniceHelixAdmin.java | 453 ++++++++++-- .../controller/VeniceParentHelixAdmin.java | 24 +- .../init/SystemStoreInitializationHelper.java | 5 +- .../controller/server/CreateVersion.java | 688 ++++++++++-------- .../control/RealTimeTopicSwitcher.java | 26 +- .../controller/TestVeniceHelixAdmin.java | 4 +- .../controller/server/CreateVersionTest.java | 517 ++++++++++++- .../router/api/TestVeniceDelegateMode.java | 2 +- .../api/path/TestVeniceComputePath.java | 2 +- 38 files changed, 1952 insertions(+), 555 deletions(-) create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequest.java create mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequestTest.java diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 311156d00a7..6fc6685e4e5 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -846,6 +846,7 @@ private boolean canSwitchToLeaderTopic(PartitionConsumptionState pcs) { private boolean isLocalVersionTopicPartitionFullyConsumed(PartitionConsumptionState pcs) { long localVTOff = pcs.getLatestProcessedLocalVersionTopicOffset(); long localVTEndOffset = getTopicPartitionEndOffSet(localKafkaServer, versionTopic, pcs.getPartition()); + if (localVTEndOffset == StatsErrorCode.LAG_MEASUREMENT_FAILURE.code) { return false; } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 65d7c95fce2..4a6d8b7c733 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -12,6 +12,7 @@ import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; import static com.linkedin.venice.LogMessages.KILLED_JOB_MESSAGE; import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.START_OF_SEGMENT; +import static com.linkedin.venice.pubsub.PubSubConstants.UNKNOWN_LATEST_OFFSET; import static com.linkedin.venice.utils.Utils.FATAL_DATA_VALIDATION_ERROR; import static com.linkedin.venice.utils.Utils.getReplicaId; import static java.util.Comparator.comparingInt; @@ -2319,16 +2320,17 @@ private void reportStoreVersionTopicOffsetRewindMetrics(PartitionConsumptionStat * written to, the end offset is 0. */ protected long getTopicPartitionEndOffSet(String kafkaUrl, PubSubTopic pubSubTopic, int partition) { - long offsetFromConsumer = aggKafkaConsumerService - .getLatestOffsetBasedOnMetrics(kafkaUrl, versionTopic, new PubSubTopicPartitionImpl(pubSubTopic, partition)); + PubSubTopicPartition topicPartition = new PubSubTopicPartitionImpl(pubSubTopic, partition); + long offsetFromConsumer = + aggKafkaConsumerService.getLatestOffsetBasedOnMetrics(kafkaUrl, versionTopic, topicPartition); if (offsetFromConsumer >= 0) { return offsetFromConsumer; } try { return RetryUtils.executeWithMaxAttemptAndExponentialBackoff(() -> { long offset = getTopicManager(kafkaUrl).getLatestOffsetCachedNonBlocking(pubSubTopic, partition); - if (offset == -1) { - throw new VeniceException("Found latest offset -1"); + if (offset == UNKNOWN_LATEST_OFFSET) { + throw new VeniceException("Latest offset is unknown. Check if the topic: " + topicPartition + " exists."); } return offset; }, @@ -3076,6 +3078,13 @@ private boolean processControlMessage( processEndOfIncrementalPush(controlMessage, partitionConsumptionState); break; case TOPIC_SWITCH: + TopicSwitch topicSwitch = (TopicSwitch) controlMessage.controlMessageUnion; + LOGGER.info( + "Received {} control message. Replica: {}, Offset: {} NewSource: {}", + type.name(), + partitionConsumptionState.getReplicaId(), + offset, + topicSwitch.getSourceKafkaServers()); checkReadyToServeAfterProcess = processTopicSwitch(controlMessage, partition, offset, partitionConsumptionState); break; diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequest.java b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequest.java new file mode 100644 index 00000000000..5d14b42ca63 --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequest.java @@ -0,0 +1,187 @@ +package com.linkedin.venice.controllerapi; + +import com.linkedin.venice.meta.Version.PushType; +import java.security.cert.X509Certificate; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + + +public class RequestTopicForPushRequest { + private final String clusterName; + private final String storeName; + private final PushType pushType; + private final String pushJobId; + + private boolean sendStartOfPush = false; + private boolean sorted = false; // an inefficient but safe default + private boolean isWriteComputeEnabled = false; + private boolean separateRealTimeTopicEnabled = false; + private long rewindTimeInSecondsOverride = -1L; + private boolean deferVersionSwap = false; + private String targetedRegions = null; + private int repushSourceVersion = -1; + private Set partitioners = Collections.emptySet(); + private String compressionDictionary = null; + private X509Certificate certificateInRequest = null; + private String sourceGridFabric = null; + private String emergencySourceRegion = null; + + public RequestTopicForPushRequest(String clusterName, String storeName, PushType pushType, String pushJobId) { + if (clusterName == null || clusterName.isEmpty()) { + throw new IllegalArgumentException("clusterName is required"); + } + if (storeName == null || storeName.isEmpty()) { + throw new IllegalArgumentException("storeName is required"); + } + if (pushType == null) { + throw new IllegalArgumentException("pushType is required"); + } + + if (pushJobId == null || pushJobId.isEmpty()) { + throw new IllegalArgumentException("pushJobId is required"); + } + + this.clusterName = clusterName; + this.storeName = storeName; + this.pushType = pushType; + this.pushJobId = pushJobId; + } + + public static PushType extractPushType(String pushTypeString) { + try { + return PushType.valueOf(pushTypeString); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException( + pushTypeString + " is an invalid push type. Valid push types are: " + Arrays.toString(PushType.values())); + } + } + + public String getClusterName() { + return clusterName; + } + + public String getStoreName() { + return storeName; + } + + public PushType getPushType() { + return pushType; + } + + public String getPushJobId() { + return pushJobId; + } + + public boolean isSendStartOfPush() { + return sendStartOfPush; + } + + public boolean isSorted() { + return sorted; + } + + public boolean isWriteComputeEnabled() { + return isWriteComputeEnabled; + } + + public String getSourceGridFabric() { + return sourceGridFabric; + } + + public long getRewindTimeInSecondsOverride() { + return rewindTimeInSecondsOverride; + } + + public boolean isDeferVersionSwap() { + return deferVersionSwap; + } + + public String getTargetedRegions() { + return targetedRegions; + } + + public int getRepushSourceVersion() { + return repushSourceVersion; + } + + public Set getPartitioners() { + return partitioners; + } + + public String getCompressionDictionary() { + return compressionDictionary; + } + + public X509Certificate getCertificateInRequest() { + return certificateInRequest; + } + + public String getEmergencySourceRegion() { + return emergencySourceRegion; + } + + public void setSendStartOfPush(boolean sendStartOfPush) { + this.sendStartOfPush = sendStartOfPush; + } + + public void setSorted(boolean sorted) { + this.sorted = sorted; + } + + public void setWriteComputeEnabled(boolean writeComputeEnabled) { + isWriteComputeEnabled = writeComputeEnabled; + } + + public void setSourceGridFabric(String sourceGridFabric) { + this.sourceGridFabric = sourceGridFabric; + } + + public void setRewindTimeInSecondsOverride(long rewindTimeInSecondsOverride) { + this.rewindTimeInSecondsOverride = rewindTimeInSecondsOverride; + } + + public void setDeferVersionSwap(boolean deferVersionSwap) { + this.deferVersionSwap = deferVersionSwap; + } + + public void setTargetedRegions(String targetedRegions) { + this.targetedRegions = targetedRegions; + } + + public void setRepushSourceVersion(int repushSourceVersion) { + this.repushSourceVersion = repushSourceVersion; + } + + public void setPartitioners(String commaSeparatedPartitioners) { + if (commaSeparatedPartitioners == null || commaSeparatedPartitioners.isEmpty()) { + return; + } + setPartitioners(new HashSet<>(Arrays.asList(commaSeparatedPartitioners.split(",")))); + } + + public void setPartitioners(Set partitioners) { + this.partitioners = partitioners; + } + + public void setCompressionDictionary(String compressionDictionary) { + this.compressionDictionary = compressionDictionary; + } + + public void setCertificateInRequest(X509Certificate certificateInRequest) { + this.certificateInRequest = certificateInRequest; + } + + public void setEmergencySourceRegion(String emergencySourceRegion) { + this.emergencySourceRegion = emergencySourceRegion; + } + + public boolean isSeparateRealTimeTopicEnabled() { + return separateRealTimeTopicEnabled; + } + + public void setSeparateRealTimeTopicEnabled(boolean separateRealTimeTopicEnabled) { + this.separateRealTimeTopicEnabled = separateRealTimeTopicEnabled; + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubConstants.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubConstants.java index 011e468acb7..1d7cb1224b5 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubConstants.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubConstants.java @@ -65,6 +65,7 @@ public class PubSubConstants { * Default value of sleep interval for polling topic deletion status from ZK. */ public static final int PUBSUB_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS_DEFAULT_VALUE = 2 * Time.MS_PER_SECOND; + public static final long UNKNOWN_LATEST_OFFSET = -12345; private static final Duration PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE_DEFAULT = Duration.ofMinutes(1); private static Duration PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE = diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java index c91d3c452ef..a44df4bd763 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java @@ -272,6 +272,17 @@ public boolean updateTopicRetention( return false; } + public boolean updateTopicRetentionWithRetries(PubSubTopic topicName, long expectedRetentionInMs) { + PubSubTopicConfiguration topicConfiguration = getCachedTopicConfig(topicName); + return RetryUtils.executeWithMaxAttemptAndExponentialBackoff( + () -> updateTopicRetention(topicName, expectedRetentionInMs, topicConfiguration), + 5, + Duration.ofMillis(200), + Duration.ofSeconds(1), + Duration.ofMillis(2 * topicManagerContext.getPubSubOperationTimeoutMs()), + CREATE_TOPIC_RETRIABLE_EXCEPTIONS); + } + public void updateTopicCompactionPolicy(PubSubTopic topic, boolean expectedLogCompacted) { updateTopicCompactionPolicy(topic, expectedLogCompacted, -1, Optional.empty()); } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java index 4a4e7ac85e4..e34602e4733 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java @@ -392,7 +392,7 @@ long getLatestOffsetCachedNonBlocking(PubSubTopicPartition pubSubTopicPartition) if (cachedValue == null) { cachedValue = latestOffsetCache.get(pubSubTopicPartition); if (cachedValue == null) { - return -1; + return PubSubConstants.UNKNOWN_LATEST_OFFSET; } } return cachedValue.getValue(); 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 72b8fe5e119..11d3e3a95d9 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 @@ -309,6 +309,12 @@ public static long parseLongFromString(String value, String fieldName) { * any string that are not equal to 'true', We validate the string by our own. */ public static boolean parseBooleanFromString(String value, String fieldName) { + if (value == null) { + throw new VeniceHttpException( + HttpStatus.SC_BAD_REQUEST, + fieldName + " must be a boolean, but value is null", + ErrorType.BAD_REQUEST); + } if (value.equalsIgnoreCase("true") || value.equalsIgnoreCase("false")) { return Boolean.parseBoolean(value); } else { diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequestTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequestTest.java new file mode 100644 index 00000000000..2c8b2596ee0 --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequestTest.java @@ -0,0 +1,114 @@ +package com.linkedin.venice.controllerapi; + +import static com.linkedin.venice.meta.Version.PushType.BATCH; +import static com.linkedin.venice.meta.Version.PushType.STREAM; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +public class RequestTopicForPushRequestTest { + private RequestTopicForPushRequest request; + + @BeforeMethod + public void setUp() { + request = new RequestTopicForPushRequest("clusterA", "storeA", BATCH, "job123"); + } + + @Test + public void testRequestTopicForPushRequestConstructorArgs() { + assertEquals(request.getClusterName(), "clusterA"); + assertEquals(request.getStoreName(), "storeA"); + assertEquals(request.getPushType(), BATCH); + assertEquals(request.getPushJobId(), "job123"); + + // Invalid clusterName + IllegalArgumentException ex1 = Assert.expectThrows( + IllegalArgumentException.class, + () -> new RequestTopicForPushRequest("", "storeA", BATCH, "job123")); + assertEquals(ex1.getMessage(), "clusterName is required"); + + // Invalid storeName + IllegalArgumentException ex2 = Assert.expectThrows( + IllegalArgumentException.class, + () -> new RequestTopicForPushRequest("clusterA", "", BATCH, "job123")); + assertEquals(ex2.getMessage(), "storeName is required"); + + // Null pushType + IllegalArgumentException ex3 = Assert.expectThrows( + IllegalArgumentException.class, + () -> new RequestTopicForPushRequest("clusterA", "storeA", null, "job123")); + assertEquals(ex3.getMessage(), "pushType is required"); + + // Invalid pushJobId + IllegalArgumentException ex4 = Assert.expectThrows( + IllegalArgumentException.class, + () -> new RequestTopicForPushRequest("clusterA", "storeA", BATCH, "")); + assertEquals(ex4.getMessage(), "pushJobId is required"); + } + + @Test + public void testExtractPushTypeValidAndInvalidValues() { + // Valid cases + assertEquals(RequestTopicForPushRequest.extractPushType("BATCH"), BATCH); + assertEquals(RequestTopicForPushRequest.extractPushType("STREAM"), STREAM); + + // Invalid case + IllegalArgumentException ex = Assert + .expectThrows(IllegalArgumentException.class, () -> RequestTopicForPushRequest.extractPushType("INVALID")); + assertTrue(ex.getMessage().contains("INVALID is an invalid push type")); + } + + @Test + public void testRequestTopicForPushRequestSettersAndGetters() { + request.setSendStartOfPush(true); + request.setSorted(true); + request.setWriteComputeEnabled(true); + request.setSourceGridFabric("fabricA"); + request.setRewindTimeInSecondsOverride(3600); + request.setDeferVersionSwap(true); + request.setTargetedRegions("regionA,regionB"); + request.setRepushSourceVersion(42); + request.setPartitioners("partitioner1,partitioner2"); + request.setCompressionDictionary("compressionDict"); + request.setEmergencySourceRegion("regionX"); + + assertTrue(request.isSendStartOfPush()); + assertTrue(request.isSorted()); + assertTrue(request.isWriteComputeEnabled()); + assertEquals(request.getSourceGridFabric(), "fabricA"); + assertEquals(request.getRewindTimeInSecondsOverride(), 3600); + assertTrue(request.isDeferVersionSwap()); + assertEquals(request.getTargetedRegions(), "regionA,regionB"); + assertEquals(request.getRepushSourceVersion(), 42); + assertEquals(request.getPartitioners(), new HashSet<>(Arrays.asList("partitioner1", "partitioner2"))); + assertEquals(request.getCompressionDictionary(), "compressionDict"); + assertEquals(request.getEmergencySourceRegion(), "regionX"); + } + + @Test + public void testSetPartitionersValidAndEmptyCases() { + // Valid partitioners + request.setPartitioners("partitioner1"); + assertEquals(request.getPartitioners(), new HashSet<>(Collections.singletonList("partitioner1"))); + request.setPartitioners("partitioner1,partitioner2"); + assertEquals(request.getPartitioners(), new HashSet<>(Arrays.asList("partitioner1", "partitioner2"))); + + // Empty set + request.setPartitioners(Collections.emptySet()); + assertEquals(request.getPartitioners(), Collections.emptySet()); + + // Null and empty string + request.setPartitioners((String) null); + assertEquals(request.getPartitioners(), Collections.emptySet()); + + request.setPartitioners(""); + assertEquals(request.getPartitioners(), Collections.emptySet()); + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/helix/HelixCustomizedViewOfflinePushRepositoryTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/helix/HelixCustomizedViewOfflinePushRepositoryTest.java index 7ee08c747ff..152032f6868 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/helix/HelixCustomizedViewOfflinePushRepositoryTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/helix/HelixCustomizedViewOfflinePushRepositoryTest.java @@ -1,7 +1,12 @@ package com.linkedin.venice.helix; -import static org.mockito.ArgumentMatchers.*; -import static org.mockito.Mockito.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import com.linkedin.venice.meta.ReadOnlyStoreRepository; import com.linkedin.venice.meta.Store; diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/helix/TestHelixLiveInstanceMonitor.java b/internal/venice-common/src/test/java/com/linkedin/venice/helix/TestHelixLiveInstanceMonitor.java index a59add061cb..7710a9c8c26 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/helix/TestHelixLiveInstanceMonitor.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/helix/TestHelixLiveInstanceMonitor.java @@ -1,7 +1,9 @@ package com.linkedin.venice.helix; -import static org.mockito.ArgumentMatchers.*; -import static org.mockito.Mockito.*; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import org.apache.helix.zookeeper.impl.client.ZkClient; import org.mockito.Mockito; 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 1f2a998d7ba..6058c5cf5ce 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 @@ -1,6 +1,6 @@ package com.linkedin.venice.pubsub.adapter.kafka.admin; -import static org.testng.Assert.*; +import static org.testng.Assert.assertEquals; import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import com.linkedin.venice.utils.VeniceProperties; diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java index bca7d156296..e2f3ff2ad9e 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java @@ -29,6 +29,7 @@ import com.linkedin.venice.kafka.protocol.ProducerMetadata; import com.linkedin.venice.message.KafkaKey; import com.linkedin.venice.pubsub.ImmutablePubSubMessage; +import com.linkedin.venice.pubsub.PubSubConstants; import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.PubSubTopicPartitionInfo; import com.linkedin.venice.pubsub.PubSubTopicRepository; @@ -262,7 +263,7 @@ public void testGetTopicLatestOffsets() { assertEquals(res.get(1), 222L); assertEquals( topicMetadataFetcher.getLatestOffsetCachedNonBlocking(new PubSubTopicPartitionImpl(pubSubTopic, 0)), - -1); + PubSubConstants.UNKNOWN_LATEST_OFFSET); verify(consumerMock, times(3)).partitionsFor(pubSubTopic); verify(consumerMock, times(1)).endOffsets(eq(offsetsMap.keySet()), any(Duration.class)); diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pushstatushelper/PushStatusStoreReaderTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pushstatushelper/PushStatusStoreReaderTest.java index 888ad175aa2..1c25cebaa45 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/pushstatushelper/PushStatusStoreReaderTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pushstatushelper/PushStatusStoreReaderTest.java @@ -3,7 +3,7 @@ import static com.linkedin.venice.common.PushStatusStoreUtils.SERVER_INCREMENTAL_PUSH_PREFIX; import static com.linkedin.venice.common.PushStatusStoreUtils.getServerIncrementalPushKey; import static com.linkedin.venice.pushmonitor.ExecutionStatus.END_OF_INCREMENTAL_PUSH_RECEIVED; -import static org.mockito.ArgumentMatchers.*; +import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anySet; import static org.mockito.Mockito.doCallRealMethod; 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 ef67ef05aee..03bbfb87914 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 @@ -1,6 +1,7 @@ package com.linkedin.venice.utils; -import static org.testng.Assert.*; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; import com.linkedin.venice.pubsub.api.PubSubSecurityProtocol; import org.testng.annotations.Test; 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 ca0abdd4aa0..6c348f382c1 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 @@ -9,6 +9,7 @@ import static org.testng.Assert.fail; import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.exceptions.VeniceHttpException; import com.linkedin.venice.meta.HybridStoreConfig; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.StoreInfo; @@ -28,7 +29,9 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; +import org.apache.http.HttpStatus; import org.testng.Assert; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import org.testng.collections.Lists; @@ -393,4 +396,34 @@ public void testGetLeaderTopicFromPubSubTopic() { Utils.resolveLeaderTopicFromPubSubTopic(pubSubTopicRepository, separateRealTimeTopic), realTimeTopic); } + + @DataProvider(name = "booleanParsingData") + public Object[][] booleanParsingData() { + return new Object[][] { + // Valid cases + { "true", "testField", true }, // Valid "true" + { "false", "testField", false }, // Valid "false" + { "TRUE", "testField", true }, // Valid case-insensitive "TRUE" + { "FALSE", "testField", false }, // Valid case-insensitive "FALSE" + + // Invalid cases + { "notABoolean", "testField", null }, // Invalid string + { "123", "testField", null }, // Non-boolean numeric string + { "", "testField", null }, // Empty string + { null, "testField", null }, // Null input + }; + } + + @Test(dataProvider = "booleanParsingData") + public void testParseBooleanFromString(String value, String fieldName, Boolean expectedResult) { + if (expectedResult != null) { + // For valid cases + boolean result = Utils.parseBooleanFromString(value, fieldName); + assertEquals((boolean) expectedResult, result, "Parsed boolean value does not match expected value."); + return; + } + VeniceHttpException e = + expectThrows(VeniceHttpException.class, () -> Utils.parseBooleanFromString(value, fieldName)); + assertEquals(e.getHttpStatusCode(), HttpStatus.SC_BAD_REQUEST, "Invalid status code."); + } } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumer.java index d12b085e990..f0062c1e9b1 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumer.java @@ -112,7 +112,6 @@ public class TestChangelogConsumer { private String clusterName; private VeniceClusterWrapper clusterWrapper; private ControllerClient parentControllerClient; - private static final List SCHEMA_HISTORY = Arrays.asList( NAME_RECORD_V1_SCHEMA, NAME_RECORD_V2_SCHEMA, diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/AbstractTestVeniceHelixAdmin.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/AbstractTestVeniceHelixAdmin.java index 6a73a694466..4c98479879a 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/AbstractTestVeniceHelixAdmin.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/AbstractTestVeniceHelixAdmin.java @@ -16,6 +16,9 @@ import static com.linkedin.venice.ConfigKeys.TOPIC_CLEANUP_SLEEP_INTERVAL_BETWEEN_TOPIC_LIST_FETCH_MS; import static com.linkedin.venice.ConfigKeys.UNREGISTER_METRIC_FOR_DELETED_STORE_ENABLED; import static com.linkedin.venice.ConfigKeys.ZOOKEEPER_ADDRESS; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; import com.linkedin.venice.common.VeniceSystemStoreUtils; import com.linkedin.venice.controller.kafka.TopicCleanupService; @@ -30,6 +33,8 @@ import com.linkedin.venice.integration.utils.ZkServerWrapper; import com.linkedin.venice.meta.Store; import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.stats.HelixMessageChannelStats; import com.linkedin.venice.utils.HelixUtils; import com.linkedin.venice.utils.MockTestStateModelFactory; @@ -90,7 +95,9 @@ public void setupCluster(boolean createParticipantStore, MetricsRepository metri pubSubBrokerWrapper = ServiceFactory.getPubSubBroker(); clusterName = Utils.getUniqueString("test-cluster"); Properties properties = getControllerProperties(clusterName); - if (!createParticipantStore) { + if (createParticipantStore) { + properties.put(PARTICIPANT_MESSAGE_STORE_ENABLED, true); + } else { properties.put(PARTICIPANT_MESSAGE_STORE_ENABLED, false); properties.put(ADMIN_HELIX_MESSAGING_CHANNEL_ENABLED, true); } @@ -259,17 +266,16 @@ VeniceHelixAdmin getFollower(List admins, String cluster) { * Participant store should be set up by child controller. */ private void verifyParticipantMessageStoreSetup() { + TopicManager topicManager = veniceAdmin.getTopicManager(); String participantStoreName = VeniceSystemStoreUtils.getParticipantStoreNameForCluster(clusterName); + PubSubTopic participantStoreRt = pubSubTopicRepository.getTopic(Utils.composeRealTimeTopic(participantStoreName)); TestUtils.waitForNonDeterministicAssertion(5, TimeUnit.SECONDS, () -> { Store store = veniceAdmin.getStore(clusterName, participantStoreName); - Assert.assertNotNull(store); - Assert.assertEquals(store.getVersions().size(), 1); + assertNotNull(store); + assertEquals(store.getVersions().size(), 1); + }); + TestUtils.waitForNonDeterministicAssertion(60, TimeUnit.SECONDS, () -> { + assertTrue(topicManager.containsTopic(participantStoreRt)); }); - TestUtils.waitForNonDeterministicAssertion( - 3, - TimeUnit.SECONDS, - () -> Assert.assertEquals( - veniceAdmin.getRealTimeTopic(clusterName, participantStoreName), - Utils.composeRealTimeTopic(participantStoreName))); } } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDeleteStoreDeletesRealtimeTopic.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDeleteStoreDeletesRealtimeTopic.java index 0ad26693291..d25262c69d4 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDeleteStoreDeletesRealtimeTopic.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDeleteStoreDeletesRealtimeTopic.java @@ -15,6 +15,7 @@ import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.integration.utils.ServiceFactory; +import com.linkedin.venice.integration.utils.VeniceClusterCreateOptions; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; import com.linkedin.venice.meta.StoreInfo; import com.linkedin.venice.meta.Version; @@ -40,48 +41,46 @@ public class TestDeleteStoreDeletesRealtimeTopic { private static final Logger LOGGER = LogManager.getLogger(TestDeleteStoreDeletesRealtimeTopic.class); - private VeniceClusterWrapper venice = null; - private AvroGenericStoreClient client = null; + private VeniceClusterWrapper veniceCluster = null; private ControllerClient controllerClient = null; private TopicManagerRepository topicManagerRepository = null; - private String storeName = null; private final PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository(); - @BeforeClass + @BeforeClass(alwaysRun = true) public void setUp() { - venice = ServiceFactory.getVeniceCluster(); - controllerClient = - ControllerClient.constructClusterControllerClient(venice.getClusterName(), venice.getRandomRouterURL()); + veniceCluster = ServiceFactory.getVeniceCluster( + new VeniceClusterCreateOptions.Builder().numberOfControllers(1).numberOfServers(1).numberOfRouters(1).build()); + controllerClient = ControllerClient + .constructClusterControllerClient(veniceCluster.getClusterName(), veniceCluster.getRandomRouterURL()); topicManagerRepository = IntegrationTestPushUtils.getTopicManagerRepo( PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100, 0l, - venice.getPubSubBrokerWrapper(), + veniceCluster.getPubSubBrokerWrapper(), pubSubTopicRepository); - storeName = Utils.getUniqueString("hybrid-store"); - venice.getNewStore(storeName); - makeStoreHybrid(venice, storeName, 100L, 5L); - client = ClientFactory.getAndStartGenericAvroClient( - ClientConfig.defaultGenericClientConfig(storeName).setVeniceURL(venice.getRandomRouterURL())); } - @AfterClass + @AfterClass(alwaysRun = true) public void cleanUp() { Utils.closeQuietlyWithErrorLogged(topicManagerRepository); - Utils.closeQuietlyWithErrorLogged(client); - Utils.closeQuietlyWithErrorLogged(venice); + Utils.closeQuietlyWithErrorLogged(veniceCluster); Utils.closeQuietlyWithErrorLogged(controllerClient); } @Test(timeOut = 60 * Time.MS_PER_SECOND) public void deletingHybridStoreDeletesRealtimeTopic() { - TestUtils.assertCommand(controllerClient.emptyPush(storeName, Utils.getUniqueString("push-id"), 1L)); + String storeName = Utils.getUniqueString("hybrid-store"); + veniceCluster.getNewStore(storeName); + makeStoreHybrid(veniceCluster, storeName, 100L, 5L); + + TestUtils + .assertCommand(controllerClient.sendEmptyPushAndWait(storeName, Utils.getUniqueString("push-id"), 1000, 60000)); // write streaming records SystemProducer veniceProducer = null; try { - veniceProducer = getSamzaProducer(venice, storeName, Version.PushType.STREAM); + veniceProducer = getSamzaProducer(veniceCluster, storeName, Version.PushType.STREAM); for (int i = 1; i <= 10; i++) { sendStreamingRecord(veniceProducer, storeName, i); } @@ -98,13 +97,16 @@ public void deletingHybridStoreDeletesRealtimeTopic() { assertEquals(storeResponse.getStore().getCurrentVersion(), 1, "The empty push has not activated yet..."); }); - TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, () -> { - try { - assertEquals(client.get("9").get(), new Utf8("stream_9")); - } catch (Exception e) { - throw new VeniceException(e); - } - }); + try (AvroGenericStoreClient client = ClientFactory.getAndStartGenericAvroClient( + ClientConfig.defaultGenericClientConfig(storeName).setVeniceURL(veniceCluster.getRandomRouterURL()))) { + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, () -> { + try { + assertEquals(client.get("9").get(), new Utf8("stream_9")); + } catch (Exception e) { + throw new VeniceException(e); + } + }); + } // verify realtime topic exists PubSubTopic rtTopic = pubSubTopicRepository.getTopic(Utils.getRealTimeTopicName(storeInfo.get())); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestInstanceRemovable.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestInstanceRemovable.java index a6121f6e132..17e5ddb70d0 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestInstanceRemovable.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestInstanceRemovable.java @@ -1,10 +1,12 @@ package com.linkedin.venice.controller; +import com.linkedin.venice.ConfigKeys; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.StoppableNodeStatusResponse; import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; import com.linkedin.venice.controllerapi.VersionCreationResponse; import com.linkedin.venice.integration.utils.ServiceFactory; +import com.linkedin.venice.integration.utils.VeniceClusterCreateOptions; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; import com.linkedin.venice.integration.utils.VeniceServerWrapper; import com.linkedin.venice.meta.PartitionAssignment; @@ -34,17 +36,16 @@ public class TestInstanceRemovable { int replicaFactor = 3; private void setupCluster(int numberOfServer) { - int numberOfController = 1; - int numberOfRouter = 1; - + Properties properties = new Properties(); + properties.setProperty(ConfigKeys.PARTICIPANT_MESSAGE_STORE_ENABLED, "false"); cluster = ServiceFactory.getVeniceCluster( - numberOfController, - numberOfServer, - numberOfRouter, - replicaFactor, - partitionSize, - false, - false); + new VeniceClusterCreateOptions.Builder().numberOfControllers(1) + .numberOfServers(numberOfServer) + .numberOfRouters(1) + .replicationFactor(replicaFactor) + .partitionSize(partitionSize) + .extraProperties(properties) + .build()); } @AfterMethod @@ -169,7 +170,7 @@ public void testIsInstanceRemovableAfterPush() throws Exception { // Wait push completed. TestUtils.waitForNonDeterministicCompletion( - 3, + 30, TimeUnit.SECONDS, () -> cluster.getLeaderVeniceController() .getVeniceAdmin() diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java index cfb1e8cb9e6..dea8d2573ce 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java @@ -645,7 +645,7 @@ public void testGetRealTimeTopic() { String storeName = Utils.getUniqueString("store"); // Must not be able to get a real time topic until the store is created - Assert.assertThrows(VeniceNoStoreException.class, () -> veniceAdmin.getRealTimeTopic(clusterName, storeName)); + Assert.assertThrows(VeniceNoStoreException.class, () -> veniceAdmin.getRealTimeTopic(clusterName, storeName, null)); veniceAdmin.createStore(clusterName, storeName, "owner", KEY_SCHEMA, VALUE_SCHEMA); Store store = veniceAdmin.getStore(clusterName, storeName); @@ -656,7 +656,7 @@ public void testGetRealTimeTopic() { // hybrid try { - veniceAdmin.getRealTimeTopic(clusterName, storeName); + veniceAdmin.getRealTimeTopic(clusterName, storeName, null); Assert.fail("Must not be able to get a real time topic until the store is initialized with a version"); } catch (VeniceException e) { Assert.assertTrue( @@ -667,7 +667,7 @@ public void testGetRealTimeTopic() { int partitions = 2; // TODO verify partition count for RT topic. veniceAdmin.incrementVersionIdempotent(clusterName, storeName, Version.guidBasedDummyPushId(), partitions, 1); - String rtTopic = veniceAdmin.getRealTimeTopic(clusterName, storeName); + String rtTopic = veniceAdmin.getRealTimeTopic(clusterName, storeName, partitions); Assert.assertEquals(rtTopic, Utils.getRealTimeTopicName(store)); } @@ -1685,7 +1685,7 @@ public void testGetIncrementalPushVersion() { Version.guidBasedDummyPushId(), 1, 1); - String rtTopic = veniceAdmin.getRealTimeTopic(clusterName, incrementalAndHybridEnabledStoreName); + String rtTopic = veniceAdmin.getRealTimeTopic(clusterName, incrementalAndHybridEnabledStoreName, 1); TestUtils.waitForNonDeterministicCompletion( TOTAL_TIMEOUT_FOR_SHORT_TEST_MS, TimeUnit.MILLISECONDS, @@ -1694,14 +1694,14 @@ public void testGetIncrementalPushVersion() { // For incremental push policy INCREMENTAL_PUSH_SAME_AS_REAL_TIME, incremental push should succeed even if version // topic is truncated veniceAdmin.truncateKafkaTopic(Version.composeKafkaTopic(incrementalAndHybridEnabledStoreName, 1)); - veniceAdmin.getIncrementalPushVersion(clusterName, incrementalAndHybridEnabledStoreName); + veniceAdmin.getIncrementalPushVersion(clusterName, incrementalAndHybridEnabledStoreName, "test-job-1"); // For incremental push policy INCREMENTAL_PUSH_SAME_AS_REAL_TIME, incremental push should fail if rt topic is // truncated veniceAdmin.truncateKafkaTopic(rtTopic); Assert.assertThrows( VeniceException.class, - () -> veniceAdmin.getIncrementalPushVersion(clusterName, incrementalAndHybridEnabledStoreName)); + () -> veniceAdmin.getIncrementalPushVersion(clusterName, incrementalAndHybridEnabledStoreName, "test-job-1")); } @Test(timeOut = TOTAL_TIMEOUT_FOR_LONG_TEST_MS) @@ -1863,8 +1863,8 @@ public void testHybridStoreToBatchOnly() { Assert.assertTrue(veniceAdmin.getStore(clusterName, storeName).isSeparateRealTimeTopicEnabled()); Assert.assertTrue(veniceAdmin.getStore(clusterName, storeName).getVersion(1).isSeparateRealTimeTopicEnabled()); - String rtTopic = veniceAdmin.getRealTimeTopic(clusterName, storeName); - String incrementalPushRealTimeTopic = veniceAdmin.getSeparateRealTimeTopic(clusterName, storeName); + String rtTopic = veniceAdmin.getRealTimeTopic(clusterName, storeName, 1); + String incrementalPushRealTimeTopic = veniceAdmin.getSeparateRealTimeTopic(clusterName, storeName, 1); Assert.assertFalse(veniceAdmin.isTopicTruncated(rtTopic)); Assert.assertFalse(veniceAdmin.isTopicTruncated(incrementalPushRealTimeTopic)); veniceAdmin.updateStore( diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/VeniceParentHelixAdminTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/VeniceParentHelixAdminTest.java index cfcfb0d59e2..99e920d0790 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/VeniceParentHelixAdminTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/VeniceParentHelixAdminTest.java @@ -10,6 +10,7 @@ import static com.linkedin.venice.controller.SchemaConstants.VALUE_SCHEMA_FOR_WRITE_COMPUTE_V3; import static com.linkedin.venice.controller.SchemaConstants.VALUE_SCHEMA_FOR_WRITE_COMPUTE_V4; import static com.linkedin.venice.controller.SchemaConstants.VALUE_SCHEMA_FOR_WRITE_COMPUTE_V5; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.utils.ByteUtils.BYTES_PER_MB; import static com.linkedin.venice.utils.TestUtils.assertCommand; import static com.linkedin.venice.utils.TestUtils.waitForNonDeterministicAssertion; @@ -31,6 +32,7 @@ import com.linkedin.venice.controllerapi.StoreResponse; import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; import com.linkedin.venice.controllerapi.VersionCreationResponse; +import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; import com.linkedin.venice.integration.utils.VeniceControllerWrapper; @@ -40,9 +42,14 @@ import com.linkedin.venice.meta.HybridStoreConfig; import com.linkedin.venice.meta.StoreInfo; import com.linkedin.venice.meta.Version; +import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.schema.AvroSchemaParseUtils; import com.linkedin.venice.schema.writecompute.WriteComputeSchemaConverter; import com.linkedin.venice.security.SSLFactory; +import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.SslUtils; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.TestWriteUtils; @@ -298,7 +305,23 @@ public void testResourceCleanupCheckForStoreRecreation() { TimeUnit.SECONDS); // Delete the store and try re-creation. - assertFalse(parentControllerClient.disableAndDeleteStore(storeName).isError(), "Delete store shouldn't fail"); + TestUtils.assertCommand(parentControllerClient.disableAndDeleteStore(storeName), "Delete store shouldn't fail"); + + PubSubBrokerWrapper parentPubSub = twoLayerMultiRegionMultiClusterWrapper.getParentKafkaBrokerWrapper(); + PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository(); + // Manually create an RT topic in the parent region to simulate its existence, + // as RT topics are no longer created for regional system stores like meta and ps3. + try (TopicManagerRepository topicManagerRepo = IntegrationTestPushUtils + .getTopicManagerRepo(PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100, 0l, parentPubSub, pubSubTopicRepository); + TopicManager topicManager = topicManagerRepo.getLocalTopicManager()) { + PubSubTopic metaStoreRT = pubSubTopicRepository.getTopic(Version.composeRealTimeTopic(metaSystemStoreName)); + topicManager.createTopic(metaStoreRT, 1, 1, true); + TestUtils.waitForNonDeterministicAssertion( + 30, + TimeUnit.SECONDS, + () -> assertTrue(topicManager.containsTopic(metaStoreRT))); + } + // Re-create the same store right away will fail because of lingering system store resources controllerResponse = parentControllerClient.createNewStore(storeName, "test", "\"string\"", "\"string\""); assertTrue( diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkServerWithMultiServers.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkServerWithMultiServers.java index 0e8dc96bde9..1d52e718f60 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkServerWithMultiServers.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkServerWithMultiServers.java @@ -254,7 +254,9 @@ public void requestTopicIsIdempotent() { controllerClient.updateStore( storeName, new UpdateStoreQueryParams().setHybridRewindSeconds(1000).setHybridOffsetLagThreshold(1000)); - controllerClient.emptyPush(storeName, Utils.getUniqueString("emptyPushId"), 10000); + TestUtils.assertCommand( + controllerClient + .sendEmptyPushAndWait(storeName, Utils.getUniqueString("emptyPushId"), 10000, TEST_TIMEOUT)); } // Both diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkWithMocks.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkWithMocks.java index 925625ab4c5..c44ae5a4b7b 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkWithMocks.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/server/TestAdminSparkWithMocks.java @@ -1,6 +1,7 @@ package com.linkedin.venice.controller.server; import static com.linkedin.venice.meta.Store.NON_EXISTING_VERSION; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.anyString; @@ -24,11 +25,11 @@ import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; import com.linkedin.venice.meta.VersionImpl; +import com.linkedin.venice.meta.VersionStatus; import com.linkedin.venice.meta.ZKStore; import com.linkedin.venice.utils.DataProviderUtils; import com.linkedin.venice.utils.ObjectMapperFactory; import com.linkedin.venice.utils.SslUtils; -import com.linkedin.venice.utils.Utils; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -53,7 +54,7 @@ */ public class TestAdminSparkWithMocks { @Test - public void testGetRealTimeTopicUsesAdmin() throws Exception { + public void testGetRealTimeTopicForStreamPushJobUsesAdmin() throws Exception { // setup server with mock admin, note returns topic "store_rt" VeniceHelixAdmin admin = Mockito.mock(VeniceHelixAdmin.class); Store mockStore = new ZKStore( @@ -72,13 +73,21 @@ public void testGetRealTimeTopicUsesAdmin() throws Exception { HybridStoreConfigImpl.DEFAULT_HYBRID_TIME_LAG_THRESHOLD, DataReplicationPolicy.NON_AGGREGATE, BufferReplayPolicy.REWIND_FROM_EOP)); + Version hybridVersion = new VersionImpl("store", 1, "pushJobId-1234", 33); + hybridVersion.setHybridStoreConfig(mockStore.getHybridStoreConfig()); + hybridVersion.setStatus(VersionStatus.ONLINE); + mockStore.addVersion(hybridVersion); + + // check store partition count is different from hybrid version partition count so that we can verify the + // partition count is updated to the hybrid version partition count in response + Assert.assertNotEquals(mockStore.getPartitionCount(), hybridVersion.getPartitionCount()); + doReturn(mockStore).when(admin).getStore(anyString(), anyString()); doReturn(true).when(admin).isLeaderControllerFor(anyString()); doReturn(1).when(admin).getReplicationFactor(anyString(), anyString()); doReturn(1).when(admin).calculateNumberOfPartitions(anyString(), anyString()); doReturn("kafka-bootstrap").when(admin).getKafkaBootstrapServers(anyBoolean()); - doReturn("store_rt").when(admin).getRealTimeTopic(anyString(), anyString()); - doReturn("store_rt").when(admin).getRealTimeTopic(anyString(), any(Store.class)); + doReturn(hybridVersion).when(admin).getReferenceVersionForStreamingWrites(anyString(), anyString(), any()); // Add a banned route not relevant to the test just to make sure theres coverage for unbanned routes still be // accessible AdminSparkServer server = @@ -110,6 +119,7 @@ public void testGetRealTimeTopicUsesAdmin() throws Exception { // verify response, note we expect same topic, "store_rt" Assert.assertFalse(responseObject.isError(), "unexpected error: " + responseObject.getError()); Assert.assertEquals(responseObject.getKafkaTopic(), "store_rt"); + Assert.assertEquals(responseObject.getPartitions(), hybridVersion.getPartitionCount()); server.stop(); } @@ -139,8 +149,7 @@ public void testBannedRoutesAreRejected() throws Exception { doReturn(1).when(admin).getReplicationFactor(anyString(), anyString()); doReturn(1).when(admin).calculateNumberOfPartitions(anyString(), anyString()); doReturn("kafka-bootstrap").when(admin).getKafkaBootstrapServers(anyBoolean()); - doReturn("store_rt").when(admin).getRealTimeTopic(anyString(), anyString()); - doReturn("store_rt").when(admin).getRealTimeTopic(anyString(), any(Store.class)); + doReturn("store_rt").when(admin).getRealTimeTopic(anyString(), anyString(), any()); AdminSparkServer server = ServiceFactory.getMockAdminSparkServer(admin, "clustername", Arrays.asList(ControllerRoute.REQUEST_TOPIC)); int port = server.getPort(); @@ -217,8 +226,7 @@ public void testAAIncrementalPushRTSourceRegion(boolean sourceGridFabricPresent, doReturn(corpRegionKafka).when(admin).getKafkaBootstrapServers(anyBoolean()); doReturn(true).when(admin).whetherEnableBatchPushFromAdmin(anyString()); doReturn(true).when(admin).isActiveActiveReplicationEnabledInAllRegion(clusterName, storeName, false); - doReturn(Utils.getRealTimeTopicName(mockStore)).when(admin).getRealTimeTopic(anyString(), anyString()); - doReturn(Utils.getRealTimeTopicName(mockStore)).when(admin).getRealTimeTopic(anyString(), any(Store.class)); + doReturn(Version.composeRealTimeTopic(storeName)).when(admin).getRealTimeTopic(anyString(), anyString(), any()); doReturn(corpRegionKafka).when(admin).getNativeReplicationKafkaBootstrapServerAddress(corpRegion); doReturn(emergencySourceRegionKafka).when(admin) .getNativeReplicationKafkaBootstrapServerAddress(emergencySourceRegion); @@ -330,19 +338,23 @@ public void testSamzaReplicationPolicyMode(boolean samzaPolicy, boolean storePol DataReplicationPolicy.NON_AGGREGATE, BufferReplayPolicy.REWIND_FROM_EOP)); } + Version hybridVersion = new VersionImpl("store", 1, "pushJobId-1234", 33); + hybridVersion.setHybridStoreConfig(mockStore.getHybridStoreConfig()); + hybridVersion.setStatus(VersionStatus.ONLINE); + mockStore.addVersion(hybridVersion); + doReturn(mockStore).when(admin).getStore(anyString(), anyString()); doReturn(true).when(admin).isLeaderControllerFor(anyString()); doReturn(1).when(admin).getReplicationFactor(anyString(), anyString()); doReturn(1).when(admin).calculateNumberOfPartitions(anyString(), anyString()); doReturn("kafka-bootstrap").when(admin).getKafkaBootstrapServers(anyBoolean()); - doReturn("store_rt").when(admin).getRealTimeTopic(anyString(), anyString()); - doReturn("store_rt").when(admin).getRealTimeTopic(anyString(), any(Store.class)); + doReturn(hybridVersion).when(admin).getReferenceVersionForStreamingWrites(anyString(), anyString(), anyString()); doReturn(samzaPolicy).when(admin).isParent(); doReturn(ParentControllerRegionState.ACTIVE).when(admin).getParentControllerRegionState(); doReturn(aaEnabled).when(admin).isActiveActiveReplicationEnabledInAllRegion(anyString(), anyString(), eq(true)); mockStore.setActiveActiveReplicationEnabled(aaEnabled); - // Add a banned route not relevant to the test just to make sure theres coverage for unbanned routes still be + // Add a banned route not relevant to the test just to make sure there is coverage for unbanned routes still be // accessible AdminSparkServer server = ServiceFactory.getMockAdminSparkServer(admin, "clustername", Arrays.asList(ControllerRoute.ADD_DERIVED_SCHEMA)); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/BlobP2PTransferAmongServersTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/BlobP2PTransferAmongServersTest.java index 621e4a8a6ee..e04d4c4c389 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/BlobP2PTransferAmongServersTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/BlobP2PTransferAmongServersTest.java @@ -342,9 +342,9 @@ public void testBlobP2PTransferAmongServersForHybridStore() throws Exception { veniceProducer.stop(); } + cluster.restartVeniceServer(server1.getPort()); // restart server 1 TestUtils.waitForNonDeterministicAssertion(2, TimeUnit.MINUTES, () -> { - cluster.restartVeniceServer(server1.getPort()); Assert.assertTrue(server1.isRunning()); }); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java index b50bc298565..63fb50cbaf5 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java @@ -21,6 +21,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.StoreResponse; @@ -37,8 +38,11 @@ import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.PubSubTopicRepository; +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.samza.VeniceSystemFactory; +import com.linkedin.venice.samza.VeniceSystemProducer; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.TestWriteUtils; @@ -47,6 +51,7 @@ import java.io.File; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.concurrent.CompletableFuture; @@ -55,6 +60,7 @@ import org.apache.avro.Schema; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.samza.config.MapConfig; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -134,6 +140,135 @@ public void testAAReplicationForIncPush() throws Exception { testAAReplicationForIncPush(false); } + /** + * This test reproduces an issue where the real-time topic partition count did not match the hybrid version + * partition count under the following scenario: + * + * 1. Create a store with 1 partition. + * 2. Perform a batch push, resulting in a batch version with 1 partition. + * 3. Update the store to have 3 partitions and convert it into a hybrid store. + * 4. Start real-time writes using push type {@link com.linkedin.venice.meta.Version.PushType#STREAM}. + * 5. Perform a full push, which creates a hybrid version with 3 partitions. This push results in an error + * because, after the topic switch to real-time consumers, partitions 1 and 2 of the real-time topic cannot + * be found, as it has only 1 partition (partition: 0). + * + * The root cause of the issue lies in step 4, where the real-time topic was created if it did not already exist. + * The partition count for the real-time topic was derived from the largest existing version, which in this case + * was the batch version with 1 partition. This caused the real-time topic to have incorrect partition count (1 + * instead of 3). + * + * To resolve this issue: + * - STREAM push type is no longer allowed if there is no online hybrid version. + * - If there is an online hybrid version, it is safe to assume that the real-time topic partition count matches + * the hybrid version partition count. + * - The real-time topic is no longer created if it does not exist as part of the `requestTopicForPushing` method. + */ + @Test(timeOut = TEST_TIMEOUT) + public void testRealTimeTopicPartitionCountMatchesHybridVersion() throws Exception { + File inputDirBatch = getTempDataDirectory(); + String parentControllerUrls = multiRegionMultiClusterWrapper.getControllerConnectString(); + String inputDirPathBatch = "file:" + inputDirBatch.getAbsolutePath(); + try (ControllerClient parentControllerClient = new ControllerClient(clusterName, parentControllerUrls)) { + String storeName = Utils.getUniqueString("store"); + Properties propsBatch = + IntegrationTestPushUtils.defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPathBatch, storeName); + propsBatch.put(SEND_CONTROL_MESSAGES_DIRECTLY, true); + Schema recordSchema = TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema(inputDirBatch); + String keySchemaStr = recordSchema.getField(DEFAULT_KEY_FIELD_PROP).schema().toString(); + String valueSchemaStr = recordSchema.getField(DEFAULT_VALUE_FIELD_PROP).schema().toString(); + + TestUtils.assertCommand(parentControllerClient.createNewStore(storeName, "owner", keySchemaStr, valueSchemaStr)); + UpdateStoreQueryParams updateStoreParams1 = + new UpdateStoreQueryParams().setStorageQuotaInByte(Store.UNLIMITED_STORAGE_QUOTA).setPartitionCount(1); + TestUtils.assertCommand(parentControllerClient.updateStore(storeName, updateStoreParams1)); + + // Run a batch push first to create a batch version with 1 partition + try (VenicePushJob job = new VenicePushJob("Test push job batch with NR + A/A all fabrics", propsBatch)) { + job.run(); + } + + // wait until version is created and verify the partition count + TestUtils.waitForNonDeterministicAssertion(1, TimeUnit.MINUTES, () -> { + StoreResponse storeResponse = assertCommand(parentControllerClient.getStore(storeName)); + StoreInfo storeInfo = storeResponse.getStore(); + assertNotNull(storeInfo, "Store info is null."); + assertNull(storeInfo.getHybridStoreConfig(), "Hybrid store config is not null."); + assertNotNull(storeInfo.getVersion(1), "Version 1 is not present."); + Optional version = storeInfo.getVersion(1); + assertTrue(version.isPresent(), "Version 1 is not present."); + assertNull(version.get().getHybridStoreConfig(), "Version level hybrid store config is not null."); + assertEquals(version.get().getPartitionCount(), 1, "Partition count is not 1."); + }); + + // Update the store to have 3 partitions and convert it into a hybrid store + UpdateStoreQueryParams updateStoreParams = + new UpdateStoreQueryParams().setStorageQuotaInByte(Store.UNLIMITED_STORAGE_QUOTA) + .setPartitionCount(3) + .setHybridOffsetLagThreshold(TEST_TIMEOUT / 2) + .setHybridRewindSeconds(2L); + TestUtils.assertCommand(parentControllerClient.updateStore(storeName, updateStoreParams)); + + TestUtils.waitForNonDeterministicAssertion(1, TimeUnit.MINUTES, () -> { + StoreResponse storeResponse = assertCommand(parentControllerClient.getStore(storeName)); + StoreInfo storeInfo = storeResponse.getStore(); + assertNotNull(storeInfo, "Store info is null."); + assertNotNull(storeInfo.getHybridStoreConfig(), "Hybrid store config is null."); + // verify that there is just one version and it is batch version + assertEquals(storeInfo.getVersions().size(), 1, "Version count is not 1."); + Optional version = storeInfo.getVersion(1); + assertTrue(version.isPresent(), "Version 1 is not present."); + assertNull(version.get().getHybridStoreConfig(), "Version level hybrid store config is not null."); + assertEquals(version.get().getPartitionCount(), 1, "Partition count is not 1."); + }); + + // Push job step was disabled to reproduce the issue + // Run a full push to create a hybrid version with 3 partitions + try (VenicePushJob job = new VenicePushJob("push_job_to_create_hybrid_version", propsBatch)) { + job.run(); + } + + // wait until hybrid version is created and verify the partition count + TestUtils.waitForNonDeterministicAssertion(1, TimeUnit.MINUTES, () -> { + StoreResponse storeResponse = assertCommand(parentControllerClient.getStore(storeName)); + StoreInfo storeInfo = storeResponse.getStore(); + assertNotNull(storeInfo, "Store info is null."); + assertNotNull(storeInfo.getHybridStoreConfig(), "Hybrid store config is null."); + assertNotNull(storeInfo.getVersion(2), "Version 2 is not present."); + Optional version = storeInfo.getVersion(2); + assertTrue(version.isPresent(), "Version 2 is not present."); + assertNotNull(version.get().getHybridStoreConfig(), "Version level hybrid store config is null."); + assertEquals(version.get().getPartitionCount(), 3, "Partition count is not 3."); + }); + + VeniceSystemFactory factory = new VeniceSystemFactory(); + Map samzaConfig = IntegrationTestPushUtils.getSamzaProducerConfig(childDatacenters, 1, storeName); + VeniceSystemProducer veniceProducer = factory.getClosableProducer("venice", new MapConfig(samzaConfig), null); + veniceProducer.start(); + + PubSubTopicRepository pubSubTopicRepository = + childDatacenters.get(1).getClusters().get(clusterName).getPubSubTopicRepository(); + PubSubTopic realTimeTopic = pubSubTopicRepository.getTopic(Version.composeRealTimeTopic(storeName)); + + // wait for 120 secs and check producer getTopicName + TestUtils.waitForNonDeterministicAssertion(2, TimeUnit.MINUTES, () -> { + Assert.assertEquals(veniceProducer.getTopicName(), realTimeTopic.getName()); + }); + + try (TopicManager topicManager = + IntegrationTestPushUtils + .getTopicManagerRepo( + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, + 100, + 0l, + childDatacenters.get(1).getClusters().get(clusterName).getPubSubBrokerWrapper(), + pubSubTopicRepository) + .getLocalTopicManager()) { + int partitionCount = topicManager.getPartitionCount(realTimeTopic); + assertEquals(partitionCount, 3, "Partition count is not 3."); + } + } + } + /** * The purpose of this test is to verify that incremental push with RT policy succeeds when A/A is enabled in all * regions. And also incremental push can push to the closes kafka cluster from the grid using the SOURCE_GRID_CONFIG. diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/router/TestRead.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/router/TestRead.java index 690c35a9ba8..79968b4f759 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/router/TestRead.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/router/TestRead.java @@ -156,6 +156,7 @@ public void setUp() throws VeniceClientException, ExecutionException, Interrupte extraProperties.put(ConfigKeys.ROUTER_HTTP2_INBOUND_ENABLED, isRouterHttp2Enabled()); extraProperties.put(ConfigKeys.SERVER_HTTP2_INBOUND_ENABLED, true); extraProperties.put(ConfigKeys.ROUTER_PER_STORE_ROUTER_QUOTA_BUFFER, 0.0); + extraProperties.put(ConfigKeys.PARTICIPANT_MESSAGE_STORE_ENABLED, false); veniceCluster = ServiceFactory.getVeniceCluster(1, 1, 1, 2, 100, true, false, extraProperties); routerAddr = veniceCluster.getRandomRouterSslURL(); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/server/VeniceServerTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/server/VeniceServerTest.java index 68aa27f4960..6fb1c6a615c 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/server/VeniceServerTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/server/VeniceServerTest.java @@ -118,15 +118,13 @@ public void testCheckBeforeJoinCluster() { try (VeniceClusterWrapper cluster = ServiceFactory.getVeniceCluster(1, 1, 0)) { VeniceServerWrapper server = cluster.getVeniceServers().get(0); StorageEngineRepository repository = server.getVeniceServer().getStorageService().getStorageEngineRepository(); - Assert - .assertTrue(repository.getAllLocalStorageEngines().isEmpty(), "New node should not have any storage engine."); // Create a storage engine. String storeName = cluster.createStore(10); - Assert.assertNotEquals( - repository.getAllLocalStorageEngines().size(), - 0, - "We have created one storage engine for store: " + storeName); + String storeVersionName = Version.composeKafkaTopic(storeName, 1); + Assert.assertNotNull( + repository.getLocalStorageEngine(storeVersionName), + "Storage engine should be created for: " + storeVersionName); // Restart server, as server's info leave in Helix cluster, so we expect that all local storage would NOT be // deleted @@ -136,6 +134,9 @@ public void testCheckBeforeJoinCluster() { repository = server.getVeniceServer().getStorageService().getStorageEngineRepository(); Assert .assertNotEquals(repository.getAllLocalStorageEngines().size(), 0, "We should not cleanup the local storage"); + Assert.assertNotNull( + repository.getLocalStorageEngine(storeVersionName), + "Storage engine should be created for: " + storeVersionName); // Stop server, remove it from the cluster then restart. We expect that all local storage would be deleted. Once // the server join again. @@ -163,12 +164,11 @@ public void testCheckBeforeJoinCluster() { TestUtils.waitForNonDeterministicAssertion( 30, TimeUnit.SECONDS, - () -> Assert.assertTrue( + () -> Assert.assertNull( server.getVeniceServer() .getStorageService() .getStorageEngineRepository() - .getAllLocalStorageEngines() - .isEmpty(), + .getLocalStorageEngine(storeVersionName), "After removing the node from cluster, local storage should be cleaned up once the server join the cluster again.")); } } @@ -210,12 +210,12 @@ public void testStartServerAndShutdownWithPartitionAssignmentVerification() { Assert.assertTrue(server.getVeniceServer().isStarted()); StorageService storageService = server.getVeniceServer().getStorageService(); StorageEngineRepository repository = storageService.getStorageEngineRepository(); - Assert - .assertTrue(repository.getAllLocalStorageEngines().isEmpty(), "New node should not have any storage engine."); // Create a storage engine. String storeName = Version.composeKafkaTopic(cluster.createStore(1), 1); - Assert.assertEquals(repository.getAllLocalStorageEngines().size(), 1); + Assert.assertNotNull( + repository.getLocalStorageEngine(storeName), + "Storage engine should be created for: " + storeName); Assert.assertTrue(server.getVeniceServer().getHelixParticipationService().isRunning()); Assert.assertEquals(storageService.getStorageEngine(storeName).getPartitionIds().size(), 3); @@ -404,18 +404,14 @@ public void testDropStorePartitionAsynchronously() { StorageService storageService = server.getVeniceServer().getStorageService(); Assert.assertTrue(server.getVeniceServer().isStarted()); final StorageEngineRepository repository = storageService.getStorageEngineRepository(); - Assert - .assertTrue(repository.getAllLocalStorageEngines().isEmpty(), "New node should not have any storage engine."); - // Create a new store String storeName = cluster.createStore(1); String storeVersionName = Version.composeKafkaTopic(storeName, 1); - Assert.assertEquals(repository.getAllLocalStorageEngines().size(), 1); + Assert.assertNotNull( + storageService.getStorageEngine(storeVersionName), + "Storage engine should be created for: " + storeVersionName); Assert.assertTrue(server.getVeniceServer().getHelixParticipationService().isRunning()); Assert.assertEquals(storageService.getStorageEngine(storeVersionName).getPartitionIds().size(), 3); - - Assert.assertEquals(repository.getAllLocalStorageEngines().size(), 1); - String helixInstanceName = Utils.getHelixNodeIdentifier(Utils.getHostName(), server.getPort()); String instanceOperationReason = "Disable instance to remove all partitions assigned to it"; cluster.getLeaderVeniceController() @@ -429,6 +425,9 @@ public void testDropStorePartitionAsynchronously() { TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, () -> { // All partitions should have been dropped asynchronously due to instance being disabled + Assert.assertNull( + storageService.getStorageEngine(storeVersionName), + "Storage engine: " + storeVersionName + " should have been dropped"); Assert.assertEquals(repository.getAllLocalStorageEngines().size(), 0); }); } @@ -445,14 +444,13 @@ public void testDropStorePartitionSynchronously() { StorageService storageService = server.getVeniceServer().getStorageService(); Assert.assertTrue(server.getVeniceServer().isStarted()); - final StorageEngineRepository repository = storageService.getStorageEngineRepository(); - Assert - .assertTrue(repository.getAllLocalStorageEngines().isEmpty(), "New node should not have any storage engine."); // Create a new store String storeName = cluster.createStore(1); String storeVersionName = Version.composeKafkaTopic(storeName, 1); - Assert.assertEquals(repository.getAllLocalStorageEngines().size(), 1); + Assert.assertNotNull( + storageService.getStorageEngine(storeVersionName), + "Storage engine should be created for: " + storeVersionName); Assert.assertTrue(server.getVeniceServer().getHelixParticipationService().isRunning()); Assert.assertEquals(storageService.getStorageEngine(storeVersionName).getPartitionIds().size(), 3); @@ -462,7 +460,9 @@ public void testDropStorePartitionSynchronously() { TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, () -> { // All partitions should have been dropped synchronously - Assert.assertEquals(repository.getAllLocalStorageEngines().size(), 0); + Assert.assertNull( + storageService.getStorageEngine(storeVersionName), + "Storage engine: " + storeVersionName + " should have been dropped"); }); } } diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java index 2358c09762b..9ea86030a1f 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java @@ -9,7 +9,6 @@ import com.linkedin.venice.controllerapi.UpdateClusterConfigQueryParams; import com.linkedin.venice.controllerapi.UpdateStoragePersonaQueryParams; import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; -import com.linkedin.venice.exceptions.VeniceNoStoreException; import com.linkedin.venice.helix.HelixReadOnlyStoreConfigRepository; import com.linkedin.venice.helix.HelixReadOnlyZKSharedSchemaRepository; import com.linkedin.venice.helix.HelixReadOnlyZKSharedSystemStoreRepository; @@ -289,26 +288,9 @@ Version incrementVersionIdempotent( String targetedRegions, int repushSourceVersion); - String getRealTimeTopic(String clusterName, Store store); + Version getIncrementalPushVersion(String clusterName, String storeName, String pushJobId); - default String getRealTimeTopic(String clusterName, String storeName) { - Store store = getStore(clusterName, storeName); - if (store == null) { - throw new VeniceNoStoreException(storeName, clusterName); - } - return getRealTimeTopic(clusterName, store); - } - - String getSeparateRealTimeTopic(String clusterName, String storeName); - - /** - * Right now, it will return the latest version recorded in parent controller. There are a couple of edge cases. - * 1. If a push fails in some colos, the version will be inconsistent among colos - * 2. If rollback happens, latest version will not be the current version. - * - * TODO: figure out how we'd like to cover these edge cases - */ - Version getIncrementalPushVersion(String clusterName, String storeName); + Version getReferenceVersionForStreamingWrites(String clusterName, String storeName, String pushJobId); int getCurrentVersion(String clusterName, String storeName); 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 827c9024e37..4ee6b4cfea7 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 @@ -826,7 +826,7 @@ public VeniceControllerClusterConfig(VeniceProperties props) { this.parentControllerMaxErroredTopicNumToKeep = props.getInt(PARENT_CONTROLLER_MAX_ERRORED_TOPIC_NUM_TO_KEEP, 0); this.pushJobStatusStoreClusterName = props.getString(PUSH_JOB_STATUS_STORE_CLUSTER_NAME, ""); - this.participantMessageStoreEnabled = props.getBoolean(PARTICIPANT_MESSAGE_STORE_ENABLED, false); + this.participantMessageStoreEnabled = props.getBoolean(PARTICIPANT_MESSAGE_STORE_ENABLED, true); this.adminHelixMessagingChannelEnabled = props.getBoolean(ADMIN_HELIX_MESSAGING_CHANNEL_ENABLED, true); if (!adminHelixMessagingChannelEnabled && !participantMessageStoreEnabled) { throw new VeniceException( diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceHelixAdmin.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceHelixAdmin.java index 66329256d61..9b8ab681ea4 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceHelixAdmin.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceHelixAdmin.java @@ -641,6 +641,7 @@ public VeniceHelixAdmin( // Participant stores are not read or written in parent colo. Parent controller skips participant store // initialization. if (!isParent() && multiClusterConfigs.isParticipantMessageStoreEnabled()) { + LOGGER.info("Adding PerClusterInternalRTStoreInitializationRoutine for ParticipantMessageStore"); initRoutines.add( new PerClusterInternalRTStoreInitializationRoutine( PARTICIPANT_MESSAGE_SYSTEM_STORE_VALUE, @@ -2776,46 +2777,8 @@ private Pair addVersion( version.setPushStreamSourceAddress(sourceKafkaBootstrapServers); version.setNativeReplicationSourceFabric(sourceFabric); } - if (isParent() && ((store.isHybrid() - && store.getHybridStoreConfig().getDataReplicationPolicy() == DataReplicationPolicy.AGGREGATE) - || store.isIncrementalPushEnabled())) { - // Create rt topic in parent colo if the store is aggregate mode hybrid store - PubSubTopic realTimeTopic = pubSubTopicRepository.getTopic(Utils.getRealTimeTopicName(store)); - if (!getTopicManager().containsTopic(realTimeTopic)) { - getTopicManager().createTopic( - realTimeTopic, - numberOfPartitions, - clusterConfig.getKafkaReplicationFactorRTTopics(), - StoreUtils.getExpectedRetentionTimeInMs(store, store.getHybridStoreConfig()), - false, - // Note: do not enable RT compaction! Might make jobs in Online/Offline model stuck - clusterConfig.getMinInSyncReplicasRealTimeTopics(), - false); - if (version.isSeparateRealTimeTopicEnabled()) { - getTopicManager().createTopic( - pubSubTopicRepository.getTopic(Version.composeSeparateRealTimeTopic(storeName)), - numberOfPartitions, - clusterConfig.getKafkaReplicationFactorRTTopics(), - StoreUtils.getExpectedRetentionTimeInMs(store, store.getHybridStoreConfig()), - false, - // Note: do not enable RT compaction! Might make jobs in Online/Offline model stuck - clusterConfig.getMinInSyncReplicasRealTimeTopics(), - false); - } - } else { - // If real-time topic already exists, check whether its retention time is correct. - PubSubTopicConfiguration pubSubTopicConfiguration = - getTopicManager().getCachedTopicConfig(realTimeTopic); - long topicRetentionTimeInMs = TopicManager.getTopicRetention(pubSubTopicConfiguration); - long expectedRetentionTimeMs = - StoreUtils.getExpectedRetentionTimeInMs(store, store.getHybridStoreConfig()); - if (topicRetentionTimeInMs != expectedRetentionTimeMs) { - getTopicManager() - .updateTopicRetention(realTimeTopic, expectedRetentionTimeMs, pubSubTopicConfiguration); - } - } - } } + /** * Version-level rewind time override. */ @@ -2847,6 +2810,9 @@ private Pair addVersion( constructViewResources(veniceViewProperties, store, version.getNumber()); repository.updateStore(store); + if (isRealTimeTopicRequired(store, version)) { + createOrUpdateRealTimeTopics(clusterName, store, version); + } LOGGER.info("Add version: {} for store: {}", version.getNumber(), storeName); /** @@ -3005,6 +2971,172 @@ private Pair addVersion( } } + /** + * Determines whether real-time topics should be created for the given store and version. + * + *

Real-time topics are created based on the following conditions: + *

    + *
  • The store and version must both be hybrid ({@code store.isHybrid()} and {@code version.isHybrid()}).
  • + *
  • If the controller is a child, real-time topics are always created.
  • + *
  • If the controller is a parent, real-time topics are created only if: + *
      + *
    • Active-active replication is disabled for the store, and
    • + *
    • Either the store's data replication policy is {@code DataReplicationPolicy.AGGREGATE}, or
    • + *
    • Incremental push is enabled for the store.
    • + *
    + *
  • + *
+ * + * @param store the store being evaluated + * @param version the version being evaluated + * @return {@code true} if real-time topics should be created; {@code false} otherwise + */ + boolean isRealTimeTopicRequired(Store store, Version version) { + if (!store.isHybrid() || !version.isHybrid()) { + return false; + } + + // Child controllers always create real-time topics for hybrid stores in their region + if (!isParent()) { + return true; + } + + // Parent controllers create real-time topics in the parent region only under certain conditions + return !store.isActiveActiveReplicationEnabled() + && (store.getHybridStoreConfig().getDataReplicationPolicy() == DataReplicationPolicy.AGGREGATE + || store.isIncrementalPushEnabled()); + } + + /** + * Creates or updates real-time topics for the specified store (using reference hybrid version) in the given cluster. + * + *

This method ensures that real-time topics (primary and separate, if applicable) are configured + * correctly for a hybrid store. It creates the topics if they do not exist and updates their retention + * time if necessary. For stores with separate real-time topics enabled, the method handles the creation + * or update of those topics as well. + * + * @param clusterName the name of the cluster where the topics are managed + * @param store the {@link Store} associated with the topics + * @param version the {@link Version} containing the configuration for the topics, including partition count + * and hybrid store settings + */ + void createOrUpdateRealTimeTopics(String clusterName, Store store, Version version) { + LOGGER.info( + "Setting up real-time topics for store: {} with reference hybrid version: {} in cluster: {}", + store.getName(), + version.getNumber(), + clusterName); + String storeName = store.getName(); + // Create real-time topic if it doesn't exist; otherwise, update the retention time if necessary + PubSubTopic realTimeTopic = pubSubTopicRepository.getTopic(Utils.getRealTimeTopicName(version)); + createOrUpdateRealTimeTopic(clusterName, store, version, realTimeTopic); + + // Create separate real-time topic if it doesn't exist; otherwise, update the retention time if necessary + if (version.isSeparateRealTimeTopicEnabled()) { + // TODO: Add support for repartitioning separate real-time topics, primarily needed for incremental push jobs. + createOrUpdateRealTimeTopic( + clusterName, + store, + version, + pubSubTopicRepository.getTopic(Version.composeSeparateRealTimeTopic(storeName))); + } + } + + /** + * Creates or updates a real-time topic for a given store in the specified cluster. + * + *

This method ensures that the real-time topic matches the expected configuration based on + * the store's hybrid settings and the associated version. If the topic already exists: + *

    + *
  • It validates the partition count against the expected partition count for the version.
  • + *
  • It updates the retention time if necessary.
  • + *
+ * If the topic does not exist, it creates the topic with the required configuration. + * + * @param clusterName the name of the cluster where the topic resides + * @param store the {@link Store} store to which the topic belongs + * @param version the reference hybrid {@link Version} containing + * @param realTimeTopic the {@link PubSubTopic} representing the real-time topic + * @throws VeniceException if the partition count of an existing topic does not match the expected value + */ + void createOrUpdateRealTimeTopic(String clusterName, Store store, Version version, PubSubTopic realTimeTopic) { + int expectedNumOfPartitions = version.getPartitionCount(); + TopicManager topicManager = getTopicManager(); + if (topicManager.containsTopic(realTimeTopic)) { + validateAndUpdateTopic(realTimeTopic, store, version, expectedNumOfPartitions, getTopicManager()); + } else { + VeniceControllerClusterConfig clusterConfig = multiClusterConfigs.getControllerConfig(clusterName); + topicManager.createTopic( + realTimeTopic, + expectedNumOfPartitions, + clusterConfig.getKafkaReplicationFactorRTTopics(), + StoreUtils.getExpectedRetentionTimeInMs(store, store.getHybridStoreConfig()), + false, + // Note: do not enable RT compaction! Might make jobs in Online/Offline model stuck + clusterConfig.getMinInSyncReplicasRealTimeTopics(), + false); + } + LOGGER.info( + "Completed setup for real-time topic: {} for store: {} with reference hybrid version: {} and partition count: {}", + realTimeTopic.getName(), + store.getName(), + version.getNumber(), + expectedNumOfPartitions); + } + + /** + * Validates the real-time topic's configuration and updates its retention time if necessary. + * + *

This method checks if the partition count of the real-time topic matches the expected partition count + * for the specified version. If the counts do not match, an exception is thrown. Additionally, it validates + * the topic's retention time against the expected retention time and updates it if required. + * + * @param realTimeTopic the {@link PubSubTopic} representing the real-time topic to validate + * @param store the {@link Store} store to which the topic belongs + * @param version the reference hybrid {@link Version} + * @param expectedNumOfPartitions the expected number of partitions for the real-time topic + * @param topicManager the {@link TopicManager} used for topic management operations + * @throws VeniceException if the partition count of the topic does not match the expected partition count + */ + private void validateAndUpdateTopic( + PubSubTopic realTimeTopic, + Store store, + Version version, + int expectedNumOfPartitions, + TopicManager topicManager) { + int actualNumOfPartitions = topicManager.getPartitionCount(realTimeTopic); + // Validate partition count + if (actualNumOfPartitions != expectedNumOfPartitions) { + LOGGER.error( + "Real-time topic: {} for store: {} has different partition count: {} from version partition count: {} version: {} store: {}", + realTimeTopic.getName(), + store.getName(), + actualNumOfPartitions, + expectedNumOfPartitions, + version, + store); + String errorMessage = String.format( + "Real-time topic: %s for store: %s has different partition count: %d from version partition count: %d", + realTimeTopic.getName(), + store.getName(), + actualNumOfPartitions, + expectedNumOfPartitions); + throw new VeniceException(errorMessage); + } + + // Validate and update retention time if necessary + HybridStoreConfig hybridStoreConfig = store.getHybridStoreConfig(); + long expectedRetentionTimeMs = StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig); + boolean isUpdated = topicManager.updateTopicRetentionWithRetries(realTimeTopic, expectedRetentionTimeMs); + LOGGER.info( + "{} retention time for real-time topic: {} for store: {} (hybrid version: {}, partition count: {})", + isUpdated ? "Updated" : "Validated", + realTimeTopic.getName(), + store.getName(), + version.getNumber(), + expectedNumOfPartitions); + } + /** * During store migration, skip a version if: * This is the child controller of the destination cluster @@ -3073,7 +3205,7 @@ public Version incrementVersionIdempotent( VeniceControllerClusterConfig clusterConfig = getHelixVeniceClusterResources(clusterName).getConfig(); int replicationMetadataVersionId = clusterConfig.getReplicationMetadataVersion(); return pushType.isIncremental() - ? getIncrementalPushVersion(clusterName, storeName) + ? getIncrementalPushVersion(clusterName, storeName, pushJobId) : addVersion( clusterName, storeName, @@ -3163,36 +3295,61 @@ private Optional getVersionWithPushId(String clusterName, String storeN * Get the real time topic name for a given store. If the topic is not created in Kafka, it creates the * real time topic and returns the topic name. * @param clusterName name of the Venice cluster. - * @param store store. + * @param store name of the store. * @return name of the store's real time topic name. */ - @Override - public String getRealTimeTopic(String clusterName, Store store) { - checkControllerLeadershipFor(clusterName); + public String getRealTimeTopic(String clusterName, Store store, Integer expectedPartitionCount) { PubSubTopic realTimeTopic = pubSubTopicRepository.getTopic(Utils.getRealTimeTopicName(store)); - ensureRealTimeTopicIsReady(clusterName, realTimeTopic); + ensureRealTimeTopicIsReady(clusterName, realTimeTopic, expectedPartitionCount); return realTimeTopic.getName(); } - @Override - public String getSeparateRealTimeTopic(String clusterName, String storeName) { - checkControllerLeadershipFor(clusterName); + public String getRealTimeTopic(String clusterName, String storeName, Integer expectedPartitionCount) { + Store store = getStore(clusterName, storeName); + if (store == null) { + throw new VeniceNoStoreException(storeName, clusterName); + } + return getRealTimeTopic(clusterName, store, expectedPartitionCount); + } + + public String getSeparateRealTimeTopic(String clusterName, String storeName, Integer expectedPartitionCount) { PubSubTopic incrementalPushRealTimeTopic = pubSubTopicRepository.getTopic(Version.composeSeparateRealTimeTopic(storeName)); - ensureRealTimeTopicIsReady(clusterName, incrementalPushRealTimeTopic); + ensureRealTimeTopicIsReady(clusterName, incrementalPushRealTimeTopic, expectedPartitionCount); return incrementalPushRealTimeTopic.getName(); } - private void ensureRealTimeTopicIsReady(String clusterName, PubSubTopic realTimeTopic) { + private void ensureRealTimeTopicIsReady( + String clusterName, + PubSubTopic realTimeTopic, + Integer expectedPartitionCount) { + checkControllerLeadershipFor(clusterName); TopicManager topicManager = getTopicManager(); String storeName = realTimeTopic.getStoreName(); if (!topicManager.containsTopic(realTimeTopic)) { HelixVeniceClusterResources resources = getHelixVeniceClusterResources(clusterName); try (AutoCloseableLock ignore = resources.getClusterLockManager().createStoreWriteLock(storeName)) { - // The topic might be created by another thread already. Check before creating. - if (topicManager.containsTopic(realTimeTopic)) { + boolean isRealTimeTopicCreated = topicManager.containsTopic(realTimeTopic); + if (isRealTimeTopicCreated && expectedPartitionCount != null) { + int actualPartitionCount = topicManager.getPartitionCount(realTimeTopic); + if (actualPartitionCount == expectedPartitionCount) { + // Topic is already created with the expected partition count + return; + } + + LOGGER.error( + "Real time topic: {} has partition count: {} but expected partition count is: {}", + realTimeTopic.getName(), + actualPartitionCount, + expectedPartitionCount); + throw new VeniceException( + "Real time topic " + realTimeTopic.getName() + " has partition count " + actualPartitionCount + + " but expected partition count is " + expectedPartitionCount); + } else if (isRealTimeTopicCreated) { + // real-time topic exists but return; } + ReadWriteStoreRepository repository = resources.getStoreMetadataRepository(); Store store = repository.getStore(storeName); if (store == null) { @@ -3217,6 +3374,16 @@ private void ensureRealTimeTopicIsReady(String clusterName, PubSubTopic realTime } } + if (expectedPartitionCount != null && partitionCount != expectedPartitionCount) { + LOGGER.error( + "Version partition count: {} does not match expected partition count: {} " + + "for store: {}. Will use expected partition count to create real time topic", + partitionCount, + expectedPartitionCount, + storeName); + partitionCount = expectedPartitionCount; + } + VeniceControllerClusterConfig clusterConfig = getHelixVeniceClusterResources(clusterName).getConfig(); getTopicManager().createTopic( realTimeTopic, @@ -3257,15 +3424,20 @@ public Optional getReplicationMetadataSchema( } /** - * @see Admin#getIncrementalPushVersion(String, String) + * @see Admin#getIncrementalPushVersion(String, String, String) */ @Override - public Version getIncrementalPushVersion(String clusterName, String storeName) { + public Version getIncrementalPushVersion(String clusterName, String storeName, String pushJobId) { checkControllerLeadershipFor(clusterName); HelixVeniceClusterResources resources = getHelixVeniceClusterResources(clusterName); try (AutoCloseableLock ignore = resources.getClusterLockManager().createStoreReadLock(storeName)) { Store store = resources.getStoreMetadataRepository().getStore(storeName); if (store == null) { + LOGGER.error( + "Unable to locate version for incremental push: {}. Store: {} does not exist in cluster: {}", + pushJobId, + storeName, + clusterName); throwStoreDoesNotExist(clusterName, storeName); } @@ -3273,29 +3445,159 @@ public Version getIncrementalPushVersion(String clusterName, String storeName) { throw new VeniceException("Incremental push is not enabled for store: " + storeName); } - List versions = store.getVersions(); + List versions = new ArrayList<>(store.getVersions()); if (versions.isEmpty()) { throw new VeniceException("Store: " + storeName + " is not initialized with a version yet"); } - /** - * Don't use {@link Store#getCurrentVersion()} here since it is always 0 in parent controller - */ - Version version = versions.get(versions.size() - 1); - if (version.getStatus() == ERROR) { + Version hybridVersion = null; + versions.sort(Comparator.comparingInt(Version::getNumber).reversed()); + for (Version version: versions) { + if (version.getHybridStoreConfig() != null && version.getStatus() == ONLINE) { + hybridVersion = version; + break; + } + } + + if (hybridVersion == null) { + LOGGER.error( + "Could not find an online hybrid store version for incremental push: {} on store: {} in cluster: {}", + pushJobId, + storeName, + clusterName); throw new VeniceException( - "cannot have incremental push because current version is in error status. " + "Version: " - + version.getNumber() + " Store:" + storeName); + "No ONLINE hybrid store version found for store: " + storeName + " in cluster: " + clusterName); + } + LOGGER.info( + "Found hybrid version: {} for store: {} in cluster: {}. Will use it as a reference for incremental push: {}", + hybridVersion.getNumber(), + storeName, + clusterName, + pushJobId); + + // If real-time topic is not required, no need to check for its presence + if (!isRealTimeTopicRequired(store, hybridVersion)) { + return hybridVersion; } PubSubTopic rtTopic = pubSubTopicRepository.getTopic(Utils.getRealTimeTopicName(store)); if (!getTopicManager().containsTopicAndAllPartitionsAreOnline(rtTopic) || isTopicTruncated(rtTopic.getName())) { + LOGGER.error( + "Incremental push: {} cannot be started for store: {} in cluster: {} because the topic: {} is either absent or being truncated", + pushJobId, + storeName, + clusterName, + rtTopic); resources.getVeniceAdminStats().recordUnexpectedTopicAbsenceCount(); throw new VeniceException( "Incremental push cannot be started for store: " + storeName + " in cluster: " + clusterName + " because the topic: " + rtTopic + " is either absent or being truncated"); } - return version; + + if (hybridVersion.isSeparateRealTimeTopicEnabled()) { + PubSubTopic separateRtTopic = pubSubTopicRepository.getTopic(Version.composeSeparateRealTimeTopic(storeName)); + if (!getTopicManager().containsTopicAndAllPartitionsAreOnline(separateRtTopic) + || isTopicTruncated(separateRtTopic.getName())) { + LOGGER.error( + "Incremental push: {} cannot be started for store: {} in cluster: {} because the topic: {} is either absent or being truncated", + pushJobId, + storeName, + clusterName, + separateRtTopic); + resources.getVeniceAdminStats().recordUnexpectedTopicAbsenceCount(); + throw new VeniceException( + "Incremental push cannot be started for store: " + storeName + " in cluster: " + clusterName + + " because the topic: " + separateRtTopic + " is either absent or being truncated"); + } + } + return hybridVersion; + } + } + + @Override + public Version getReferenceVersionForStreamingWrites(String clusterName, String storeName, String pushJobId) { + boolean requiresVersionToBeOnline = true; + checkControllerLeadershipFor(clusterName); + HelixVeniceClusterResources resources = getHelixVeniceClusterResources(clusterName); + try (AutoCloseableLock ignore = resources.getClusterLockManager().createStoreReadLock(storeName)) { + Store store = resources.getStoreMetadataRepository().getStore(storeName); + if (store == null) { + throwStoreDoesNotExist(clusterName, storeName); + } + if (!store.isHybrid()) { + LOGGER.warn( + "Rejecting request for streaming writes with pushJobId: {} for store: {} in cluster: {} because it is not a hybrid store", + pushJobId, + storeName, + clusterName); + throw new VeniceException( + "Store: " + storeName + " is not a hybrid store and cannot be used for streaming writes"); + } + List versions = new ArrayList<>(store.getVersions()); + if (versions.isEmpty()) { + LOGGER.warn( + "Rejecting request for streaming writes with pushJobId: {} for store: {} in cluster: {} because it is not initialized with a version yet", + pushJobId, + storeName, + clusterName); + throw new VeniceException( + "Streaming writes cannot be started for store: " + storeName + + " because it is not initialized with a version yet"); + } + + Version hybridVersion = null; + + versions.sort(Comparator.comparingInt(Version::getNumber).reversed()); + for (Version version: versions) { + if (version.getHybridStoreConfig() != null && (!requiresVersionToBeOnline || version.getStatus() == ONLINE)) { + hybridVersion = version; + break; + } + } + if (hybridVersion == null) { + String logMessage = String.format( + "Could not find %s hybrid store version for streaming writes with pushJobId: %s on store: %s in cluster: %s", + requiresVersionToBeOnline ? "an ONLINE" : "a", + pushJobId, + storeName, + clusterName); + LOGGER.error(logMessage); + throw new VeniceException(logMessage); + } + + LOGGER.info( + "Found {} hybrid version: {} for store: {} in cluster: {}. Will use it as a reference for streaming writes with pushJobId: {}", + requiresVersionToBeOnline ? "an ONLINE" : "a", + hybridVersion.getNumber(), + storeName, + clusterName, + pushJobId); + + int partitionCount = hybridVersion.getPartitionCount(); + if (isParent()) { + /** + * Create RT topic here in parent region. For child regions, RT should always be created as part of + * {@link RealTimeTopicSwitcher#ensurePreconditions(PubSubTopic, PubSubTopic, Store, Optional)} + */ + // getRealTimeTopic(clusterName, storeName, partitionCount); + System.out.println("getRealTimeTopic"); + } + PubSubTopic rtTopic = pubSubTopicRepository.getTopic(Version.composeRealTimeTopic(storeName)); + if (!getTopicManager().containsTopicAndAllPartitionsAreOnline(rtTopic, partitionCount) + || isTopicTruncated(rtTopic.getName())) { + LOGGER.error( + "Streaming writes: {} cannot be started for store: {} in cluster: {} because the topic: {} is " + + "either absent or being truncated or has different partition count than hybrid version partition count", + pushJobId, + storeName, + clusterName, + rtTopic); + resources.getVeniceAdminStats().recordUnexpectedTopicAbsenceCount(); + throw new VeniceException( + "Streaming writes cannot be started for store: " + storeName + " in cluster: " + clusterName + + " because the topic: " + rtTopic + " is either absent or being truncated"); + } + return hybridVersion; } } @@ -3517,7 +3819,7 @@ private void deleteOneStoreVersion(String clusterName, String storeName, int ver } cleanUpViewResources(new Properties(), store, deletedVersion.get().getNumber()); } - if (store.isDaVinciPushStatusStoreEnabled()) { + if (store.isDaVinciPushStatusStoreEnabled() && !isParent()) { ExecutorService executor = Executors.newSingleThreadExecutor(); Future future = executor.submit( () -> getPushStatusStoreWriter().deletePushStatus( @@ -6161,7 +6463,7 @@ public OfflinePushStatusInfo getOffLinePushStatus( // if status is not SOIP remove incremental push version from the supposedlyOngoingIncrementalPushVersions if (incrementalPushVersion.isPresent() && (status == ExecutionStatus.END_OF_INCREMENTAL_PUSH_RECEIVED || status == ExecutionStatus.NOT_CREATED) - && store.isDaVinciPushStatusStoreEnabled()) { + && store.isDaVinciPushStatusStoreEnabled() && !isParent()) { getPushStatusStoreWriter().removeFromSupposedlyOngoingIncrementalPushVersions( store.getName(), versionNumber, @@ -7351,7 +7653,7 @@ public void close() { void checkControllerLeadershipFor(String clusterName) { if (!isLeaderControllerFor(clusterName)) { throw new VeniceException( - "This controller:" + controllerName + " is not the leader controller for " + clusterName); + "This controller:" + controllerName + " is not the leader controller for cluster: " + clusterName); } } @@ -8017,7 +8319,11 @@ private void setUpDaVinciPushStatusStore(String clusterName, String storeName) { throwStoreDoesNotExist(clusterName, storeName); } String daVinciPushStatusStoreName = VeniceSystemStoreType.DAVINCI_PUSH_STATUS_STORE.getSystemStoreName(storeName); - getRealTimeTopic(clusterName, daVinciPushStatusStoreName); + + if (!isParent()) { + // We do not materialize PS3 for parent region. Hence, skip RT topic creation. + getRealTimeTopic(clusterName, daVinciPushStatusStoreName, null); + } if (!store.isDaVinciPushStatusStoreEnabled()) { storeMetadataUpdate(clusterName, storeName, (s) -> { s.setDaVinciPushStatusStoreEnabled(true); @@ -8026,6 +8332,11 @@ private void setUpDaVinciPushStatusStore(String clusterName, String storeName) { } } + /** + * Set up the meta store and produce snapshot to meta store RT. Should be called in the child controllers. + * @param clusterName The cluster name. + * @param regularStoreName The regular user store name. + */ void setUpMetaStoreAndMayProduceSnapshot(String clusterName, String regularStoreName) { checkControllerLeadershipFor(clusterName); ReadWriteStoreRepository repository = getHelixVeniceClusterResources(clusterName).getStoreMetadataRepository(); @@ -8036,7 +8347,10 @@ void setUpMetaStoreAndMayProduceSnapshot(String clusterName, String regularStore // Make sure RT topic exists before producing. There's no write to parent region meta store RT, but we still create // the RT topic to be consistent in case it was not auto-materialized - getRealTimeTopic(clusterName, VeniceSystemStoreType.META_STORE.getSystemStoreName(regularStoreName)); + if (!isParent()) { + // We do not materialize PS3 for parent region. Hence, skip RT topic creation. + getRealTimeTopic(clusterName, VeniceSystemStoreType.META_STORE.getSystemStoreName(regularStoreName), null); + } // Update the store flag to enable meta system store. if (!store.isStoreMetaSystemStoreEnabled()) { @@ -8415,6 +8729,9 @@ public PushStatusStoreWriter getPushStatusStoreWriter() { @Override public void sendHeartbeatToSystemStore(String clusterName, String storeName, long heartbeatTimeStamp) { + if (isParent()) { + return; + } VeniceSystemStoreType systemStoreType = VeniceSystemStoreType.getSystemStoreType(storeName); String userStoreName = systemStoreType.extractRegularStoreName(storeName); long currentTimestamp = System.currentTimeMillis(); diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java index 74365eb9b50..aec8260b4a0 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java @@ -1532,7 +1532,7 @@ public Version incrementVersionIdempotent( Version newVersion; if (pushType.isIncremental()) { - newVersion = getVeniceHelixAdmin().getIncrementalPushVersion(clusterName, storeName); + newVersion = getVeniceHelixAdmin().getIncrementalPushVersion(clusterName, storeName, pushJobId); } else { validateTargetedRegions(targetedRegions, clusterName); @@ -1709,19 +1709,6 @@ private AddVersion getAddVersionMessage( return addVersion; } - /** - * @see VeniceHelixAdmin#getRealTimeTopic(String, Store) - */ - @Override - public String getRealTimeTopic(String clusterName, Store store) { - return getVeniceHelixAdmin().getRealTimeTopic(clusterName, store); - } - - @Override - public String getSeparateRealTimeTopic(String clusterName, String storeName) { - return getVeniceHelixAdmin().getSeparateRealTimeTopic(clusterName, storeName); - } - /** * A couple of extra checks are needed in parent controller * 1. check batch job statuses across child controllers. (We cannot only check the version status @@ -1730,14 +1717,19 @@ public String getSeparateRealTimeTopic(String clusterName, String storeName) { * preserve incremental push topic in parent Kafka anymore */ @Override - public Version getIncrementalPushVersion(String clusterName, String storeName) { - Version incrementalPushVersion = getVeniceHelixAdmin().getIncrementalPushVersion(clusterName, storeName); + public Version getIncrementalPushVersion(String clusterName, String storeName, String pushJobId) { + Version incrementalPushVersion = getVeniceHelixAdmin().getIncrementalPushVersion(clusterName, storeName, pushJobId); String incrementalPushTopic = incrementalPushVersion.kafkaTopicName(); ExecutionStatus status = getOffLinePushStatus(clusterName, incrementalPushTopic).getExecutionStatus(); return getIncrementalPushVersion(incrementalPushVersion, status); } + @Override + public Version getReferenceVersionForStreamingWrites(String clusterName, String storeName, String pushJobId) { + return getVeniceHelixAdmin().getReferenceVersionForStreamingWrites(clusterName, storeName, pushJobId); + } + // This method is only for internal / test use case Version getIncrementalPushVersion(Version incrementalPushVersion, ExecutionStatus status) { String storeName = incrementalPushVersion.getStoreName(); diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/init/SystemStoreInitializationHelper.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/init/SystemStoreInitializationHelper.java index 60b2589ef02..9dca7d2af3a 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/init/SystemStoreInitializationHelper.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/init/SystemStoreInitializationHelper.java @@ -64,6 +64,7 @@ public static void setupSystemStore( UpdateStoreQueryParams updateStoreQueryParams, Admin admin, VeniceControllerMultiClusterConfig multiClusterConfigs) { + LOGGER.info("Setting up system store: {} in cluster: {}", systemStoreName, clusterName); Map protocolSchemaMap = Utils.getAllSchemasFromResources(protocolDefinition); Store store = admin.getStore(clusterName, systemStoreName); String keySchemaString = keySchema != null ? keySchema.toString() : DEFAULT_KEY_SCHEMA_STR; @@ -86,7 +87,7 @@ public static void setupSystemStore( throw new VeniceException("Unable to create or fetch store " + systemStoreName); } } else { - LOGGER.info("Internal store {} already exists in cluster {}", systemStoreName, clusterName); + LOGGER.info("Internal store: {} already exists in cluster: {}", systemStoreName, clusterName); if (keySchema != null) { /** * Only verify the key schema if it is explicitly specified by the caller, and we don't care @@ -203,6 +204,8 @@ public static void setupSystemStore( LOGGER.info("Created a version for internal store {} in cluster {}", systemStoreName, clusterName); } + + LOGGER.info("System store: {} in cluster: {} is set up", systemStoreName, clusterName); } // Visible for testing diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java index 887710f6d8a..9886dab7534 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java @@ -33,6 +33,7 @@ import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.controller.Admin; import com.linkedin.venice.controllerapi.ControllerResponse; +import com.linkedin.venice.controllerapi.RequestTopicForPushRequest; import com.linkedin.venice.controllerapi.VersionCreationResponse; import com.linkedin.venice.controllerapi.VersionResponse; import com.linkedin.venice.exceptions.ErrorType; @@ -46,11 +47,13 @@ import com.linkedin.venice.meta.Version; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.lazy.Lazy; -import java.security.cert.X509Certificate; +import java.util.Collections; import java.util.Optional; +import java.util.Set; import org.apache.http.HttpStatus; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import spark.Request; import spark.Route; @@ -72,6 +75,362 @@ public CreateVersion( this.disableParentRequestTopicForStreamPushes = disableParentRequestTopicForStreamPushes; } + static void extractOptionalParamsFromRequestTopicRequest( + Request httpRequest, + RequestTopicForPushRequest request, + boolean isAclEnabled) { + request.setPartitioners(httpRequest.queryParamOrDefault(PARTITIONERS, null)); + + request.setSendStartOfPush( + Utils.parseBooleanFromString(httpRequest.queryParamOrDefault(SEND_START_OF_PUSH, "false"), SEND_START_OF_PUSH)); + + request.setSorted( + Utils.parseBooleanFromString( + httpRequest.queryParamOrDefault(PUSH_IN_SORTED_ORDER, "false"), + PUSH_IN_SORTED_ORDER)); + + request.setWriteComputeEnabled( + Utils.parseBooleanFromString( + httpRequest.queryParamOrDefault(IS_WRITE_COMPUTE_ENABLED, "false"), + IS_WRITE_COMPUTE_ENABLED)); + + request.setSeparateRealTimeTopicEnabled( + Utils.parseBooleanFromString( + httpRequest.queryParamOrDefault(SEPARATE_REAL_TIME_TOPIC_ENABLED, "false"), + SEPARATE_REAL_TIME_TOPIC_ENABLED)); + + /* + * Version-level rewind time override, and it is only valid for hybrid stores. + */ + request.setRewindTimeInSecondsOverride( + Long.parseLong(httpRequest.queryParamOrDefault(REWIND_TIME_IN_SECONDS_OVERRIDE, "-1"))); + + /* + * Version level override to defer marking this new version to the serving version post push completion. + */ + request.setDeferVersionSwap( + Utils.parseBooleanFromString(httpRequest.queryParamOrDefault(DEFER_VERSION_SWAP, "false"), DEFER_VERSION_SWAP)); + + request.setTargetedRegions(httpRequest.queryParamOrDefault(TARGETED_REGIONS, null)); + + request.setRepushSourceVersion(Integer.parseInt(httpRequest.queryParamOrDefault(REPUSH_SOURCE_VERSION, "-1"))); + + request.setSourceGridFabric(httpRequest.queryParamOrDefault(SOURCE_GRID_FABRIC, null)); + + request.setCompressionDictionary(httpRequest.queryParamOrDefault(COMPRESSION_DICTIONARY, null)); + + // Retrieve certificate from request if ACL is enabled + request.setCertificateInRequest(isAclEnabled ? getCertificate(httpRequest) : null); + } + + private static void verifyPartitioner(PartitionerConfig storePartitionerConfig, Set partitionersFromRequest) { + // If partitioners are provided, check if the store partitioner is in the list + if (partitionersFromRequest != null && !partitionersFromRequest.isEmpty() + && !partitionersFromRequest.contains(storePartitionerConfig.getPartitionerClass())) { + throw new VeniceException( + "Expected partitioner class " + storePartitionerConfig.getPartitionerClass() + " cannot be found."); + } + } + + static void verifyAndConfigurePartitionerSettings( + PartitionerConfig storePartitionerConfig, + Set partitionersFromRequest, + VersionCreationResponse response) { + verifyPartitioner(storePartitionerConfig, partitionersFromRequest); + partitionersFromRequest = partitionersFromRequest != null ? partitionersFromRequest : Collections.emptySet(); + // Get the first partitioner that matches the store partitioner + for (String partitioner: partitionersFromRequest) { + if (storePartitionerConfig.getPartitionerClass().equals(partitioner)) { + response.setPartitionerClass(partitioner); + response.setPartitionerParams(storePartitionerConfig.getPartitionerParams()); + response.setAmplificationFactor(storePartitionerConfig.getAmplificationFactor()); + return; + } + } + response.setPartitionerClass(storePartitionerConfig.getPartitionerClass()); + response.setPartitionerParams(storePartitionerConfig.getPartitionerParams()); + response.setAmplificationFactor(storePartitionerConfig.getAmplificationFactor()); + } + + private Lazy getActiveActiveReplicationCheck( + Admin admin, + Store store, + String clusterName, + String storeName, + boolean checkCurrentVersion) { + return Lazy.of( + () -> admin.isParent() && store.isActiveActiveReplicationEnabled() + && admin.isActiveActiveReplicationEnabledInAllRegion(clusterName, storeName, checkCurrentVersion)); + } + + private static String resolveConfig( + String configType, + String configValue, + String storeName, + Lazy isActiveActiveReplicationEnabledInAllRegion) { + if (configValue != null && !isActiveActiveReplicationEnabledInAllRegion.get()) { + LOGGER.info( + "Ignoring config {} : {}, as store {} is not set up for Active/Active replication in all regions", + configType, + configValue, + storeName); + return null; + } + return configValue; + } + + /** + * Configures the source fabric to align with the native replication source fabric selection. + *

+ * For incremental pushes using a real-time (RT) policy, the push job produces to the parent Kafka cluster. + * In such cases, this method ensures that the source fabric is not overridden with the native replication (NR) + * source fabric to maintain proper configuration. + */ + public static void configureSourceFabric( + Admin admin, + Version version, + Lazy isActiveActiveReplicationEnabledInAllRegions, + RequestTopicForPushRequest request, + VersionCreationResponse response) { + PushType pushType = request.getPushType(); + // Handle native replication for non-incremental push types + if (version.isNativeReplicationEnabled() && !pushType.isIncremental()) { + String childDataCenterKafkaBootstrapServer = version.getPushStreamSourceAddress(); + if (childDataCenterKafkaBootstrapServer != null) { + response.setKafkaBootstrapServers(childDataCenterKafkaBootstrapServer); + } + response.setKafkaSourceRegion(version.getNativeReplicationSourceFabric()); + } + + // Handle incremental push with override for source region + if (admin.isParent() && pushType.isIncremental()) { + overrideSourceRegionAddressForIncrementalPushJob( + admin, + response, + request.getClusterName(), + request.getStoreName(), + request.getEmergencySourceRegion(), + request.getSourceGridFabric(), + isActiveActiveReplicationEnabledInAllRegions.get(), + version.isNativeReplicationEnabled()); + LOGGER.info( + "Using source region: {} for incremental push job: {} on store: {} cluster: {}", + response.getKafkaBootstrapServers(), + request.getPushJobId(), + request.getStoreName(), + request.getClusterName()); + } + } + + static CompressionStrategy getCompressionStrategy(Version version, String responseTopic) { + if (Version.isRealTimeTopic(responseTopic)) { + return CompressionStrategy.NO_OP; + } + return version.getCompressionStrategy(); + } + + static String determineResponseTopic(String storeName, Version version, RequestTopicForPushRequest request) { + String responseTopic; + PushType pushType = request.getPushType(); + if (pushType == PushType.INCREMENTAL) { + // If incremental push with a dedicated real-time topic is enabled then use the separate real-time topic + if (version.isSeparateRealTimeTopicEnabled() && request.isSeparateRealTimeTopicEnabled()) { + responseTopic = Version.composeSeparateRealTimeTopic(storeName); + } else { + responseTopic = Version.composeRealTimeTopic(storeName); + } + } else if (pushType == PushType.STREAM) { + responseTopic = Version.composeRealTimeTopic(storeName); + } else if (pushType == PushType.STREAM_REPROCESSING) { + responseTopic = Version.composeStreamReprocessingTopic(storeName, version.getNumber()); + } else { + responseTopic = version.kafkaTopicName(); + } + return responseTopic; + } + + private void handleNonStreamPushType( + Admin admin, + Store store, + RequestTopicForPushRequest request, + VersionCreationResponse response, + Lazy isActiveActiveReplicationEnabledInAllRegions) { + String clusterName = request.getClusterName(); + String storeName = request.getStoreName(); + PushType pushType = request.getPushType(); + // Check if requestTopicForPush can be handled by child controllers for the given store + if (!admin.whetherEnableBatchPushFromAdmin(storeName)) { + throw new VeniceUnsupportedOperationException( + request.getPushType().name(), + "Please push data to Venice Parent Colo instead"); + } + response.setPartitions(admin.calculateNumberOfPartitions(clusterName, storeName)); + int computedPartitionCount = admin.calculateNumberOfPartitions(clusterName, storeName); + final Version version = admin.incrementVersionIdempotent( + clusterName, + storeName, + request.getPushJobId(), + computedPartitionCount, + response.getReplicas(), + pushType, + request.isSendStartOfPush(), + request.isSorted(), + request.getCompressionDictionary(), + Optional.ofNullable(request.getSourceGridFabric()), + Optional.ofNullable(request.getCertificateInRequest()), + request.getRewindTimeInSecondsOverride(), + Optional.ofNullable(request.getEmergencySourceRegion()), + request.isDeferVersionSwap(), + request.getTargetedRegions(), + request.getRepushSourceVersion()); + + // Set the partition count + response.setPartitions(version.getPartitionCount()); + // Set the version number + response.setVersion(version.getNumber()); + // Set the response topic + response.setKafkaTopic(determineResponseTopic(storeName, version, request)); + // Set the compression strategy + response.setCompressionStrategy(getCompressionStrategy(version, response.getKafkaTopic())); + // Set the bootstrap servers + configureSourceFabric(admin, version, isActiveActiveReplicationEnabledInAllRegions, request, response); + } + + /** + * Method handle request to get a topic for pushing data to Venice with {@link PushType#STREAM} + */ + void handleStreamPushType( + Admin admin, + Store store, + RequestTopicForPushRequest request, + VersionCreationResponse response, + Lazy isActiveActiveReplicationEnabledInAllRegionAllVersions) { + DataReplicationPolicy dataReplicationPolicy = store.getHybridStoreConfig().getDataReplicationPolicy(); + boolean isAggregateMode = DataReplicationPolicy.AGGREGATE.equals(dataReplicationPolicy); + if (admin.isParent()) { + // Conditionally check if the controller allows for fetching this information + if (disableParentRequestTopicForStreamPushes) { + throw new VeniceException( + "Write operations to the parent region are not permitted with push type: STREAM, as this feature is currently disabled."); + } + + // Conditionally check if this store has aggregate mode enabled. If not, throw an exception (as aggregate + // mode is required to produce to parent colo) + // We check the store config instead of the version config because we want this policy to go into effect + // without needing to perform empty pushes everywhere + if (!isAggregateMode) { + if (!isActiveActiveReplicationEnabledInAllRegionAllVersions.get()) { + throw new VeniceException( + "Store is not in aggregate mode! Cannot push data to parent topic!!. Current store setup: non-aggregate mode, AA is not enabled in all regions"); + } else { + // TODO: maybe throw exception here since this mode (REGION: PARENT, PUSH: STREAM, REPLICATION: AA-ENABLED) + // doesn't seem valid anymore + LOGGER.info( + "Store: {} samza job running in Aggregate mode; Store config is in Non-Aggregate mode; " + + "AA is enabled in all regions, letting the job continue", + store.getName()); + } + } + } else { + if (isAggregateMode) { + if (!store.isActiveActiveReplicationEnabled()) { + throw new VeniceException( + "Store is in aggregate mode and AA is not enabled. Cannot push data to child topic!!"); + } else { + LOGGER.info( + "Store: {} samza job running in Non-Aggregate mode, Store config is in Aggregate mode, " + + "AA is enabled in the local region, letting the job continue", + store.getName()); + } + } + } + + Version referenceHybridVersion = admin.getReferenceVersionForStreamingWrites( + request.getClusterName(), + request.getStoreName(), + request.getPushJobId()); + if (referenceHybridVersion == null) { + LOGGER.error( + "Request to get topic for STREAM push: {} for store: {} in cluster: {} is rejected as no hybrid version found", + request.getPushJobId(), + store.getName(), + request.getClusterName()); + throw new VeniceException( + "No hybrid version found for store: " + store.getName() + " in cluster: " + request.getClusterName() + + ". Create a hybrid version before starting a stream push job."); + } + response.setPartitions(referenceHybridVersion.getPartitionCount()); + response.setCompressionStrategy(CompressionStrategy.NO_OP); + response.setKafkaTopic(Version.composeRealTimeTopic(store.getName())); + } + + /** + * This method is used to handle the request to get a topic for pushing data to Venice. + */ + void handleRequestTopicForPushing(Admin admin, RequestTopicForPushRequest request, VersionCreationResponse response) { + String clusterName = request.getClusterName(); + String storeName = request.getStoreName(); + response.setCluster(clusterName); + response.setName(storeName); + + // Check if the store exists + Store store = admin.getStore(clusterName, storeName); + if (store == null) { + throw new VeniceNoStoreException(storeName, clusterName); + } + + // Verify and configure the partitioner + verifyAndConfigurePartitionerSettings(store.getPartitionerConfig(), request.getPartitioners(), response); + + // Validate push type + validatePushType(request.getPushType(), store); + + // Create aa replication checks with lazy evaluation + Lazy isActiveActiveReplicationEnabledInAllRegions = + getActiveActiveReplicationCheck(admin, store, clusterName, storeName, false); + Lazy isActiveActiveReplicationEnabledInAllRegionAllVersions = + getActiveActiveReplicationCheck(admin, store, clusterName, storeName, true); + + // Validate source and emergency region details and update request object + String sourceGridFabric = resolveConfig( + SOURCE_GRID_FABRIC, + request.getSourceGridFabric(), + storeName, + isActiveActiveReplicationEnabledInAllRegions); + String emergencySourceRegion = resolveConfig( + EMERGENCY_SOURCE_REGION, + admin.getEmergencySourceRegion(clusterName).orElse(null), + storeName, + isActiveActiveReplicationEnabledInAllRegions); + + request.setSourceGridFabric(sourceGridFabric); + request.setEmergencySourceRegion(emergencySourceRegion); + LOGGER.info( + "Request to push to store: {} in cluster: {} with source grid fabric: {} and emergency source region: {}", + storeName, + clusterName, + sourceGridFabric != null ? sourceGridFabric : "N/A", + emergencySourceRegion != null ? emergencySourceRegion : "N/A"); + + // Set the store's replication factor and partition count + response.setReplicas(admin.getReplicationFactor(clusterName, storeName)); + + boolean isSSL = admin.isSSLEnabledForPush(clusterName, storeName); + response.setKafkaBootstrapServers(admin.getKafkaBootstrapServers(isSSL)); + response.setKafkaSourceRegion(admin.getRegionName()); + response.setEnableSSL(isSSL); + + PushType pushType = request.getPushType(); + if (pushType == PushType.STREAM) { + handleStreamPushType(admin, store, request, response, isActiveActiveReplicationEnabledInAllRegionAllVersions); + } else { + handleNonStreamPushType(admin, store, request, response, isActiveActiveReplicationEnabledInAllRegions); + } + + response.setDaVinciPushStatusStoreEnabled(store.isDaVinciPushStatusStoreEnabled()); + response.setAmplificationFactor(1); + } + /** * Instead of asking Venice to create a version, pushes should ask venice which topic to write into. * The logic below includes the ability to respond with an existing topic for the same push, allowing requests @@ -108,302 +467,20 @@ public Route requestTopicForPushing(Admin admin) { return AdminSparkServer.OBJECT_MAPPER.writeValueAsString(responseObject); } + // Validate the request parameters AdminSparkServer.validateParams(request, REQUEST_TOPIC.getParams(), admin); - // Query params - String clusterName = request.queryParams(CLUSTER); - String storeName = request.queryParams(NAME); - Store store = admin.getStore(clusterName, storeName); - if (store == null) { - throw new VeniceNoStoreException(storeName); - } - responseObject.setCluster(clusterName); - responseObject.setName(storeName); - responseObject.setDaVinciPushStatusStoreEnabled(store.isDaVinciPushStatusStoreEnabled()); - - // Retrieve partitioner config from the store - PartitionerConfig storePartitionerConfig = store.getPartitionerConfig(); - if (request.queryParams(PARTITIONERS) == null) { - // Request does not contain partitioner info - responseObject.setPartitionerClass(storePartitionerConfig.getPartitionerClass()); - responseObject.setAmplificationFactor(storePartitionerConfig.getAmplificationFactor()); - responseObject.setPartitionerParams(storePartitionerConfig.getPartitionerParams()); - } else { - // Retrieve provided partitioner class list from the request - boolean hasMatchedPartitioner = false; - for (String partitioner: request.queryParams(PARTITIONERS).split(",")) { - if (partitioner.equals(storePartitionerConfig.getPartitionerClass())) { - responseObject.setPartitionerClass(storePartitionerConfig.getPartitionerClass()); - responseObject.setAmplificationFactor(storePartitionerConfig.getAmplificationFactor()); - responseObject.setPartitionerParams(storePartitionerConfig.getPartitionerParams()); - hasMatchedPartitioner = true; - break; - } - } - if (!hasMatchedPartitioner) { - throw new VeniceException( - "Expected partitioner class " + storePartitionerConfig.getPartitionerClass() + " cannot be found."); - } - } - - String pushTypeString = request.queryParams(PUSH_TYPE); - PushType pushType; - try { - pushType = PushType.valueOf(pushTypeString); - } catch (RuntimeException e) { - throw new VeniceHttpException( - HttpStatus.SC_BAD_REQUEST, - pushTypeString + " is an invalid " + PUSH_TYPE, - e, - ErrorType.BAD_REQUEST); - } - validatePushType(pushType, store); - - boolean sendStartOfPush = false; - // Make this optional so that it is compatible with old version controller client - if (request.queryParams().contains(SEND_START_OF_PUSH)) { - sendStartOfPush = Utils.parseBooleanFromString(request.queryParams(SEND_START_OF_PUSH), SEND_START_OF_PUSH); - } - - int replicationFactor = admin.getReplicationFactor(clusterName, storeName); - int partitionCount = admin.calculateNumberOfPartitions(clusterName, storeName); - responseObject.setReplicas(replicationFactor); - responseObject.setPartitions(partitionCount); - - boolean isSSL = admin.isSSLEnabledForPush(clusterName, storeName); - responseObject.setKafkaBootstrapServers(admin.getKafkaBootstrapServers(isSSL)); - responseObject.setKafkaSourceRegion(admin.getRegionName()); - responseObject.setEnableSSL(isSSL); - - String pushJobId = request.queryParams(PUSH_JOB_ID); - - boolean sorted = false; // an inefficient but safe default - String sortedParam = request.queryParams(PUSH_IN_SORTED_ORDER); - if (sortedParam != null) { - sorted = Utils.parseBooleanFromString(sortedParam, PUSH_IN_SORTED_ORDER); - } - - boolean isWriteComputeEnabled = false; - String wcEnabledParam = request.queryParams(IS_WRITE_COMPUTE_ENABLED); - if (wcEnabledParam != null) { - isWriteComputeEnabled = Utils.parseBooleanFromString(wcEnabledParam, IS_WRITE_COMPUTE_ENABLED); - } - - Optional sourceGridFabric = Optional.ofNullable(request.queryParams(SOURCE_GRID_FABRIC)); - - /** - * We can't honor source grid fabric and emergency source region config untill the store is A/A enabled in all regions. This is because - * if push job start producing to a different prod region then non A/A enabled region will not have the capability to consume from that region. - * This resets this config in such cases. - */ - Lazy isActiveActiveReplicationEnabledInAllRegion = Lazy.of(() -> { - if (admin.isParent() && store.isActiveActiveReplicationEnabled()) { - return admin.isActiveActiveReplicationEnabledInAllRegion(clusterName, storeName, false); - } else { - return false; - } - }); - - Lazy isActiveActiveReplicationEnabledInAllRegionAllVersions = Lazy.of(() -> { - if (admin.isParent() && store.isActiveActiveReplicationEnabled()) { - return admin.isActiveActiveReplicationEnabledInAllRegion(clusterName, storeName, true); - } else { - return false; - } - }); - - if (sourceGridFabric.isPresent() && !isActiveActiveReplicationEnabledInAllRegion.get()) { - LOGGER.info( - "Ignoring config {} : {}, as store {} is not set up for Active/Active replication in all regions", - SOURCE_GRID_FABRIC, - sourceGridFabric.get(), - storeName); - sourceGridFabric = Optional.empty(); - } - Optional emergencySourceRegion = admin.getEmergencySourceRegion(clusterName); - if (emergencySourceRegion.isPresent() && !isActiveActiveReplicationEnabledInAllRegion.get()) { - LOGGER.info( - "Ignoring config {} : {}, as store {} is not set up for Active/Active replication in all regions", - EMERGENCY_SOURCE_REGION, - emergencySourceRegion.get(), - storeName); - } - LOGGER.info( - "requestTopicForPushing: source grid fabric: {}, emergency source region: {}", - sourceGridFabric.orElse(""), - emergencySourceRegion.orElse("")); - - /** - * Version-level rewind time override, and it is only valid for hybrid stores. - */ - Optional rewindTimeInSecondsOverrideOptional = - Optional.ofNullable(request.queryParams(REWIND_TIME_IN_SECONDS_OVERRIDE)); - long rewindTimeInSecondsOverride = -1; - if (rewindTimeInSecondsOverrideOptional.isPresent()) { - rewindTimeInSecondsOverride = Long.parseLong(rewindTimeInSecondsOverrideOptional.get()); - } - - /** - * Version level override to defer marking this new version to the serving version post push completion. - */ - boolean deferVersionSwap = Boolean.parseBoolean(request.queryParams(DEFER_VERSION_SWAP)); - - String targetedRegions = request.queryParams(TARGETED_REGIONS); - - int repushSourceVersion = Integer.parseInt(request.queryParamOrDefault(REPUSH_SOURCE_VERSION, "-1")); - - switch (pushType) { - case BATCH: - case INCREMENTAL: - case STREAM_REPROCESSING: - if (!admin.whetherEnableBatchPushFromAdmin(storeName)) { - throw new VeniceUnsupportedOperationException( - pushTypeString, - "Please push data to Venice Parent Colo instead"); - } - String dictionaryStr = request.queryParams(COMPRESSION_DICTIONARY); - - /** - * Before trying to get the version, create the RT topic in parent kafka since it's needed anyway in following cases. - * Otherwise topic existence check fails internally. - */ - if (pushType.isIncremental() && isWriteComputeEnabled) { - admin.getRealTimeTopic(clusterName, store); - } - - final Optional certInRequest = - isAclEnabled() ? Optional.of(getCertificate(request)) : Optional.empty(); - final Version version = admin.incrementVersionIdempotent( - clusterName, - storeName, - pushJobId, - partitionCount, - replicationFactor, - pushType, - sendStartOfPush, - sorted, - dictionaryStr, - sourceGridFabric, - certInRequest, - rewindTimeInSecondsOverride, - emergencySourceRegion, - deferVersionSwap, - targetedRegions, - repushSourceVersion); - - // If Version partition count different from calculated partition count use the version count as store count - // may have been updated later. - if (version.getPartitionCount() != partitionCount) { - responseObject.setPartitions(version.getPartitionCount()); - } - String responseTopic; - /** - * Override the source fabric to respect the native replication source fabric selection. - */ - boolean overrideSourceFabric = true; - boolean isTopicRT = false; - if (pushType.isStreamReprocessing()) { - responseTopic = Version.composeStreamReprocessingTopic(storeName, version.getNumber()); - } else if (pushType.isIncremental()) { - isTopicRT = true; - if (version.isSeparateRealTimeTopicEnabled() - && Boolean.parseBoolean(request.queryParamOrDefault(SEPARATE_REAL_TIME_TOPIC_ENABLED, "false"))) { - admin.getSeparateRealTimeTopic(clusterName, storeName); - responseTopic = Version.composeSeparateRealTimeTopic(storeName); - } else { - responseTopic = Utils.getRealTimeTopicName(store); - } - // disable amplificationFactor logic on real-time topic - responseObject.setAmplificationFactor(1); - - if (version.isNativeReplicationEnabled()) { - /** - * For incremental push with RT policy store the push job produces to parent corp kafka cluster. We should not override the - * source fabric in such cases with NR source fabric. - */ - overrideSourceFabric = false; - } - } else { - responseTopic = version.kafkaTopicName(); - } - - responseObject.setVersion(version.getNumber()); - responseObject.setKafkaTopic(responseTopic); - if (isTopicRT) { - // RT topic only supports NO_OP compression - responseObject.setCompressionStrategy(CompressionStrategy.NO_OP); - } else { - responseObject.setCompressionStrategy(version.getCompressionStrategy()); - } - if (version.isNativeReplicationEnabled() && overrideSourceFabric) { - String childDataCenterKafkaBootstrapServer = version.getPushStreamSourceAddress(); - if (childDataCenterKafkaBootstrapServer != null) { - responseObject.setKafkaBootstrapServers(childDataCenterKafkaBootstrapServer); - } - responseObject.setKafkaSourceRegion(version.getNativeReplicationSourceFabric()); - } - - if (pushType.isIncremental() && admin.isParent()) { - overrideSourceRegionAddressForIncrementalPushJob( - admin, - responseObject, - clusterName, - emergencySourceRegion.orElse(null), - sourceGridFabric.orElse(null), - isActiveActiveReplicationEnabledInAllRegion.get(), - version.isNativeReplicationEnabled()); - LOGGER.info( - "Incremental push job final source region address is: {}", - responseObject.getKafkaBootstrapServers()); - } - break; - case STREAM: - - if (admin.isParent()) { - - // Conditionally check if the controller allows for fetching this information - if (disableParentRequestTopicForStreamPushes) { - throw new VeniceException( - String.format( - "Parent request topic is disabled!! Cannot push data to topic in parent colo for store %s. Aborting!!", - storeName)); - } - - // Conditionally check if this store has aggregate mode enabled. If not, throw an exception (as aggregate - // mode is required to produce to parent colo) - // We check the store config instead of the version config because we want this policy to go into affect - // without needing to perform empty pushes everywhere - if (!store.getHybridStoreConfig().getDataReplicationPolicy().equals(DataReplicationPolicy.AGGREGATE)) { - if (!isActiveActiveReplicationEnabledInAllRegionAllVersions.get()) { - throw new VeniceException("Store is not in aggregate mode! Cannot push data to parent topic!!"); - } else { - LOGGER.info( - "Store: {} samza job running in Aggregate mode, Store config is in Non-Aggregate mode, " - + "AA is enabled in all regions, letting the job continue", - storeName); - } - } - } else { - if (store.getHybridStoreConfig().getDataReplicationPolicy().equals(DataReplicationPolicy.AGGREGATE)) { - if (!store.isActiveActiveReplicationEnabled()) { - throw new VeniceException("Store is in aggregate mode! Cannot push data to child topic!!"); - } else { - LOGGER.info( - "Store: {} samza job running in Non-Aggregate mode, Store config is in Aggregate mode, " - + "AA is enabled in the local region, letting the job continue", - storeName); - } - } - } - - String realTimeTopic = admin.getRealTimeTopic(clusterName, store); - responseObject.setKafkaTopic(realTimeTopic); - // disable amplificationFactor logic on real-time topic - responseObject.setAmplificationFactor(1); - break; - default: - throw new VeniceException(pushTypeString + " is an unrecognized " + PUSH_TYPE); - } + // Extract request parameters and create a RequestTopicForPushRequest object + RequestTopicForPushRequest requestTopicForPushRequest = new RequestTopicForPushRequest( + request.queryParams(CLUSTER), + request.queryParams(NAME), + RequestTopicForPushRequest.extractPushType(request.queryParams(PUSH_TYPE)), + request.queryParams(PUSH_JOB_ID)); + + // populate the request object with optional parameters + extractOptionalParamsFromRequestTopicRequest(request, requestTopicForPushRequest, isAclEnabled()); + // Invoke the handler to get the topic for pushing data + handleRequestTopicForPushing(admin, requestTopicForPushRequest, responseObject); } catch (Throwable e) { responseObject.setError(e); AdminSparkServer.handleError(e, request, response); @@ -428,6 +505,7 @@ static void overrideSourceRegionAddressForIncrementalPushJob( Admin admin, VersionCreationResponse response, String clusterName, + String storeName, String emergencySourceRegion, String pushJobSourceGridFabric, boolean isAAEnabledInAllRegions, @@ -435,7 +513,17 @@ static void overrideSourceRegionAddressForIncrementalPushJob( if (!isAAEnabledInAllRegions && isNativeReplicationEnabled) { // P2: When AA is not enabled in all the regions we use aggregate RT address, if it is available, // for inc-pushes if native-replication is enabled. - admin.getAggregateRealTimeTopicSource(clusterName).ifPresent(response::setKafkaBootstrapServers); + Optional aggregateRealTimeTopicSource = admin.getAggregateRealTimeTopicSource(clusterName); + if (aggregateRealTimeTopicSource.isPresent()) { + response.setKafkaBootstrapServers(aggregateRealTimeTopicSource.get()); + LOGGER.info( + "Incremental push job source region is being overridden with: {} address: {} for store: {} in cluster: {}", + aggregateRealTimeTopicSource.get(), + response.getKafkaBootstrapServers(), + storeName, + clusterName); + } + return; } else if (!isAAEnabledInAllRegions) { // When AA is not enabled in all regions and native replication is also disabled, don't do anything. @@ -457,13 +545,15 @@ static void overrideSourceRegionAddressForIncrementalPushJob( throw new VeniceException("Failed to get the broker server URL for the source region: " + overRideSourceRegion); } LOGGER.info( - "Incremental push job source region is being overridden with: {} address: {}", + "Incremental push job source region is being overridden with: {} address: {} for store: {} in cluster: {}", overRideSourceRegion, - bootstrapServerAddress); + bootstrapServerAddress, + storeName, + clusterName); response.setKafkaBootstrapServers(bootstrapServerAddress); } - void validatePushType(PushType pushType, Store store) { + static void validatePushType(PushType pushType, Store store) { if (pushType.equals(PushType.STREAM) && !store.isHybrid()) { throw new VeniceHttpException( HttpStatus.SC_BAD_REQUEST, diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java b/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java index ecb0605e990..877312224ff 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java @@ -4,6 +4,7 @@ import static com.linkedin.venice.ConfigKeys.KAFKA_REPLICATION_FACTOR; import static com.linkedin.venice.ConfigKeys.KAFKA_REPLICATION_FACTOR_RT_TOPICS; import static com.linkedin.venice.VeniceConstants.REWIND_TIME_DECIDED_BY_SERVER; +import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.TOPIC_SWITCH; import static com.linkedin.venice.pubsub.PubSubConstants.DEFAULT_KAFKA_REPLICATION_FACTOR; import com.linkedin.venice.ConfigKeys; @@ -114,9 +115,11 @@ void sendTopicSwitch( .broadcastTopicSwitch(sourceClusters, realTimeTopic.getName(), rewindStartTimestamp, Collections.emptyMap()); } LOGGER.info( - "Successfully sent TopicSwitch into '{}' instructing to switch to '{}' with a rewindStartTimestamp of {}.", + "Successfully sent {} into '{}' instructing to switch to {} at {} with a rewindStartTimestamp of {}.", + TOPIC_SWITCH, topicWhereToSendTheTopicSwitch, realTimeTopic, + remoteKafkaUrls, rewindStartTimestamp); } @@ -135,19 +138,10 @@ void ensurePreconditions( Version version = store.getVersion(Version.parseVersionFromKafkaTopicName(topicWhereToSendTheTopicSwitch.getName())); /** - * TopicReplicator is used in child fabrics to create real-time (RT) topic when a child fabric - * is ready to start buffer replay but RT topic doesn't exist. This scenario could happen for a - * hybrid store when users haven't started any Samza job yet. In this case, RT topic should be - * created with proper retention time instead of the default 5 days retention. - * - * Potential race condition: If both rewind-time update operation and buffer-replay - * start at the same time, RT topic might not be created with the expected retention time, - * which can be fixed by sending another rewind-time update command. - * - * TODO: RT topic should be created in both parent and child fabrics when the store is converted to - * hybrid (update store command handling). However, if a store is converted to hybrid when it - * doesn't have any existing version or a correct storage quota, we cannot decide the partition - * number for it. + * We create the real-time topics when creating hybrid version for the first time. This is to ensure that the + * real-time topics are created with the correct partition count. Here we'll only check retention time and update + * it if necessary. + * TODO: Remove topic creation logic from here once new code is deployed to all regions. */ createRealTimeTopicIfNeeded(store, version, srcTopicName, hybridStoreConfig.get()); if (version != null && version.isSeparateRealTimeTopicEnabled()) { @@ -189,7 +183,6 @@ void createRealTimeTopicIfNeeded( getTopicManager().updateTopicRetention(realTimeTopic, expectedRetentionTimeMs); } } - } long getRewindStartTime( @@ -301,7 +294,8 @@ public void switchToRealTimeTopic( remoteKafkaUrls.add(aggregateRealTimeSourceKafkaUrl); } LOGGER.info( - "Will send TopicSwitch into '{}' instructing to switch to '{}' with a rewindStartTimestamp of {}.", + "Will send {} into '{}' instructing to switch to '{}' with a rewindStartTimestamp of {}.", + TOPIC_SWITCH, topicWhereToSendTheTopicSwitch, realTimeTopic, rewindStartTimestamp); diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceHelixAdmin.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceHelixAdmin.java index 5ed9d3a2452..783584910c2 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceHelixAdmin.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceHelixAdmin.java @@ -75,7 +75,7 @@ public void testDropResources() { @Test public void enforceRealTimeTopicCreationBeforeWriting() { VeniceHelixAdmin veniceHelixAdmin = mock(VeniceHelixAdmin.class); - doReturn("test_rt").when(veniceHelixAdmin).getRealTimeTopic(anyString(), anyString()); + doReturn("test_rt").when(veniceHelixAdmin).getRealTimeTopic(anyString(), anyString(), any()); doCallRealMethod().when(veniceHelixAdmin).setUpMetaStoreAndMayProduceSnapshot(anyString(), anyString()); InOrder inorder = inOrder(veniceHelixAdmin); @@ -92,7 +92,7 @@ public void enforceRealTimeTopicCreationBeforeWriting() { veniceHelixAdmin.setUpMetaStoreAndMayProduceSnapshot(anyString(), anyString()); // Enforce that getRealTimeTopic happens before storeMetadataUpdate. See the above comments for the reasons. - inorder.verify(veniceHelixAdmin).getRealTimeTopic(anyString(), anyString()); + inorder.verify(veniceHelixAdmin).getRealTimeTopic(anyString(), anyString(), any()); inorder.verify(veniceHelixAdmin).storeMetadataUpdate(anyString(), anyString(), any()); } diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java index 39e48528682..87ea27fd5b3 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java @@ -4,55 +4,82 @@ import static com.linkedin.venice.VeniceConstants.CONTROLLER_SSL_CERTIFICATE_ATTRIBUTE_NAME; import static com.linkedin.venice.controller.server.CreateVersion.overrideSourceRegionAddressForIncrementalPushJob; import static com.linkedin.venice.controllerapi.ControllerApiConstants.CLUSTER; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.COMPRESSION_DICTIONARY; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.DEFER_VERSION_SWAP; import static com.linkedin.venice.controllerapi.ControllerApiConstants.HOSTNAME; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.IS_WRITE_COMPUTE_ENABLED; import static com.linkedin.venice.controllerapi.ControllerApiConstants.NAME; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.PARTITIONERS; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.PUSH_IN_SORTED_ORDER; import static com.linkedin.venice.controllerapi.ControllerApiConstants.PUSH_JOB_ID; import static com.linkedin.venice.controllerapi.ControllerApiConstants.PUSH_TYPE; import static com.linkedin.venice.controllerapi.ControllerApiConstants.REPUSH_SOURCE_VERSION; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.REWIND_TIME_IN_SECONDS_OVERRIDE; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.SEND_START_OF_PUSH; import static com.linkedin.venice.controllerapi.ControllerApiConstants.SEPARATE_REAL_TIME_TOPIC_ENABLED; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.SOURCE_GRID_FABRIC; import static com.linkedin.venice.controllerapi.ControllerApiConstants.STORE_SIZE; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.TARGETED_REGIONS; import static com.linkedin.venice.controllerapi.ControllerRoute.REQUEST_TOPIC; import static com.linkedin.venice.meta.BufferReplayPolicy.REWIND_FROM_EOP; import static com.linkedin.venice.meta.DataReplicationPolicy.ACTIVE_ACTIVE; import static com.linkedin.venice.meta.DataReplicationPolicy.AGGREGATE; import static com.linkedin.venice.meta.DataReplicationPolicy.NONE; import static com.linkedin.venice.meta.DataReplicationPolicy.NON_AGGREGATE; +import static com.linkedin.venice.meta.Version.PushType.*; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.Mockito.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doCallRealMethod; 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.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; +import static org.testng.Assert.fail; import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.venice.acl.DynamicAccessController; +import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.controller.Admin; +import com.linkedin.venice.controllerapi.RequestTopicForPushRequest; import com.linkedin.venice.controllerapi.VersionCreationResponse; import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.exceptions.VeniceHttpException; +import com.linkedin.venice.meta.DataReplicationPolicy; +import com.linkedin.venice.meta.HybridStoreConfig; import com.linkedin.venice.meta.HybridStoreConfigImpl; import com.linkedin.venice.meta.OfflinePushStrategy; +import com.linkedin.venice.meta.PartitionerConfig; import com.linkedin.venice.meta.PersistenceType; import com.linkedin.venice.meta.ReadStrategy; import com.linkedin.venice.meta.RoutingStrategy; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; -import com.linkedin.venice.meta.Version.PushType; import com.linkedin.venice.meta.VersionImpl; import com.linkedin.venice.meta.ZKStore; import com.linkedin.venice.utils.DataProviderUtils; import com.linkedin.venice.utils.ObjectMapperFactory; import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.lazy.Lazy; import java.security.cert.X509Certificate; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Optional; +import java.util.Set; import javax.security.auth.x500.X500Principal; import javax.servlet.http.HttpServletRequest; +import org.apache.http.HttpStatus; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -95,7 +122,7 @@ public void setUp() { queryMap.put(NAME, new String[] { STORE_NAME }); queryMap.put(STORE_SIZE, new String[] { "0" }); queryMap.put(REPUSH_SOURCE_VERSION, new String[] { "0" }); - queryMap.put(PUSH_TYPE, new String[] { PushType.INCREMENTAL.name() }); + queryMap.put(PUSH_TYPE, new String[] { INCREMENTAL.name() }); queryMap.put(PUSH_JOB_ID, new String[] { JOB_ID }); queryMap.put(HOSTNAME, new String[] { "localhost" }); @@ -168,7 +195,7 @@ public void testRequestTopicForHybridIncPushEnabled( JOB_ID, 0, 0, - PushType.INCREMENTAL, + INCREMENTAL, false, false, null, @@ -227,7 +254,7 @@ public void testRequestTopicForIncPushReturnsErrorWhenStoreIsNotHybridAndIncPush JOB_ID, 0, 0, - PushType.INCREMENTAL, + INCREMENTAL, false, false, null, @@ -253,7 +280,7 @@ public void testRequestTopicForIncPushReturnsErrorWhenStoreIsNotHybridAndIncPush OBJECT_MAPPER.readValue(result.toString(), VersionCreationResponse.class); assertTrue(versionCreateResponse.isError()); assertTrue(versionCreateResponse.getError().contains("which does not have hybrid mode enabled")); - Assert.assertNull(versionCreateResponse.getKafkaTopic()); + assertNull(versionCreateResponse.getKafkaTopic()); } @Test @@ -281,7 +308,7 @@ public void testRequestTopicForIncPushCanUseEmergencyRegionWhenItIsSet() throws JOB_ID, 0, 0, - PushType.INCREMENTAL, + INCREMENTAL, false, false, null, @@ -336,39 +363,87 @@ public void testOverrideSourceRegionAddressForIncrementalPushJob() { creationResponse = new VersionCreationResponse(); creationResponse.setKafkaBootstrapServers("default.src.region.com"); doReturn(Optional.empty()).when(admin).getAggregateRealTimeTopicSource(CLUSTER_NAME); - overrideSourceRegionAddressForIncrementalPushJob(admin, creationResponse, CLUSTER_NAME, null, null, false, true); + overrideSourceRegionAddressForIncrementalPushJob( + admin, + creationResponse, + CLUSTER_NAME, + STORE_NAME, + null, + null, + false, + true); assertEquals(creationResponse.getKafkaBootstrapServers(), "default.src.region.com"); // AA-all-region is disabled & NR is enabled * AGG RT address is set creationResponse = new VersionCreationResponse(); creationResponse.setKafkaBootstrapServers("default.src.region.com"); doReturn(Optional.of("agg.rt.region.com")).when(admin).getAggregateRealTimeTopicSource(CLUSTER_NAME); - overrideSourceRegionAddressForIncrementalPushJob(admin, creationResponse, CLUSTER_NAME, null, null, false, true); + overrideSourceRegionAddressForIncrementalPushJob( + admin, + creationResponse, + CLUSTER_NAME, + STORE_NAME, + null, + null, + false, + true); assertEquals(creationResponse.getKafkaBootstrapServers(), "agg.rt.region.com"); // AA-all-region and NR are disabled creationResponse = new VersionCreationResponse(); creationResponse.setKafkaBootstrapServers("default.src.region.com"); - overrideSourceRegionAddressForIncrementalPushJob(admin, creationResponse, CLUSTER_NAME, null, null, false, false); + overrideSourceRegionAddressForIncrementalPushJob( + admin, + creationResponse, + CLUSTER_NAME, + STORE_NAME, + null, + null, + false, + false); assertEquals(creationResponse.getKafkaBootstrapServers(), "default.src.region.com"); // AA-all-region is enabled and NR is disabled creationResponse = new VersionCreationResponse(); creationResponse.setKafkaBootstrapServers("default.src.region.com"); - overrideSourceRegionAddressForIncrementalPushJob(admin, creationResponse, CLUSTER_NAME, null, null, true, false); + overrideSourceRegionAddressForIncrementalPushJob( + admin, + creationResponse, + CLUSTER_NAME, + STORE_NAME, + null, + null, + true, + false); assertEquals(creationResponse.getKafkaBootstrapServers(), "default.src.region.com"); // AA-all-region and NR are enabled AND emergencySourceRegion and pushJobSourceGridFabric are null creationResponse = new VersionCreationResponse(); creationResponse.setKafkaBootstrapServers("default.src.region.com"); - overrideSourceRegionAddressForIncrementalPushJob(admin, creationResponse, CLUSTER_NAME, null, null, true, true); + overrideSourceRegionAddressForIncrementalPushJob( + admin, + creationResponse, + CLUSTER_NAME, + STORE_NAME, + null, + null, + true, + true); assertEquals(creationResponse.getKafkaBootstrapServers(), "default.src.region.com"); // AA-all-region and NR are enabled AND emergencySourceRegion is not set but pushJobSourceGridFabric is provided creationResponse = new VersionCreationResponse(); creationResponse.setKafkaBootstrapServers("default.src.region.com"); doReturn("vpj.src.region.com").when(admin).getNativeReplicationKafkaBootstrapServerAddress("dc-vpj"); - overrideSourceRegionAddressForIncrementalPushJob(admin, creationResponse, CLUSTER_NAME, null, "dc-vpj", true, true); + overrideSourceRegionAddressForIncrementalPushJob( + admin, + creationResponse, + CLUSTER_NAME, + STORE_NAME, + null, + "dc-vpj", + true, + true); assertEquals(creationResponse.getKafkaBootstrapServers(), "vpj.src.region.com"); // AA-all-region and NR are enabled AND emergencySourceRegion is set and pushJobSourceGridFabric is provided @@ -379,6 +454,7 @@ public void testOverrideSourceRegionAddressForIncrementalPushJob() { admin, creationResponse, CLUSTER_NAME, + STORE_NAME, "dc-e", "dc-vpj", true, @@ -389,7 +465,15 @@ public void testOverrideSourceRegionAddressForIncrementalPushJob() { creationResponse = new VersionCreationResponse(); creationResponse.setKafkaBootstrapServers("emergency.src.region.com"); doReturn("emergency.src.region.com").when(admin).getNativeReplicationKafkaBootstrapServerAddress("dc-e"); - overrideSourceRegionAddressForIncrementalPushJob(admin, creationResponse, CLUSTER_NAME, "dc-e", null, true, true); + overrideSourceRegionAddressForIncrementalPushJob( + admin, + creationResponse, + CLUSTER_NAME, + STORE_NAME, + "dc-e", + null, + true, + true); assertEquals(creationResponse.getKafkaBootstrapServers(), "emergency.src.region.com"); } @@ -398,7 +482,15 @@ public void testOverrideSourceRegionAddressForIncrementalPushJobWhenOverrideRegi VersionCreationResponse creationResponse = new VersionCreationResponse(); creationResponse.setKafkaBootstrapServers("default.src.region.com"); doReturn(null).when(admin).getNativeReplicationKafkaBootstrapServerAddress("dc1"); - overrideSourceRegionAddressForIncrementalPushJob(admin, creationResponse, CLUSTER_NAME, "dc1", null, true, true); + overrideSourceRegionAddressForIncrementalPushJob( + admin, + creationResponse, + CLUSTER_NAME, + STORE_NAME, + "dc1", + null, + true, + true); } @Test @@ -408,35 +500,35 @@ public void testValidatePushTypeForStreamPushType() { // push type is STREAM and store is not hybrid Store store1 = mock(Store.class); when(store1.isHybrid()).thenReturn(false); - Exception e = expectThrows(VeniceException.class, () -> createVersion.validatePushType(PushType.STREAM, store1)); + Exception e = expectThrows(VeniceException.class, () -> createVersion.validatePushType(STREAM, store1)); assertTrue(e.getMessage().contains("which is not configured to be a hybrid store")); // push type is STREAM and store is AA enabled hybrid Store store2 = mock(Store.class); when(store2.isHybrid()).thenReturn(true); when(store2.isActiveActiveReplicationEnabled()).thenReturn(true); - createVersion.validatePushType(PushType.STREAM, store2); + createVersion.validatePushType(STREAM, store2); // push type is STREAM and store is not AA enabled hybrid but has NON_AGGREGATE replication policy Store store3 = mock(Store.class); when(store3.isHybrid()).thenReturn(true); when(store3.isActiveActiveReplicationEnabled()).thenReturn(false); when(store3.getHybridStoreConfig()).thenReturn(new HybridStoreConfigImpl(0, 1, 0, NON_AGGREGATE, REWIND_FROM_EOP)); - createVersion.validatePushType(PushType.STREAM, store3); + createVersion.validatePushType(STREAM, store3); // push type is STREAM and store is not AA enabled hybrid but has AGGREGATE replication policy Store store4 = mock(Store.class); when(store4.isHybrid()).thenReturn(true); when(store4.isActiveActiveReplicationEnabled()).thenReturn(false); when(store4.getHybridStoreConfig()).thenReturn(new HybridStoreConfigImpl(0, 1, 0, AGGREGATE, REWIND_FROM_EOP)); - createVersion.validatePushType(PushType.STREAM, store4); + createVersion.validatePushType(STREAM, store4); // push type is STREAM and store is not AA enabled hybrid but has NONE replication policy Store store5 = mock(Store.class); when(store5.isHybrid()).thenReturn(true); when(store5.isActiveActiveReplicationEnabled()).thenReturn(false); when(store5.getHybridStoreConfig()).thenReturn(new HybridStoreConfigImpl(0, 1, 0, NONE, REWIND_FROM_EOP)); - Exception e5 = expectThrows(VeniceException.class, () -> createVersion.validatePushType(PushType.STREAM, store5)); + Exception e5 = expectThrows(VeniceException.class, () -> createVersion.validatePushType(STREAM, store5)); assertTrue(e5.getMessage().contains("which is configured to have a hybrid data replication policy")); // push type is STREAM and store is not AA enabled hybrid but has ACTIVE_ACTIVE replication policy @@ -444,7 +536,7 @@ public void testValidatePushTypeForStreamPushType() { when(store6.isHybrid()).thenReturn(true); when(store6.isActiveActiveReplicationEnabled()).thenReturn(false); when(store6.getHybridStoreConfig()).thenReturn(new HybridStoreConfigImpl(0, 1, 0, ACTIVE_ACTIVE, REWIND_FROM_EOP)); - Exception e6 = expectThrows(VeniceException.class, () -> createVersion.validatePushType(PushType.STREAM, store6)); + Exception e6 = expectThrows(VeniceException.class, () -> createVersion.validatePushType(STREAM, store6)); assertTrue(e6.getMessage().contains("which is configured to have a hybrid data replication policy")); } @@ -455,16 +547,391 @@ public void testValidatePushTypeForIncrementalPushPushType() { // push type is INCREMENTAL and store is not hybrid Store store1 = mock(Store.class); when(store1.isHybrid()).thenReturn(false); - Exception e = - expectThrows(VeniceException.class, () -> createVersion.validatePushType(PushType.INCREMENTAL, store1)); + Exception e = expectThrows(VeniceException.class, () -> createVersion.validatePushType(INCREMENTAL, store1)); assertTrue(e.getMessage().contains("which does not have hybrid mode enabled")); // push type is INCREMENTAL and store is hybrid but incremental push is not enabled Store store2 = mock(Store.class); when(store2.isHybrid()).thenReturn(true); when(store2.isIncrementalPushEnabled()).thenReturn(false); - Exception e2 = - expectThrows(VeniceException.class, () -> createVersion.validatePushType(PushType.INCREMENTAL, store2)); + Exception e2 = expectThrows(VeniceException.class, () -> createVersion.validatePushType(INCREMENTAL, store2)); assertTrue(e2.getMessage().contains("which does not have incremental push enabled")); } + + @Test + public void testExtractOptionalParamsFromRequestTopicForPushingRequest() { + // Test case 1: Default values + Request mockRequest = mock(Request.class); + doCallRealMethod().when(mockRequest).queryParamOrDefault(anyString(), anyString()); + doReturn(null).when(mockRequest).queryParams(any()); + + RequestTopicForPushRequest requestDetails = new RequestTopicForPushRequest(CLUSTER_NAME, STORE_NAME, BATCH, JOB_ID); + + CreateVersion.extractOptionalParamsFromRequestTopicRequest(mockRequest, requestDetails, false); + + assertNotNull(requestDetails.getPartitioners(), "Default partitioners should not be null"); + assertTrue(requestDetails.getPartitioners().isEmpty(), "Default partitioners should be empty"); + assertFalse(requestDetails.isSendStartOfPush(), "Default sendStartOfPush should be false"); + assertFalse(requestDetails.isSorted(), "Default sorted should be false"); + assertFalse(requestDetails.isWriteComputeEnabled(), "Default writeComputeEnabled should be false"); + assertEquals( + requestDetails.getRewindTimeInSecondsOverride(), + -1L, + "Default rewindTimeInSecondsOverride should be -1"); + assertFalse(requestDetails.isDeferVersionSwap(), "Default deferVersionSwap should be false"); + assertNull(requestDetails.getTargetedRegions(), "Default targetedRegions should be null"); + assertEquals(requestDetails.getRepushSourceVersion(), -1, "Default repushSourceVersion should be -1"); + assertNull(requestDetails.getSourceGridFabric(), "Default sourceGridFabric should be null"); + assertNull(requestDetails.getCompressionDictionary(), "Default compressionDictionary should be null"); + assertNull(requestDetails.getCertificateInRequest(), "Default certificateInRequest should be null"); + + // Test case 2: All optional parameters are set + mockRequest = mock(Request.class); + doCallRealMethod().when(mockRequest).queryParamOrDefault(any(), any()); + String customPartitioners = "f.q.c.n.P1,f.q.c.n.P2"; + Set expectedPartitioners = new HashSet<>(Arrays.asList("f.q.c.n.P1", "f.q.c.n.P2")); + + when(mockRequest.queryParams(eq(PARTITIONERS))).thenReturn(customPartitioners); + when(mockRequest.queryParams(SEND_START_OF_PUSH)).thenReturn("true"); + when(mockRequest.queryParams(PUSH_IN_SORTED_ORDER)).thenReturn("true"); + when(mockRequest.queryParams(IS_WRITE_COMPUTE_ENABLED)).thenReturn("true"); + when(mockRequest.queryParams(REWIND_TIME_IN_SECONDS_OVERRIDE)).thenReturn("120"); + when(mockRequest.queryParams(DEFER_VERSION_SWAP)).thenReturn("true"); + when(mockRequest.queryParams(TARGETED_REGIONS)).thenReturn("region-1"); + when(mockRequest.queryParams(REPUSH_SOURCE_VERSION)).thenReturn("5"); + when(mockRequest.queryParams(SOURCE_GRID_FABRIC)).thenReturn("grid-fabric"); + when(mockRequest.queryParams(COMPRESSION_DICTIONARY)).thenReturn("XYZ"); + + requestDetails = new RequestTopicForPushRequest(CLUSTER_NAME, STORE_NAME, BATCH, JOB_ID); + + CreateVersion.extractOptionalParamsFromRequestTopicRequest(mockRequest, requestDetails, false); + + assertEquals(requestDetails.getPartitioners(), expectedPartitioners); + assertTrue(requestDetails.isSendStartOfPush()); + assertTrue(requestDetails.isSorted()); + assertTrue(requestDetails.isWriteComputeEnabled()); + assertEquals(requestDetails.getRewindTimeInSecondsOverride(), 120L); + assertTrue(requestDetails.isDeferVersionSwap()); + assertEquals(requestDetails.getTargetedRegions(), "region-1"); + assertEquals(requestDetails.getRepushSourceVersion(), 5); + assertEquals(requestDetails.getSourceGridFabric(), "grid-fabric"); + assertEquals(requestDetails.getCompressionDictionary(), "XYZ"); + + // Test case 3: check that the certificate is set in the request details when access control is enabled + HttpServletRequest mockHttpServletRequest = mock(HttpServletRequest.class); + X509Certificate[] mockCertificates = { mock(X509Certificate.class) }; + when(mockHttpServletRequest.getAttribute(CONTROLLER_SSL_CERTIFICATE_ATTRIBUTE_NAME)).thenReturn(mockCertificates); + when(mockRequest.raw()).thenReturn(mockHttpServletRequest); + CreateVersion.extractOptionalParamsFromRequestTopicRequest(mockRequest, requestDetails, true); + assertEquals(requestDetails.getCertificateInRequest(), mockCertificates[0]); + + // Test case 4: Invalid values for optional parameters + when(mockRequest.queryParams(SEND_START_OF_PUSH)).thenReturn("notBoolean"); + when(mockRequest.queryParams(REWIND_TIME_IN_SECONDS_OVERRIDE)).thenReturn("invalidLong"); + + requestDetails = new RequestTopicForPushRequest(CLUSTER_NAME, STORE_NAME, BATCH, JOB_ID); + Request finalMockRequest = mockRequest; + RequestTopicForPushRequest finalRequestDetails = requestDetails; + VeniceHttpException e = expectThrows( + VeniceHttpException.class, + () -> CreateVersion.extractOptionalParamsFromRequestTopicRequest(finalMockRequest, finalRequestDetails, false)); + assertEquals(e.getHttpStatusCode(), HttpStatus.SC_BAD_REQUEST); + } + + @Test + public void testVerifyAndConfigurePartitionerSettings() { + VersionCreationResponse response = new VersionCreationResponse(); + PartitionerConfig storePartitionerConfig = mock(PartitionerConfig.class); + when(storePartitionerConfig.getPartitionerClass()).thenReturn("f.q.c.n.DefaultPartitioner"); + + // Test Case 1: Null partitionersFromRequest (should pass) + try { + CreateVersion.verifyAndConfigurePartitionerSettings(storePartitionerConfig, null, response); + } catch (Exception e) { + fail("Null partitionersFromRequest should not throw an exception."); + } + assertEquals(response.getPartitionerClass(), "f.q.c.n.DefaultPartitioner"); + + // Test Case 2: Empty partitionersFromRequest (should pass) + response = new VersionCreationResponse(); + Set partitionersFromRequest = Collections.emptySet(); + try { + CreateVersion.verifyAndConfigurePartitionerSettings(storePartitionerConfig, partitionersFromRequest, response); + } catch (Exception e) { + fail("Empty partitionersFromRequest should not throw an exception."); + } + assertEquals(response.getPartitionerClass(), "f.q.c.n.DefaultPartitioner"); + + // Test Case 3: Matching partitioner in partitionersFromRequest (should pass) + response = new VersionCreationResponse(); + partitionersFromRequest = new HashSet<>(Arrays.asList("f.q.c.n.DefaultPartitioner", "f.q.c.n.CustomPartitioner")); + try { + CreateVersion.verifyAndConfigurePartitionerSettings(storePartitionerConfig, partitionersFromRequest, response); + } catch (Exception e) { + fail("Matching partitioner should not throw an exception."); + } + assertEquals(response.getPartitionerClass(), "f.q.c.n.DefaultPartitioner"); + + // Test Case 4: Non-matching partitioner in partitionersFromRequest (should throw exception) + final VersionCreationResponse finalResponse = new VersionCreationResponse(); + partitionersFromRequest = new HashSet<>(Collections.singletonList("f.q.c.n.CustomPartitioner")); + Set finalPartitionersFromRequest = partitionersFromRequest; + Exception e = expectThrows( + VeniceException.class, + () -> CreateVersion.verifyAndConfigurePartitionerSettings( + storePartitionerConfig, + finalPartitionersFromRequest, + finalResponse)); + assertTrue(e.getMessage().contains("cannot be found")); + } + + @Test + public void testDetermineResponseTopic() { + String storeName = "test_store"; + String vtName = Version.composeKafkaTopic(storeName, 1); + String rtName = Version.composeRealTimeTopic(storeName); + String srTopicName = Version.composeStreamReprocessingTopic(storeName, 1); + String separateRtName = Version.composeSeparateRealTimeTopic(storeName); + + RequestTopicForPushRequest request = new RequestTopicForPushRequest("v0", storeName, INCREMENTAL, "JOB_ID"); + + // Test Case: PushType.INCREMENTAL with separate real-time topic enabled + Version mockVersion1 = mock(Version.class); + when(mockVersion1.kafkaTopicName()).thenReturn(vtName); + when(mockVersion1.isSeparateRealTimeTopicEnabled()).thenReturn(true); + request.setSeparateRealTimeTopicEnabled(true); + String result1 = CreateVersion.determineResponseTopic(storeName, mockVersion1, request); + assertEquals(result1, separateRtName); + + // Test Case: PushType.INCREMENTAL with separate real-time topic enabled, but the request does not have the separate + // real-time topic flag + mockVersion1 = mock(Version.class); + when(mockVersion1.kafkaTopicName()).thenReturn(vtName); + when(mockVersion1.isSeparateRealTimeTopicEnabled()).thenReturn(true); + request.setSeparateRealTimeTopicEnabled(false); + result1 = CreateVersion.determineResponseTopic(storeName, mockVersion1, request); + assertEquals(result1, rtName); + + // Test Case: PushType.INCREMENTAL without separate real-time topic enabled + Version mockVersion2 = mock(Version.class); + when(mockVersion2.kafkaTopicName()).thenReturn(vtName); + when(mockVersion2.isSeparateRealTimeTopicEnabled()).thenReturn(true); + request = new RequestTopicForPushRequest("v0", storeName, INCREMENTAL, "JOB_ID"); + String result2 = CreateVersion.determineResponseTopic(storeName, mockVersion2, request); + assertEquals(result2, rtName); + + // Test Case: PushType.STREAM + Version mockVersion3 = mock(Version.class); + when(mockVersion3.kafkaTopicName()).thenReturn(vtName); + request = new RequestTopicForPushRequest("v0", storeName, STREAM, "JOB_ID"); + String result3 = CreateVersion.determineResponseTopic(storeName, mockVersion3, request); + assertEquals(result3, rtName); + + // Test Case: PushType.STREAM_REPROCESSING + Version mockVersion4 = mock(Version.class); + when(mockVersion4.kafkaTopicName()).thenReturn(vtName); + when(mockVersion4.getNumber()).thenReturn(1); + request = new RequestTopicForPushRequest("v0", storeName, STREAM_REPROCESSING, "JOB_ID"); + String result4 = CreateVersion.determineResponseTopic(storeName, mockVersion4, request); + assertEquals(result4, srTopicName); + + // Test Case: Default case with a Kafka topic + Version mockVersion5 = mock(Version.class); + when(mockVersion5.kafkaTopicName()).thenReturn(vtName); + request = new RequestTopicForPushRequest("v0", storeName, BATCH, "JOB_ID"); + String result5 = CreateVersion.determineResponseTopic(storeName, mockVersion5, request); + assertEquals(result5, vtName); + } + + @Test + public void testGetCompressionStrategy() { + // Test Case 1: Real-time topic returns NO_OP + Version mockVersion1 = mock(Version.class); + String responseTopic1 = Version.composeRealTimeTopic("test_store"); + CompressionStrategy result1 = CreateVersion.getCompressionStrategy(mockVersion1, responseTopic1); + assertEquals(result1, CompressionStrategy.NO_OP); + + // Test Case 2: Non-real-time topic returns version's compression strategy + Version mockVersion2 = mock(Version.class); + String responseTopic2 = Version.composeKafkaTopic("test_store", 1); + when(mockVersion2.getCompressionStrategy()).thenReturn(CompressionStrategy.GZIP); + CompressionStrategy result2 = CreateVersion.getCompressionStrategy(mockVersion2, responseTopic2); + assertEquals(result2, CompressionStrategy.GZIP); + } + + @Test + public void testConfigureSourceFabric() { + // Test Case 1: Native replication enabled and non-incremental push type + Admin mockAdmin1 = mock(Admin.class); + Version mockVersion1 = mock(Version.class); + Lazy mockLazy1 = mock(Lazy.class); + RequestTopicForPushRequest mockRequest1 = mock(RequestTopicForPushRequest.class); + VersionCreationResponse mockResponse1 = new VersionCreationResponse(); + + when(mockVersion1.isNativeReplicationEnabled()).thenReturn(true); + when(mockVersion1.getPushStreamSourceAddress()).thenReturn("bootstrapServer1"); + when(mockVersion1.getNativeReplicationSourceFabric()).thenReturn("sourceFabric1"); + when(mockRequest1.getPushType()).thenReturn(BATCH); + + CreateVersion.configureSourceFabric(mockAdmin1, mockVersion1, mockLazy1, mockRequest1, mockResponse1); + + assertEquals(mockResponse1.getKafkaBootstrapServers(), "bootstrapServer1"); + assertEquals(mockResponse1.getKafkaSourceRegion(), "sourceFabric1"); + + // Test Case 2: Native replication enabled with null PushStreamSourceAddress + Admin mockAdmin2 = mock(Admin.class); + Version mockVersion2 = mock(Version.class); + Lazy mockLazy2 = mock(Lazy.class); + RequestTopicForPushRequest mockRequest2 = mock(RequestTopicForPushRequest.class); + VersionCreationResponse mockResponse2 = new VersionCreationResponse(); + + when(mockVersion2.isNativeReplicationEnabled()).thenReturn(true); + when(mockVersion2.getPushStreamSourceAddress()).thenReturn(null); + when(mockVersion2.getNativeReplicationSourceFabric()).thenReturn("sourceFabric2"); + when(mockRequest2.getPushType()).thenReturn(BATCH); + + CreateVersion.configureSourceFabric(mockAdmin2, mockVersion2, mockLazy2, mockRequest2, mockResponse2); + + assertNull(mockResponse2.getKafkaBootstrapServers()); + assertEquals(mockResponse2.getKafkaSourceRegion(), "sourceFabric2"); + + // Test Case 3: Incremental push with parent admin and override source region + Admin mockAdmin3 = mock(Admin.class); + Version mockVersion3 = mock(Version.class); + Lazy mockLazy3 = mock(Lazy.class); + RequestTopicForPushRequest mockRequest3 = mock(RequestTopicForPushRequest.class); + VersionCreationResponse mockResponse3 = new VersionCreationResponse(); + + when(mockAdmin3.isParent()).thenReturn(true); + when(mockVersion3.isNativeReplicationEnabled()).thenReturn(true); + when(mockRequest3.getPushType()).thenReturn(INCREMENTAL); + when(mockRequest3.getClusterName()).thenReturn("testCluster"); + when(mockRequest3.getStoreName()).thenReturn("testStore"); + when(mockRequest3.getEmergencySourceRegion()).thenReturn("emergencyRegion"); + when(mockRequest3.getSourceGridFabric()).thenReturn("gridFabric"); + when(mockLazy3.get()).thenReturn(true); + + when(mockAdmin3.getNativeReplicationKafkaBootstrapServerAddress("emergencyRegion")) + .thenReturn("emergencyRegionAddress"); + + CreateVersion.configureSourceFabric(mockAdmin3, mockVersion3, mockLazy3, mockRequest3, mockResponse3); + + assertEquals(mockResponse3.getKafkaBootstrapServers(), "emergencyRegionAddress"); + + // No specific assertions here since `overrideSourceRegionAddressForIncrementalPushJob` is mocked, + // but we can verify if the mock was called with appropriate parameters. + verify(mockAdmin3, times(1)).isParent(); + } + + @Test + public void testHandleStreamPushTypeInParentController() { + Admin admin = mock(Admin.class); + Store store = mock(Store.class); + when(store.getName()).thenReturn(STORE_NAME); + HybridStoreConfig hybridStoreConfig = mock(HybridStoreConfig.class); + when(store.getHybridStoreConfig()).thenReturn(hybridStoreConfig); + RequestTopicForPushRequest request = new RequestTopicForPushRequest("CLUSTER_NAME", STORE_NAME, STREAM, "JOB_ID"); + VersionCreationResponse response = new VersionCreationResponse(); + + // Case 1: Parent region; With stream pushes disabled + when(admin.isParent()).thenReturn(true); + CreateVersion createVersionNotOk = new CreateVersion(true, Optional.of(accessClient), false, true); + VeniceException ex1 = expectThrows( + VeniceException.class, + () -> createVersionNotOk.handleStreamPushType(admin, store, request, response, Lazy.of(() -> false))); + assertTrue( + ex1.getMessage().contains("Write operations to the parent region are not permitted with push type: STREAM")); + + CreateVersion createVersionOk = new CreateVersion(true, Optional.of(accessClient), false, false); + + // Case 2: Parent region; Non-aggregate mode in parent with no AA replication + when(admin.isParent()).thenReturn(true); + when(store.getHybridStoreConfig().getDataReplicationPolicy()).thenReturn(NON_AGGREGATE); + VeniceException ex2 = expectThrows( + VeniceException.class, + () -> createVersionOk.handleStreamPushType(admin, store, request, response, Lazy.of(() -> false))); + assertTrue(ex2.getMessage().contains("Store is not in aggregate mode!")); + + // Case 3: Parent region; Non-aggregate mode but AA replication enabled and no hybrid version + when(admin.isParent()).thenReturn(true); + when(store.getHybridStoreConfig().getDataReplicationPolicy()).thenReturn(NON_AGGREGATE); + when(store.isActiveActiveReplicationEnabled()).thenReturn(true); + when(admin.getReferenceVersionForStreamingWrites(anyString(), anyString(), anyString())).thenReturn(null); + VeniceException ex3 = expectThrows( + VeniceException.class, + () -> createVersionOk.handleStreamPushType(admin, store, request, response, Lazy.of(() -> true))); + assertTrue(ex3.getMessage().contains("No hybrid version found for store"), "Got: " + ex3.getMessage()); + + // Case 4: Parent region; Aggregate mode but no hybrid version + when(admin.isParent()).thenReturn(true); + when(store.getHybridStoreConfig().getDataReplicationPolicy()).thenReturn(AGGREGATE); + when(store.isActiveActiveReplicationEnabled()).thenReturn(false); + when(admin.getReferenceVersionForStreamingWrites(anyString(), anyString(), anyString())).thenReturn(null); + VeniceException ex4 = expectThrows( + VeniceException.class, + () -> createVersionOk.handleStreamPushType(admin, store, request, response, Lazy.of(() -> true))); + assertTrue(ex4.getMessage().contains("No hybrid version found for store"), "Got: " + ex4.getMessage()); + + // Case 5: Parent region; Aggregate mode and there is a hybrid version + Version mockVersion = mock(Version.class); + when(mockVersion.getPartitionCount()).thenReturn(42); + when(admin.isParent()).thenReturn(true); + when(store.getHybridStoreConfig().getDataReplicationPolicy()).thenReturn(AGGREGATE); + when(admin.getReferenceVersionForStreamingWrites(anyString(), anyString(), anyString())).thenReturn(mockVersion); + createVersionOk.handleStreamPushType(admin, store, request, response, Lazy.of(() -> true)); + assertEquals(response.getPartitions(), 42); + assertEquals(response.getCompressionStrategy(), CompressionStrategy.NO_OP); + assertEquals(response.getKafkaTopic(), Version.composeRealTimeTopic(STORE_NAME)); + } + + @Test + public void testHandleStreamPushTypeInChildController() { + Admin admin = mock(Admin.class); + Store store = mock(Store.class); + when(store.getName()).thenReturn(STORE_NAME); + HybridStoreConfig hybridStoreConfig = mock(HybridStoreConfig.class); + when(store.getHybridStoreConfig()).thenReturn(hybridStoreConfig); + RequestTopicForPushRequest request = new RequestTopicForPushRequest("CLUSTER_NAME", STORE_NAME, STREAM, "JOB_ID"); + VersionCreationResponse response = new VersionCreationResponse(); + CreateVersion createVersionOk = new CreateVersion(true, Optional.of(accessClient), false, false); + + // Case 1: Child region; Aggregate mode in child and AA not enabled + when(admin.isParent()).thenReturn(false); + when(store.getHybridStoreConfig().getDataReplicationPolicy()).thenReturn(DataReplicationPolicy.AGGREGATE); + when(store.isActiveActiveReplicationEnabled()).thenReturn(false); + VeniceException ex5 = expectThrows( + VeniceException.class, + () -> createVersionOk.handleStreamPushType(admin, store, request, response, Lazy.of(() -> false))); + assertTrue(ex5.getMessage().contains("Store is in aggregate mode and AA is not enabled")); + + // Case 2: Child region; Aggregate mode but AA is enabled in all regions but no hybrid version + when(admin.isParent()).thenReturn(false); + when(store.isActiveActiveReplicationEnabled()).thenReturn(true); + when(store.getHybridStoreConfig().getDataReplicationPolicy()).thenReturn(DataReplicationPolicy.AGGREGATE); + when(admin.getReferenceVersionForStreamingWrites(anyString(), anyString(), anyString())).thenReturn(null); + VeniceException ex6 = expectThrows( + VeniceException.class, + () -> createVersionOk.handleStreamPushType(admin, store, request, response, Lazy.of(() -> true))); + assertTrue(ex6.getMessage().contains("No hybrid version found"), "Got: " + ex6.getMessage()); + + // Case 3: Child region; Non-aggregate mode but no hybrid version + when(admin.isParent()).thenReturn(false); + when(store.getHybridStoreConfig().getDataReplicationPolicy()).thenReturn(DataReplicationPolicy.NON_AGGREGATE); + when(admin.getReferenceVersionForStreamingWrites(anyString(), anyString(), anyString())).thenReturn(null); + VeniceException ex7 = expectThrows( + VeniceException.class, + () -> createVersionOk.handleStreamPushType(admin, store, request, response, Lazy.of(() -> true))); + assertTrue(ex7.getMessage().contains("No hybrid version found"), "Got: " + ex7.getMessage()); + + // Case 4: Child region; Non-aggregate mode and there is a hybrid version + Version mockVersion = mock(Version.class); + when(mockVersion.getPartitionCount()).thenReturn(42); + when(admin.isParent()).thenReturn(false); + when(store.getHybridStoreConfig().getDataReplicationPolicy()).thenReturn(DataReplicationPolicy.NON_AGGREGATE); + when(admin.getReferenceVersionForStreamingWrites(anyString(), anyString(), anyString())).thenReturn(mockVersion); + createVersionOk.handleStreamPushType(admin, store, request, response, Lazy.of(() -> true)); + assertEquals(response.getPartitions(), 42); + assertEquals(response.getCompressionStrategy(), CompressionStrategy.NO_OP); + assertEquals(response.getKafkaTopic(), Version.composeRealTimeTopic(STORE_NAME)); + } } diff --git a/services/venice-router/src/test/java/com/linkedin/venice/router/api/TestVeniceDelegateMode.java b/services/venice-router/src/test/java/com/linkedin/venice/router/api/TestVeniceDelegateMode.java index 36696226916..e9983fc7b2a 100644 --- a/services/venice-router/src/test/java/com/linkedin/venice/router/api/TestVeniceDelegateMode.java +++ b/services/venice-router/src/test/java/com/linkedin/venice/router/api/TestVeniceDelegateMode.java @@ -2,7 +2,7 @@ import static com.linkedin.venice.router.api.VeniceMultiKeyRoutingStrategy.HELIX_ASSISTED_ROUTING; import static com.linkedin.venice.router.api.VeniceMultiKeyRoutingStrategy.LEAST_LOADED_ROUTING; -import static org.mockito.ArgumentMatchers.*; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.atLeastOnce; diff --git a/services/venice-router/src/test/java/com/linkedin/venice/router/api/path/TestVeniceComputePath.java b/services/venice-router/src/test/java/com/linkedin/venice/router/api/path/TestVeniceComputePath.java index 494f3031421..f4672683dc1 100644 --- a/services/venice-router/src/test/java/com/linkedin/venice/router/api/path/TestVeniceComputePath.java +++ b/services/venice-router/src/test/java/com/linkedin/venice/router/api/path/TestVeniceComputePath.java @@ -1,7 +1,7 @@ package com.linkedin.venice.router.api.path; import static com.linkedin.venice.compute.ComputeRequestWrapper.LATEST_SCHEMA_VERSION_FOR_COMPUTE_REQUEST; -import static org.mockito.ArgumentMatchers.*; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.mock; From 7f27513d2f0281279b26d5bd261092003bf58f5f Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Wed, 18 Dec 2024 00:43:49 -0800 Subject: [PATCH 2/2] Address review comments and more test --- .../RequestTopicForPushRequest.java | 9 -- .../com/linkedin/venice/meta/Version.java | 49 ++++++-- .../RequestTopicForPushRequestTest.java | 13 --- .../com/linkedin/venice/meta/TestVersion.java | 108 ++++++++++++++---- .../controller/server/CreateVersion.java | 2 +- 5 files changed, 129 insertions(+), 52 deletions(-) diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequest.java b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequest.java index 5d14b42ca63..580ee2de74c 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequest.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequest.java @@ -49,15 +49,6 @@ public RequestTopicForPushRequest(String clusterName, String storeName, PushType this.pushJobId = pushJobId; } - public static PushType extractPushType(String pushTypeString) { - try { - return PushType.valueOf(pushTypeString); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException( - pushTypeString + " is an invalid push type. Valid push types are: " + Arrays.toString(PushType.values())); - } - } - public String getClusterName() { return clusterName; } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/meta/Version.java b/internal/venice-common/src/main/java/com/linkedin/venice/meta/Version.java index 406422e8ed2..1f9f08e42e1 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/meta/Version.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/meta/Version.java @@ -10,10 +10,9 @@ import com.linkedin.venice.systemstore.schemas.StoreVersion; import com.linkedin.venice.views.VeniceView; import java.time.Duration; -import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; /** @@ -41,11 +40,21 @@ public interface Version extends Comparable, DataModelBackedStructure VALUE_TO_TYPE_MAP = new HashMap<>(4); + private static final Map NAME_TO_TYPE_MAP = new HashMap<>(4); + + // Static initializer for map population + static { + for (PushType type: PushType.values()) { + VALUE_TO_TYPE_MAP.put(type.value, type); + NAME_TO_TYPE_MAP.put(type.name(), type); + } + } PushType(int value) { this.value = value; @@ -68,15 +77,41 @@ public boolean isStreamReprocessing() { } public boolean isBatchOrStreamReprocessing() { - return isBatch() || isStreamReprocessing(); + return this == BATCH || this == STREAM_REPROCESSING; } + /** + * Retrieve the PushType based on its integer value. + * + * @param value the integer value of the PushType + * @return the corresponding PushType + * @throws VeniceException if the value is invalid + */ public static PushType valueOf(int value) { - Optional pushType = Arrays.stream(values()).filter(p -> p.value == value).findFirst(); - if (!pushType.isPresent()) { + PushType pushType = VALUE_TO_TYPE_MAP.get(value); + if (pushType == null) { throw new VeniceException("Invalid push type with int value: " + value); } - return pushType.get(); + return pushType; + } + + /** + * Extracts the PushType from its string name. + * + * @param pushTypeString the string representation of the PushType + * @return the corresponding PushType + * @throws IllegalArgumentException if the string is invalid + */ + public static PushType extractPushType(String pushTypeString) { + PushType pushType = NAME_TO_TYPE_MAP.get(pushTypeString); + if (pushType == null) { + throw new IllegalArgumentException( + String.format( + "%s is an invalid push type. Valid push types are: %s", + pushTypeString, + String.join(", ", NAME_TO_TYPE_MAP.keySet()))); + } + return pushType; } } diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequestTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequestTest.java index 2c8b2596ee0..34e4be44c8b 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequestTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/controllerapi/RequestTopicForPushRequestTest.java @@ -1,7 +1,6 @@ package com.linkedin.venice.controllerapi; import static com.linkedin.venice.meta.Version.PushType.BATCH; -import static com.linkedin.venice.meta.Version.PushType.STREAM; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; @@ -53,18 +52,6 @@ public void testRequestTopicForPushRequestConstructorArgs() { assertEquals(ex4.getMessage(), "pushJobId is required"); } - @Test - public void testExtractPushTypeValidAndInvalidValues() { - // Valid cases - assertEquals(RequestTopicForPushRequest.extractPushType("BATCH"), BATCH); - assertEquals(RequestTopicForPushRequest.extractPushType("STREAM"), STREAM); - - // Invalid case - IllegalArgumentException ex = Assert - .expectThrows(IllegalArgumentException.class, () -> RequestTopicForPushRequest.extractPushType("INVALID")); - assertTrue(ex.getMessage().contains("INVALID is an invalid push type")); - } - @Test public void testRequestTopicForPushRequestSettersAndGetters() { request.setSendStartOfPush(true); diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/meta/TestVersion.java b/internal/venice-common/src/test/java/com/linkedin/venice/meta/TestVersion.java index e261b956b88..f0f25ca0103 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/meta/TestVersion.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/meta/TestVersion.java @@ -1,7 +1,13 @@ package com.linkedin.venice.meta; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; + import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.meta.Version.PushType; import com.linkedin.venice.utils.ObjectMapperFactory; import com.linkedin.venice.utils.Utils; import java.io.IOException; @@ -25,7 +31,7 @@ public class TestVersion { @Test public void identifiesValidTopicNames() { String goodTopic = "my_very_good_store_v4"; - Assert.assertTrue( + assertTrue( Version.isVersionTopicOrStreamReprocessingTopic(goodTopic), goodTopic + " should parse as a valid store-version topic"); @@ -41,7 +47,7 @@ public void serializes() throws IOException { int versionNumber = 17; Version version = new VersionImpl(storeName, versionNumber); String serialized = OBJECT_MAPPER.writeValueAsString(version); - Assert.assertTrue(serialized.contains(storeName)); + assertTrue(serialized.contains(storeName)); } /** @@ -52,21 +58,21 @@ public void serializes() throws IOException { @Test public void deserializeWithWrongFields() throws IOException { Version oldParsedVersion = OBJECT_MAPPER.readValue(OLD_SERIALIZED, Version.class); - Assert.assertEquals(oldParsedVersion.getStoreName(), "store-1492637190910-78714331"); + assertEquals(oldParsedVersion.getStoreName(), "store-1492637190910-78714331"); Version newParsedVersion = OBJECT_MAPPER.readValue(EXTRA_FIELD_SERIALIZED, Version.class); - Assert.assertEquals(newParsedVersion.getStoreName(), "store-1492637190910-12345678"); + assertEquals(newParsedVersion.getStoreName(), "store-1492637190910-12345678"); Version legacyParsedVersion = OBJECT_MAPPER.readValue(MISSING_FIELD_SERIALIZED, Version.class); - Assert.assertEquals(legacyParsedVersion.getStoreName(), "store-missing"); - Assert.assertNotNull(legacyParsedVersion.getPushJobId()); // missing final field can still deserialize, just gets - // arbitrary value from constructor + assertEquals(legacyParsedVersion.getStoreName(), "store-missing"); + assertNotNull(legacyParsedVersion.getPushJobId()); // missing final field can still deserialize, just gets + // arbitrary value from constructor } @Test public void testParseStoreFromRealTimeTopic() { String validRealTimeTopic = "abc_rt"; - Assert.assertEquals(Version.parseStoreFromRealTimeTopic(validRealTimeTopic), "abc"); + assertEquals(Version.parseStoreFromRealTimeTopic(validRealTimeTopic), "abc"); String invalidRealTimeTopic = "abc"; try { Version.parseStoreFromRealTimeTopic(invalidRealTimeTopic); @@ -80,19 +86,19 @@ public void testParseStoreFromRealTimeTopic() { public void testIsTopic() { String topic = "abc_rt"; Assert.assertFalse(Version.isVersionTopic(topic)); - Assert.assertTrue(Version.isRealTimeTopic(topic)); + assertTrue(Version.isRealTimeTopic(topic)); topic = "abc"; Assert.assertFalse(Version.isVersionTopic(topic)); topic = "abc_v12df"; Assert.assertFalse(Version.isVersionTopic(topic)); topic = "abc_v123"; - Assert.assertTrue(Version.isVersionTopic(topic)); + assertTrue(Version.isVersionTopic(topic)); Assert.assertFalse(Version.isRealTimeTopic(topic)); - Assert.assertTrue(Version.isVersionTopicOrStreamReprocessingTopic(topic)); + assertTrue(Version.isVersionTopicOrStreamReprocessingTopic(topic)); topic = "abc_v123_sr"; Assert.assertFalse(Version.isVersionTopic(topic)); - Assert.assertTrue(Version.isStreamReprocessingTopic(topic)); - Assert.assertTrue(Version.isVersionTopicOrStreamReprocessingTopic(topic)); + assertTrue(Version.isStreamReprocessingTopic(topic)); + assertTrue(Version.isVersionTopicOrStreamReprocessingTopic(topic)); topic = "abc_v12ab3_sr"; Assert.assertFalse(Version.isVersionTopic(topic)); Assert.assertFalse(Version.isStreamReprocessingTopic(topic)); @@ -108,41 +114,99 @@ public void testIsATopicThatIsVersioned() { String topic = "abc_rt"; Assert.assertFalse(Version.isATopicThatIsVersioned(topic)); topic = "abc_v1_sr"; - Assert.assertTrue(Version.isATopicThatIsVersioned(topic)); + assertTrue(Version.isATopicThatIsVersioned(topic)); topic = "abc_v1"; - Assert.assertTrue(Version.isATopicThatIsVersioned(topic)); + assertTrue(Version.isATopicThatIsVersioned(topic)); topic = "abc_v1_cc"; - Assert.assertTrue(Version.isATopicThatIsVersioned(topic)); + assertTrue(Version.isATopicThatIsVersioned(topic)); } @Test public void testParseStoreFromKafkaTopicName() { String storeName = "abc"; String topic = "abc_rt"; - Assert.assertEquals(Version.parseStoreFromKafkaTopicName(topic), storeName); + assertEquals(Version.parseStoreFromKafkaTopicName(topic), storeName); topic = "abc_v1"; - Assert.assertEquals(Version.parseStoreFromKafkaTopicName(topic), storeName); + assertEquals(Version.parseStoreFromKafkaTopicName(topic), storeName); topic = "abc_v1_cc"; - Assert.assertEquals(Version.parseStoreFromKafkaTopicName(topic), storeName); + assertEquals(Version.parseStoreFromKafkaTopicName(topic), storeName); } @Test public void testParseVersionFromKafkaTopicName() { int version = 1; String topic = "abc_v1"; - Assert.assertEquals(Version.parseVersionFromVersionTopicName(topic), version); + assertEquals(Version.parseVersionFromVersionTopicName(topic), version); topic = "abc_v1_cc"; - Assert.assertEquals(Version.parseVersionFromKafkaTopicName(topic), version); + assertEquals(Version.parseVersionFromKafkaTopicName(topic), version); } @Test void testVersionStatus() { for (VersionStatus status: VersionStatus.values()) { if (status == VersionStatus.KILLED) { - Assert.assertTrue(VersionStatus.isVersionKilled(status)); + assertTrue(VersionStatus.isVersionKilled(status)); } else { Assert.assertFalse(VersionStatus.isVersionKilled(status)); } } } + + @Test + public void testExtractPushType() { + // Case 1: Valid push types + assertEquals(PushType.extractPushType("BATCH"), PushType.BATCH); + assertEquals(PushType.extractPushType("STREAM_REPROCESSING"), PushType.STREAM_REPROCESSING); + assertEquals(PushType.extractPushType("STREAM"), PushType.STREAM); + assertEquals(PushType.extractPushType("INCREMENTAL"), PushType.INCREMENTAL); + + // Case 2: Invalid push type + String invalidType = "INVALID_TYPE"; + IllegalArgumentException invalidException = + expectThrows(IllegalArgumentException.class, () -> PushType.extractPushType(invalidType)); + assertTrue(invalidException.getMessage().contains(invalidType)); + assertTrue(invalidException.getMessage().contains("Valid push types are")); + + // Case 3: Case sensitivity + String lowerCaseType = "batch"; + IllegalArgumentException caseException = + expectThrows(IllegalArgumentException.class, () -> PushType.extractPushType(lowerCaseType)); + assertTrue(caseException.getMessage().contains(lowerCaseType)); + + // Case 4: Empty string + String emptyInput = ""; + IllegalArgumentException emptyException = + expectThrows(IllegalArgumentException.class, () -> PushType.extractPushType(emptyInput)); + assertTrue(emptyException.getMessage().contains(emptyInput)); + + // Case 5: Null input + IllegalArgumentException exception = + expectThrows(IllegalArgumentException.class, () -> PushType.extractPushType(null)); + assertNotNull(exception); + } + + @Test + public void testValueOfIntReturnsPushType() { + // Case 1: Valid integer values + assertEquals(PushType.valueOf(0), PushType.BATCH); + assertEquals(PushType.valueOf(1), PushType.STREAM_REPROCESSING); + assertEquals(PushType.valueOf(2), PushType.STREAM); + assertEquals(PushType.valueOf(3), PushType.INCREMENTAL); + + // Case 2: Invalid integer value (negative) + int invalidNegative = -1; + VeniceException negativeException = expectThrows(VeniceException.class, () -> PushType.valueOf(invalidNegative)); + assertTrue(negativeException.getMessage().contains("Invalid push type with int value: " + invalidNegative)); + + // Case 3: Invalid integer value (positive out of range) + int invalidPositive = 999; + VeniceException positiveException = expectThrows(VeniceException.class, () -> PushType.valueOf(invalidPositive)); + assertTrue(positiveException.getMessage().contains("Invalid push type with int value: " + invalidPositive)); + + // Case 4: Edge case - Valid minimum value + assertEquals(PushType.valueOf(0), PushType.BATCH); + + // Case 5: Edge case - Valid maximum value + assertEquals(PushType.valueOf(3), PushType.INCREMENTAL); + } } diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java index 9886dab7534..86e807f2123 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java @@ -474,7 +474,7 @@ public Route requestTopicForPushing(Admin admin) { RequestTopicForPushRequest requestTopicForPushRequest = new RequestTopicForPushRequest( request.queryParams(CLUSTER), request.queryParams(NAME), - RequestTopicForPushRequest.extractPushType(request.queryParams(PUSH_TYPE)), + PushType.extractPushType(request.queryParams(PUSH_TYPE)), request.queryParams(PUSH_JOB_ID)); // populate the request object with optional parameters