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/07/15 20:42:47 UTC

[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6113: [HUDI-4400] Fix missing bloom filters in metadata table in non-partitioned table

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


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java:
##########
@@ -110,24 +116,35 @@ public void tearDown() throws Exception {
     cleanupResources();
   }
 
-  private HoodieWriteConfig makeConfig(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) {
+  private HoodieWriteConfig makeConfig(
+      boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking, boolean useMetadataTable) {
+    // For the bloom index to use column stats and bloom filters from metadata table,
+    // the following configs must be set to true:
+    // "hoodie.bloom.index.use.metadata"
+    // "hoodie.metadata.enable" (by default is true)
+    // "hoodie.metadata.index.column.stats.enable"
+    // "hoodie.metadata.index.bloom.filter.enable"
     return HoodieWriteConfig.newBuilder().withPath(basePath)
         .withIndexConfig(HoodieIndexConfig.newBuilder().bloomIndexPruneByRanges(rangePruning)
             .bloomIndexTreebasedFilter(treeFiltering).bloomIndexBucketizedChecking(bucketizedChecking)
-            .bloomIndexKeysPerBucket(2).build())
+            .bloomIndexKeysPerBucket(2).bloomIndexUseMetadata(useMetadataTable).build())

Review Comment:
   Can we please line up them on individual lines so that it's easier to understand which config attributes where



##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java:
##########
@@ -428,7 +455,99 @@ public void testTagLocation(boolean rangePruning, boolean treeFiltering, boolean
 
   @ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
   @MethodSource("configParams")
-  public void testCheckExists(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
+  public void testTagLocationOnNonpartitionedTable(
+      boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking,
+      boolean useMetadataTable) throws Exception {
+    // We have some records to be tagged (two different partitions)
+    String rowKey1 = UUID.randomUUID().toString();

Review Comment:
   Let's generate UUID from a random w/ fixed seed (so that they don't change from run to run), there's `genPseudoRandomUUID` specifically for that



##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java:
##########
@@ -428,7 +455,99 @@ public void testTagLocation(boolean rangePruning, boolean treeFiltering, boolean
 
   @ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
   @MethodSource("configParams")
-  public void testCheckExists(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
+  public void testTagLocationOnNonpartitionedTable(
+      boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking,
+      boolean useMetadataTable) throws Exception {
+    // We have some records to be tagged (two different partitions)
+    String rowKey1 = UUID.randomUUID().toString();
+    String rowKey2 = UUID.randomUUID().toString();
+    String rowKey3 = UUID.randomUUID().toString();
+    String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
+    String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
+    String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
+
+    String emptyPartitionPath = "";
+    RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
+    HoodieRecord record1 =
+        new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), emptyPartitionPath), rowChange1);
+    RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
+    HoodieRecord record2 =
+        new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), emptyPartitionPath), rowChange2);
+    RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
+    HoodieRecord record3 =
+        new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), emptyPartitionPath), rowChange3);
+
+    JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3));
+
+    // Also create the metadata and config
+    HoodieWriteConfig config =
+        makeConfig(rangePruning, treeFiltering, bucketizedChecking, useMetadataTable);
+    HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
+    metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context);
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
+
+    // Let's tag
+    HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
+    JavaRDD<HoodieRecord> taggedRecordRDD = tagLocation(bloomIndex, recordRDD, hoodieTable);
+
+    // Should not find any files
+    for (HoodieRecord record : taggedRecordRDD.collect()) {
+      assertFalse(record.isCurrentLocationKnown());
+    }
+
+    final Map<String, List<Pair<String, Integer>>> partitionToFilesNameLengthMap = new HashMap<>();
+
+    // We create three parquet file, each having one record
+    final String fileId1 = UUID.randomUUID().toString();

Review Comment:
   Same comment as above



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