yihua commented on code in PR #13295:
URL: https://github.com/apache/hudi/pull/13295#discussion_r2113002622


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDMetadataWriteClient.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.client.embedded.EmbeddedTimelineService;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.table.HoodieSparkMergeOnReadMetadataTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+
+import org.apache.spark.api.java.JavaRDD;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+
+/**
+ * Write client to assist with writing to metadata table.
+ * @param <T>
+ */
+public class SparkRDDMetadataWriteClient<T> extends SparkRDDWriteClient<T> {
+
+  // tracks the instants for which preWrite has been invoked.
+  private final Set<String> preWriteCompletedInstants = new HashSet<>();
+
+  public SparkRDDMetadataWriteClient(HoodieEngineContext context, 
HoodieWriteConfig clientConfig) {
+    super(context, clientConfig);
+  }
+
+  public SparkRDDMetadataWriteClient(HoodieEngineContext context, 
HoodieWriteConfig writeConfig,
+                                     Option<EmbeddedTimelineService> 
timelineService) {
+    super(context, writeConfig, timelineService);
+  }
+
+  /**
+   * Upserts the given prepared records into the Hoodie table, at the supplied 
instantTime.
+   * <p>
+   * This implementation requires that the input records are already tagged, 
and de-duped if needed.
+   *
+   * @param preppedRecords Prepared HoodieRecords to upsert
+   * @param instantTime Instant time of the commit
+   * @return Collection of WriteStatus to inspect errors and counts
+   */
+  public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> 
preppedRecords, String instantTime, Option<List<HoodieFileGroupId>> 
partitionFileIdPairsOpt) {
+    HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<HoodieKey>, 
HoodieData<WriteStatus>> table =
+        initTable(WriteOperationType.UPSERT_PREPPED, 
Option.ofNullable(instantTime));
+    table.validateUpsertSchema();
+    boolean initialCall = !preWriteCompletedInstants.contains(instantTime);
+    if (initialCall) {
+      // we do not want to call prewrite more than once for the same instant, 
since we could be writing to metadata table more than once w/ streaming writes.
+      preWrite(instantTime, WriteOperationType.UPSERT_PREPPED, 
table.getMetaClient());
+      preWriteCompletedInstants.add(instantTime);
+    }
+    HoodieWriteMetadata<HoodieData<WriteStatus>> result = 
((HoodieSparkMergeOnReadMetadataTable) table).upsertPrepped(context, 
instantTime, HoodieJavaRDD.of(preppedRecords),
+        partitionFileIdPairsOpt, initialCall);
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> resultRDD = 
result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses()));
+    return postWrite(resultRDD, instantTime, table);
+  }
+
+  /**
+   * Complete changes performed at the given instantTime marker with specified 
action.
+   */
+  @Override
+  public boolean commit(String instantTime, JavaRDD<WriteStatus> 
writeStatuses, Option<Map<String, String>> extraMetadata,

Review Comment:
   Now I understand this: `JavaRDD<WriteStatus> writeStatuses` needs a better 
naming for readability.  Or override the `commitStats`.



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