From b3dc21efe86445107e0111fd7e6397f085ee0a40 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Mon, 5 May 2025 10:47:14 -0700 Subject: [PATCH 01/25] initial commit --- ...gConsumerDaVinciRecordTransformerImpl.java | 2 + .../consumer/ChangelogClientConfig.java | 14 +- .../venice/client/store/ClientConfig.java | 11 + .../TestBootstrappingChangelogConsumer.java | 269 ++++++++++++++++-- 4 files changed, 278 insertions(+), 18 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java index 35ed28e4b39..477f657ac72 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java @@ -96,6 +96,8 @@ public BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl(Changelo * DVRT implmentation. This is to prevent the local state from being wiped everytime a change is deployed */ .setSkipCompatibilityChecks(true) + .setOutputValueClass(innerClientConfig.getSpecificValueClass()) + .setOutputValueSchema(innerClientConfig.getSpecificValueSchema()) .build(); daVinciConfig.setRecordTransformerConfig(recordTransformerConfig); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java index 7e5c8c2d159..ab56751d2e8 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java @@ -8,6 +8,7 @@ import java.util.Objects; import java.util.Properties; import javax.annotation.Nonnull; +import org.apache.avro.Schema; import org.apache.avro.specific.SpecificRecord; @@ -240,6 +241,11 @@ public ChangelogClientConfig setSpecificValue(Class specificValue) { return this; } + public ChangelogClientConfig setSpecificValueSchema(Schema specificValueSchema) { + this.innerClientConfig.setSpecificValueSchema(specificValueSchema); + return this; + } + public ChangelogClientConfig setShouldSkipFailedToAssembleRecords(boolean skipFailedToAssembleRecords) { this.skipFailedToAssembleRecords = skipFailedToAssembleRecords; return this; @@ -271,7 +277,8 @@ public static ChangelogClientConfig cloneConfig(Ch .setIsExperimentalClientEnabled(config.isExperimentalClientEnabled()) .setMaxBufferSize(config.getMaxBufferSize()) .setSeekThreadPoolSize(config.getSeekThreadPoolSize()) - .setShouldSkipFailedToAssembleRecords(config.shouldSkipFailedToAssembleRecords()); + .setShouldSkipFailedToAssembleRecords(config.shouldSkipFailedToAssembleRecords()) + .setInnerClientConfig(config.getInnerClientConfig()); return newConfig; } @@ -312,4 +319,9 @@ public ChangelogClientConfig setMaxBufferSize(int maxBufferSize) { this.maxBufferSize = maxBufferSize; return this; } + + private ChangelogClientConfig setInnerClientConfig(ClientConfig innerClientConfig) { + this.innerClientConfig = innerClientConfig; + return this; + } } diff --git a/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java b/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java index 03651cbb5ca..0921de7dfd6 100644 --- a/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java +++ b/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java @@ -32,6 +32,7 @@ public class ClientConfig { private String veniceURL; private String statsPrefix; private Class specificValueClass = null; + private Schema specificValueSchema = null; private boolean isVsonClient = false; // D2 specific settings @@ -98,6 +99,7 @@ public static ClientConfig cloneConfig(ClientConfi .setStoreName(config.getStoreName()) .setVeniceURL(config.getVeniceURL()) .setSpecificValueClass(config.getSpecificValueClass()) + .setSpecificValueSchema(config.getSpecificValueSchema()) .setVsonClient(config.isVsonClient()) // D2 specific settings @@ -191,6 +193,15 @@ public ClientConfig setStatsPrefix(String statsPrefix) { return this; } + public Schema getSpecificValueSchema() { + return specificValueSchema; + } + + public ClientConfig setSpecificValueSchema(Schema specificValueSchema) { + this.specificValueSchema = specificValueSchema; + return this; + } + public Class getSpecificValueClass() { return specificValueClass; } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index 4831e948000..cd1f4681e0d 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -32,7 +32,11 @@ import static com.linkedin.venice.utils.SslUtils.LOCAL_KEYSTORE_JKS; import static com.linkedin.venice.utils.SslUtils.LOCAL_PASSWORD; import static com.linkedin.venice.utils.TestWriteUtils.DEFAULT_USER_DATA_RECORD_COUNT; +import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V1_SCHEMA; +import static com.linkedin.venice.utils.TestWriteUtils.STRING_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.utils.TestWriteUtils.renderNameRecord; +import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFile; import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; @@ -54,6 +58,7 @@ import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.MultiStoreTopicsResponse; import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; +import com.linkedin.venice.endToEnd.TestChangelogValue; import com.linkedin.venice.integration.utils.PubSubBrokerConfigs; import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; import com.linkedin.venice.integration.utils.ServiceFactory; @@ -67,6 +72,7 @@ import com.linkedin.venice.store.rocksdb.RocksDBUtils; import com.linkedin.venice.utils.DataProviderUtils; import com.linkedin.venice.utils.ForkedJavaProcess; +import com.linkedin.venice.utils.PushInputSchemaBuilder; import com.linkedin.venice.utils.SslUtils; import com.linkedin.venice.utils.TestMockTime; import com.linkedin.venice.utils.TestUtils; @@ -151,7 +157,7 @@ public void cleanUp() { @Test(timeOut = TEST_TIMEOUT, dataProvider = "changelogConsumer", dataProviderClass = DataProviderUtils.class, priority = 3) public void testVeniceChangelogConsumer(int consumerCount) throws Exception { String storeName = Utils.getUniqueString("store"); - String inputDirPath = setUpStore(storeName); + String inputDirPath = setUpStore(storeName, false); Map samzaConfig = getSamzaProducerConfig(clusterWrapper, storeName, Version.PushType.STREAM); VeniceSystemFactory factory = new VeniceSystemFactory(); @@ -185,7 +191,7 @@ public void testVeniceChangelogConsumer(int consumerCount) throws Exception { factory.getClosableProducer("venice", new MapConfig(samzaConfig), null)) { veniceProducer.start(); // Run Samza job to send PUT and DELETE requests. - runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100); + runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, false); // Produce a DELETE record with large timestamp sendStreamingRecordWithLogicalTimestamp(veniceProducer, storeName, deleteWithRmdKeyIndex, 1000, true); } @@ -262,7 +268,7 @@ public void testVeniceChangelogConsumer(int consumerCount) throws Exception { @Test(timeOut = TEST_TIMEOUT * 3, priority = 3) public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { String storeName = Utils.getUniqueString("store"); - String inputDirPath = setUpStore(storeName); + String inputDirPath = setUpStore(storeName, false); Map samzaConfig = getSamzaProducerConfig(clusterWrapper, storeName, Version.PushType.STREAM); VeniceSystemFactory factory = new VeniceSystemFactory(); @@ -298,7 +304,7 @@ public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exc factory.getClosableProducer("venice", new MapConfig(samzaConfig), null)) { veniceProducer.start(); // Run Samza job to send PUT and DELETE requests. - runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100); + runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, false); // Produce a DELETE record with large timestamp sendStreamingRecordWithLogicalTimestamp(veniceProducer, storeName, deleteWithRmdKeyIndex, 1000, true); } @@ -403,7 +409,7 @@ public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exc @Test(timeOut = TEST_TIMEOUT, priority = 3) public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { String storeName = Utils.getUniqueString("store"); - String inputDirPath1 = setUpStore(storeName); + String inputDirPath1 = setUpStore(storeName, false); String inputDirPath2 = Utils.getTempDataDirectory().getAbsolutePath(); Map samzaConfig = getSamzaProducerConfig(clusterWrapper, storeName, Version.PushType.STREAM); VeniceSystemFactory factory = new VeniceSystemFactory(); @@ -461,7 +467,7 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( factory.getClosableProducer("venice", new MapConfig(samzaConfig), null)) { veniceProducer.start(); // Run Samza job to send PUT and DELETE requests. - runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100); + runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, true); } // Spin up a DVRT CDC instance and wait for it to consume everything, then perform blob transfer @@ -520,6 +526,87 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( } } + @Test(timeOut = TEST_TIMEOUT, priority = 3) + public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { + String storeName = Utils.getUniqueString("store"); + String inputDirPath = setUpStore(storeName, true); + Map samzaConfig = getSamzaProducerConfig(clusterWrapper, storeName, Version.PushType.STREAM); + VeniceSystemFactory factory = new VeniceSystemFactory(); + + try (PubSubBrokerWrapper localKafka = ServiceFactory.getPubSubBroker( + new PubSubBrokerConfigs.Builder().setZkWrapper(clusterWrapper.getZk()) + .setMockTime(testMockTime) + .setRegionName(REGION_NAME) + .build())) { + Properties consumerProperties = new Properties(); + String localKafkaUrl = localKafka.getAddress(); + consumerProperties.put(KAFKA_BOOTSTRAP_SERVERS, localKafkaUrl); + consumerProperties.put(CLUSTER_NAME, clusterName); + consumerProperties.put(ZOOKEEPER_ADDRESS, zkAddress); + ChangelogClientConfig globalChangelogClientConfig = + new ChangelogClientConfig().setConsumerProperties(consumerProperties) + .setControllerD2ServiceName(D2_SERVICE_NAME) + .setD2ServiceName(VeniceRouterWrapper.CLUSTER_DISCOVERY_D2_SERVICE_NAME) + .setLocalD2ZkHosts(zkAddress) + .setControllerRequestRetryCount(3) + .setBootstrapFileSystemPath(inputDirPath) + .setIsExperimentalClientEnabled(true) + .setSpecificValue(TestChangelogValue.class) + .setSpecificValueSchema(NAME_RECORD_V1_SCHEMA) + .setD2Client(d2Client); + VeniceChangelogConsumerClientFactory veniceChangelogConsumerClientFactory = + new VeniceChangelogConsumerClientFactory(globalChangelogClientConfig, metricsRepository); + List> bootstrappingVeniceChangelogConsumerList = + Collections.singletonList( + veniceChangelogConsumerClientFactory + .getBootstrappingChangelogConsumer(storeName, Integer.toString(0), TestChangelogValue.class)); + + try (VeniceSystemProducer veniceProducer = + factory.getClosableProducer("venice", new MapConfig(samzaConfig), null)) { + veniceProducer.start(); + // Run Samza job to send PUT and DELETE requests. + runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, true); + } + + bootstrappingVeniceChangelogConsumerList.get(0).start().get(); + + Map, VeniceChangeCoordinate>> polledChangeEventsMap = + new HashMap<>(); + List, VeniceChangeCoordinate>> polledChangeEventsList = + new ArrayList<>(); + // 20 changes in near-line. 10 puts, 10 deletes + TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + pollChangeEventsFromSpecificChangeCaptureConsumer( + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumerList); + // 20 events for near-line events + int expectedRecordCount = DEFAULT_USER_DATA_RECORD_COUNT + 20; + Assert.assertEquals(polledChangeEventsList.size(), expectedRecordCount); + verifySpecificPut(polledChangeEventsMap, 100, 110, 1); + verifySpecificDelete(polledChangeEventsMap, 110, 120, 1); + }); + polledChangeEventsList.clear(); + polledChangeEventsMap.clear(); + + runSpecificNearlineJobAndVerifyConsumption( + 120, + storeName, + 1, + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumerList); + + // Since nothing is produced, so no changed events generated. + verifyNoSpecificRecordsProduced( + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumerList); + + cleanUpStoreAndVerify(storeName); + } + } + private void pollChangeEventsFromChangeCaptureConsumerToList( List, VeniceChangeCoordinate>> polledChangeEvents, VeniceChangelogConsumer veniceChangelogConsumer) { @@ -545,21 +632,50 @@ private void pollChangeEventsFromChangeCaptureConsumer( } } + private void pollChangeEventsFromSpecificChangeCaptureConsumer( + Map, VeniceChangeCoordinate>> keyToMessageMap, + List, VeniceChangeCoordinate>> polledMessageList, + List> bootstrappingVeniceChangelogConsumerList) { + for (BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer: bootstrappingVeniceChangelogConsumerList) { + Collection, VeniceChangeCoordinate>> pubSubMessages = + bootstrappingVeniceChangelogConsumer.poll(1000); + for (PubSubMessage, VeniceChangeCoordinate> pubSubMessage: pubSubMessages) { + String key = pubSubMessage.getKey() == null ? null : pubSubMessage.getKey().toString(); + keyToMessageMap.put(key, pubSubMessage); + } + polledMessageList.addAll(pubSubMessages); + } + } + private void runSamzaStreamJob( VeniceSystemProducer veniceProducer, String storeName, Time mockedTime, int numPuts, int numDels, - int startIdx) { + int startIdx, + boolean useSpecificRecord) { // Send PUT requests. for (int i = startIdx; i < startIdx + numPuts; i++) { - sendStreamingRecord( - veniceProducer, - storeName, - Integer.toString(i), - "stream_" + i, - mockedTime == null ? null : mockedTime.getMilliseconds()); + if (useSpecificRecord) { + TestChangelogValue value = new TestChangelogValue(); + value.put("firstName", "first_name_stream_" + i); + value.put("lastName", "last_name_stream_" + i); + + sendStreamingRecord( + veniceProducer, + storeName, + Integer.toString(i), + value, + mockedTime == null ? null : mockedTime.getMilliseconds()); + } else { + sendStreamingRecord( + veniceProducer, + storeName, + Integer.toString(i), + "stream_" + i, + mockedTime == null ? null : mockedTime.getMilliseconds()); + } } // Send DELETE requests. for (int i = startIdx + numPuts; i < startIdx + numPuts + numDels; i++) { @@ -573,15 +689,33 @@ private void runSamzaStreamJob( /** * @param storeName the name of the store + * @param useSpecificRecord Whether to push data using a specific record * @return the path that's being used for the test */ - private String setUpStore(String storeName) throws Exception { + private String setUpStore(String storeName, boolean useSpecificRecord) throws Exception { File inputDir = getTempDataDirectory(); - Schema recordSchema = TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema(inputDir); String inputDirPath = "file://" + inputDir.getAbsolutePath(); Properties props = defaultVPJProps(clusterWrapper, inputDirPath, storeName); + + Schema recordSchema; + String valueSchemaStr; + if (useSpecificRecord) { + Schema recordSchema1 = + new PushInputSchemaBuilder().setKeySchema(STRING_SCHEMA).setValueSchema(TestChangelogValue.SCHEMA$).build(); + // recordSchema = TestWriteUtils.writeSimpleAvroFileWithStringToNameRecordV1Schema(inputDir); + recordSchema = writeSimpleAvroFile( + inputDir, + recordSchema1, + i -> renderNameRecord(recordSchema1, i), + DEFAULT_USER_DATA_RECORD_COUNT); + valueSchemaStr = TestChangelogValue.SCHEMA$.toString(); + } else { + recordSchema = TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema(inputDir); + valueSchemaStr = recordSchema.getField(DEFAULT_VALUE_FIELD_PROP).schema().toString(); + } String keySchemaStr = recordSchema.getField(DEFAULT_KEY_FIELD_PROP).schema().toString(); - String valueSchemaStr = recordSchema.getField(DEFAULT_VALUE_FIELD_PROP).schema().toString(); + ; + UpdateStoreQueryParams storeParms = new UpdateStoreQueryParams().setHybridRewindSeconds(500) .setHybridOffsetLagThreshold(8) .setChunkingEnabled(true) @@ -594,6 +728,7 @@ private String setUpStore(String storeName) throws Exception { clusterWrapper.createMetaSystemStore(storeName); clusterWrapper.createPushStatusSystemStore(storeName); TestUtils.assertCommand(controllerClient.updateStore(storeName, storeParms)); + TestUtils.assertCommand(controllerClient.addValueSchema(storeName, valueSchemaStr)); TestWriteUtils.runPushJob("Run push job", props); } @@ -621,7 +756,7 @@ private int runNearlineJobAndVerifyConsumption( VeniceSystemProducer veniceProducer = factory.getClosableProducer("venice", new MapConfig(samzaConfig), null)) { veniceProducer.start(); // Run Samza job to send PUT and DELETE requests. - runSamzaStreamJob(veniceProducer, storeName, null, numPuts, numDeletes, startIndex); + runSamzaStreamJob(veniceProducer, storeName, null, numPuts, numDeletes, startIndex, false); } try (AvroGenericStoreClient client = ClientFactory.getAndStartGenericAvroClient( @@ -650,6 +785,56 @@ private int runNearlineJobAndVerifyConsumption( return startIndex + recordsToProduce; } + /** + * @return the end index + */ + private int runSpecificNearlineJobAndVerifyConsumption( + int startIndex, + String storeName, + int version, + Map, VeniceChangeCoordinate>> polledChangeEventsMap, + List, VeniceChangeCoordinate>> polledChangeEventsList, + List> bootstrappingVeniceChangelogConsumerList) { + Map samzaConfig = getSamzaProducerConfig(clusterWrapper, storeName, Version.PushType.STREAM); + VeniceSystemFactory factory = new VeniceSystemFactory(); + // Half puts and half deletes + int recordsToProduce = 20; + int numPuts = recordsToProduce / 2; + int numDeletes = recordsToProduce / 2; + + try ( + VeniceSystemProducer veniceProducer = factory.getClosableProducer("venice", new MapConfig(samzaConfig), null)) { + veniceProducer.start(); + // Run Samza job to send PUT and DELETE requests. + runSamzaStreamJob(veniceProducer, storeName, null, numPuts, numDeletes, startIndex, true); + } + + try (AvroGenericStoreClient client = ClientFactory.getAndStartGenericAvroClient( + ClientConfig.defaultGenericClientConfig(storeName) + .setVeniceURL(clusterWrapper.getRandomRouterURL()) + .setMetricsRepository(metricsRepository))) { + TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + Assert.assertNotNull(client.get(Integer.toString(startIndex + numPuts - 1)).get()); + }); + } + + // 20 changes in near-line. 10 puts, 10 deletes + TestUtils.waitForNonDeterministicAssertion(60, TimeUnit.SECONDS, true, () -> { + pollChangeEventsFromSpecificChangeCaptureConsumer( + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumerList); + Assert.assertEquals(polledChangeEventsMap.size(), recordsToProduce); + + verifySpecificPut(polledChangeEventsMap, startIndex, startIndex + numPuts, version); + verifySpecificDelete(polledChangeEventsMap, startIndex + numPuts, startIndex + numDeletes, version); + }); + polledChangeEventsList.clear(); + polledChangeEventsMap.clear(); + + return startIndex + recordsToProduce; + } + private void verifyPut( Map, VeniceChangeCoordinate>> polledChangeEventsMap, int startIndex, @@ -667,6 +852,27 @@ private void verifyPut( } } + private void verifySpecificPut( + Map, VeniceChangeCoordinate>> polledChangeEventsMap, + int startIndex, + int endIndex, + int version) { + for (int i = startIndex; i < endIndex; i++) { + String key = Integer.toString(i); + PubSubMessage, VeniceChangeCoordinate> message = + polledChangeEventsMap.get((key)); + ChangeEvent changeEvent = message.getValue(); + int versionFromMessage = Version.parseVersionFromVersionTopicName(message.getTopicPartition().getTopicName()); + Assert.assertEquals(versionFromMessage, version); + Assert.assertNotNull(changeEvent); + Assert.assertNull(changeEvent.getPreviousValue()); + + TestChangelogValue value = changeEvent.getCurrentValue(); + Assert.assertEquals(value.firstName, "first_name_" + i); + Assert.assertEquals(value.lastName, "last_name_" + i); + } + } + private void verifyDelete( Map, VeniceChangeCoordinate>> polledChangeEventsMap, int startIndex, @@ -684,6 +890,24 @@ private void verifyDelete( } } + private void verifySpecificDelete( + Map, VeniceChangeCoordinate>> polledChangeEventsMap, + int startIndex, + int endIndex, + int version) { + for (int i = startIndex; i < endIndex; i++) { + String key = Integer.toString(i); + PubSubMessage, VeniceChangeCoordinate> message = + polledChangeEventsMap.get((key)); + ChangeEvent changeEvent = message.getValue(); + int versionFromMessage = Version.parseVersionFromVersionTopicName(message.getTopicPartition().getTopicName()); + Assert.assertEquals(versionFromMessage, version); + Assert.assertNotNull(changeEvent); + Assert.assertNull(changeEvent.getPreviousValue()); + Assert.assertNull(changeEvent.getCurrentValue()); + } + } + private void verifyNoRecordsProduced( Map, VeniceChangeCoordinate>> polledChangeEventsMap, List, VeniceChangeCoordinate>> polledChangeEventsList, @@ -695,6 +919,17 @@ private void verifyNoRecordsProduced( Assert.assertEquals(polledChangeEventsList.size(), 0); } + private void verifyNoSpecificRecordsProduced( + Map, VeniceChangeCoordinate>> polledChangeEventsMap, + List, VeniceChangeCoordinate>> polledChangeEventsList, + List> bootstrappingVeniceChangelogConsumerList) { + pollChangeEventsFromSpecificChangeCaptureConsumer( + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumerList); + Assert.assertEquals(polledChangeEventsList.size(), 0); + } + private void cleanUpStoreAndVerify(String storeName) { clusterWrapper.useControllerClient(controllerClient -> { // Verify that topics and store is cleaned up From 1da7e77bdcb48494658c9ddc924c7344605d8c21 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Tue, 6 May 2025 16:33:29 -0700 Subject: [PATCH 02/25] Add support for specific records in dvrt --- .../DaVinciRecordTransformerConfig.java | 7 ++ .../DaVinciRecordTransformerUtility.java | 19 +++- ...gConsumerDaVinciRecordTransformerImpl.java | 2 +- .../kafka/consumer/StoreIngestionTask.java | 86 +++++++++++-------- .../TestBootstrappingChangelogConsumer.java | 27 +++--- 5 files changed, 85 insertions(+), 56 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java index 627e369a081..d96c2467283 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java @@ -47,6 +47,13 @@ public Class getOutputValueClass() { return outputValueClass; } + /** + * @return Whether this is a specific client + */ + public boolean isSpecificClient() { + return outputValueClass != null; + } + /** * @return {@link #outputValueSchema} */ diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java index 73302a86ad8..4f4dfb3cd37 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java @@ -8,6 +8,9 @@ import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; import com.linkedin.venice.serializer.AvroGenericDeserializer; import com.linkedin.venice.serializer.AvroSerializer; +import com.linkedin.venice.serializer.FastSerializerDeserializerFactory; +import com.linkedin.venice.serializer.RecordDeserializer; +import com.linkedin.venice.serializer.RecordSerializer; import com.linkedin.venice.utils.lazy.Lazy; import java.io.IOException; import java.nio.ByteBuffer; @@ -27,9 +30,9 @@ public class DaVinciRecordTransformerUtility { private static final Logger LOGGER = LogManager.getLogger(DaVinciRecordTransformerUtility.class); private final DaVinciRecordTransformer recordTransformer; private final DaVinciRecordTransformerConfig recordTransformerConfig; - private final AvroGenericDeserializer keyDeserializer; - private final AvroGenericDeserializer outputValueDeserializer; - private final AvroSerializer outputValueSerializer; + private final RecordDeserializer keyDeserializer; + private final RecordDeserializer outputValueDeserializer; + private final RecordSerializer outputValueSerializer; public DaVinciRecordTransformerUtility( DaVinciRecordTransformer recordTransformer, @@ -40,7 +43,15 @@ public DaVinciRecordTransformerUtility( Schema keySchema = recordTransformer.getKeySchema(); Schema outputValueSchema = recordTransformer.getOutputValueSchema(); this.keyDeserializer = new AvroGenericDeserializer<>(keySchema, keySchema); - this.outputValueDeserializer = new AvroGenericDeserializer<>(outputValueSchema, outputValueSchema); + + if (recordTransformerConfig.isSpecificClient()) { + this.outputValueDeserializer = FastSerializerDeserializerFactory + .getFastAvroSpecificDeserializer(outputValueSchema, recordTransformerConfig.getOutputValueClass()); + } else { + this.outputValueDeserializer = + FastSerializerDeserializerFactory.getFastAvroGenericDeserializer(outputValueSchema, outputValueSchema); + } + this.outputValueSerializer = new AvroSerializer<>(outputValueSchema); } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java index 477f657ac72..9e2556adc66 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java @@ -65,7 +65,7 @@ public class BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl, VeniceChangeCoordinate>> pubSubMessages; // Determines what version per partition is currently serving - private final ConcurrentHashMap partitionToVersionToServe; + private final Map partitionToVersionToServe; private final DaVinciRecordTransformerConfig recordTransformerConfig; // CachingDaVinciClientFactory used instead of DaVinciClientFactory, so we have the ability to close down the client private final CachingDaVinciClientFactory daVinciClientFactory; 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 44dae2b8d6a..1d62f27f2b3 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 @@ -49,6 +49,7 @@ import com.linkedin.davinci.utils.InMemoryChunkAssembler; import com.linkedin.davinci.validation.KafkaDataIntegrityValidator; import com.linkedin.davinci.validation.PartitionTracker; +import com.linkedin.venice.ConfigKeys; import com.linkedin.venice.common.VeniceSystemStoreType; import com.linkedin.venice.common.VeniceSystemStoreUtils; import com.linkedin.venice.compression.CompressionStrategy; @@ -151,7 +152,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.PriorityBlockingQueue; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -165,6 +165,7 @@ import javax.annotation.Nonnull; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; +import org.apache.avro.specific.SpecificRecord; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -355,9 +356,10 @@ public abstract class StoreIngestionTask implements Runnable, Closeable { protected final InMemoryChunkAssembler chunkAssembler; private final Optional cacheBackend; + private final DaVinciRecordTransformerConfig recordTransformerConfig; private final Schema recordTransformerInputValueSchema; private final AvroGenericDeserializer recordTransformerKeyDeserializer; - private final SparseConcurrentList recordTransformerDeserializersByPutSchemaId; + private final Map schemaIdToSchemaMap; private BlockingDaVinciRecordTransformer recordTransformer; protected final String localKafkaServer; @@ -520,6 +522,7 @@ public StoreIngestionTask( this.missingSOPCheckExecutor.execute(() -> waitForStateVersion(kafkaVersionTopic)); this.cacheBackend = cacheBackend; + this.recordTransformerConfig = recordTransformerConfig; if (recordTransformerConfig != null && recordTransformerConfig.getRecordTransformerFunction() != null) { this.chunkAssembler = new InMemoryChunkAssembler(new InMemoryStorageEngine(storeName)); Schema keySchema = schemaRepository.getKeySchema(storeName).getSchema(); @@ -546,7 +549,7 @@ public StoreIngestionTask( this.recordTransformerInputValueSchema, outputValueSchema, recordTransformerConfig); - this.recordTransformerDeserializersByPutSchemaId = new SparseConcurrentList<>(); + this.schemaIdToSchemaMap = new VeniceConcurrentHashMap<>(); daVinciRecordTransformerStats = builder.getDaVinciRecordTransformerStats(); @@ -559,9 +562,9 @@ public StoreIngestionTask( LatencyUtils.getElapsedTimeFromNSToMS(startTime), storeVersionName); } else { + this.schemaIdToSchemaMap = null; this.recordTransformerKeyDeserializer = null; this.recordTransformerInputValueSchema = null; - this.recordTransformerDeserializersByPutSchemaId = null; this.chunkAssembler = null; } @@ -862,7 +865,7 @@ private boolean checkDatabaseIntegrity( String topic, OffsetRecord offsetRecord, PartitionConsumptionState partitionConsumptionState) { - String replicaId = Utils.getReplicaId(topic, partitionId); + String replicaId = getReplicaId(topic, partitionId); boolean returnStatus = true; if (offsetRecord.getLocalVersionTopicOffset() > 0) { StoreVersionState storeVersionState = storageEngine.getStoreVersionState(); @@ -1035,7 +1038,7 @@ protected boolean isReadyToServe(PartitionConsumptionState partitionConsumptionS * If timestamp lag threshold is set to -1, offset lag threshold will be the only criterion for going online. */ if (producerTimeLagThresholdInSeconds > 0) { - long producerTimeLagThresholdInMS = TimeUnit.SECONDS.toMillis(producerTimeLagThresholdInSeconds); + long producerTimeLagThresholdInMS = SECONDS.toMillis(producerTimeLagThresholdInSeconds); long latestConsumedProducerTimestamp = partitionConsumptionState.getOffsetRecord().getLatestProducerProcessingTimeInMs(); boolean timestampLagIsAcceptable = checkAndLogIfLagIsAcceptableForHybridStore( @@ -1103,7 +1106,7 @@ protected boolean isReadyToServe(PartitionConsumptionState partitionConsumptionS } else { LOGGER.info( "[Time Lag] The producer timestamp of the last message in topic-partition: {} is {}, which is smaller or equal to the latest known producer time: {}. Consumption lag is already caught up for replica {}.", - Utils.getReplicaId(lagMeasurementTopic, partitionId), + getReplicaId(lagMeasurementTopic, partitionId), latestProducerTimestampInTopic, latestConsumedProducerTimestamp, partitionConsumptionState.getReplicaId()); @@ -1481,7 +1484,7 @@ private void processIngestionException() { if (partitionConsumptionState == null || !partitionConsumptionState.isSubscribed()) { LOGGER.warn( "Ignoring exception for replica: {} since the topic-partition has been unsubscribed already.", - Utils.getReplicaId(kafkaVersionTopic, exceptionPartition), + getReplicaId(kafkaVersionTopic, exceptionPartition), partitionException); /** * Since the partition is already unsubscribed, we will clear the exception to avoid excessive logging, and in theory, @@ -1506,9 +1509,7 @@ private void processIngestionException() { pauseConsumption(pubSubTopicPartition.getPubSubTopic().getName(), pubSubTopicPartition.getPartitionNumber()); LOGGER.info( "Memory limit reached. Pausing consumption of topic-partition: {}", - Utils.getReplicaId( - pubSubTopicPartition.getPubSubTopic().getName(), - pubSubTopicPartition.getPartitionNumber())); + getReplicaId(pubSubTopicPartition.getPubSubTopic().getName(), pubSubTopicPartition.getPartitionNumber())); runnableForKillIngestionTasksForNonCurrentVersions.run(); if (storageEngine.hasMemorySpaceLeft()) { unSubscribePartition(pubSubTopicPartition, false); @@ -1522,7 +1523,7 @@ private void processIngestionException() { */ LOGGER.info( "Ingestion for topic-partition: {} can resume since more space has been reclaimed.", - Utils.getReplicaId(kafkaVersionTopic, exceptionPartition)); + getReplicaId(kafkaVersionTopic, exceptionPartition)); storageEngine.reopenStoragePartition(exceptionPartition); // DaVinci is always a follower. subscribePartition(pubSubTopicPartition, false); @@ -1537,7 +1538,7 @@ private void processIngestionException() { Collections.singletonList(HelixUtils.getPartitionName(kafkaVersionTopic, exceptionPartition))); LOGGER.error( "Marking current version replica status to ERROR for replica: {}", - Utils.getReplicaId(kafkaVersionTopic, exceptionPartition), + getReplicaId(kafkaVersionTopic, exceptionPartition), partitionException); // No need to reset again, clearing out the exception. partitionIngestionExceptionList.set(exceptionPartition, null); @@ -1547,7 +1548,7 @@ private void processIngestionException() { } else { LOGGER.error( "Ignoring exception for replica: {} since it is already online. The replica will continue serving reads, but the data may be stale as it is not actively ingesting data. Please engage the Venice DEV team immediately.", - Utils.getReplicaId(kafkaVersionTopic, exceptionPartition), + getReplicaId(kafkaVersionTopic, exceptionPartition), partitionException); } // Unsubscribe the partition to avoid more damages. @@ -1681,7 +1682,7 @@ private void maybeUnsubscribeCompletedPartitions(Store store) { if (state.isCompletionReported() && consumerHasSubscription(versionTopic, state)) { LOGGER.info( "Unsubscribing completed topic-partition: {}. Current version at this time: {}", - Utils.getReplicaId(versionTopic, state.getPartition()), + getReplicaId(versionTopic, state.getPartition()), store.getCurrentVersion()); topicPartitionsToUnsubscribe.add(new PubSubTopicPartitionImpl(versionTopic, state.getPartition())); forceUnSubscribedCount++; @@ -1955,7 +1956,7 @@ private void internalClose(boolean doFlush) { ConsumerActionType opType = message.getType(); String topic = message.getTopic(); int partition = message.getPartition(); - String replica = Utils.getReplicaId(message.getTopic(), message.getPartition()); + String replica = getReplicaId(message.getTopic(), message.getPartition()); try { switch (opType) { case RESET_OFFSET: @@ -2221,7 +2222,7 @@ private void checkConsumptionStateWhenStart( if (previousOffsetLag != OffsetRecord.DEFAULT_OFFSET_LAG) { LOGGER.info( "Checking offset lag behavior for replica: {}. Current offset lag: {}, previous offset lag: {}, offset lag threshold: {}", - Utils.getReplicaId(versionTopic, partition), + getReplicaId(versionTopic, partition), offsetLag, previousOffsetLag, offsetLagThreshold); @@ -2273,7 +2274,7 @@ protected void processCommonConsumerAction(ConsumerAction consumerAction) throws LOGGER.info( "DaVinciRecordTransformer onRecovery took {} ms for replica: {}", LatencyUtils.getElapsedTimeFromNSToMS(startTime), - Utils.getReplicaId(topic, partition)); + getReplicaId(topic, partition)); } // Get the last persisted Offset record from metadata service @@ -2281,7 +2282,7 @@ protected void processCommonConsumerAction(ConsumerAction consumerAction) throws // Let's try to restore the state retrieved from the OffsetManager PartitionConsumptionState newPartitionConsumptionState = new PartitionConsumptionState( - Utils.getReplicaId(versionTopic, partition), + getReplicaId(versionTopic, partition), partition, offsetRecord, hybridStoreConfig.isPresent()); @@ -2301,7 +2302,7 @@ protected void processCommonConsumerAction(ConsumerAction consumerAction) throws if (!checkDatabaseIntegrity(partition, topic, offsetRecord, newPartitionConsumptionState)) { LOGGER.warn( "Restart ingestion from the beginning by resetting OffsetRecord for topic-partition: {}. Replica: {}", - Utils.getReplicaId(topic, partition), + getReplicaId(topic, partition), newPartitionConsumptionState.getReplicaId()); resetOffset(partition, topicPartition, true); newPartitionConsumptionState = partitionConsumptionStateMap.get(partition); @@ -2438,7 +2439,7 @@ private void resetOffset(int partition, PubSubTopicPartition topicPartition, boo partitionConsumptionState.getReplicaId()); } PartitionConsumptionState consumptionState = new PartitionConsumptionState( - Utils.getReplicaId(versionTopic, partition), + getReplicaId(versionTopic, partition), partition, new OffsetRecord(partitionStateSerializer), hybridStoreConfig.isPresent()); @@ -2481,7 +2482,7 @@ private void reportStoreVersionTopicOffsetRewindMetrics(PartitionConsumptionStat // report offset rewind. LOGGER.warn( "Offset rewind for version topic-partition: {}, persisted record offset: {}, Kafka topic partition end-offset: {}", - Utils.getReplicaId(kafkaVersionTopic, pcs.getPartition()), + getReplicaId(kafkaVersionTopic, pcs.getPartition()), offset, endOffset); versionedIngestionStats.recordVersionTopicEndOffsetRewind(storeName, versionNumber); @@ -2570,7 +2571,7 @@ protected boolean shouldProcessRecord(DefaultPubSubMessage record) { PartitionConsumptionState partitionConsumptionState = partitionConsumptionStateMap.get(record.getPartition()); if (partitionConsumptionState == null) { - String msg = "PCS for replica: " + Utils.getReplicaId(kafkaVersionTopic, record.getPartition()) + String msg = "PCS for replica: " + getReplicaId(kafkaVersionTopic, record.getPartition()) + " is null. Skipping incoming record with topic-partition: {} and offset: {}"; if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { LOGGER.info(msg, record.getTopicPartition(), record.getPosition()); @@ -2628,7 +2629,7 @@ protected boolean shouldPersistRecord( DefaultPubSubMessage record, PartitionConsumptionState partitionConsumptionState) { int partitionId = record.getTopicPartition().getPartitionNumber(); - String replicaId = Utils.getReplicaId(kafkaVersionTopic, partitionId); + String replicaId = getReplicaId(kafkaVersionTopic, partitionId); if (failedPartitions.contains(partitionId)) { String msg = "Errors already exist for replica: " + replicaId + ", skipping incoming record"; if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msg)) { @@ -2709,7 +2710,7 @@ public void processConsumerRecord( beforeProcessingRecordTimestampNs); } catch (FatalDataValidationException e) { int faultyPartition = record.getTopicPartition().getPartitionNumber(); - String replicaId = Utils.getReplicaId(versionTopic, faultyPartition); + String replicaId = getReplicaId(versionTopic, faultyPartition); String errorMessage; errorMessage = FATAL_DATA_VALIDATION_ERROR + " for replica: " + replicaId + ". Incoming record topic-partition: " + record.getTopicPartition() + " offset: " + record.getPosition(); @@ -3086,7 +3087,7 @@ private void processStartOfPush( final boolean sorted; if (serverConfig.getRocksDBServerConfig().isBlobFilesEnabled() && isHybridMode()) { /** - * We would like to skip {@link com.linkedin.davinci.store.rocksdb.RocksDBSstFileWriter} for hybrid stores + * We would like to skip {@link RocksDBSstFileWriter} for hybrid stores * when RocksDB blob mode is enabled and here are the reasons: * 1. Hybrid stores will use the same amount of MemTables eventually even not in batch processing phase. * 2. SSTFileWriter + RocksDB blob mode will introduce additional space overhead in the following way: @@ -3799,7 +3800,7 @@ protected void logStorageOperationWhileUnsubscribed(int partition) { "Attempted to interact with the storage engine for partition: {} while the " + "partitionConsumptionStateMap does not contain this partition. " + "Will ignore the operation as it probably indicates the partition was unsubscribed.", - Utils.getReplicaId(versionTopic, partition)); + getReplicaId(versionTopic, partition)); } public boolean consumerHasAnySubscription() { @@ -3974,15 +3975,24 @@ private int processKafkaDataMessage( Lazy lazyValue = Lazy.of(() -> { try { ByteBuffer decompressedAssembledObject = compressor.get().decompress(assembledObject); - RecordDeserializer recordDeserializer = - this.recordTransformerDeserializersByPutSchemaId.computeIfAbsent(readerSchemaId, i -> { - Schema valueSchema = schemaRepository.getValueSchema(storeName, readerSchemaId).getSchema(); - if (this.recordTransformer.useUniformInputValueSchema()) { - return new AvroGenericDeserializer<>(valueSchema, this.recordTransformerInputValueSchema); - } else { - return new AvroGenericDeserializer<>(valueSchema, valueSchema); - } - }); + Schema valueSchema = this.schemaIdToSchemaMap.computeIfAbsent( + readerSchemaId, + i -> schemaRepository.getValueSchema(storeName, readerSchemaId).getSchema()); + RecordDeserializer recordDeserializer; + + if (recordTransformerConfig.isSpecificClient() + && SpecificRecord.class.isAssignableFrom(recordTransformerConfig.getOutputValueClass())) { + recordDeserializer = FastSerializerDeserializerFactory + .getFastAvroSpecificDeserializer(valueSchema, recordTransformerConfig.getOutputValueClass()); + } else { + if (this.recordTransformer.useUniformInputValueSchema()) { + recordDeserializer = FastSerializerDeserializerFactory + .getFastAvroGenericDeserializer(valueSchema, this.recordTransformerInputValueSchema); + } else { + recordDeserializer = + FastSerializerDeserializerFactory.getFastAvroGenericDeserializer(valueSchema, valueSchema); + } + } return recordDeserializer.deserialize(decompressedAssembledObject); } catch (IOException e) { @@ -4284,7 +4294,7 @@ private void deserializeValue(int schemaId, ByteBuffer value, DefaultPubSubMessa "Value deserialization succeeded with schema id {} for incoming record from: {} for replica: {}", schemaId, record.getTopicPartition(), - Utils.getReplicaId(versionTopic, record.getPartition())); + getReplicaId(versionTopic, record.getPartition())); deserializedSchemaIds.set(schemaId, new Object()); } } @@ -4422,7 +4432,7 @@ public boolean isPartitionConsumingOrHasPendingIngestionAction(int userPartition } /** - * Override the {@link com.linkedin.venice.ConfigKeys#KAFKA_BOOTSTRAP_SERVERS} config with a remote Kafka bootstrap url. + * Override the {@link ConfigKeys#KAFKA_BOOTSTRAP_SERVERS} config with a remote Kafka bootstrap url. */ protected Properties createKafkaConsumerProperties( Properties localConsumerProps, diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index cd1f4681e0d..36cc5eae148 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -268,7 +268,8 @@ public void testVeniceChangelogConsumer(int consumerCount) throws Exception { @Test(timeOut = TEST_TIMEOUT * 3, priority = 3) public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { String storeName = Utils.getUniqueString("store"); - String inputDirPath = setUpStore(storeName, false); + boolean useSpecificRecord = false; + String inputDirPath = setUpStore(storeName, useSpecificRecord); Map samzaConfig = getSamzaProducerConfig(clusterWrapper, storeName, Version.PushType.STREAM); VeniceSystemFactory factory = new VeniceSystemFactory(); @@ -304,7 +305,7 @@ public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exc factory.getClosableProducer("venice", new MapConfig(samzaConfig), null)) { veniceProducer.start(); // Run Samza job to send PUT and DELETE requests. - runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, false); + runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, useSpecificRecord); // Produce a DELETE record with large timestamp sendStreamingRecordWithLogicalTimestamp(veniceProducer, storeName, deleteWithRmdKeyIndex, 1000, true); } @@ -409,7 +410,8 @@ public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exc @Test(timeOut = TEST_TIMEOUT, priority = 3) public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { String storeName = Utils.getUniqueString("store"); - String inputDirPath1 = setUpStore(storeName, false); + boolean useSpecificRecord = false; + String inputDirPath1 = setUpStore(storeName, useSpecificRecord); String inputDirPath2 = Utils.getTempDataDirectory().getAbsolutePath(); Map samzaConfig = getSamzaProducerConfig(clusterWrapper, storeName, Version.PushType.STREAM); VeniceSystemFactory factory = new VeniceSystemFactory(); @@ -467,7 +469,7 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( factory.getClosableProducer("venice", new MapConfig(samzaConfig), null)) { veniceProducer.start(); // Run Samza job to send PUT and DELETE requests. - runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, true); + runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, useSpecificRecord); } // Spin up a DVRT CDC instance and wait for it to consume everything, then perform blob transfer @@ -529,7 +531,8 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( @Test(timeOut = TEST_TIMEOUT, priority = 3) public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { String storeName = Utils.getUniqueString("store"); - String inputDirPath = setUpStore(storeName, true); + boolean useSpecificRecord = true; + String inputDirPath = setUpStore(storeName, useSpecificRecord); Map samzaConfig = getSamzaProducerConfig(clusterWrapper, storeName, Version.PushType.STREAM); VeniceSystemFactory factory = new VeniceSystemFactory(); @@ -565,7 +568,7 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp factory.getClosableProducer("venice", new MapConfig(samzaConfig), null)) { veniceProducer.start(); // Run Samza job to send PUT and DELETE requests. - runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, true); + runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, useSpecificRecord); } bootstrappingVeniceChangelogConsumerList.get(0).start().get(); @@ -700,13 +703,12 @@ private String setUpStore(String storeName, boolean useSpecificRecord) throws Ex Schema recordSchema; String valueSchemaStr; if (useSpecificRecord) { - Schema recordSchema1 = + Schema tempRecordSchema = new PushInputSchemaBuilder().setKeySchema(STRING_SCHEMA).setValueSchema(TestChangelogValue.SCHEMA$).build(); - // recordSchema = TestWriteUtils.writeSimpleAvroFileWithStringToNameRecordV1Schema(inputDir); recordSchema = writeSimpleAvroFile( inputDir, - recordSchema1, - i -> renderNameRecord(recordSchema1, i), + tempRecordSchema, + i -> renderNameRecord(tempRecordSchema, i), DEFAULT_USER_DATA_RECORD_COUNT); valueSchemaStr = TestChangelogValue.SCHEMA$.toString(); } else { @@ -714,7 +716,6 @@ private String setUpStore(String storeName, boolean useSpecificRecord) throws Ex valueSchemaStr = recordSchema.getField(DEFAULT_VALUE_FIELD_PROP).schema().toString(); } String keySchemaStr = recordSchema.getField(DEFAULT_KEY_FIELD_PROP).schema().toString(); - ; UpdateStoreQueryParams storeParms = new UpdateStoreQueryParams().setHybridRewindSeconds(500) .setHybridOffsetLagThreshold(8) @@ -868,8 +869,8 @@ private void verifySpecificPut( Assert.assertNull(changeEvent.getPreviousValue()); TestChangelogValue value = changeEvent.getCurrentValue(); - Assert.assertEquals(value.firstName, "first_name_" + i); - Assert.assertEquals(value.lastName, "last_name_" + i); + Assert.assertEquals(value.firstName.toString(), "first_name_stream_" + i); + Assert.assertEquals(value.lastName.toString(), "last_name_stream_" + i); } } From 54add59e1f05ed5a39b5e046c78d926f102d6798 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Thu, 8 May 2025 10:35:00 -0700 Subject: [PATCH 03/25] Add more tests for specific record --- ...nsumerDaVinciRecordTransformerUserApp.java | 64 ++++++-- .../TestBootstrappingChangelogConsumer.java | 148 ++++++++++++++++-- .../DaVinciClientRecordTransformerTest.java | 35 ++++- .../TestIntToStringRecordTransformer.java | 53 ++++++- .../endToEnd/TestStringRecordTransformer.java | 2 +- 5 files changed, 279 insertions(+), 23 deletions(-) diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ChangelogConsumerDaVinciRecordTransformerUserApp.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ChangelogConsumerDaVinciRecordTransformerUserApp.java index e4c3b511aff..8f9d76af75c 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ChangelogConsumerDaVinciRecordTransformerUserApp.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ChangelogConsumerDaVinciRecordTransformerUserApp.java @@ -32,6 +32,7 @@ import com.linkedin.davinci.consumer.VeniceChangeCoordinate; import com.linkedin.davinci.consumer.VeniceChangelogConsumerClientFactory; import com.linkedin.venice.D2.D2ClientUtils; +import com.linkedin.venice.endToEnd.TestChangelogValue; import com.linkedin.venice.integration.utils.VeniceRouterWrapper; import com.linkedin.venice.pubsub.api.PubSubMessage; import com.linkedin.venice.utils.SslUtils; @@ -68,6 +69,7 @@ public static void main(String[] args) throws InterruptedException, ExecutionExc int blobTransferServerPort = Integer.parseInt(args[5]); int blobTransferClientPort = Integer.parseInt(args[6]); int eventsToPoll = Integer.parseInt(args[7]); + boolean useSpecificRecord = Boolean.parseBoolean(args[8]); Utils.thisIsLocalhost(); @@ -117,23 +119,50 @@ public static void main(String[] args) throws InterruptedException, ExecutionExc .setD2Client(d2Client) .setIsExperimentalClientEnabled(true); + if (useSpecificRecord) { + globalChangelogClientConfig.setSpecificValue(TestChangelogValue.class) + .setSpecificValueSchema(TestChangelogValue.SCHEMA$); + } + VeniceChangelogConsumerClientFactory veniceChangelogConsumerClientFactory = new VeniceChangelogConsumerClientFactory(globalChangelogClientConfig, metricsRepository); - BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer = - veniceChangelogConsumerClientFactory.getBootstrappingChangelogConsumer(storeName, Integer.toString(0)); + BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer; + + if (useSpecificRecord) { + bootstrappingVeniceChangelogConsumer = veniceChangelogConsumerClientFactory + .getBootstrappingChangelogConsumer(storeName, Integer.toString(0), TestChangelogValue.class); + } else { + bootstrappingVeniceChangelogConsumer = + veniceChangelogConsumerClientFactory.getBootstrappingChangelogConsumer(storeName, Integer.toString(0)); + } bootstrappingVeniceChangelogConsumer.start().get(); LOGGER.info("DVRT CDC user app has come online."); - Map, VeniceChangeCoordinate>> polledChangeEventsMap = new HashMap<>(); - List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); - TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { - pollChangeEventsFromChangeCaptureConsumer( - polledChangeEventsMap, - polledChangeEventsList, - bootstrappingVeniceChangelogConsumer); - Assert.assertEquals(polledChangeEventsList.size(), eventsToPoll); - }); + if (useSpecificRecord) { + Map, VeniceChangeCoordinate>> polledChangeEventsMap = + new HashMap<>(); + List, VeniceChangeCoordinate>> polledChangeEventsList = + new ArrayList<>(); + TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + pollChangeEventsFromSpecificChangeCaptureConsumer( + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumer); + Assert.assertEquals(polledChangeEventsList.size(), eventsToPoll); + }); + } else { + Map, VeniceChangeCoordinate>> polledChangeEventsMap = + new HashMap<>(); + List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); + TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + pollChangeEventsFromChangeCaptureConsumer( + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumer); + Assert.assertEquals(polledChangeEventsList.size(), eventsToPoll); + }); + } LOGGER.info("DVRT CDC user app has consumed all events"); } @@ -150,4 +179,17 @@ private static void pollChangeEventsFromChangeCaptureConsumer( } polledMessageList.addAll(pubSubMessages); } + + private static void pollChangeEventsFromSpecificChangeCaptureConsumer( + Map, VeniceChangeCoordinate>> keyToMessageMap, + List, VeniceChangeCoordinate>> polledMessageList, + BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer) { + Collection, VeniceChangeCoordinate>> pubSubMessages = + bootstrappingVeniceChangelogConsumer.poll(1000); + for (PubSubMessage, VeniceChangeCoordinate> pubSubMessage: pubSubMessages) { + String key = pubSubMessage.getKey() == null ? null : pubSubMessage.getKey().toString(); + keyToMessageMap.put(key, pubSubMessage); + } + polledMessageList.addAll(pubSubMessages); + } } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index 36cc5eae148..07d540bda8f 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -32,7 +32,6 @@ import static com.linkedin.venice.utils.SslUtils.LOCAL_KEYSTORE_JKS; import static com.linkedin.venice.utils.SslUtils.LOCAL_PASSWORD; import static com.linkedin.venice.utils.TestWriteUtils.DEFAULT_USER_DATA_RECORD_COUNT; -import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V1_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.STRING_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; import static com.linkedin.venice.utils.TestWriteUtils.renderNameRecord; @@ -482,7 +481,8 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( storeName, Integer.toString(port2), Integer.toString(port1), - Integer.toString(DEFAULT_USER_DATA_RECORD_COUNT + 20)); + Integer.toString(DEFAULT_USER_DATA_RECORD_COUNT + 20), + Boolean.toString(useSpecificRecord)); Thread.sleep(30000); bootstrappingVeniceChangelogConsumerList.get(0).start().get(); @@ -555,7 +555,7 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp .setBootstrapFileSystemPath(inputDirPath) .setIsExperimentalClientEnabled(true) .setSpecificValue(TestChangelogValue.class) - .setSpecificValueSchema(NAME_RECORD_V1_SCHEMA) + .setSpecificValueSchema(TestChangelogValue.SCHEMA$) .setD2Client(d2Client); VeniceChangelogConsumerClientFactory veniceChangelogConsumerClientFactory = new VeniceChangelogConsumerClientFactory(globalChangelogClientConfig, metricsRepository); @@ -610,21 +610,151 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp } } + @Test(timeOut = TEST_TIMEOUT, priority = 3) + public void testSpecificRecordBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { + String storeName = Utils.getUniqueString("store"); + boolean useSpecificRecord = true; + String inputDirPath1 = setUpStore(storeName, useSpecificRecord); + String inputDirPath2 = Utils.getTempDataDirectory().getAbsolutePath(); + Map samzaConfig = getSamzaProducerConfig(clusterWrapper, storeName, Version.PushType.STREAM); + VeniceSystemFactory factory = new VeniceSystemFactory(); + int port1 = TestUtils.getFreePort(); + int port2 = TestUtils.getFreePort(); + while (port1 == port2) { + port2 = TestUtils.getFreePort(); + } + + try (PubSubBrokerWrapper localKafka = ServiceFactory.getPubSubBroker( + new PubSubBrokerConfigs.Builder().setZkWrapper(clusterWrapper.getZk()) + .setMockTime(testMockTime) + .setRegionName(REGION_NAME) + .build())) { + String localKafkaUrl = localKafka.getAddress(); + Properties consumerProperties = new Properties(); + consumerProperties.put(KAFKA_BOOTSTRAP_SERVERS, localKafkaUrl); + consumerProperties.put(CLUSTER_NAME, clusterName); + consumerProperties.put(ZOOKEEPER_ADDRESS, zkAddress); + consumerProperties.put(BLOB_TRANSFER_MANAGER_ENABLED, true); + consumerProperties.put(DAVINCI_P2P_BLOB_TRANSFER_SERVER_PORT, port1); + consumerProperties.put(DAVINCI_P2P_BLOB_TRANSFER_CLIENT_PORT, port2); + consumerProperties.put(BLOB_TRANSFER_SSL_ENABLED, true); + consumerProperties.put(BLOB_TRANSFER_ACL_ENABLED, true); + + String keyStorePath = SslUtils.getPathForResource(LOCAL_KEYSTORE_JKS); + consumerProperties.put(SSL_KEYSTORE_TYPE, "JKS"); + consumerProperties.put(SSL_KEYSTORE_LOCATION, keyStorePath); + consumerProperties.put(SSL_KEYSTORE_PASSWORD, LOCAL_PASSWORD); + consumerProperties.put(SSL_TRUSTSTORE_TYPE, "JKS"); + consumerProperties.put(SSL_TRUSTSTORE_LOCATION, keyStorePath); + consumerProperties.put(SSL_TRUSTSTORE_PASSWORD, LOCAL_PASSWORD); + consumerProperties.put(SSL_KEY_PASSWORD, LOCAL_PASSWORD); + consumerProperties.put(SSL_KEYMANAGER_ALGORITHM, "SunX509"); + consumerProperties.put(SSL_TRUSTMANAGER_ALGORITHM, "SunX509"); + consumerProperties.put(SSL_SECURE_RANDOM_IMPLEMENTATION, "SHA1PRNG"); + + ChangelogClientConfig globalChangelogClientConfig = + new ChangelogClientConfig().setConsumerProperties(consumerProperties) + .setControllerD2ServiceName(D2_SERVICE_NAME) + .setD2ServiceName(VeniceRouterWrapper.CLUSTER_DISCOVERY_D2_SERVICE_NAME) + .setLocalD2ZkHosts(zkAddress) + .setControllerRequestRetryCount(3) + .setBootstrapFileSystemPath(inputDirPath1) + .setD2Client(d2Client) + .setIsExperimentalClientEnabled(true) + .setSpecificValue(TestChangelogValue.class) + .setSpecificValueSchema(TestChangelogValue.SCHEMA$); + + VeniceChangelogConsumerClientFactory veniceChangelogConsumerClientFactory = + new VeniceChangelogConsumerClientFactory(globalChangelogClientConfig, metricsRepository); + List> bootstrappingVeniceChangelogConsumerList = + Collections.singletonList( + veniceChangelogConsumerClientFactory + .getBootstrappingChangelogConsumer(storeName, Integer.toString(0), TestChangelogValue.class)); + + try (VeniceSystemProducer veniceProducer = + factory.getClosableProducer("venice", new MapConfig(samzaConfig), null)) { + veniceProducer.start(); + // Run Samza job to send PUT and DELETE requests. + runSamzaStreamJob(veniceProducer, storeName, null, 10, 10, 100, useSpecificRecord); + } + + // Spin up a DVRT CDC instance and wait for it to consume everything, then perform blob transfer + ForkedJavaProcess.exec( + ChangelogConsumerDaVinciRecordTransformerUserApp.class, + inputDirPath2, + zkAddress, + localKafkaUrl, + clusterName, + storeName, + Integer.toString(port2), + Integer.toString(port1), + Integer.toString(DEFAULT_USER_DATA_RECORD_COUNT + 20), + Boolean.toString(useSpecificRecord)); + Thread.sleep(30000); + + bootstrappingVeniceChangelogConsumerList.get(0).start().get(); + + // Verify snapshots exists + for (int i = 0; i < PARTITION_COUNT; i++) { + String snapshotPath = RocksDBUtils.composeSnapshotDir(inputDirPath2 + "/rocksdb", storeName + "_v1", i); + Assert.assertTrue(Files.exists(Paths.get(snapshotPath))); + } + + Map, VeniceChangeCoordinate>> polledChangeEventsMap = + new HashMap<>(); + List, VeniceChangeCoordinate>> polledChangeEventsList = + new ArrayList<>(); + TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + pollChangeEventsFromSpecificChangeCaptureConsumer( + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumerList); + // 20 changes in near-line. 10 puts, 10 deletes. But one of the puts overwrites a key from batch push, and the + // 10 deletes are against non-existant keys. So there should only be 109 events total + int expectedRecordCount = DEFAULT_USER_DATA_RECORD_COUNT + 9; + Assert.assertEquals(polledChangeEventsList.size(), expectedRecordCount); + verifySpecificPut(polledChangeEventsMap, 100, 110, 1); + }); + polledChangeEventsList.clear(); + polledChangeEventsMap.clear(); + + // Since nothing is produced, so no changed events generated. + verifyNoSpecificRecordsProduced( + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumerList); + + runSpecificNearlineJobAndVerifyConsumption( + 120, + storeName, + 1, + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumerList); + + // Since nothing is produced, so no changed events generated. + verifyNoSpecificRecordsProduced( + polledChangeEventsMap, + polledChangeEventsList, + bootstrappingVeniceChangelogConsumerList); + + cleanUpStoreAndVerify(storeName); + } + } + private void pollChangeEventsFromChangeCaptureConsumerToList( List, VeniceChangeCoordinate>> polledChangeEvents, - VeniceChangelogConsumer veniceChangelogConsumer) { + VeniceChangelogConsumer veniceChangelogConsumer) { Collection, VeniceChangeCoordinate>> pubSubMessages = veniceChangelogConsumer.poll(1000); - for (PubSubMessage, VeniceChangeCoordinate> pubSubMessage: pubSubMessages) { - polledChangeEvents.add(pubSubMessage); - } + polledChangeEvents.addAll(pubSubMessages); } private void pollChangeEventsFromChangeCaptureConsumer( Map, VeniceChangeCoordinate>> keyToMessageMap, List, VeniceChangeCoordinate>> polledMessageList, List> bootstrappingVeniceChangelogConsumerList) { - for (BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer: bootstrappingVeniceChangelogConsumerList) { + for (BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer: bootstrappingVeniceChangelogConsumerList) { Collection, VeniceChangeCoordinate>> pubSubMessages = bootstrappingVeniceChangelogConsumer.poll(1000); for (PubSubMessage, VeniceChangeCoordinate> pubSubMessage: pubSubMessages) { @@ -639,7 +769,7 @@ private void pollChangeEventsFromSpecificChangeCaptureConsumer( Map, VeniceChangeCoordinate>> keyToMessageMap, List, VeniceChangeCoordinate>> polledMessageList, List> bootstrappingVeniceChangelogConsumerList) { - for (BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer: bootstrappingVeniceChangelogConsumerList) { + for (BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer: bootstrappingVeniceChangelogConsumerList) { Collection, VeniceChangeCoordinate>> pubSubMessages = bootstrappingVeniceChangelogConsumer.poll(1000); for (PubSubMessage, VeniceChangeCoordinate> pubSubMessage: pubSubMessages) { diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java index 7144dabcae2..ee76cd43f6e 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java @@ -230,8 +230,23 @@ public void testTypeChangeRecordTransformer() throws Exception { metricsRepository, backendConfig)) { - DaVinciRecordTransformerConfig recordTransformerConfig = new DaVinciRecordTransformerConfig.Builder() + DaVinciRecordTransformerConfig dummyRecordTransformerConfig = new DaVinciRecordTransformerConfig.Builder() .setRecordTransformerFunction(TestIntToStringRecordTransformer::new) + .build(); + + Schema myKeySchema = Schema.create(Schema.Type.INT); + Schema myInputValueSchema = Schema.create(Schema.Type.INT); + Schema myOutputValueSchema = Schema.create(Schema.Type.STRING); + TestIntToStringRecordTransformer recordTransformer = new TestIntToStringRecordTransformer( + 1, + myKeySchema, + myInputValueSchema, + myOutputValueSchema, + dummyRecordTransformerConfig); + + DaVinciRecordTransformerConfig recordTransformerConfig = new DaVinciRecordTransformerConfig.Builder() + .setRecordTransformerFunction( + (storeVersion, keySchema, inputValueSchema, outputValueSchema, config) -> recordTransformer) .setOutputValueClass(String.class) .setOutputValueSchema(Schema.create(Schema.Type.STRING)) .build(); @@ -250,6 +265,24 @@ public void testTypeChangeRecordTransformer() throws Exception { String expectedValue = k + "Transformed"; assertEquals(valueObj.toString(), expectedValue); } + + /* + * Simulates a client restart. During this process, the DVRT will use the on-disk state + * to repopulate the inMemoryDB, avoiding the need for re-ingestion after clearing. + */ + clientWithRecordTransformer.close(); + recordTransformer.clearInMemoryDB(); + assertTrue(recordTransformer.isInMemoryDBEmpty()); + + clientWithRecordTransformer.start(); + clientWithRecordTransformer.subscribeAll().get(); + + for (int k = 1; k <= numKeys; ++k) { + Object valueObj = clientWithRecordTransformer.get(k).get(); + String expectedValue = k + "Transformed"; + assertEquals(valueObj.toString(), expectedValue); + } + clientWithRecordTransformer.unsubscribeAll(); } } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestIntToStringRecordTransformer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestIntToStringRecordTransformer.java index a31d58c15fd..cb11c4804b7 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestIntToStringRecordTransformer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestIntToStringRecordTransformer.java @@ -5,13 +5,19 @@ import com.linkedin.davinci.client.DaVinciRecordTransformerResult; import com.linkedin.venice.utils.lazy.Lazy; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; import org.apache.avro.Schema; +import org.apache.avro.util.Utf8; /** * Transforms int values to strings */ public class TestIntToStringRecordTransformer extends DaVinciRecordTransformer { + private final Map inMemoryDB = new HashMap<>(); + private int transformInvocationCount = 0; + public TestIntToStringRecordTransformer( int storeVersion, Schema keySchema, @@ -25,12 +31,57 @@ public TestIntToStringRecordTransformer( public DaVinciRecordTransformerResult transform(Lazy key, Lazy value, int partitionId) { String valueStr = value.get().toString(); String transformedValue = valueStr + "Transformed"; + transformInvocationCount++; return new DaVinciRecordTransformerResult<>(DaVinciRecordTransformerResult.Result.TRANSFORMED, transformedValue); } @Override public void processPut(Lazy key, Lazy value, int partitionId) { - return; + String valueStr = convertUtf8ToString(value.get()); + put(key.get(), valueStr); + } + + @Override + public void processDelete(Lazy key, int partitionId) { + delete(key.get()); + }; + + private String convertUtf8ToString(Object valueObj) { + String valueStr; + if (valueObj instanceof Utf8) { + valueStr = valueObj.toString(); + } else { + valueStr = (String) valueObj; + } + + return valueStr; + } + + public void clearInMemoryDB() { + inMemoryDB.clear(); + } + + public boolean isInMemoryDBEmpty() { + return inMemoryDB.isEmpty(); + } + + public String get(Integer key) { + return inMemoryDB.get(key); + } + + private void put(Integer key, String value) { + inMemoryDB.put(key, value); + } + + private void delete(Integer key) { + if (!inMemoryDB.containsKey(key)) { + throw new IllegalArgumentException("Key not found: " + key); + } + inMemoryDB.remove(key); + } + + public int getTransformInvocationCount() { + return transformInvocationCount; } @Override diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStringRecordTransformer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStringRecordTransformer.java index 4f5433c5a22..d71fcd03441 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStringRecordTransformer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStringRecordTransformer.java @@ -41,7 +41,7 @@ public void processPut(Lazy key, Lazy value, int partitionId) { @Override public void processDelete(Lazy key, int partitionId) { delete(key.get()); - }; + } private String convertUtf8ToString(Object valueObj) { String valueStr; From acf2ceb7b1e9c6566ae7ef0020411dbb51ef365c Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Thu, 8 May 2025 21:00:08 -0700 Subject: [PATCH 04/25] Add isCaughtUp support for dvrt cdc --- .../BootstrappingVeniceChangelogConsumer.java | 5 +++++ ...logConsumerDaVinciRecordTransformerImpl.java | 13 +++++++++++++ .../TestBootstrappingChangelogConsumer.java | 17 +++++++++++++++++ 3 files changed, 35 insertions(+) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java index 14b287452e1..b48ed0bcc18 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java @@ -46,4 +46,9 @@ public interface BootstrappingVeniceChangelogConsumer { */ Collection, VeniceChangeCoordinate>> poll(long timeoutInMs); + /** + * @return True if all subscribed partitions have caught up. + */ + boolean isCaughtUp(); + } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java index 9e2556adc66..e7c04a91c91 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java @@ -46,6 +46,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; import org.apache.avro.Schema; @@ -80,6 +81,7 @@ public class BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl start(Set partitions) { LOGGER.error("Failed to subscribe to partitions: {} for store: {}", partitions, storeName, error); throw new VeniceException(error); } + + isCaughtUp.set(true); + LOGGER.info( + "BootstrappingVeniceChangelogConsumer is caught up for store: {} for partitions: {}", + storeName, + partitions); }); return CompletableFuture.supplyAsync(() -> { @@ -206,6 +214,11 @@ public Collection, VeniceChangeCoordinate>> poll return drainedPubSubMessages; } + @Override + public boolean isCaughtUp() { + return isCaughtUp.get(); + } + private void internalStart() { if (isStarted) { return; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index 07d540bda8f..570ee9cbef9 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -41,6 +41,8 @@ import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static org.testng.AssertJUnit.assertFalse; +import static org.testng.AssertJUnit.assertTrue; import com.linkedin.d2.balancer.D2Client; import com.linkedin.d2.balancer.D2ClientBuilder; @@ -310,12 +312,14 @@ public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exc } bootstrappingVeniceChangelogConsumerList.get(0).start().get(); + assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); Map, VeniceChangeCoordinate>> polledChangeEventsMap = new HashMap<>(); List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); // 21 changes in near-line. 10 puts, 10 deletes, and 1 record with a producer timestamp TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + // assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); pollChangeEventsFromChangeCaptureConsumer( polledChangeEventsMap, polledChangeEventsList, @@ -326,6 +330,7 @@ public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exc verifyPut(polledChangeEventsMap, 100, 110, 1); verifyDelete(polledChangeEventsMap, 110, 120, 1); }); + assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); polledChangeEventsList.clear(); polledChangeEventsMap.clear(); @@ -486,6 +491,7 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( Thread.sleep(30000); bootstrappingVeniceChangelogConsumerList.get(0).start().get(); + assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); // Verify snapshots exists for (int i = 0; i < PARTITION_COUNT; i++) { @@ -497,6 +503,7 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( new HashMap<>(); List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); pollChangeEventsFromChangeCaptureConsumer( polledChangeEventsMap, polledChangeEventsList, @@ -507,6 +514,7 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( Assert.assertEquals(polledChangeEventsList.size(), expectedRecordCount); verifyPut(polledChangeEventsMap, 100, 110, 1); }); + assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); polledChangeEventsList.clear(); polledChangeEventsMap.clear(); @@ -572,6 +580,7 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp } bootstrappingVeniceChangelogConsumerList.get(0).start().get(); + assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); Map, VeniceChangeCoordinate>> polledChangeEventsMap = new HashMap<>(); @@ -579,6 +588,7 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp new ArrayList<>(); // 20 changes in near-line. 10 puts, 10 deletes TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); pollChangeEventsFromSpecificChangeCaptureConsumer( polledChangeEventsMap, polledChangeEventsList, @@ -589,6 +599,7 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp verifySpecificPut(polledChangeEventsMap, 100, 110, 1); verifySpecificDelete(polledChangeEventsMap, 110, 120, 1); }); + assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); polledChangeEventsList.clear(); polledChangeEventsMap.clear(); @@ -693,6 +704,7 @@ public void testSpecificRecordBlobTransferVeniceChangelogConsumerDaVinciRecordTr Thread.sleep(30000); bootstrappingVeniceChangelogConsumerList.get(0).start().get(); + assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); // Verify snapshots exists for (int i = 0; i < PARTITION_COUNT; i++) { @@ -705,6 +717,7 @@ public void testSpecificRecordBlobTransferVeniceChangelogConsumerDaVinciRecordTr List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); pollChangeEventsFromSpecificChangeCaptureConsumer( polledChangeEventsMap, polledChangeEventsList, @@ -715,6 +728,7 @@ public void testSpecificRecordBlobTransferVeniceChangelogConsumerDaVinciRecordTr Assert.assertEquals(polledChangeEventsList.size(), expectedRecordCount); verifySpecificPut(polledChangeEventsMap, 100, 110, 1); }); + assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); polledChangeEventsList.clear(); polledChangeEventsMap.clear(); @@ -906,6 +920,9 @@ private int runNearlineJobAndVerifyConsumption( polledChangeEventsList, bootstrappingVeniceChangelogConsumerList); Assert.assertEquals(polledChangeEventsMap.size(), recordsToProduce); + if (polledChangeEventsMap.size() == 22) { + int i = 0; + } verifyPut(polledChangeEventsMap, startIndex, startIndex + numPuts, version); verifyDelete(polledChangeEventsMap, startIndex + numPuts, startIndex + numDeletes, version); From b6e98a8f807d9073d6fce0467d80c8ea9fc973ea Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Thu, 8 May 2025 21:05:33 -0700 Subject: [PATCH 05/25] Cleanup --- .../venice/consumer/TestBootstrappingChangelogConsumer.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index 570ee9cbef9..26e8c56b1d4 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -920,9 +920,6 @@ private int runNearlineJobAndVerifyConsumption( polledChangeEventsList, bootstrappingVeniceChangelogConsumerList); Assert.assertEquals(polledChangeEventsMap.size(), recordsToProduce); - if (polledChangeEventsMap.size() == 22) { - int i = 0; - } verifyPut(polledChangeEventsMap, startIndex, startIndex + numPuts, version); verifyDelete(polledChangeEventsMap, startIndex + numPuts, startIndex + numDeletes, version); From 08cb1ca79232b8c1485934ccb06186a4bd8c2485 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Fri, 9 May 2025 16:55:25 -0700 Subject: [PATCH 06/25] test --- .../TestBootstrappingChangelogConsumer.java | 33 ++++++++++++++----- 1 file changed, 25 insertions(+), 8 deletions(-) diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index 26e8c56b1d4..68843258109 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -99,11 +99,12 @@ import org.apache.avro.util.Utf8; import org.apache.samza.config.MapConfig; import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +@Test(singleThreaded = true) public class TestBootstrappingChangelogConsumer { private static final int TEST_TIMEOUT = 2 * Time.MS_PER_MINUTE; private static final String REGION_NAME = "local-pubsub"; @@ -118,7 +119,7 @@ public class TestBootstrappingChangelogConsumer { private MetricsRepository metricsRepository; private String zkAddress; - @BeforeClass(alwaysRun = true) + @BeforeMethod(alwaysRun = true) public void setUp() { Utils.thisIsLocalhost(); @@ -149,7 +150,7 @@ public void setUp() { metricsRepository = new MetricsRepository(); } - @AfterClass(alwaysRun = true) + @AfterMethod(alwaysRun = true) public void cleanUp() { clusterWrapper.close(); TestView.resetCounters(); @@ -330,7 +331,11 @@ public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exc verifyPut(polledChangeEventsMap, 100, 110, 1); verifyDelete(polledChangeEventsMap, 110, 120, 1); }); - assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); + + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { + assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); + }); + polledChangeEventsList.clear(); polledChangeEventsMap.clear(); @@ -514,7 +519,11 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( Assert.assertEquals(polledChangeEventsList.size(), expectedRecordCount); verifyPut(polledChangeEventsMap, 100, 110, 1); }); - assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); + + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { + assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); + }); + polledChangeEventsList.clear(); polledChangeEventsMap.clear(); @@ -599,7 +608,11 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp verifySpecificPut(polledChangeEventsMap, 100, 110, 1); verifySpecificDelete(polledChangeEventsMap, 110, 120, 1); }); - assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); + + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { + assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); + }); + polledChangeEventsList.clear(); polledChangeEventsMap.clear(); @@ -728,7 +741,11 @@ public void testSpecificRecordBlobTransferVeniceChangelogConsumerDaVinciRecordTr Assert.assertEquals(polledChangeEventsList.size(), expectedRecordCount); verifySpecificPut(polledChangeEventsMap, 100, 110, 1); }); - assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); + + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { + assertTrue(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); + }); + polledChangeEventsList.clear(); polledChangeEventsMap.clear(); From 0fc01b2a056995e46cb276e9217b38f377c78d78 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Fri, 9 May 2025 17:07:18 -0700 Subject: [PATCH 07/25] Fix tests --- .../endToEnd/DaVinciClientRecordTransformerTest.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java index 9370bc005f4..2b929680668 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java @@ -85,18 +85,19 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +@Test(singleThreaded = true) public class DaVinciClientRecordTransformerTest { private static final Logger LOGGER = LogManager.getLogger(DaVinciClientRecordTransformerTest.class); private static final int TEST_TIMEOUT = 120_000; private VeniceClusterWrapper cluster; private D2Client d2Client; - @BeforeClass + @BeforeMethod(alwaysRun = true) public void setUp() { Utils.thisIsLocalhost(); Properties clusterConfig = new Properties(); @@ -116,7 +117,7 @@ public void setUp() { D2ClientUtils.startClient(d2Client); } - @AfterClass + @AfterMethod(alwaysRun = true) public void cleanUp() { if (d2Client != null) { D2ClientUtils.shutdownClient(d2Client); From 8e4d59ef83050b752b2b7b1bce27b49069ab4e30 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Fri, 9 May 2025 19:37:59 -0700 Subject: [PATCH 08/25] revert --- .../consumer/TestBootstrappingChangelogConsumer.java | 8 ++++---- .../endToEnd/DaVinciClientRecordTransformerTest.java | 9 ++++----- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index 7d66d51df99..504732a0ed1 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -98,8 +98,8 @@ import org.apache.avro.util.Utf8; import org.apache.samza.config.MapConfig; import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -118,7 +118,7 @@ public class TestBootstrappingChangelogConsumer { private MetricsRepository metricsRepository; private String zkAddress; - @BeforeMethod(alwaysRun = true) + @BeforeClass(alwaysRun = true) public void setUp() { Utils.thisIsLocalhost(); @@ -148,7 +148,7 @@ public void setUp() { metricsRepository = new MetricsRepository(); } - @AfterMethod(alwaysRun = true) + @AfterClass(alwaysRun = true) public void cleanUp() { clusterWrapper.close(); TestView.resetCounters(); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java index 2b929680668..671aed3a29d 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java @@ -85,19 +85,18 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -@Test(singleThreaded = true) public class DaVinciClientRecordTransformerTest { private static final Logger LOGGER = LogManager.getLogger(DaVinciClientRecordTransformerTest.class); private static final int TEST_TIMEOUT = 120_000; private VeniceClusterWrapper cluster; private D2Client d2Client; - @BeforeMethod(alwaysRun = true) + @BeforeClass(alwaysRun = true) public void setUp() { Utils.thisIsLocalhost(); Properties clusterConfig = new Properties(); @@ -117,7 +116,7 @@ public void setUp() { D2ClientUtils.startClient(d2Client); } - @AfterMethod(alwaysRun = true) + @AfterClass(alwaysRun = true) public void cleanUp() { if (d2Client != null) { D2ClientUtils.shutdownClient(d2Client); From 2f8e0ad2e461a1071d7e9e169573d4ea69de5e0f Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Fri, 9 May 2025 21:14:45 -0700 Subject: [PATCH 09/25] remove assertions --- .../TestBootstrappingChangelogConsumer.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index 504732a0ed1..b06d1084518 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -154,7 +154,7 @@ public void cleanUp() { TestView.resetCounters(); } - @Test(timeOut = TEST_TIMEOUT, dataProvider = "changelogConsumer", dataProviderClass = DataProviderUtils.class, priority = 3) + @Test(timeOut = TEST_TIMEOUT, dataProvider = "changelogConsumer", dataProviderClass = DataProviderUtils.class) public void testVeniceChangelogConsumer(int consumerCount) throws Exception { String storeName = Utils.getUniqueString("store"); String inputDirPath = setUpStore(storeName, false); @@ -265,7 +265,7 @@ public void testVeniceChangelogConsumer(int consumerCount) throws Exception { } } - @Test(timeOut = TEST_TIMEOUT * 3, priority = 3) + @Test(timeOut = TEST_TIMEOUT * 3) public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { String storeName = Utils.getUniqueString("store"); boolean useSpecificRecord = false; @@ -318,7 +318,6 @@ public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exc List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); // 21 changes in near-line. 10 puts, 10 deletes, and 1 record with a producer timestamp TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { - // assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); pollChangeEventsFromChangeCaptureConsumer( polledChangeEventsMap, polledChangeEventsList, @@ -414,7 +413,7 @@ public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exc } } - @Test(timeOut = TEST_TIMEOUT, priority = 3) + @Test(timeOut = TEST_TIMEOUT) public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { String storeName = Utils.getUniqueString("store"); boolean useSpecificRecord = false; @@ -506,7 +505,6 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( new HashMap<>(); List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { - assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); pollChangeEventsFromChangeCaptureConsumer( polledChangeEventsMap, polledChangeEventsList, @@ -543,7 +541,7 @@ public void testBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl( } } - @Test(timeOut = TEST_TIMEOUT, priority = 3) + @Test(timeOut = TEST_TIMEOUT) public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { String storeName = Utils.getUniqueString("store"); boolean useSpecificRecord = true; @@ -595,7 +593,6 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp new ArrayList<>(); // 20 changes in near-line. 10 puts, 10 deletes TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { - assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); pollChangeEventsFromSpecificChangeCaptureConsumer( polledChangeEventsMap, polledChangeEventsList, @@ -632,7 +629,7 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp } } - @Test(timeOut = TEST_TIMEOUT, priority = 3) + @Test(timeOut = TEST_TIMEOUT) public void testSpecificRecordBlobTransferVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { String storeName = Utils.getUniqueString("store"); boolean useSpecificRecord = true; @@ -728,7 +725,6 @@ public void testSpecificRecordBlobTransferVeniceChangelogConsumerDaVinciRecordTr List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { - assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); pollChangeEventsFromSpecificChangeCaptureConsumer( polledChangeEventsMap, polledChangeEventsList, From 748ee90cf5191395d859f81e187decee5fd25e52 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Mon, 12 May 2025 16:02:04 -0700 Subject: [PATCH 10/25] Update BootstrappingVeniceChangelogConsumer JavaDoc --- .../consumer/BootstrappingVeniceChangelogConsumer.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java index b48ed0bcc18..7191bfa5383 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java @@ -29,6 +29,11 @@ public interface BootstrappingVeniceChangelogConsumer { * NOTE: This future may take some time to complete depending on how much data needs to be ingested in order to catch * up with the time that this client started. * + * NOTE: In the experimental client, the future will complete when there is at least one message to be polled. + * We don't wait for all partitions to catch up, as loading every message into a buffer will result in an + * Out Of Memory error. Instead, use the {@link #isCaughtUp()} method to determine once all subscribed partitions have + * caught up. + * * @param partitions which partition id's to catch up with * @return a future that completes once catch up is complete for all passed in partitions. */ @@ -41,6 +46,10 @@ public interface BootstrappingVeniceChangelogConsumer { /** * polls for the next batch of change events. The first records returned following calling 'start()' will be from the bootstrap state. * Once this state is consumed, subsequent calls to poll will be based off of recent updates to the Venice store. + * + * In the experimental client, records will be returned in batches configured to the MAX_BUFFER_SIZE. So the initial + * calls to poll will be from records from the bootstrap state, until the partitions have caught up. + * * @param timeoutInMs * @return */ From bd1dd6727b4ffb6f06e826284e48021335de718e Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Mon, 12 May 2025 18:11:05 -0700 Subject: [PATCH 11/25] Add unit tests for isCaughtUp --- ...sumerDaVinciRecordTransformerImplTest.java | 78 ++++++++++++++----- 1 file changed, 60 insertions(+), 18 deletions(-) diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java index 88813470d63..54551b6c22e 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java @@ -24,6 +24,7 @@ import com.linkedin.davinci.client.DaVinciRecordTransformerResult; import com.linkedin.davinci.client.factory.CachingDaVinciClientFactory; import com.linkedin.venice.controllerapi.D2ControllerClient; +import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.pubsub.api.PubSubMessage; import com.linkedin.venice.schema.SchemaReader; import com.linkedin.venice.utils.TestUtils; @@ -60,6 +61,8 @@ public class BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTes private static final int CURRENT_STORE_VERSION = 1; private static final int FUTURE_STORE_VERSION = 2; private static final int MAX_BUFFER_SIZE = 10; + private static final int value = 2; + private static final Lazy lazyValue = Lazy.of(() -> value); private Schema keySchema; private Schema valueSchema; @@ -68,7 +71,9 @@ public class BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTes private ChangelogClientConfig changelogClientConfig; private DaVinciRecordTransformerConfig mockDaVinciRecordTransformerConfig; private DaVinciClient mockDaVinciClient; + private CompletableFuture daVinciClientSubscribeFuture; private List> keys; + private Set partitionSet; @BeforeMethod public void setUp() throws NoSuchFieldException, IllegalAccessException { @@ -109,8 +114,9 @@ public void setUp() throws NoSuchFieldException, IllegalAccessException { // Replace daVinciClient with a mock mockDaVinciClient = mock(DaVinciClient.class); + daVinciClientSubscribeFuture = new CompletableFuture<>(); when(mockDaVinciClient.getPartitionCount()).thenReturn(PARTITION_COUNT); - when(mockDaVinciClient.subscribe(any())).thenReturn(mock(CompletableFuture.class)); + when(mockDaVinciClient.subscribe(any())).thenReturn(daVinciClientSubscribeFuture); AccessController.doPrivileged((PrivilegedAction) () -> { try { @@ -125,9 +131,11 @@ public void setUp() throws NoSuchFieldException, IllegalAccessException { }); keys = new ArrayList<>(); + partitionSet = new HashSet<>(); for (int i = 0; i < PARTITION_COUNT; i++) { int tempI = i; keys.add(Lazy.of(() -> tempI)); + partitionSet.add(i); } } @@ -142,11 +150,6 @@ public void testStartAllPartitions() throws IllegalAccessException, NoSuchFieldE verify(mockDaVinciClient).start(); - Set partitionSet = new HashSet<>(); - for (int i = 0; i < PARTITION_COUNT; i++) { - partitionSet.add(i); - } - Field subscribedPartitionsField = BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.class.getDeclaredField("subscribedPartitions"); subscribedPartitionsField.setAccessible(true); @@ -197,9 +200,6 @@ public void testPutAndDelete() { bootstrappingVeniceChangelogConsumer.start(); recordTransformer.onStartVersionIngestion(true); - int value = 2; - Lazy lazyValue = Lazy.of(() -> value); - for (int partitionId = 0; partitionId < PARTITION_COUNT; partitionId++) { recordTransformer.processPut(keys.get(partitionId), lazyValue, partitionId); } @@ -279,22 +279,15 @@ bootstrappingVeniceChangelogConsumer.new DaVinciRecordTransformerBootstrappingCh // CompletableFuture should not be finished until a record has been pushed to the buffer by the current version assertFalse(startCompletableFuture.isDone()); - int value = 2; - Lazy lazyValue = Lazy.of(() -> value); - // Future version should not cause the CompletableFuture to complete for (int partitionId = 0; partitionId < PARTITION_COUNT; partitionId++) { futureRecordTransformer.processPut(keys.get(partitionId), lazyValue, partitionId); } - assertFalse(startCompletableFuture.isDone()); // CompletableFuture should be finished when the current version produces to the buffer for (int partitionId = 0; partitionId < PARTITION_COUNT; partitionId++) { recordTransformer.processPut(keys.get(partitionId), lazyValue, partitionId); } - TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { - assertTrue(startCompletableFuture.isDone()); - }); } @Test @@ -328,8 +321,6 @@ public void testMaxBufferSize() throws NoSuchFieldException, IllegalAccessExcept recordTransformer.onStartVersionIngestion(true); int partitionId = 1; - int value = 2; - Lazy lazyValue = Lazy.of(() -> value); List completableFutureList = new ArrayList<>(); for (int i = 0; i <= MAX_BUFFER_SIZE; i++) { @@ -409,6 +400,57 @@ public void testMaxBufferSize() throws NoSuchFieldException, IllegalAccessExcept }); } + @Test + public void testIsCaughtUp() { + assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); + + CompletableFuture startCompletableFuture = bootstrappingVeniceChangelogConsumer.start(partitionSet); + recordTransformer.onStartVersionIngestion(true); + + // Add records for all but 1 partition to complete the start future, but to not complete the subscribe future. + for (int partitionId = 0; partitionId < PARTITION_COUNT - 1; partitionId++) { + recordTransformer.processPut(keys.get(partitionId), lazyValue, partitionId); + } + + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { + verify(mockDaVinciClient).subscribe(partitionSet); + assertTrue(startCompletableFuture.isDone()); + }); + assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); + + // Add record for last partition + recordTransformer.processPut(keys.get(PARTITION_COUNT - 1), lazyValue, PARTITION_COUNT - 1); + daVinciClientSubscribeFuture.complete(null); + + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { + assertTrue(bootstrappingVeniceChangelogConsumer.isCaughtUp()); + }); + } + + @Test + public void testIsCaughtUpException() { + assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); + + CompletableFuture startCompletableFuture = bootstrappingVeniceChangelogConsumer.start(); + recordTransformer.onStartVersionIngestion(true); + + // Add records for all but 1 partition to complete the start future, but to not complete the subscribe future. + for (int partitionId = 0; partitionId < PARTITION_COUNT - 1; partitionId++) { + recordTransformer.processPut(keys.get(partitionId), lazyValue, partitionId); + } + + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { + verify(mockDaVinciClient).subscribe(partitionSet); + assertTrue(startCompletableFuture.isDone()); + }); + assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); + + // Add record for last partition + recordTransformer.processPut(keys.get(PARTITION_COUNT - 1), lazyValue, PARTITION_COUNT - 1); + daVinciClientSubscribeFuture.completeExceptionally(new VeniceException("Test exception")); + assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); + } + private void verifyPuts(int value) { Collection, VeniceChangeCoordinate>> pubSubMessages = bootstrappingVeniceChangelogConsumer.poll(POLL_TIMEOUT); From 2539842fb08c6b28b3080b1755fa2002743d8b26 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Mon, 12 May 2025 21:16:23 -0700 Subject: [PATCH 12/25] Fix tests --- ...ChangelogConsumerDaVinciRecordTransformerImplTest.java | 5 +++++ .../endToEnd/DaVinciClientRecordTransformerTest.java | 8 +++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java index 54551b6c22e..45471baa22a 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java @@ -283,11 +283,15 @@ bootstrappingVeniceChangelogConsumer.new DaVinciRecordTransformerBootstrappingCh for (int partitionId = 0; partitionId < PARTITION_COUNT; partitionId++) { futureRecordTransformer.processPut(keys.get(partitionId), lazyValue, partitionId); } + assertFalse(startCompletableFuture.isDone()); // CompletableFuture should be finished when the current version produces to the buffer for (int partitionId = 0; partitionId < PARTITION_COUNT; partitionId++) { recordTransformer.processPut(keys.get(partitionId), lazyValue, partitionId); } + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { + assertTrue(startCompletableFuture.isDone()); + }); } @Test @@ -447,6 +451,7 @@ public void testIsCaughtUpException() { // Add record for last partition recordTransformer.processPut(keys.get(PARTITION_COUNT - 1), lazyValue, PARTITION_COUNT - 1); + daVinciClientSubscribeFuture.completeExceptionally(new VeniceException("Test exception")); assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java index 671aed3a29d..17586c628fa 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java @@ -124,7 +124,13 @@ public void cleanUp() { Utils.closeQuietlyWithErrorLogged(cluster); } - @Test(timeOut = TEST_TIMEOUT) + /* + * Lower priority ensures that this test runs first. This is needed because this test fails if it doesn't go first, + * even when this class is set to single threaded and we recreate the resources before every test. + * It fails because the DVRT metrics get emitted but they're not queryable from the metrics repository. The root + * cause is unknown, but to mitigate for now we need this test to run first. + */ + @Test(timeOut = TEST_TIMEOUT, priority = -1) public void testRecordTransformer() throws Exception { DaVinciConfig clientConfig = new DaVinciConfig(); From db7e3144105a2af4bfd623be705111bfec634007 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Tue, 13 May 2025 11:48:05 -0700 Subject: [PATCH 13/25] Make sure dvc subscription doesn't die silently --- ...gConsumerDaVinciRecordTransformerImpl.java | 37 ++++++++------- ...sumerDaVinciRecordTransformerImplTest.java | 46 +++++++++---------- .../TestBootstrappingChangelogConsumer.java | 2 +- .../DaVinciClientRecordTransformerTest.java | 4 +- 4 files changed, 44 insertions(+), 45 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java index e7c04a91c91..88ea8801d56 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java @@ -81,7 +81,7 @@ public class BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl start(Set partitions) { internalStart(); subscribedPartitions.addAll(partitions); + CompletableFuture startFuture = CompletableFuture.supplyAsync(() -> { + try { + /* + * When this latch gets released, this means there's at least one message in pubSubMessages. So when the user + * calls poll, they don't get an empty response. This also signals that blob transfer was completed + * for at least one partition. + */ + startLatch.await(); + } catch (InterruptedException e) { + LOGGER.info("Thread was interrupted", e); + // Restore the interrupt status + Thread.currentThread().interrupt(); + } + return null; + }, completableFutureThreadPool); + /* * Avoid waiting on the CompletableFuture to prevent a circular dependency. * When subscribe is called, DVRT scans the entire storage engine and fills pubSubMessages. @@ -132,7 +148,8 @@ public CompletableFuture start(Set partitions) { daVinciClient.subscribe(partitions).whenComplete((result, error) -> { if (error != null) { LOGGER.error("Failed to subscribe to partitions: {} for store: {}", partitions, storeName, error); - throw new VeniceException(error); + startFuture.completeExceptionally(new VeniceException(error)); + return; } isCaughtUp.set(true); @@ -142,21 +159,7 @@ public CompletableFuture start(Set partitions) { partitions); }); - return CompletableFuture.supplyAsync(() -> { - try { - /* - * When this latch gets released, this means there's at least one message in pubSubMessages. So when the user - * calls poll, they don't get an empty response. This also signals that blob transfer was completed - * for at least one partition. - */ - startLatch.await(); - } catch (InterruptedException e) { - LOGGER.info("Thread was interrupted", e); - // Restore the interrupt status - Thread.currentThread().interrupt(); - } - return null; - }, completableFutureThreadPool); + return startFuture; } @Override diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java index 45471baa22a..e0bc4e4fdac 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java @@ -15,6 +15,7 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; import com.linkedin.d2.balancer.D2Client; @@ -41,6 +42,7 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; @@ -294,6 +296,25 @@ bootstrappingVeniceChangelogConsumer.new DaVinciRecordTransformerBootstrappingCh }); } + @Test + public void testCompletableFutureFromStartException() { + assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); + + CompletableFuture startCompletableFuture = bootstrappingVeniceChangelogConsumer.start(); + recordTransformer.onStartVersionIngestion(true); + + TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { + verify(mockDaVinciClient).subscribe(partitionSet); + assertFalse(startCompletableFuture.isDone()); + }); + assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); + + daVinciClientSubscribeFuture.completeExceptionally(new VeniceException("Test exception")); + assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); + + assertThrows(ExecutionException.class, startCompletableFuture::get); + } + @Test public void testTransformResult() { int value = 2; @@ -431,31 +452,6 @@ public void testIsCaughtUp() { }); } - @Test - public void testIsCaughtUpException() { - assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); - - CompletableFuture startCompletableFuture = bootstrappingVeniceChangelogConsumer.start(); - recordTransformer.onStartVersionIngestion(true); - - // Add records for all but 1 partition to complete the start future, but to not complete the subscribe future. - for (int partitionId = 0; partitionId < PARTITION_COUNT - 1; partitionId++) { - recordTransformer.processPut(keys.get(partitionId), lazyValue, partitionId); - } - - TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { - verify(mockDaVinciClient).subscribe(partitionSet); - assertTrue(startCompletableFuture.isDone()); - }); - assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); - - // Add record for last partition - recordTransformer.processPut(keys.get(PARTITION_COUNT - 1), lazyValue, PARTITION_COUNT - 1); - - daVinciClientSubscribeFuture.completeExceptionally(new VeniceException("Test exception")); - assertFalse(bootstrappingVeniceChangelogConsumer.isCaughtUp()); - } - private void verifyPuts(int value) { Collection, VeniceChangeCoordinate>> pubSubMessages = bootstrappingVeniceChangelogConsumer.poll(POLL_TIMEOUT); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index b06d1084518..55cb8035f08 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -265,7 +265,7 @@ public void testVeniceChangelogConsumer(int consumerCount) throws Exception { } } - @Test(timeOut = TEST_TIMEOUT * 3) + @Test(timeOut = TEST_TIMEOUT * 2) public void testVeniceChangelogConsumerDaVinciRecordTransformerImpl() throws Exception { String storeName = Utils.getUniqueString("store"); boolean useSpecificRecord = false; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java index 17586c628fa..d610b59efb8 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientRecordTransformerTest.java @@ -96,7 +96,7 @@ public class DaVinciClientRecordTransformerTest { private VeniceClusterWrapper cluster; private D2Client d2Client; - @BeforeClass(alwaysRun = true) + @BeforeClass public void setUp() { Utils.thisIsLocalhost(); Properties clusterConfig = new Properties(); @@ -116,7 +116,7 @@ public void setUp() { D2ClientUtils.startClient(d2Client); } - @AfterClass(alwaysRun = true) + @AfterClass public void cleanUp() { if (d2Client != null) { D2ClientUtils.shutdownClient(d2Client); From 62ead2cfb24ee1661e55049607fcc240d963a3b3 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Tue, 13 May 2025 16:36:45 -0700 Subject: [PATCH 14/25] Add unit test for specific record in dvrt --- .../transformer/RecordTransformerTest.java | 57 +++++++++++++++++-- .../TestSpecificRecordTransformer.java | 43 ++++++++++++++ .../transformer/TestSpecificValue.java | 46 +++++++++++++++ 3 files changed, 140 insertions(+), 6 deletions(-) create mode 100644 clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificRecordTransformer.java create mode 100644 clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificValue.java diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java index a56a7edeec4..8a8a0179545 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java @@ -27,7 +27,9 @@ import com.linkedin.venice.offsets.OffsetRecord; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; +import com.linkedin.venice.serializer.AvroSpecificDeserializer; import com.linkedin.venice.utils.lazy.Lazy; +import java.lang.reflect.Field; import java.util.Optional; import org.apache.avro.Schema; import org.testng.annotations.Test; @@ -38,6 +40,9 @@ public class RecordTransformerTest { static final int partitionId = 0; static final InternalAvroSpecificSerializer partitionStateSerializer = AvroProtocolDefinition.PARTITION_STATE.getSerializer(); + static final Lazy lazyKey = Lazy.of(() -> 42); + static final String value = "SampleValue"; + static final Lazy lazyValue = Lazy.of(() -> value); static final Schema keySchema = Schema.create(Schema.Type.INT); static final Schema valueSchema = Schema.create(Schema.Type.STRING); @@ -62,13 +67,11 @@ public void testRecordTransformer() { assertEquals(recordTransformer.getKeySchema().getType(), Schema.Type.INT); assertEquals(recordTransformer.getOutputValueSchema().getType(), Schema.Type.STRING); - Lazy lazyKey = Lazy.of(() -> 42); - Lazy lazyValue = Lazy.of(() -> "SampleValue"); DaVinciRecordTransformerResult transformerResult = recordTransformer.transform(lazyKey, lazyValue, partitionId); recordTransformer.processPut(lazyKey, lazyValue, partitionId); assertEquals(transformerResult.getResult(), DaVinciRecordTransformerResult.Result.TRANSFORMED); - assertEquals(transformerResult.getValue(), "SampleValueTransformed"); + assertEquals(transformerResult.getValue(), value + "Transformed"); assertNull(recordTransformer.transformAndProcessPut(lazyKey, lazyValue, partitionId)); recordTransformer.processDelete(lazyKey, partitionId); @@ -229,13 +232,11 @@ public void testBlockingRecordTransformer() { assertEquals(blockingRecordTransformer.getOutputValueSchema().getType(), Schema.Type.STRING); - Lazy lazyKey = Lazy.of(() -> 42); - Lazy lazyValue = Lazy.of(() -> "SampleValue"); DaVinciRecordTransformerResult recordTransformerResult = blockingRecordTransformer.transformAndProcessPut(lazyKey, lazyValue, partitionId); verify(clientRecordTransformer).transform(lazyKey, lazyValue, partitionId); verify(clientRecordTransformer).processPut(eq(lazyKey), any(), eq(partitionId)); - assertEquals(recordTransformerResult.getValue(), "SampleValueTransformed"); + assertEquals(recordTransformerResult.getValue(), value + "Transformed"); blockingRecordTransformer.processDelete(lazyKey, partitionId); verify(clientRecordTransformer).processDelete(lazyKey, partitionId); @@ -275,4 +276,48 @@ public void testBlockingRecordTransformerVersionSwap() { blockingRecordTransformer.onVersionSwap(currentVersion, futureVersion, partitionId); verify(clientRecordTransformer).onVersionSwap(currentVersion, futureVersion, partitionId); } + + @Test + public void testSpecificRecordTransformer() throws NoSuchFieldException, IllegalAccessException { + Schema valueSchema = TestSpecificValue.SCHEMA$; + + DaVinciRecordTransformerConfig dummyRecordTransformerConfig = + new DaVinciRecordTransformerConfig.Builder().setRecordTransformerFunction(TestSpecificRecordTransformer::new) + .setOutputValueSchema(valueSchema) + .setOutputValueClass(TestSpecificValue.class) + .build(); + + assertTrue(dummyRecordTransformerConfig.isSpecificClient()); + + DaVinciRecordTransformer recordTransformer = + new TestSpecificRecordTransformer( + storeVersion, + keySchema, + valueSchema, + valueSchema, + dummyRecordTransformerConfig); + + DaVinciRecordTransformerUtility recordTransformerUtility = + recordTransformer.getRecordTransformerUtility(); + + Field outputValueDeserializerField = + recordTransformerUtility.getClass().getDeclaredField("outputValueDeserializer"); + outputValueDeserializerField.setAccessible(true); + assertTrue(outputValueDeserializerField.get(recordTransformerUtility) instanceof AvroSpecificDeserializer); + + TestSpecificValue specificValue = new TestSpecificValue(); + String firstName = "first"; + String lastName = "last"; + specificValue.firstName = firstName; + specificValue.lastName = lastName; + + Lazy lazyValue = Lazy.of(() -> specificValue); + + DaVinciRecordTransformerResult transformerResult = + recordTransformer.transform(lazyKey, lazyValue, partitionId); + assertEquals(transformerResult.getResult(), DaVinciRecordTransformerResult.Result.TRANSFORMED); + TestSpecificValue transformedSpecificValue = transformerResult.getValue(); + assertEquals(transformedSpecificValue.firstName, firstName.toUpperCase()); + assertEquals(transformedSpecificValue.lastName, lastName.toUpperCase()); + } } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificRecordTransformer.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificRecordTransformer.java new file mode 100644 index 00000000000..afc9ce0961a --- /dev/null +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificRecordTransformer.java @@ -0,0 +1,43 @@ +package com.linkedin.davinci.transformer; + +import com.linkedin.davinci.client.DaVinciRecordTransformer; +import com.linkedin.davinci.client.DaVinciRecordTransformerConfig; +import com.linkedin.davinci.client.DaVinciRecordTransformerResult; +import com.linkedin.venice.utils.lazy.Lazy; +import java.io.IOException; +import org.apache.avro.Schema; + + +public class TestSpecificRecordTransformer + extends DaVinciRecordTransformer { + public TestSpecificRecordTransformer( + int storeVersion, + Schema keySchema, + Schema inputValueSchema, + Schema outputValueSchema, + DaVinciRecordTransformerConfig recordTransformerConfig) { + super(storeVersion, keySchema, inputValueSchema, outputValueSchema, recordTransformerConfig); + } + + @Override + public DaVinciRecordTransformerResult transform( + Lazy key, + Lazy lazyValue, + int partitionId) { + TestSpecificValue value = lazyValue.get(); + value.firstName = value.firstName.toString().toUpperCase(); + value.lastName = value.lastName.toString().toUpperCase(); + + return new DaVinciRecordTransformerResult<>(DaVinciRecordTransformerResult.Result.TRANSFORMED, value); + } + + @Override + public void processPut(Lazy key, Lazy value, int partitionId) { + return; + } + + @Override + public void close() throws IOException { + + } +} diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificValue.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificValue.java new file mode 100644 index 00000000000..ac5817e33ad --- /dev/null +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificValue.java @@ -0,0 +1,46 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package com.linkedin.davinci.transformer; + +@SuppressWarnings("all") +public class TestSpecificValue extends org.apache.avro.specific.SpecificRecordBase + implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"TestSpecificValue\",\"namespace\":\"example.avro\",\"fields\":[{\"name\":\"firstName\",\"type\":\"string\",\"default\":\"\"},{\"name\":\"lastName\",\"type\":\"string\",\"default\":\"\"}]}"); + public CharSequence firstName; + public CharSequence lastName; + + public org.apache.avro.Schema getSchema() { + return SCHEMA$; + } + + // Used by DatumWriter. Applications should not call. + public Object get(int field$) { + switch (field$) { + case 0: + return firstName; + case 1: + return lastName; + default: + throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value = "unchecked") + public void put(int field$, Object value$) { + switch (field$) { + case 0: + firstName = (CharSequence) value$; + break; + case 1: + lastName = (CharSequence) value$; + break; + default: + throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } +} From d34b0a398d1fe78dc28558d13090cdf5c2d59726 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Wed, 14 May 2025 15:33:40 -0700 Subject: [PATCH 15/25] Add support for specificrecord keys --- .../DaVinciRecordTransformerConfig.java | 41 +++++- .../DaVinciRecordTransformerUtility.java | 12 +- ...gConsumerDaVinciRecordTransformerImpl.java | 1 + .../consumer/ChangelogClientConfig.java | 5 + .../kafka/consumer/StoreIngestionTask.java | 16 +- .../transformer/RecordTransformerTest.java | 25 +++- .../davinci/transformer/TestSpecificKey.java | 41 ++++++ .../TestSpecificRecordTransformer.java | 14 +- .../venice/client/store/ClientConfig.java | 19 +++ gradle/spotbugs/exclude.xml | 3 + .../TestBootstrappingChangelogConsumer.java | 138 +++++++++++------- .../venice/endToEnd/TestChangelogKey.java | 41 ++++++ .../utils/IntegrationTestPushUtils.java | 2 +- 13 files changed, 282 insertions(+), 76 deletions(-) create mode 100644 clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificKey.java create mode 100644 internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestChangelogKey.java diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java index d96c2467283..90ccd738c0a 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java @@ -4,6 +4,7 @@ import com.linkedin.venice.utils.lazy.Lazy; import java.util.Optional; import org.apache.avro.Schema; +import org.apache.avro.specific.SpecificRecord; /** @@ -11,6 +12,7 @@ */ public class DaVinciRecordTransformerConfig { private final DaVinciRecordTransformerFunctionalInterface recordTransformerFunction; + private final Class keyClass; private final Class outputValueClass; private final Schema outputValueSchema; private final boolean storeRecordsInDaVinci; @@ -21,6 +23,7 @@ public DaVinciRecordTransformerConfig(Builder builder) { this.recordTransformerFunction = Optional.ofNullable(builder.recordTransformerFunction) .orElseThrow(() -> new VeniceException("recordTransformerFunction cannot be null")); + this.keyClass = builder.keyClass; this.outputValueClass = builder.outputValueClass; this.outputValueSchema = builder.outputValueSchema; if ((this.outputValueClass != null && this.outputValueSchema == null) @@ -40,6 +43,20 @@ public DaVinciRecordTransformerFunctionalInterface getRecordTransformerFunction( return recordTransformerFunction; } + /** + * @return {@link #keyClass} + */ + public Class getKeyClass() { + return keyClass; + } + + /** + * @return Whether the SpecificRecord deserializer should be used for keys + */ + public boolean useSpecificRecordKeyDeserializer() { + return keyClass != null && SpecificRecord.class.isAssignableFrom(keyClass); + } + /** * @return {@link #outputValueClass} */ @@ -48,10 +65,10 @@ public Class getOutputValueClass() { } /** - * @return Whether this is a specific client + * @return Whether the SpecificRecord deserializer should be used for values */ - public boolean isSpecificClient() { - return outputValueClass != null; + public boolean useSpecificRecordValueDeserializer() { + return outputValueClass != null && SpecificRecord.class.isAssignableFrom(outputValueClass); } /** @@ -84,6 +101,7 @@ public boolean shouldSkipCompatibilityChecks() { public static class Builder { private DaVinciRecordTransformerFunctionalInterface recordTransformerFunction; + private Class keyClass; private Class outputValueClass; private Schema outputValueSchema; private Boolean storeRecordsInDaVinci = true; @@ -99,7 +117,18 @@ public Builder setRecordTransformerFunction(DaVinciRecordTransformerFunctionalIn } /** - * Set this if you modify the schema during transformation. Must be used in conjunction with {@link #setOutputValueSchema(Schema)} + * Set this if you want to deserialize keys into {@link org.apache.avro.specific.SpecificRecord}. + * @param keyClass the class of the key + */ + public Builder setKeyClass(Class keyClass) { + this.keyClass = keyClass; + return this; + } + + /** + * Set this if you modify the schema during transformation, or you want to deserialize values + * into {@link org.apache.avro.specific.SpecificRecord}. + * Must be used in conjunction with {@link #setOutputValueSchema(Schema)} * @param outputValueClass the class of the output value */ public Builder setOutputValueClass(Class outputValueClass) { @@ -108,7 +137,9 @@ public Builder setOutputValueClass(Class outputValueClass) { } /** - * Set this if you modify the schema during transformation. Must be used in conjunction with {@link #setOutputValueClass(Class)} + * Set this if you modify the schema during transformation, or you want to deserialize values + * into {@link org.apache.avro.specific.SpecificRecord}. + * Must be used in conjunction with {@link #setOutputValueClass(Class)} * @param outputValueSchema the schema of the output value */ public Builder setOutputValueSchema(Schema outputValueSchema) { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java index 4f4dfb3cd37..629d1d975da 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java @@ -6,7 +6,6 @@ import com.linkedin.venice.kafka.protocol.state.PartitionState; import com.linkedin.venice.offsets.OffsetRecord; import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; -import com.linkedin.venice.serializer.AvroGenericDeserializer; import com.linkedin.venice.serializer.AvroSerializer; import com.linkedin.venice.serializer.FastSerializerDeserializerFactory; import com.linkedin.venice.serializer.RecordDeserializer; @@ -41,10 +40,15 @@ public DaVinciRecordTransformerUtility( this.recordTransformerConfig = recordTransformerConfig; Schema keySchema = recordTransformer.getKeySchema(); - Schema outputValueSchema = recordTransformer.getOutputValueSchema(); - this.keyDeserializer = new AvroGenericDeserializer<>(keySchema, keySchema); + if (recordTransformerConfig.useSpecificRecordKeyDeserializer()) { + this.keyDeserializer = FastSerializerDeserializerFactory + .getFastAvroSpecificDeserializer(keySchema, recordTransformerConfig.getKeyClass()); + } else { + this.keyDeserializer = FastSerializerDeserializerFactory.getFastAvroGenericDeserializer(keySchema, keySchema); + } - if (recordTransformerConfig.isSpecificClient()) { + Schema outputValueSchema = recordTransformer.getOutputValueSchema(); + if (recordTransformerConfig.useSpecificRecordValueDeserializer()) { this.outputValueDeserializer = FastSerializerDeserializerFactory .getFastAvroSpecificDeserializer(outputValueSchema, recordTransformerConfig.getOutputValueClass()); } else { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java index 88ea8801d56..2f886933605 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java @@ -98,6 +98,7 @@ public BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl(Changelo * DVRT implmentation. This is to prevent the local state from being wiped everytime a change is deployed */ .setSkipCompatibilityChecks(true) + .setKeyClass(innerClientConfig.getSpecificKeyClass()) .setOutputValueClass(innerClientConfig.getSpecificValueClass()) .setOutputValueSchema(innerClientConfig.getSpecificValueSchema()) .build(); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java index d2f4574f452..aee9a7fd6fe 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java @@ -246,6 +246,11 @@ public ChangelogClientConfig setRocksDBBlockCacheSizeInBytes(long rocksDBBlockCa return this; } + public ChangelogClientConfig setSpecificKey(Class specificKey) { + this.innerClientConfig.setSpecificKeyClass(specificKey); + return this; + } + public ChangelogClientConfig setSpecificValue(Class specificValue) { this.innerClientConfig.setSpecificValueClass(specificValue); return this; 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 bdfe544e9f5..445ee23507d 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 @@ -167,7 +167,6 @@ import javax.annotation.Nonnull; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; -import org.apache.avro.specific.SpecificRecord; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -360,7 +359,7 @@ public abstract class StoreIngestionTask implements Runnable, Closeable { private final Optional cacheBackend; private final DaVinciRecordTransformerConfig recordTransformerConfig; private final Schema recordTransformerInputValueSchema; - private final AvroGenericDeserializer recordTransformerKeyDeserializer; + private final RecordDeserializer recordTransformerKeyDeserializer; private final Map schemaIdToSchemaMap; private BlockingDaVinciRecordTransformer recordTransformer; @@ -527,8 +526,16 @@ public StoreIngestionTask( this.recordTransformerConfig = recordTransformerConfig; if (recordTransformerConfig != null && recordTransformerConfig.getRecordTransformerFunction() != null) { this.chunkAssembler = new InMemoryChunkAssembler(new InMemoryStorageEngine(storeName)); + Schema keySchema = schemaRepository.getKeySchema(storeName).getSchema(); - this.recordTransformerKeyDeserializer = new AvroGenericDeserializer(keySchema, keySchema); + if (recordTransformerConfig.useSpecificRecordKeyDeserializer()) { + this.recordTransformerKeyDeserializer = FastSerializerDeserializerFactory + .getFastAvroSpecificDeserializer(keySchema, recordTransformerConfig.getKeyClass()); + } else { + this.recordTransformerKeyDeserializer = + FastSerializerDeserializerFactory.getFastAvroGenericDeserializer(keySchema, keySchema); + } + this.recordTransformerInputValueSchema = schemaRepository.getSupersetOrLatestValueSchema(storeName).getSchema(); Schema outputValueSchema = recordTransformerConfig.getOutputValueSchema(); @@ -3992,8 +3999,7 @@ private int processKafkaDataMessage( i -> schemaRepository.getValueSchema(storeName, readerSchemaId).getSchema()); RecordDeserializer recordDeserializer; - if (recordTransformerConfig.isSpecificClient() - && SpecificRecord.class.isAssignableFrom(recordTransformerConfig.getOutputValueClass())) { + if (recordTransformerConfig.useSpecificRecordValueDeserializer()) { recordDeserializer = FastSerializerDeserializerFactory .getFastAvroSpecificDeserializer(valueSchema, recordTransformerConfig.getOutputValueClass()); } else { diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java index 8a8a0179545..37b0b9d3c46 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java @@ -55,6 +55,8 @@ public void testRecordTransformer() { assertFalse( dummyRecordTransformerConfig.shouldSkipCompatibilityChecks(), "Default for skipCompatibilityChecks should be false"); + assertFalse(dummyRecordTransformerConfig.useSpecificRecordKeyDeserializer()); + assertFalse(dummyRecordTransformerConfig.useSpecificRecordValueDeserializer()); DaVinciRecordTransformer recordTransformer = new TestStringRecordTransformer( storeVersion, @@ -279,17 +281,20 @@ public void testBlockingRecordTransformerVersionSwap() { @Test public void testSpecificRecordTransformer() throws NoSuchFieldException, IllegalAccessException { + Schema keySchema = TestSpecificKey.SCHEMA$; Schema valueSchema = TestSpecificValue.SCHEMA$; DaVinciRecordTransformerConfig dummyRecordTransformerConfig = new DaVinciRecordTransformerConfig.Builder().setRecordTransformerFunction(TestSpecificRecordTransformer::new) + .setKeyClass(TestSpecificKey.class) .setOutputValueSchema(valueSchema) .setOutputValueClass(TestSpecificValue.class) .build(); - assertTrue(dummyRecordTransformerConfig.isSpecificClient()); + assertTrue(dummyRecordTransformerConfig.useSpecificRecordKeyDeserializer()); + assertTrue(dummyRecordTransformerConfig.useSpecificRecordValueDeserializer()); - DaVinciRecordTransformer recordTransformer = + DaVinciRecordTransformer recordTransformer = new TestSpecificRecordTransformer( storeVersion, keySchema, @@ -297,27 +302,35 @@ public void testSpecificRecordTransformer() throws NoSuchFieldException, Illegal valueSchema, dummyRecordTransformerConfig); - DaVinciRecordTransformerUtility recordTransformerUtility = + DaVinciRecordTransformerUtility recordTransformerUtility = recordTransformer.getRecordTransformerUtility(); + Field keyDeserializerField = recordTransformerUtility.getClass().getDeclaredField("keyDeserializer"); + keyDeserializerField.setAccessible(true); + assertTrue(keyDeserializerField.get(recordTransformerUtility) instanceof AvroSpecificDeserializer); + Field outputValueDeserializerField = recordTransformerUtility.getClass().getDeclaredField("outputValueDeserializer"); outputValueDeserializerField.setAccessible(true); assertTrue(outputValueDeserializerField.get(recordTransformerUtility) instanceof AvroSpecificDeserializer); + TestSpecificKey specificKey = new TestSpecificKey(); + int id = 123; + specificKey.id = id; + Lazy lazyKey = Lazy.of(() -> specificKey); + TestSpecificValue specificValue = new TestSpecificValue(); String firstName = "first"; String lastName = "last"; specificValue.firstName = firstName; specificValue.lastName = lastName; - Lazy lazyValue = Lazy.of(() -> specificValue); DaVinciRecordTransformerResult transformerResult = recordTransformer.transform(lazyKey, lazyValue, partitionId); assertEquals(transformerResult.getResult(), DaVinciRecordTransformerResult.Result.TRANSFORMED); TestSpecificValue transformedSpecificValue = transformerResult.getValue(); - assertEquals(transformedSpecificValue.firstName, firstName.toUpperCase()); - assertEquals(transformedSpecificValue.lastName, lastName.toUpperCase()); + assertEquals(transformedSpecificValue.firstName, firstName + id); + assertEquals(transformedSpecificValue.lastName, lastName + id); } } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificKey.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificKey.java new file mode 100644 index 00000000000..507e99289e6 --- /dev/null +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificKey.java @@ -0,0 +1,41 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package com.linkedin.davinci.transformer; + +@SuppressWarnings("all") +public class TestSpecificKey extends org.apache.avro.specific.SpecificRecordBase + implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"TestSpecificKey\",\"namespace\":\"example.avro\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"}]}"); + + public int id; + + public org.apache.avro.Schema getSchema() { + return SCHEMA$; + } + + // Used by DatumWriter. Applications should not call. + public Object get(int field$) { + switch (field$) { + case 0: + return id; + default: + throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + // Used by DatumWriter. Applications should not call. + @SuppressWarnings(value = "unchecked") + public void put(int field$, Object value$) { + switch (field$) { + case 0: + id = (Integer) value$; + break; + default: + throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } +} diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificRecordTransformer.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificRecordTransformer.java index afc9ce0961a..6717aca93c6 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificRecordTransformer.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/TestSpecificRecordTransformer.java @@ -9,7 +9,7 @@ public class TestSpecificRecordTransformer - extends DaVinciRecordTransformer { + extends DaVinciRecordTransformer { public TestSpecificRecordTransformer( int storeVersion, Schema keySchema, @@ -21,18 +21,22 @@ public TestSpecificRecordTransformer( @Override public DaVinciRecordTransformerResult transform( - Lazy key, + Lazy lazyKey, Lazy lazyValue, int partitionId) { + TestSpecificKey key = lazyKey.get(); + int id = key.id; + TestSpecificValue value = lazyValue.get(); - value.firstName = value.firstName.toString().toUpperCase(); - value.lastName = value.lastName.toString().toUpperCase(); + + value.firstName = value.firstName.toString() + id; + value.lastName = value.lastName.toString() + id; return new DaVinciRecordTransformerResult<>(DaVinciRecordTransformerResult.Result.TRANSFORMED, value); } @Override - public void processPut(Lazy key, Lazy value, int partitionId) { + public void processPut(Lazy key, Lazy value, int partitionId) { return; } diff --git a/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java b/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java index 0921de7dfd6..fc307ede5b8 100644 --- a/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java +++ b/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java @@ -31,6 +31,7 @@ public class ClientConfig { private String storeName; private String veniceURL; private String statsPrefix; + private Class specificKeyClass = null; private Class specificValueClass = null; private Schema specificValueSchema = null; private boolean isVsonClient = false; @@ -98,6 +99,7 @@ public static ClientConfig cloneConfig(ClientConfi // Basic settings .setStoreName(config.getStoreName()) .setVeniceURL(config.getVeniceURL()) + .setSpecificKeyClass(config.getSpecificKeyClass()) .setSpecificValueClass(config.getSpecificValueClass()) .setSpecificValueSchema(config.getSpecificValueSchema()) .setVsonClient(config.isVsonClient()) @@ -197,6 +199,10 @@ public Schema getSpecificValueSchema() { return specificValueSchema; } + /** + * If you're using the experimental client, and you want to deserialize your values into Avro SpecificRecords + * then set this configuration. + */ public ClientConfig setSpecificValueSchema(Schema specificValueSchema) { this.specificValueSchema = specificValueSchema; return this; @@ -211,6 +217,19 @@ public ClientConfig setSpecificValueClass(Class specificValueClass) { return this; } + public Class getSpecificKeyClass() { + return specificKeyClass; + } + + /** + * If you're using the experimental client, and you want to deserialize your keys into Avro SpecificRecords + * then set this configuration. + */ + public ClientConfig setSpecificKeyClass(Class specificKeyClass) { + this.specificKeyClass = specificKeyClass; + return this; + } + public boolean isSpecificClient() { return specificValueClass != null; } diff --git a/gradle/spotbugs/exclude.xml b/gradle/spotbugs/exclude.xml index 9503f77025e..9dcbc1656ac 100644 --- a/gradle/spotbugs/exclude.xml +++ b/gradle/spotbugs/exclude.xml @@ -290,6 +290,9 @@ + + + diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index 55cb8035f08..8b32bb3512e 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -31,15 +31,14 @@ import static com.linkedin.venice.utils.SslUtils.LOCAL_KEYSTORE_JKS; import static com.linkedin.venice.utils.SslUtils.LOCAL_PASSWORD; import static com.linkedin.venice.utils.TestWriteUtils.DEFAULT_USER_DATA_RECORD_COUNT; -import static com.linkedin.venice.utils.TestWriteUtils.STRING_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; -import static com.linkedin.venice.utils.TestWriteUtils.renderNameRecord; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFile; import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertFalse; import static org.testng.AssertJUnit.assertTrue; @@ -58,6 +57,7 @@ import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.MultiStoreTopicsResponse; import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; +import com.linkedin.venice.endToEnd.TestChangelogKey; import com.linkedin.venice.endToEnd.TestChangelogValue; import com.linkedin.venice.integration.utils.PubSubBrokerConfigs; import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; @@ -95,6 +95,8 @@ import java.util.Properties; import java.util.concurrent.TimeUnit; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; import org.apache.avro.util.Utf8; import org.apache.samza.config.MapConfig; import org.testng.Assert; @@ -567,12 +569,13 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp .setControllerRequestRetryCount(3) .setBootstrapFileSystemPath(inputDirPath) .setIsExperimentalClientEnabled(true) + .setSpecificKey(TestChangelogKey.class) .setSpecificValue(TestChangelogValue.class) .setSpecificValueSchema(TestChangelogValue.SCHEMA$) .setD2Client(d2Client); VeniceChangelogConsumerClientFactory veniceChangelogConsumerClientFactory = new VeniceChangelogConsumerClientFactory(globalChangelogClientConfig, metricsRepository); - List> bootstrappingVeniceChangelogConsumerList = + List> bootstrappingVeniceChangelogConsumerList = Collections.singletonList( veniceChangelogConsumerClientFactory .getBootstrappingChangelogConsumer(storeName, Integer.toString(0), TestChangelogValue.class)); @@ -587,9 +590,9 @@ public void testSpecificRecordVeniceChangelogConsumerDaVinciRecordTransformerImp bootstrappingVeniceChangelogConsumerList.get(0).start().get(); assertFalse(bootstrappingVeniceChangelogConsumerList.get(0).isCaughtUp()); - Map, VeniceChangeCoordinate>> polledChangeEventsMap = + Map, VeniceChangeCoordinate>> polledChangeEventsMap = new HashMap<>(); - List, VeniceChangeCoordinate>> polledChangeEventsList = + List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); // 20 changes in near-line. 10 puts, 10 deletes TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { @@ -680,12 +683,13 @@ public void testSpecificRecordBlobTransferVeniceChangelogConsumerDaVinciRecordTr .setBootstrapFileSystemPath(inputDirPath1) .setD2Client(d2Client) .setIsExperimentalClientEnabled(true) + .setSpecificKey(TestChangelogKey.class) .setSpecificValue(TestChangelogValue.class) .setSpecificValueSchema(TestChangelogValue.SCHEMA$); VeniceChangelogConsumerClientFactory veniceChangelogConsumerClientFactory = new VeniceChangelogConsumerClientFactory(globalChangelogClientConfig, metricsRepository); - List> bootstrappingVeniceChangelogConsumerList = + List> bootstrappingVeniceChangelogConsumerList = Collections.singletonList( veniceChangelogConsumerClientFactory .getBootstrappingChangelogConsumer(storeName, Integer.toString(0), TestChangelogValue.class)); @@ -720,9 +724,9 @@ public void testSpecificRecordBlobTransferVeniceChangelogConsumerDaVinciRecordTr Assert.assertTrue(Files.exists(Paths.get(snapshotPath))); } - Map, VeniceChangeCoordinate>> polledChangeEventsMap = + Map, VeniceChangeCoordinate>> polledChangeEventsMap = new HashMap<>(); - List, VeniceChangeCoordinate>> polledChangeEventsList = + List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { pollChangeEventsFromSpecificChangeCaptureConsumer( @@ -791,14 +795,14 @@ private void pollChangeEventsFromChangeCaptureConsumer( } private void pollChangeEventsFromSpecificChangeCaptureConsumer( - Map, VeniceChangeCoordinate>> keyToMessageMap, - List, VeniceChangeCoordinate>> polledMessageList, - List> bootstrappingVeniceChangelogConsumerList) { - for (BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer: bootstrappingVeniceChangelogConsumerList) { - Collection, VeniceChangeCoordinate>> pubSubMessages = + Map, VeniceChangeCoordinate>> keyToMessageMap, + List, VeniceChangeCoordinate>> polledMessageList, + List> bootstrappingVeniceChangelogConsumerList) { + for (BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer: bootstrappingVeniceChangelogConsumerList) { + Collection, VeniceChangeCoordinate>> pubSubMessages = bootstrappingVeniceChangelogConsumer.poll(1000); - for (PubSubMessage, VeniceChangeCoordinate> pubSubMessage: pubSubMessages) { - String key = pubSubMessage.getKey() == null ? null : pubSubMessage.getKey().toString(); + for (PubSubMessage, VeniceChangeCoordinate> pubSubMessage: pubSubMessages) { + String key = pubSubMessage.getKey() == null ? null : String.valueOf(pubSubMessage.getKey().id); keyToMessageMap.put(key, pubSubMessage); } polledMessageList.addAll(pubSubMessages); @@ -816,14 +820,17 @@ private void runSamzaStreamJob( // Send PUT requests. for (int i = startIdx; i < startIdx + numPuts; i++) { if (useSpecificRecord) { + TestChangelogKey key = new TestChangelogKey(); + key.id = i; + TestChangelogValue value = new TestChangelogValue(); - value.put("firstName", "first_name_stream_" + i); - value.put("lastName", "last_name_stream_" + i); + value.firstName = "first_name_stream_" + i; + value.lastName = "last_name_stream_" + i; sendStreamingRecord( veniceProducer, storeName, - Integer.toString(i), + key, value, mockedTime == null ? null : mockedTime.getMilliseconds()); } else { @@ -837,11 +844,23 @@ private void runSamzaStreamJob( } // Send DELETE requests. for (int i = startIdx + numPuts; i < startIdx + numPuts + numDels; i++) { - sendStreamingDeleteRecord( - veniceProducer, - storeName, - Integer.toString(i), - mockedTime == null ? null : mockedTime.getMilliseconds()); + if (useSpecificRecord) { + TestChangelogKey key = new TestChangelogKey(); + key.id = i; + + sendStreamingDeleteRecord( + veniceProducer, + storeName, + key, + mockedTime == null ? null : mockedTime.getMilliseconds()); + } else { + sendStreamingDeleteRecord( + veniceProducer, + storeName, + Integer.toString(i), + mockedTime == null ? null : mockedTime.getMilliseconds()); + } + } } @@ -856,21 +875,32 @@ private String setUpStore(String storeName, boolean useSpecificRecord) throws Ex Properties props = defaultVPJProps(clusterWrapper, inputDirPath, storeName); Schema recordSchema; + String keySchemaStr; String valueSchemaStr; if (useSpecificRecord) { - Schema tempRecordSchema = - new PushInputSchemaBuilder().setKeySchema(STRING_SCHEMA).setValueSchema(TestChangelogValue.SCHEMA$).build(); - recordSchema = writeSimpleAvroFile( - inputDir, - tempRecordSchema, - i -> renderNameRecord(tempRecordSchema, i), - DEFAULT_USER_DATA_RECORD_COUNT); - valueSchemaStr = TestChangelogValue.SCHEMA$.toString(); + recordSchema = new PushInputSchemaBuilder().setKeySchema(TestChangelogKey.SCHEMA$) + .setValueSchema(TestChangelogValue.SCHEMA$) + .build(); + + writeSimpleAvroFile(inputDir, recordSchema, i -> { + GenericRecord keyValueRecord = new GenericData.Record(recordSchema); + TestChangelogKey key = new TestChangelogKey(); + key.id = i; + keyValueRecord.put(DEFAULT_KEY_FIELD_PROP, key); + + TestChangelogValue value = new TestChangelogValue(); + value.firstName = "first_name" + i; + value.lastName = "last_name" + i; + keyValueRecord.put(DEFAULT_VALUE_FIELD_PROP, value); + + return keyValueRecord; + }, DEFAULT_USER_DATA_RECORD_COUNT); } else { recordSchema = TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema(inputDir); - valueSchemaStr = recordSchema.getField(DEFAULT_VALUE_FIELD_PROP).schema().toString(); } - String keySchemaStr = recordSchema.getField(DEFAULT_KEY_FIELD_PROP).schema().toString(); + + keySchemaStr = recordSchema.getField(DEFAULT_KEY_FIELD_PROP).schema().toString(); + valueSchemaStr = recordSchema.getField(DEFAULT_VALUE_FIELD_PROP).schema().toString(); UpdateStoreQueryParams storeParms = new UpdateStoreQueryParams().setHybridRewindSeconds(500) .setHybridOffsetLagThreshold(8) @@ -948,9 +978,9 @@ private int runSpecificNearlineJobAndVerifyConsumption( int startIndex, String storeName, int version, - Map, VeniceChangeCoordinate>> polledChangeEventsMap, - List, VeniceChangeCoordinate>> polledChangeEventsList, - List> bootstrappingVeniceChangelogConsumerList) { + Map, VeniceChangeCoordinate>> polledChangeEventsMap, + List, VeniceChangeCoordinate>> polledChangeEventsList, + List> bootstrappingVeniceChangelogConsumerList) { Map samzaConfig = getSamzaProducerConfig(clusterWrapper, storeName, Version.PushType.STREAM); VeniceSystemFactory factory = new VeniceSystemFactory(); // Half puts and half deletes @@ -965,12 +995,17 @@ private int runSpecificNearlineJobAndVerifyConsumption( runSamzaStreamJob(veniceProducer, storeName, null, numPuts, numDeletes, startIndex, true); } - try (AvroGenericStoreClient client = ClientFactory.getAndStartGenericAvroClient( - ClientConfig.defaultGenericClientConfig(storeName) - .setVeniceURL(clusterWrapper.getRandomRouterURL()) - .setMetricsRepository(metricsRepository))) { + try (AvroGenericStoreClient client = + ClientFactory.getAndStartSpecificAvroClient( + ClientConfig.defaultSpecificClientConfig(storeName, TestChangelogValue.class) + .setVeniceURL(clusterWrapper.getRandomRouterURL()) + .setMetricsRepository(metricsRepository) + .setUseFastAvro(true))) { TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { - Assert.assertNotNull(client.get(Integer.toString(startIndex + numPuts - 1)).get()); + TestChangelogKey key = new TestChangelogKey(); + key.id = startIndex + numPuts - 1; + + Assert.assertNotNull(client.get(key).get()); }); } @@ -1009,13 +1044,13 @@ private void verifyPut( } private void verifySpecificPut( - Map, VeniceChangeCoordinate>> polledChangeEventsMap, + Map, VeniceChangeCoordinate>> polledChangeEventsMap, int startIndex, int endIndex, int version) { for (int i = startIndex; i < endIndex; i++) { String key = Integer.toString(i); - PubSubMessage, VeniceChangeCoordinate> message = + PubSubMessage, VeniceChangeCoordinate> message = polledChangeEventsMap.get((key)); ChangeEvent changeEvent = message.getValue(); int versionFromMessage = Version.parseVersionFromVersionTopicName(message.getTopicPartition().getTopicName()); @@ -1047,14 +1082,17 @@ private void verifyDelete( } private void verifySpecificDelete( - Map, VeniceChangeCoordinate>> polledChangeEventsMap, + Map, VeniceChangeCoordinate>> polledChangeEventsMap, int startIndex, int endIndex, int version) { for (int i = startIndex; i < endIndex; i++) { - String key = Integer.toString(i); - PubSubMessage, VeniceChangeCoordinate> message = - polledChangeEventsMap.get((key)); + PubSubMessage, VeniceChangeCoordinate> message = + polledChangeEventsMap.get((Integer.toString(i))); + + TestChangelogKey key = message.getKey(); + assertEquals(key.id, i); + ChangeEvent changeEvent = message.getValue(); int versionFromMessage = Version.parseVersionFromVersionTopicName(message.getTopicPartition().getTopicName()); Assert.assertEquals(versionFromMessage, version); @@ -1076,9 +1114,9 @@ private void verifyNoRecordsProduced( } private void verifyNoSpecificRecordsProduced( - Map, VeniceChangeCoordinate>> polledChangeEventsMap, - List, VeniceChangeCoordinate>> polledChangeEventsList, - List> bootstrappingVeniceChangelogConsumerList) { + Map, VeniceChangeCoordinate>> polledChangeEventsMap, + List, VeniceChangeCoordinate>> polledChangeEventsList, + List> bootstrappingVeniceChangelogConsumerList) { pollChangeEventsFromSpecificChangeCaptureConsumer( polledChangeEventsMap, polledChangeEventsList, diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestChangelogKey.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestChangelogKey.java new file mode 100644 index 00000000000..052e71d33b0 --- /dev/null +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestChangelogKey.java @@ -0,0 +1,41 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package com.linkedin.venice.endToEnd; + +@SuppressWarnings("all") +public class TestChangelogKey extends org.apache.avro.specific.SpecificRecordBase + implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"TestChangelogKey\",\"namespace\":\"example.avro\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"}]}"); + + public int id; + + public org.apache.avro.Schema getSchema() { + return SCHEMA$; + } + + // Used by DatumWriter. Applications should not call. + public Object get(int field$) { + switch (field$) { + case 0: + return id; + default: + throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + // Used by DatumWriter. Applications should not call. + @SuppressWarnings(value = "unchecked") + public void put(int field$, Object value$) { + switch (field$) { + case 0: + id = (Integer) value$; + break; + default: + throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } +} diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java index d5e3ba8ee96..7f15898e5bb 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java @@ -408,7 +408,7 @@ public static void sendStreamingDeleteRecord(SystemProducer producer, String sto public static void sendStreamingDeleteRecord( SystemProducer producer, String storeName, - String key, + Object key, Long logicalTimeStamp) { sendStreamingRecord(producer, storeName, key, null, logicalTimeStamp); } From 23a909aec47ea832a7e986c1913bfd0275df1651 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Wed, 14 May 2025 15:44:10 -0700 Subject: [PATCH 16/25] cleanup --- .../linkedin/davinci/consumer/ChangelogClientConfig.java | 8 ++++++++ .../com/linkedin/venice/client/store/ClientConfig.java | 8 -------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java index aee9a7fd6fe..8c83ecd0fdc 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java @@ -246,6 +246,10 @@ public ChangelogClientConfig setRocksDBBlockCacheSizeInBytes(long rocksDBBlockCa return this; } + /** + * If you're using the experimental client, and you want to deserialize your keys into + * {@link org.apache.avro.specific.SpecificRecord} set this configuration. + */ public ChangelogClientConfig setSpecificKey(Class specificKey) { this.innerClientConfig.setSpecificKeyClass(specificKey); return this; @@ -256,6 +260,10 @@ public ChangelogClientConfig setSpecificValue(Class specificValue) { return this; } + /** + * If you're using the experimental client, and you want to deserialize your values into + * {@link org.apache.avro.specific.SpecificRecord} then set this configuration. + */ public ChangelogClientConfig setSpecificValueSchema(Schema specificValueSchema) { this.innerClientConfig.setSpecificValueSchema(specificValueSchema); return this; diff --git a/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java b/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java index fc307ede5b8..8b44f2f26b2 100644 --- a/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java +++ b/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/ClientConfig.java @@ -199,10 +199,6 @@ public Schema getSpecificValueSchema() { return specificValueSchema; } - /** - * If you're using the experimental client, and you want to deserialize your values into Avro SpecificRecords - * then set this configuration. - */ public ClientConfig setSpecificValueSchema(Schema specificValueSchema) { this.specificValueSchema = specificValueSchema; return this; @@ -221,10 +217,6 @@ public Class getSpecificKeyClass() { return specificKeyClass; } - /** - * If you're using the experimental client, and you want to deserialize your keys into Avro SpecificRecords - * then set this configuration. - */ public ClientConfig setSpecificKeyClass(Class specificKeyClass) { this.specificKeyClass = specificKeyClass; return this; From c419f155838dcf2d64d38e0b013f47b4246030c5 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Wed, 14 May 2025 19:59:09 -0700 Subject: [PATCH 17/25] Update javadoc --- .../davinci/consumer/BootstrappingVeniceChangelogConsumer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java index 7191bfa5383..52825447660 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java @@ -49,6 +49,7 @@ public interface BootstrappingVeniceChangelogConsumer { * * In the experimental client, records will be returned in batches configured to the MAX_BUFFER_SIZE. So the initial * calls to poll will be from records from the bootstrap state, until the partitions have caught up. + * Additionally, if the buffer hits the MAX_BUFFER_SIZE before the timeout is hit, poll will return immediately. * * @param timeoutInMs * @return From 943722cd9f3de74e2ce39e3e9acdc6a6a3fa0b11 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Wed, 14 May 2025 21:50:49 -0700 Subject: [PATCH 18/25] make sure to use fast avro for serializer --- .../DaVinciRecordTransformerConfig.java | 14 +++++++---- .../DaVinciRecordTransformerUtility.java | 3 +-- .../consumer/ChangelogClientConfig.java | 2 +- .../transformer/RecordTransformerTest.java | 23 +++++++++++++------ ...nsumerDaVinciRecordTransformerUserApp.java | 20 ++++++++-------- .../TestBootstrappingChangelogConsumer.java | 1 - 6 files changed, 39 insertions(+), 24 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java index 90ccd738c0a..8a2951490ef 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerConfig.java @@ -18,6 +18,8 @@ public class DaVinciRecordTransformerConfig { private final boolean storeRecordsInDaVinci; private final boolean alwaysBootstrapFromVersionTopic; private final boolean skipCompatibilityChecks; + private final boolean useSpecificRecordKeyDeserializer; + private final boolean useSpecificRecordValueDeserializer; public DaVinciRecordTransformerConfig(Builder builder) { this.recordTransformerFunction = Optional.ofNullable(builder.recordTransformerFunction) @@ -31,6 +33,10 @@ public DaVinciRecordTransformerConfig(Builder builder) { throw new VeniceException("outputValueClass and outputValueSchema must be defined together"); } + this.useSpecificRecordKeyDeserializer = keyClass != null && SpecificRecord.class.isAssignableFrom(keyClass); + this.useSpecificRecordValueDeserializer = + outputValueClass != null && SpecificRecord.class.isAssignableFrom(outputValueClass); + this.storeRecordsInDaVinci = builder.storeRecordsInDaVinci; this.alwaysBootstrapFromVersionTopic = builder.alwaysBootstrapFromVersionTopic; this.skipCompatibilityChecks = builder.skipCompatibilityChecks; @@ -51,10 +57,10 @@ public Class getKeyClass() { } /** - * @return Whether the SpecificRecord deserializer should be used for keys + * @return Whether the {@link SpecificRecord} deserializer should be used for keys */ public boolean useSpecificRecordKeyDeserializer() { - return keyClass != null && SpecificRecord.class.isAssignableFrom(keyClass); + return useSpecificRecordKeyDeserializer; } /** @@ -65,10 +71,10 @@ public Class getOutputValueClass() { } /** - * @return Whether the SpecificRecord deserializer should be used for values + * @return Whether the {@link SpecificRecord} deserializer should be used for values */ public boolean useSpecificRecordValueDeserializer() { - return outputValueClass != null && SpecificRecord.class.isAssignableFrom(outputValueClass); + return useSpecificRecordValueDeserializer; } /** diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java index 629d1d975da..390a3d01793 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/client/DaVinciRecordTransformerUtility.java @@ -6,7 +6,6 @@ import com.linkedin.venice.kafka.protocol.state.PartitionState; import com.linkedin.venice.offsets.OffsetRecord; import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; -import com.linkedin.venice.serializer.AvroSerializer; import com.linkedin.venice.serializer.FastSerializerDeserializerFactory; import com.linkedin.venice.serializer.RecordDeserializer; import com.linkedin.venice.serializer.RecordSerializer; @@ -56,7 +55,7 @@ public DaVinciRecordTransformerUtility( FastSerializerDeserializerFactory.getFastAvroGenericDeserializer(outputValueSchema, outputValueSchema); } - this.outputValueSerializer = new AvroSerializer<>(outputValueSchema); + this.outputValueSerializer = FastSerializerDeserializerFactory.getFastAvroGenericSerializer(outputValueSchema); } /** diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java index 8c83ecd0fdc..f49c87071bd 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/ChangelogClientConfig.java @@ -248,7 +248,7 @@ public ChangelogClientConfig setRocksDBBlockCacheSizeInBytes(long rocksDBBlockCa /** * If you're using the experimental client, and you want to deserialize your keys into - * {@link org.apache.avro.specific.SpecificRecord} set this configuration. + * {@link org.apache.avro.specific.SpecificRecord} thenr set this configuration. */ public ChangelogClientConfig setSpecificKey(Class specificKey) { this.innerClientConfig.setSpecificKeyClass(specificKey); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java index 37b0b9d3c46..486099a55c6 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/transformer/RecordTransformerTest.java @@ -27,6 +27,7 @@ import com.linkedin.venice.offsets.OffsetRecord; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; +import com.linkedin.venice.serializer.AvroGenericDeserializer; import com.linkedin.venice.serializer.AvroSpecificDeserializer; import com.linkedin.venice.utils.lazy.Lazy; import java.lang.reflect.Field; @@ -47,7 +48,7 @@ public class RecordTransformerTest { static final Schema valueSchema = Schema.create(Schema.Type.STRING); @Test - public void testRecordTransformer() { + public void testRecordTransformer() throws NoSuchFieldException, IllegalAccessException { DaVinciRecordTransformerConfig dummyRecordTransformerConfig = new DaVinciRecordTransformerConfig.Builder().setRecordTransformerFunction(TestStringRecordTransformer::new) .setStoreRecordsInDaVinci(false) @@ -69,6 +70,20 @@ public void testRecordTransformer() { assertEquals(recordTransformer.getKeySchema().getType(), Schema.Type.INT); assertEquals(recordTransformer.getOutputValueSchema().getType(), Schema.Type.STRING); + DaVinciRecordTransformerUtility recordTransformerUtility = + recordTransformer.getRecordTransformerUtility(); + + Field keyDeserializerField = recordTransformerUtility.getClass().getDeclaredField("keyDeserializer"); + keyDeserializerField.setAccessible(true); + assertTrue(keyDeserializerField.get(recordTransformerUtility) instanceof AvroGenericDeserializer); + assertFalse(keyDeserializerField.get(recordTransformerUtility) instanceof AvroSpecificDeserializer); + + Field outputValueDeserializerField = + recordTransformerUtility.getClass().getDeclaredField("outputValueDeserializer"); + outputValueDeserializerField.setAccessible(true); + assertTrue(outputValueDeserializerField.get(recordTransformerUtility) instanceof AvroGenericDeserializer); + assertFalse(outputValueDeserializerField.get(recordTransformerUtility) instanceof AvroSpecificDeserializer); + DaVinciRecordTransformerResult transformerResult = recordTransformer.transform(lazyKey, lazyValue, partitionId); recordTransformer.processPut(lazyKey, lazyValue, partitionId); @@ -81,15 +96,9 @@ public void testRecordTransformer() { assertFalse(recordTransformer.getStoreRecordsInDaVinci()); int classHash = recordTransformer.getClassHash(); - - DaVinciRecordTransformerUtility recordTransformerUtility = - recordTransformer.getRecordTransformerUtility(); OffsetRecord offsetRecord = new OffsetRecord(partitionStateSerializer); - assertTrue(recordTransformerUtility.hasTransformerLogicChanged(classHash, offsetRecord)); - offsetRecord.setRecordTransformerClassHash(classHash); - assertFalse(recordTransformerUtility.hasTransformerLogicChanged(classHash, offsetRecord)); } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ChangelogConsumerDaVinciRecordTransformerUserApp.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ChangelogConsumerDaVinciRecordTransformerUserApp.java index 8f9d76af75c..7c0bdebcc6f 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ChangelogConsumerDaVinciRecordTransformerUserApp.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ChangelogConsumerDaVinciRecordTransformerUserApp.java @@ -32,6 +32,7 @@ import com.linkedin.davinci.consumer.VeniceChangeCoordinate; import com.linkedin.davinci.consumer.VeniceChangelogConsumerClientFactory; import com.linkedin.venice.D2.D2ClientUtils; +import com.linkedin.venice.endToEnd.TestChangelogKey; import com.linkedin.venice.endToEnd.TestChangelogValue; import com.linkedin.venice.integration.utils.VeniceRouterWrapper; import com.linkedin.venice.pubsub.api.PubSubMessage; @@ -120,7 +121,8 @@ public static void main(String[] args) throws InterruptedException, ExecutionExc .setIsExperimentalClientEnabled(true); if (useSpecificRecord) { - globalChangelogClientConfig.setSpecificValue(TestChangelogValue.class) + globalChangelogClientConfig.setSpecificKey(TestChangelogKey.class) + .setSpecificValue(TestChangelogValue.class) .setSpecificValueSchema(TestChangelogValue.SCHEMA$); } @@ -140,9 +142,9 @@ public static void main(String[] args) throws InterruptedException, ExecutionExc LOGGER.info("DVRT CDC user app has come online."); if (useSpecificRecord) { - Map, VeniceChangeCoordinate>> polledChangeEventsMap = + Map, VeniceChangeCoordinate>> polledChangeEventsMap = new HashMap<>(); - List, VeniceChangeCoordinate>> polledChangeEventsList = + List, VeniceChangeCoordinate>> polledChangeEventsList = new ArrayList<>(); TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { pollChangeEventsFromSpecificChangeCaptureConsumer( @@ -181,13 +183,13 @@ private static void pollChangeEventsFromChangeCaptureConsumer( } private static void pollChangeEventsFromSpecificChangeCaptureConsumer( - Map, VeniceChangeCoordinate>> keyToMessageMap, - List, VeniceChangeCoordinate>> polledMessageList, - BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer) { - Collection, VeniceChangeCoordinate>> pubSubMessages = + Map, VeniceChangeCoordinate>> keyToMessageMap, + List, VeniceChangeCoordinate>> polledMessageList, + BootstrappingVeniceChangelogConsumer bootstrappingVeniceChangelogConsumer) { + Collection, VeniceChangeCoordinate>> pubSubMessages = bootstrappingVeniceChangelogConsumer.poll(1000); - for (PubSubMessage, VeniceChangeCoordinate> pubSubMessage: pubSubMessages) { - String key = pubSubMessage.getKey() == null ? null : pubSubMessage.getKey().toString(); + for (PubSubMessage, VeniceChangeCoordinate> pubSubMessage: pubSubMessages) { + String key = pubSubMessage.getKey() == null ? null : String.valueOf(pubSubMessage.getKey().id); keyToMessageMap.put(key, pubSubMessage); } polledMessageList.addAll(pubSubMessages); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index 8b32bb3512e..a9747d69a06 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -105,7 +105,6 @@ import org.testng.annotations.Test; -@Test(singleThreaded = true) public class TestBootstrappingChangelogConsumer { private static final int TEST_TIMEOUT = 2 * Time.MS_PER_MINUTE; private static final String REGION_NAME = "local-pubsub"; From f37643e40799f66ba77105ebc0b06b5e81c28147 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Thu, 15 May 2025 14:24:39 -0700 Subject: [PATCH 19/25] Throw exception when user calls start multiple times --- .../BootstrappingVeniceChangelogConsumer.java | 2 ++ ...gConsumerDaVinciRecordTransformerImpl.java | 33 +++++++++++-------- ...sumerDaVinciRecordTransformerImplTest.java | 26 +++++++++++++++ 3 files changed, 48 insertions(+), 13 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java index 52825447660..8c40cbea6cd 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java @@ -34,6 +34,8 @@ public interface BootstrappingVeniceChangelogConsumer { * Out Of Memory error. Instead, use the {@link #isCaughtUp()} method to determine once all subscribed partitions have * caught up. * + * NOTE: In the experimental client, if you pass in an empty set, it will subscribe to all partitions for the store + * * @param partitions which partition id's to catch up with * @return a future that completes once catch up is complete for all passed in partitions. */ diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java index 2f886933605..0ce64ca2b24 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java @@ -32,6 +32,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; @@ -120,8 +121,21 @@ public BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl(Changelo @Override public CompletableFuture start(Set partitions) { - internalStart(); - subscribedPartitions.addAll(partitions); + if (isStarted) { + throw new VeniceException("BootstrappingVeniceChangelogConsumer is already started!"); + } + + daVinciClient.start(); + isStarted = true; + + // If a user passes in empty partitions set, we subscribe to all partitions + if (partitions.isEmpty()) { + for (int i = 0; i < daVinciClient.getPartitionCount(); i++) { + subscribedPartitions.add(i); + } + } else { + subscribedPartitions.addAll(partitions); + } CompletableFuture startFuture = CompletableFuture.supplyAsync(() -> { try { @@ -146,9 +160,9 @@ public CompletableFuture start(Set partitions) { * prevents the user from calling poll to drain pubSubMessages, so the threads populating pubSubMessages * will wait forever for capacity to become available. This leads to a deadlock. */ - daVinciClient.subscribe(partitions).whenComplete((result, error) -> { + daVinciClient.subscribe(subscribedPartitions).whenComplete((result, error) -> { if (error != null) { - LOGGER.error("Failed to subscribe to partitions: {} for store: {}", partitions, storeName, error); + LOGGER.error("Failed to subscribe to partitions: {} for store: {}", subscribedPartitions, storeName, error); startFuture.completeExceptionally(new VeniceException(error)); return; } @@ -157,7 +171,7 @@ public CompletableFuture start(Set partitions) { LOGGER.info( "BootstrappingVeniceChangelogConsumer is caught up for store: {} for partitions: {}", storeName, - partitions); + subscribedPartitions); }); return startFuture; @@ -165,14 +179,7 @@ public CompletableFuture start(Set partitions) { @Override public CompletableFuture start() { - internalStart(); - - Set allPartitions = new HashSet<>(); - for (int i = 0; i < daVinciClient.getPartitionCount(); i++) { - allPartitions.add(i); - } - - return this.start(allPartitions); + return this.start(Collections.emptySet()); } @Override diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java index e0bc4e4fdac..20ff64bd98b 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java @@ -158,6 +158,23 @@ public void testStartAllPartitions() throws IllegalAccessException, NoSuchFieldE assertEquals(subscribedPartitionsField.get(bootstrappingVeniceChangelogConsumer), partitionSet); } + @Test + public void testStartWithEmptyPartitions() throws NoSuchFieldException, IllegalAccessException { + bootstrappingVeniceChangelogConsumer.start(Collections.emptySet()); + + Field isStartedField = + BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.class.getDeclaredField("isStarted"); + isStartedField.setAccessible(true); + assertTrue((Boolean) isStartedField.get(bootstrappingVeniceChangelogConsumer), "isStarted should be true"); + + verify(mockDaVinciClient).start(); + + Field subscribedPartitionsField = + BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.class.getDeclaredField("subscribedPartitions"); + subscribedPartitionsField.setAccessible(true); + assertEquals(subscribedPartitionsField.get(bootstrappingVeniceChangelogConsumer), partitionSet); + } + @Test public void testStartSpecificPartitions() throws IllegalAccessException, NoSuchFieldException { Set partitionSet = Collections.singleton(1); @@ -176,6 +193,15 @@ public void testStartSpecificPartitions() throws IllegalAccessException, NoSuchF assertEquals(subscribedPartitionsField.get(bootstrappingVeniceChangelogConsumer), partitionSet); } + @Test + public void testStartMultipleTimes() { + Set partitionSet = Collections.singleton(1); + bootstrappingVeniceChangelogConsumer.start(); + + assertThrows(VeniceException.class, () -> bootstrappingVeniceChangelogConsumer.start()); + assertThrows(VeniceException.class, () -> bootstrappingVeniceChangelogConsumer.start(partitionSet)); + } + @Test public void testStop() throws Exception { CachingDaVinciClientFactory daVinciClientFactoryMock = mock(CachingDaVinciClientFactory.class); From e97075604108cb31d4584558a179f272b7334b10 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Thu, 15 May 2025 15:04:26 -0700 Subject: [PATCH 20/25] Update javadoc --- .../davinci/consumer/BootstrappingVeniceChangelogConsumer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java index 8c40cbea6cd..ea8759562d6 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java @@ -59,6 +59,7 @@ public interface BootstrappingVeniceChangelogConsumer { Collection, VeniceChangeCoordinate>> poll(long timeoutInMs); /** + * In the experimental client, once this becomes true it will stay true. * @return True if all subscribed partitions have caught up. */ boolean isCaughtUp(); From 49d48db127f90d08de8dcdaf6bd450f1258a81c4 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Thu, 15 May 2025 15:05:01 -0700 Subject: [PATCH 21/25] Update javadoc --- .../davinci/consumer/BootstrappingVeniceChangelogConsumer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java index ea8759562d6..5b77f938927 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumer.java @@ -59,7 +59,8 @@ public interface BootstrappingVeniceChangelogConsumer { Collection, VeniceChangeCoordinate>> poll(long timeoutInMs); /** - * In the experimental client, once this becomes true it will stay true. + * In the experimental client, once this becomes true it will stay true even if we start to lag after the + * bootstrapping phase. * @return True if all subscribed partitions have caught up. */ boolean isCaughtUp(); From 5001bb798b95995efb0950b00d1c10e55a8f75d2 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Thu, 15 May 2025 15:08:19 -0700 Subject: [PATCH 22/25] Delete internalStart --- ...iceChangelogConsumerDaVinciRecordTransformerImpl.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java index 0ce64ca2b24..cbc6a65deb6 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java @@ -230,15 +230,6 @@ public boolean isCaughtUp() { return isCaughtUp.get(); } - private void internalStart() { - if (isStarted) { - return; - } - - daVinciClient.start(); - isStarted = true; - } - private VeniceProperties buildVeniceConfig() { return new PropertyBuilder() .put(ROCKSDB_BLOCK_CACHE_SIZE_IN_BYTES, changelogClientConfig.getRocksDBBlockCacheSizeInBytes()) From a7c8598007990cf8d44d688c3cb5b1ba74971bd3 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Mon, 19 May 2025 15:41:48 -0700 Subject: [PATCH 23/25] Make start synchronized --- ...pingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java index cbc6a65deb6..a73b76d7d90 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java @@ -120,7 +120,7 @@ public BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl(Changelo } @Override - public CompletableFuture start(Set partitions) { + public synchronized CompletableFuture start(Set partitions) { if (isStarted) { throw new VeniceException("BootstrappingVeniceChangelogConsumer is already started!"); } From 6415d0af5d843e9bf821c88937ca3607a3307e68 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Mon, 19 May 2025 15:55:28 -0700 Subject: [PATCH 24/25] make isStarted an atomicboolean --- ...niceChangelogConsumerDaVinciRecordTransformerImpl.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java index a73b76d7d90..e0f23228943 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.java @@ -72,7 +72,7 @@ public class BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl daVinciClient; - private boolean isStarted = false; + private AtomicBoolean isStarted = new AtomicBoolean(false); private final CountDownLatch startLatch = new CountDownLatch(1); // Using a dedicated thread pool for CompletableFutures created by this class to avoid potential thread starvation // issues in the default ForkJoinPool @@ -121,12 +121,12 @@ public BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl(Changelo @Override public synchronized CompletableFuture start(Set partitions) { - if (isStarted) { + if (isStarted.get()) { throw new VeniceException("BootstrappingVeniceChangelogConsumer is already started!"); } daVinciClient.start(); - isStarted = true; + isStarted.set(true); // If a user passes in empty partitions set, we subscribe to all partitions if (partitions.isEmpty()) { @@ -185,7 +185,7 @@ public CompletableFuture start() { @Override public void stop() throws Exception { daVinciClientFactory.close(); - isStarted = false; + isStarted.set(false); } @Override From 355427620c52ea4f1b924be38aedbc6b75d684de Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Mon, 19 May 2025 17:41:51 -0700 Subject: [PATCH 25/25] Fix unit tests --- ...ConsumerDaVinciRecordTransformerImplTest.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java index 20ff64bd98b..4b12ae3fc90 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImplTest.java @@ -44,6 +44,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; import org.apache.avro.Schema; @@ -148,7 +149,8 @@ public void testStartAllPartitions() throws IllegalAccessException, NoSuchFieldE Field isStartedField = BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.class.getDeclaredField("isStarted"); isStartedField.setAccessible(true); - assertTrue((Boolean) isStartedField.get(bootstrappingVeniceChangelogConsumer), "isStarted should be true"); + AtomicBoolean isStarted = (AtomicBoolean) isStartedField.get(bootstrappingVeniceChangelogConsumer); + assertTrue(isStarted.get(), "isStarted should be true"); verify(mockDaVinciClient).start(); @@ -165,7 +167,8 @@ public void testStartWithEmptyPartitions() throws NoSuchFieldException, IllegalA Field isStartedField = BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.class.getDeclaredField("isStarted"); isStartedField.setAccessible(true); - assertTrue((Boolean) isStartedField.get(bootstrappingVeniceChangelogConsumer), "isStarted should be true"); + AtomicBoolean isStarted = (AtomicBoolean) isStartedField.get(bootstrappingVeniceChangelogConsumer); + assertTrue(isStarted.get(), "isStarted should be true"); verify(mockDaVinciClient).start(); @@ -183,7 +186,8 @@ public void testStartSpecificPartitions() throws IllegalAccessException, NoSuchF Field isStartedField = BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.class.getDeclaredField("isStarted"); isStartedField.setAccessible(true); - assertTrue((Boolean) isStartedField.get(bootstrappingVeniceChangelogConsumer), "isStarted should be true"); + AtomicBoolean isStarted = (AtomicBoolean) isStartedField.get(bootstrappingVeniceChangelogConsumer); + assertTrue(isStarted.get(), "isStarted should be true"); verify(mockDaVinciClient).start(); @@ -215,10 +219,12 @@ public void testStop() throws Exception { Field isStartedField = BootstrappingVeniceChangelogConsumerDaVinciRecordTransformerImpl.class.getDeclaredField("isStarted"); isStartedField.setAccessible(true); - assertTrue((Boolean) isStartedField.get(bootstrappingVeniceChangelogConsumer), "isStarted should be true"); + AtomicBoolean isStarted = (AtomicBoolean) isStartedField.get(bootstrappingVeniceChangelogConsumer); + assertTrue(isStarted.get(), "isStarted should be true"); bootstrappingVeniceChangelogConsumer.stop(); - assertFalse((Boolean) isStartedField.get(bootstrappingVeniceChangelogConsumer), "isStarted should be false"); + isStarted = (AtomicBoolean) isStartedField.get(bootstrappingVeniceChangelogConsumer); + assertFalse(isStarted.get(), "isStarted should be false"); verify(daVinciClientFactoryMock).close(); }