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 00:57:38 UTC

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

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


##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java:
##########
@@ -315,7 +312,7 @@ public Builder withPartition(String partitionName) {
     public HoodieMergedLogRecordScanner build() {
       return new HoodieMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema,
           latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader,
-          bufferSize, spillableMapBasePath, instantRange, autoScan,
+          bufferSize, spillableMapBasePath, instantRange,
           diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true,

Review Comment:
   looks like we are hardcoding force full scan to true here. guess autoScan above should be replaced w/ fullScan. 
   



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java:
##########
@@ -133,14 +141,61 @@ private void initIfNeeded() {
   }
 
   @Override
-  protected List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordsByKeys(List<String> keys,
-                                                                                             String partitionName) {
+  public HoodieData<HoodieRecord<HoodieMetadataPayload>> getRecordsByKeyPrefixes(List<String> keyPrefixes,
+                                                                                 String partitionName) {
+    // NOTE: Since we partition records to a particular file-group by full key, we will have
+    //       to scan all file-groups for all key-prefixes as each of these might contain some
+    //       records matching the key-prefix
+    List<FileSlice> partitionFileSlices =
+        HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, partitionName);
+
+    return engineContext.parallelize(partitionFileSlices)
+        .flatMap(
+            (SerializableFunction<FileSlice, Iterator<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>>>) fileSlice -> {
+              // we are moving the readers to executors in this code path. So, reusing readers may not make sense.
+              Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers =
+                  openReadersIfNeeded(partitionName, fileSlice, false);

Review Comment:
   probably we need to fix openReadersIfNeeded for the fullScan/forceFullScan config. with more partitions, may be we want to enable full scan for FILES, but not for other partitions. So, we can't rely on a HoodieMetadataConfig to derive the value for forceFullScan. Each caller might have to set the right value when instantiating the log record reader. 
   
   



##########
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:
   can we add a test to fetch multiple key prefixes? so far, all tests are trying to fetch just 1 key prefix. 



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