alexeykudinkin commented on code in PR #7914:
URL: https://github.com/apache/hudi/pull/7914#discussion_r1116333351


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java:
##########
@@ -83,7 +83,7 @@ public abstract void preCompact(
    *
    * @param writeStatus {@link HoodieData} of {@link WriteStatus}.
    */
-  public abstract void maybePersist(HoodieData<WriteStatus> writeStatus, 
HoodieWriteConfig config);
+  public abstract void maybePersist(HoodieData<WriteStatus> writeStatus, 
HoodieEngineContext context, HoodieWriteConfig config, String instantTime);

Review Comment:
   nit: Shall we place context as first arg (it's a convention)



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java:
##########
@@ -81,11 +84,22 @@ public static <K, V> JavaPairRDD<K, V> 
getJavaRDD(HoodiePairData<K, V> hoodieDat
     return ((HoodieJavaPairRDD<K, V>) hoodieData).get();
   }
 
+  @Override
+  public int getId() {
+    return rddData.id();
+  }
+
   @Override
   public void persist(String level) {
     rddData.persist(StorageLevel.fromString(level));
   }
 
+  @Override
+  public void persist(String level, HoodieEngineContext engineContext, 
HoodieDataCacheKey cacheKey) {

Review Comment:
   Why do we have 2 overrides now (one accepting context and one that doesn't)? 



##########
hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java:
##########
@@ -196,4 +212,42 @@ default <O> HoodieData<T> 
distinctWithKey(SerializableFunction<T, O> keyGetter,
         .reduceByKey((value1, value2) -> value1, parallelism)
         .values();
   }
+
+  /**
+   * The key used in a caching map to identify a {@link HoodieData}.
+   *
+   * At the end of a write operation, we manually unpersist the {@link 
HoodieData} associated with that writer.
+   * Therefore, in multi-writer scenario, we need to use both {@code basePath} 
and {@code instantTime} to identify {@link HoodieData}s.
+   */
+  class HoodieDataCacheKey implements Serializable {

Review Comment:
   We should avoid exposing this outside of the `HoodieData` class (no other 
components should be exposed to how we're caching it, so it would be easier for 
us to change if we need to)



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java:
##########
@@ -55,7 +56,7 @@ public void preCompact(
   }
 
   @Override
-  public void maybePersist(HoodieData<WriteStatus> writeStatus, 
HoodieWriteConfig config) {
+  public void maybePersist(HoodieData<WriteStatus> writeStatus, 
HoodieEngineContext context, HoodieWriteConfig config, String instantTime) {

Review Comment:
   Same comment as above



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java:
##########
@@ -180,4 +187,29 @@ public Option<String> getProperty(EngineProperty key) {
   public void setJobStatus(String activeModule, String activityDescription) {
     javaSparkContext.setJobGroup(activeModule, activityDescription);
   }
+
+  @Override
+  public void putCachedDataIds(HoodieDataCacheKey cacheKey, int... ids) {
+    synchronized (cacheLock) {

Review Comment:
   No need for separate lock, we can synchronize on the cache itself



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java:
##########
@@ -180,4 +187,29 @@ public Option<String> getProperty(EngineProperty key) {
   public void setJobStatus(String activeModule, String activityDescription) {
     javaSparkContext.setJobGroup(activeModule, activityDescription);
   }
+
+  @Override
+  public void putCachedDataIds(HoodieDataCacheKey cacheKey, int... ids) {
+    synchronized (cacheLock) {

Review Comment:
   Let's also annotates this class as `@ThreadSafe`



##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.data.HoodieData.HoodieDataCacheKey;
+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.data.HoodieJavaRDD;
+import org.apache.hudi.testutils.SparkClientFunctionalTestHarness;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.api.java.JavaRDD;
+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);
+    HoodieJavaRDD<GenericRecord> persistedRdd0 = 
HoodieJavaRDD.of(jsc().parallelize(extraRecords0, 2));
+    persistedRdd0.persist("MEMORY_AND_DISK", context(), 
HoodieDataCacheKey.of(writeConfig.getBasePath(), instant0));
+
+    String instant1 = getCommitTimeAtUTC(1);
+    List<GenericRecord> extraRecords1 = dataGen.generateGenericRecords(10);
+    HoodieJavaRDD<GenericRecord> persistedRdd1 = 
HoodieJavaRDD.of(jsc().parallelize(extraRecords1, 2));
+    persistedRdd1.persist("MEMORY_AND_DISK", context(), 
HoodieDataCacheKey.of(writeConfig.getBasePath(), instant1));
+
+    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.getId()),
+          
context().getCachedDataIds(HoodieDataCacheKey.of(writeConfig.getBasePath(), 
instant0)),
+          "RDDs cached for " + instant0 + " should be retained.");
+      assertEquals(Collections.emptyList(),
+          
context().getCachedDataIds(HoodieDataCacheKey.of(writeConfig.getBasePath(), 
instant1)),
+          "RDDs cached for " + instant1 + " should be cleared.");
+      assertTrue(jsc().getPersistentRDDs().containsKey(persistedRdd0.getId()),

Review Comment:
   Should we combine these 3 assertions into 1 that asserts all RDDs persisted 
ids?



-- 
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]

Reply via email to