nsivabalan commented on code in PR #7914: URL: https://github.com/apache/hudi/pull/7914#discussion_r1106180368
########## hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java: ########## @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; + +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.storage.StorageLevel; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.net.URI; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.getCommitTimeAtUTC; +import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class TestSparkRDDWriteClient extends SparkClientFunctionalTestHarness { + + static Stream<Arguments> testWriteClientReleaseResourcesShouldOnlyUnpersistRelevantRdds() { + return Stream.of( + Arguments.of(HoodieTableType.COPY_ON_WRITE, true), + Arguments.of(HoodieTableType.MERGE_ON_READ, true), + Arguments.of(HoodieTableType.COPY_ON_WRITE, false), + Arguments.of(HoodieTableType.MERGE_ON_READ, false) + ); + } + + @ParameterizedTest + @MethodSource + void testWriteClientReleaseResourcesShouldOnlyUnpersistRelevantRdds(HoodieTableType tableType, boolean shouldReleaseResource) throws IOException { + final HoodieTableMetaClient metaClient = getHoodieMetaClient(hadoopConf(), URI.create(basePath()).getPath(), tableType, new Properties()); + final HoodieWriteConfig writeConfig = getConfigBuilder(true) + .withPath(metaClient.getBasePathV2().toString()) + .withAutoCommit(false) + .withReleaseResourceEnabled(shouldReleaseResource) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .build(); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(0xDEED); + + String instant0 = getCommitTimeAtUTC(0); + List<GenericRecord> extraRecords0 = dataGen.generateGenericRecords(10); + JavaRDD persistedRdd0 = jsc().parallelize(extraRecords0, 2).persist(StorageLevel.MEMORY_AND_DISK()); + context().putCachedDataIds(writeConfig.getBasePath(), instant0, persistedRdd0.id()); + + String instant1 = getCommitTimeAtUTC(1); + List<GenericRecord> extraRecords1 = dataGen.generateGenericRecords(10); + JavaRDD persistedRdd1 = jsc().parallelize(extraRecords1, 2).persist(StorageLevel.MEMORY_AND_DISK()); + context().putCachedDataIds(writeConfig.getBasePath(), instant1, persistedRdd1.id()); + + SparkRDDWriteClient writeClient = getHoodieWriteClient(writeConfig); + List<HoodieRecord> records = dataGen.generateInserts(instant1, 10); + JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 2); + writeClient.startCommitWithTime(instant1); + List<WriteStatus> writeStatuses = writeClient.insert(writeRecords, instant1).collect(); + assertNoWriteErrors(writeStatuses); + writeClient.commitStats(instant1, writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()), + Option.empty(), metaClient.getCommitActionType()); + writeClient.close(); + + if (shouldReleaseResource) { + assertEquals(Collections.singletonList(persistedRdd0.id()), + context().getCachedDataIds(writeConfig.getBasePath(), instant0), + "RDDs cached for " + instant0 + " should be retained."); + assertEquals(Collections.emptyList(), Review Comment: minor. you can create two lists. expectedToRetain and expectedToCleared. and assert within a for loop for entires in the list. will reduce LOC ########## hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java: ########## @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; + +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.storage.StorageLevel; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.net.URI; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.getCommitTimeAtUTC; +import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class TestSparkRDDWriteClient extends SparkClientFunctionalTestHarness { + + static Stream<Arguments> testWriteClientReleaseResourcesShouldOnlyUnpersistRelevantRdds() { + return Stream.of( + Arguments.of(HoodieTableType.COPY_ON_WRITE, true), + Arguments.of(HoodieTableType.MERGE_ON_READ, true), + Arguments.of(HoodieTableType.COPY_ON_WRITE, false), + Arguments.of(HoodieTableType.MERGE_ON_READ, false) + ); + } + + @ParameterizedTest + @MethodSource + void testWriteClientReleaseResourcesShouldOnlyUnpersistRelevantRdds(HoodieTableType tableType, boolean shouldReleaseResource) throws IOException { + final HoodieTableMetaClient metaClient = getHoodieMetaClient(hadoopConf(), URI.create(basePath()).getPath(), tableType, new Properties()); + final HoodieWriteConfig writeConfig = getConfigBuilder(true) + .withPath(metaClient.getBasePathV2().toString()) + .withAutoCommit(false) + .withReleaseResourceEnabled(shouldReleaseResource) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .build(); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(0xDEED); + + String instant0 = getCommitTimeAtUTC(0); + List<GenericRecord> extraRecords0 = dataGen.generateGenericRecords(10); + JavaRDD persistedRdd0 = jsc().parallelize(extraRecords0, 2).persist(StorageLevel.MEMORY_AND_DISK()); + context().putCachedDataIds(writeConfig.getBasePath(), instant0, persistedRdd0.id()); + + String instant1 = getCommitTimeAtUTC(1); + List<GenericRecord> extraRecords1 = dataGen.generateGenericRecords(10); + JavaRDD persistedRdd1 = jsc().parallelize(extraRecords1, 2).persist(StorageLevel.MEMORY_AND_DISK()); + context().putCachedDataIds(writeConfig.getBasePath(), instant1, persistedRdd1.id()); + + SparkRDDWriteClient writeClient = getHoodieWriteClient(writeConfig); + List<HoodieRecord> records = dataGen.generateInserts(instant1, 10); + JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 2); + writeClient.startCommitWithTime(instant1); + List<WriteStatus> writeStatuses = writeClient.insert(writeRecords, instant1).collect(); + assertNoWriteErrors(writeStatuses); + writeClient.commitStats(instant1, writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()), + Option.empty(), metaClient.getCommitActionType()); + writeClient.close(); + + if (shouldReleaseResource) { + assertEquals(Collections.singletonList(persistedRdd0.id()), + context().getCachedDataIds(writeConfig.getBasePath(), instant0), + "RDDs cached for " + instant0 + " should be retained."); + assertEquals(Collections.emptyList(), + context().getCachedDataIds(writeConfig.getBasePath(), instant1), + "RDDs cached for " + instant1 + " should be cleared."); + assertTrue(jsc().getPersistentRDDs().containsKey(persistedRdd0.id()), + "RDDs cached for " + instant0 + " should be retained."); + assertFalse(jsc().getPersistentRDDs().containsKey(persistedRdd1.id()), + "RDDs cached for " + instant1 + " should be cleared."); + assertFalse(jsc().getPersistentRDDs().containsKey(writeRecords.id()), + "RDDs cached for " + instant1 + " should be cleared."); + } else { + assertEquals(Collections.singletonList(persistedRdd0.id()), + context().getCachedDataIds(writeConfig.getBasePath(), instant0), + "RDDs cached for " + instant0 + " should be retained."); + assertEquals(3, + context().getCachedDataIds(writeConfig.getBasePath(), instant1).size(), Review Comment: same here -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
