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


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDMetadataWriteClient.java:
##########
@@ -0,0 +1,414 @@
+/*
+ * 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.avro.HoodieAvroUtils;
+import org.apache.hudi.avro.model.HoodieMetadataRecord;
+import org.apache.hudi.common.config.HoodieCommonConfig;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
+import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieWriteStat;
+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.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ClosableIterator;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.io.storage.HoodieSeekingFileReader;
+import org.apache.hudi.metadata.HoodieMetadataLogRecordReader;
+import org.apache.hudi.metadata.HoodieMetadataPayload;
+import org.apache.hudi.metadata.HoodieMetadataWriteUtils;
+import org.apache.hudi.metadata.HoodieTableMetadataUtil;
+import org.apache.hudi.metadata.MetadataPartitionType;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.testutils.HoodieClientTestBase;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
+import static 
org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR;
+import static org.apache.hudi.common.util.CollectionUtils.toStream;
+import static 
org.apache.hudi.common.util.ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER;
+import static 
org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_LIST;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestSparkRDDMetadataWriteClient extends HoodieClientTestBase {
+
+  private Random random;
+
+  @BeforeEach
+  public void setUp() throws Exception {
+    super.setUp();
+    random = new Random(0xDEED);
+  }
+
+  @Test
+  public void testWritesViaMetadataWriteClient() throws Exception {
+
+    HoodieWriteConfig hoodieWriteConfig = getConfigBuilder()
+        
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).withMetadataIndexColumnStats(false).withEnableRecordIndex(true)
+            .withRecordIndexFileGroupCount(1, 1).build()).build();
+
+    // trigger end to end write to data table so that metadata table is also 
initialized.
+    initDataTableWithACommit(hoodieWriteConfig);
+
+    // fetch metadata file slice info
+    HoodieWriteConfig mdtWriteConfig = 
HoodieMetadataWriteUtils.createMetadataWriteConfig(hoodieWriteConfig, 
HoodieFailedWritesCleaningPolicy.EAGER);
+    Map<String, List<String>> mdtPartitionsFileIdMapping = new HashMap<>();
+    List<HoodieFileGroupId> nonFilesPartitionFileGroupIdList = new 
ArrayList<>();
+    List<HoodieFileGroupId> filesPartitionFileGroupIdList = new ArrayList<>();
+    HoodieTableMetaClient metadataMetaClient = 
HoodieTableMetaClient.builder().setBasePath(mdtWriteConfig.getBasePath()).setConf(context.getStorageConf()).build();
+    fetchMetadataFileSliceInfo(metadataMetaClient, 
filesPartitionFileGroupIdList, nonFilesPartitionFileGroupIdList, 
mdtPartitionsFileIdMapping);
+
+    List<HoodieRecord> filesPartitionExpectedRecords = new ArrayList<>();
+    Map<String, HoodieRecord> filesPartitionExpectedRecordsMap = new 
HashMap<>();
+    List<String> expectedAllPartitions = new ArrayList<>();
+    List<HoodieRecord> rliRecords = new ArrayList<>();
+    Map<String, HoodieRecord> rliPartitionExpectedRecordsMap = new HashMap<>();
+    String commitTimeOfInterest = null;
+
+    // create Write client to SparkRDDMetadataWriteClient and trigger writes.
+    try (SparkRDDMetadataWriteClient client = new 
SparkRDDMetadataWriteClient(context, mdtWriteConfig)) {
+      commitTimeOfInterest = client.createNewInstantTime();
+
+      // prepare FILES partition records.
+      
prepareFilesPartitionRecords(mdtPartitionsFileIdMapping.get(MetadataPartitionType.FILES.getPartitionPath()).get(0),
+          commitTimeOfInterest, filesPartitionExpectedRecords, 
filesPartitionExpectedRecordsMap, expectedAllPartitions);
+
+      // prepare RLI records.
+      prepareRliRecords(commitTimeOfInterest, mdtPartitionsFileIdMapping, 
expectedAllPartitions, rliRecords, rliPartitionExpectedRecordsMap);
+
+      // ingest RLI records to metadata table.
+      client.startCommitForMetadataTable(metadataMetaClient, 
commitTimeOfInterest, DELTA_COMMIT_ACTION);
+      JavaRDD<WriteStatus> partialWriteStatusesRDD = 
client.upsertPreppedRecords(jsc.parallelize(rliRecords), commitTimeOfInterest, 
Option.of(nonFilesPartitionFileGroupIdList));
+      List<WriteStatus> partialWriteStatuses = 
partialWriteStatusesRDD.collect();
+
+      // validate that the commit is still pending since we are streaming 
write to metadata table.
+      HoodieActiveTimeline reloadedMdtActiveTimeline = 
metadataMetaClient.reloadActiveTimeline();
+      
assertEquals(reloadedMdtActiveTimeline.filterCompletedInstants().getInstants().stream().count(),
 3); // files, rli instantiaton and 1 write to data table.
+      String finalCommitTimeOfInterest = commitTimeOfInterest;
+      
assertTrue(reloadedMdtActiveTimeline.filterInflightsAndRequested().getInstants().stream().anyMatch(instant
 -> instant.requestedTime().equals(finalCommitTimeOfInterest)));

Review Comment:
   yes, it will throw. by default we treat all writes to timeline as immutable. 
file creation will only succeed if its not present already. 
   



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