You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/04/05 01:00:58 UTC

[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5208: [HUDI-3760] Adding capability to fetch Metadata Records by prefix

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


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java:
##########
@@ -1266,6 +1270,139 @@ public void testTableOperationsWithRestore(HoodieTableType tableType) throws Exc
     testTableOperationsImpl(engineContext, writeConfig);
   }
 
+  @Test
+  public void testColStatsPrefixLookup() throws IOException {
+    this.tableType = COPY_ON_WRITE;
+    initPath();
+    initSparkContexts("TestHoodieMetadata");
+    initFileSystem();
+    fs.mkdirs(new Path(basePath));
+    initTimelineService();
+    initMetaClient(tableType);
+    initTestDataGenerator();
+    metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
+
+    HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
+    // disable small file handling so that every insert goes to a new file group.
+    HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false)
+        .withRollbackUsingMarkers(false)
+        .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(0)
+            .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1)
+            .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER)
+            .withAutoClean(false).retainCommits(1).retainFileVersions(1).build())
+        .withMetadataConfig(HoodieMetadataConfig.newBuilder()
+            .enable(true)
+            .withMetadataIndexColumnStats(true)
+            .enableFullScan(false)
+            .build())
+        .build();
+
+    try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) {
+
+      String firstCommit = "0000001";
+      List<HoodieRecord> records = dataGen.generateInserts(firstCommit, 20);
+
+      AtomicInteger counter = new AtomicInteger();
+      List<HoodieRecord> processedRecords = records.stream().map(entry ->
+              new HoodieAvroRecord(new HoodieKey("key1_" + counter.getAndIncrement(), entry.getPartitionPath()), (HoodieRecordPayload) entry.getData()))
+          .collect(Collectors.toList());
+
+      client.startCommitWithTime(firstCommit);
+      List<WriteStatus> writeStatuses = client.insert(jsc.parallelize(processedRecords, 1), firstCommit).collect();
+      assertNoWriteErrors(writeStatuses);
+
+      // Write 2 (inserts)
+      String secondCommit = "0000002";
+      client.startCommitWithTime(secondCommit);
+      records = dataGen.generateInserts(secondCommit, 20);
+      AtomicInteger counter1 = new AtomicInteger();
+      processedRecords = records.stream().map(entry ->
+              new HoodieAvroRecord(new HoodieKey("key2_" + counter1.getAndIncrement(), entry.getPartitionPath()), (HoodieRecordPayload) entry.getData()))
+          .collect(Collectors.toList());
+      writeStatuses = client.insert(jsc.parallelize(processedRecords, 1), secondCommit).collect();
+      assertNoWriteErrors(writeStatuses);
+
+      Map<String, Map<String, List<String>>> commitToPartitionsToFiles = new HashMap<>();
+      // populate commit -> partition -> file info to assist in validation and prefi
+      metaClient.getActiveTimeline().getInstants().forEach(entry -> {
+        try {
+          HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
+              .fromBytes(metaClient.getActiveTimeline().getInstantDetails(entry).get(), HoodieCommitMetadata.class);
+          String commitTime = entry.getTimestamp();
+          if (!commitToPartitionsToFiles.containsKey(commitTime)) {
+            commitToPartitionsToFiles.put(commitTime, new HashMap<>());
+          }
+          commitMetadata.getPartitionToWriteStats().entrySet()
+              .stream()
+              .forEach(partitionWriteStat -> {
+                String partitionStatName = partitionWriteStat.getKey();
+                List<HoodieWriteStat> writeStats = partitionWriteStat.getValue();
+                String partition = HoodieTableMetadataUtil.getPartition(partitionStatName);
+                if (!commitToPartitionsToFiles.get(commitTime).containsKey(partition)) {
+                  commitToPartitionsToFiles.get(commitTime).put(partition, new ArrayList<>());
+                }
+                writeStats.forEach(writeStat -> commitToPartitionsToFiles.get(commitTime).get(partition).add(writeStat.getPath()));
+              });
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      });
+
+      HoodieTableMetadata tableMetadata = metadata(client);
+      // prefix search for column (_hoodie_record_key)
+      ColumnIndexID columnIndexID = new ColumnIndexID(HoodieRecord.RECORD_KEY_METADATA_FIELD);

Review Comment:
   `TestColumnStatsIndex` fetches multiple



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org