yihua commented on code in PR #13295: URL: https://github.com/apache/hudi/pull/13295#discussion_r2112988014
########## 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<>(); Review Comment: If the assumption is that `SparkRDDMetadataWriteClient` is for one data table instant / action, it would be better to maintain a single instant here for validation, and throw an exception if `#upsertPreppedRecords` and `#commit` is called on a different instant. ########## 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) { Review Comment: If this is called twice, do we handle dirty file clean-up properly with marker resolution on the write status if the DAG is retried (i.e., `JavaRDD<WriteStatus>` of both invocation should be unioned)? ########## 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); + } + Review Comment: Other unused like `insert`, `delete`, etc., should be overridden to throw an exception? ########## 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: nit: one option is to let this metadata write client track multiple `JavaRDD<WriteStats>` from `#upsertPreppedRecords` and does the merging before passing to `#commit`. However, the reason we don't do this is because of how streaming write DAG is constructed. ########## hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestSparkMetadataTableUpsertCommitActionExecutor.java: ########## @@ -0,0 +1,112 @@ +/* + * 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.table.action.commit; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.model.HoodieFileGroupId; +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.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; + +import org.apache.spark.Partitioner; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; + +/** + * Tests SparkMetadataTableUpsertCommitActionExecutor. + */ +public class TestSparkMetadataTableUpsertCommitActionExecutor extends SparkClientFunctionalTestHarness { + + @Test + public void testMetadataTableUpsertCommitActionExecutor() throws IOException { + + HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ); + HoodieWriteConfig writeConfig = getConfigBuilder(false).build(); + HoodieTable table = HoodieSparkTable.create(writeConfig, context(), metaClient); + + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); + List<HoodieRecord> records = dataGen.generateInserts("0001", 100); + HoodieData<HoodieRecord> recordHoodieData = context().parallelize(records, 1); + + List<HoodieFileGroupId> hoodieFileGroupIdList = new ArrayList<>(); + hoodieFileGroupIdList.add(new HoodieFileGroupId(MetadataPartitionType.RECORD_INDEX.getPartitionPath(), "record-index-00001")); + + HoodieData<WriteStatus> statusHoodieData = mock(HoodieData.class); + + // create requested in timeline + metaClient.getActiveTimeline().createNewInstant(metaClient.createNewInstant(HoodieInstant.State.REQUESTED, DELTA_COMMIT_ACTION, + "0001")); + + SparkMetadataTableUpsertCommitActionExecutor commitActionExecutor = new MockSparkMetadataTableUpsertCommitActionExecutor(context(), + writeConfig, table, "0001", recordHoodieData, hoodieFileGroupIdList, statusHoodieData, true); + commitActionExecutor.execute(recordHoodieData); + // since this is initial call, inflight instant should be added. + assertTrue(metaClient.reloadActiveTimeline().getWriteTimeline().filterInflights().containsInstant("0001")); + + hoodieFileGroupIdList.clear(); + hoodieFileGroupIdList.add(new HoodieFileGroupId(MetadataPartitionType.FILES.getPartitionPath(), "files-00001")); + + commitActionExecutor = new MockSparkMetadataTableUpsertCommitActionExecutor(context(), + writeConfig, table, "0001", recordHoodieData, hoodieFileGroupIdList, statusHoodieData, false); + commitActionExecutor.execute(recordHoodieData); + // ensure inflight is still intact and is not complete yet unless we commit + HoodieActiveTimeline reloadedActiveTimeline = metaClient.reloadActiveTimeline(); + assertTrue(reloadedActiveTimeline.getWriteTimeline().filterInflights().containsInstant("0001")); + assertFalse(reloadedActiveTimeline.getWriteTimeline().filterCompletedInstants().containsInstant("0001")); + } + + static class MockSparkMetadataTableUpsertCommitActionExecutor<T> extends SparkMetadataTableUpsertCommitActionExecutor<T> { + private final HoodieData<WriteStatus> writeStatusHoodieData; + public MockSparkMetadataTableUpsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, + String instantTime, HoodieData<HoodieRecord<T>> preppedRecords, + List<HoodieFileGroupId> hoodieFileGroupIdList, + HoodieData<WriteStatus> writeStatusHoodieData, + boolean initialCall) { + super(context, config, table, instantTime, preppedRecords, hoodieFileGroupIdList, initialCall); + this.writeStatusHoodieData = writeStatusHoodieData; + } + + @Override + protected HoodieData<WriteStatus> mapPartitionsAsRDD(HoodieData<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) { + return writeStatusHoodieData; + } + + } + +} + + Review Comment: nit: remove redundant empty lines ########## 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); + } Review Comment: This can be tighter by checking exactly twice invocation. ########## 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, + String commitActionType, Map<String, List<String>> partitionToReplacedFileIds, + Option<BiConsumer<HoodieTableMetaClient, HoodieCommitMetadata>> extraPreCommitFunc) { + preWriteCompletedInstants.remove(instantTime); Review Comment: Similar here on validation. -- 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]
