danny0405 commented on code in PR #11043:
URL: https://github.com/apache/hudi/pull/11043#discussion_r1623048159


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBloomFiltersIndexSupport.scala:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.functional
+
+import org.apache.hudi.DataSourceWriteOptions._
+import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
+import org.apache.hudi.common.model.{FileSlice, HoodieTableType}
+import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
+import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.metadata.HoodieMetadataFileSystemView
+import org.apache.hudi.testutils.HoodieSparkClientTestBase
+import org.apache.hudi.util.{JFunction, JavaConversions}
+import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, 
HoodieFileIndex}
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, 
Expression, Literal}
+import org.apache.spark.sql.functions.{col, not}
+import org.apache.spark.sql.types.StringType
+import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession}
+import org.junit.jupiter.api.Assertions.assertTrue
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.EnumSource
+
+import java.util.concurrent.atomic.AtomicInteger
+import java.util.stream.Collectors
+import scala.collection.JavaConverters._
+import scala.collection.{JavaConverters, mutable}
+
+class TestBloomFiltersIndexSupport extends HoodieSparkClientTestBase {
+
+  val sqlTempTable = "hudi_tbl_bloom"
+  var spark: SparkSession = _
+  var instantTime: AtomicInteger = _
+  val metadataOpts: Map[String, String] = Map(
+    HoodieMetadataConfig.ENABLE.key -> "true",
+    HoodieMetadataConfig.ENABLE_METADATA_INDEX_BLOOM_FILTER.key -> "true",
+    HoodieMetadataConfig.BLOOM_FILTER_INDEX_FOR_COLUMNS.key -> "_row_key"
+  )
+  val commonOpts: Map[String, String] = Map(
+    "hoodie.insert.shuffle.parallelism" -> "4",
+    "hoodie.upsert.shuffle.parallelism" -> "4",
+    HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+    RECORDKEY_FIELD.key -> "_row_key",
+    PARTITIONPATH_FIELD.key -> "partition",
+    PRECOMBINE_FIELD.key -> "timestamp",
+    HoodieTableConfig.POPULATE_META_FIELDS.key -> "true"
+  ) ++ metadataOpts
+  var mergedDfList: List[DataFrame] = List.empty
+
+  @BeforeEach
+  override def setUp(): Unit = {
+    initPath()
+    initSparkContexts()
+    initHoodieStorage()
+    initTestDataGenerator()
+
+    setTableName("hoodie_test")
+    initMetaClient()
+
+    instantTime = new AtomicInteger(1)
+
+    spark = sqlContext.sparkSession
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    cleanupFileSystem()
+    cleanupSparkContexts()
+  }
+
+  @ParameterizedTest
+  @EnumSource(classOf[HoodieTableType])
+  def testIndexInitialization(tableType: HoodieTableType): Unit = {
+    val hudiOpts = commonOpts + (DataSourceWriteOptions.TABLE_TYPE.key -> 
tableType.name())
+    doWriteAndValidateBloomFilters(
+      hudiOpts,
+      operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+      saveMode = SaveMode.Overwrite)
+  }
+
+  /**
+   * Test case to do a write with updates and then validate file pruning using 
bloom filters.
+   */
+  @Test
+  def testBloomFiltersIndexFilePruning(): Unit = {
+    var hudiOpts = commonOpts
+    hudiOpts = hudiOpts + (
+      DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true")
+
+    doWriteAndValidateBloomFilters(
+      hudiOpts,
+      operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
+      saveMode = SaveMode.Overwrite,
+      shouldValidate = false)
+    doWriteAndValidateBloomFilters(
+      hudiOpts,
+      operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
+      saveMode = SaveMode.Append)
+
+    createTempTable(hudiOpts)
+    verifyQueryPredicate(hudiOpts, "_row_key")
+  }
+
+  private def createTempTable(hudiOpts: Map[String, String]): Unit = {
+    val readDf = spark.read.format("hudi").options(hudiOpts).load(basePath)
+    readDf.createOrReplaceTempView(sqlTempTable)
+  }
+
+  def verifyQueryPredicate(hudiOpts: Map[String, String], columnName: String): 
Unit = {
+    val reckey = mergedDfList.last.limit(1).collect().map(row => 
row.getAs(columnName).toString)
+    val dataFilter = EqualTo(attribute(columnName), Literal(reckey(0)))
+    verifyFilePruning(hudiOpts, dataFilter)
+  }
+
+  private def attribute(partition: String): AttributeReference = {
+    AttributeReference(partition, StringType, true)()
+  }
+
+
+  private def verifyFilePruning(opts: Map[String, String], dataFilter: 
Expression): Unit = {
+    // with data skipping
+    val commonOpts = opts + ("path" -> basePath)
+    metaClient = HoodieTableMetaClient.reload(metaClient)
+    var fileIndex = HoodieFileIndex(spark, metaClient, None, commonOpts, 
includeLogFiles = true)
+    val filteredPartitionDirectories = fileIndex.listFiles(Seq(), 
Seq(dataFilter))
+    val filteredFilesCount = filteredPartitionDirectories.flatMap(s => 
s.files).size
+    assertTrue(filteredFilesCount < getLatestDataFilesCount(opts))
+
+    // with no data skipping
+    fileIndex = HoodieFileIndex(spark, metaClient, None, commonOpts + 
(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "false"), includeLogFiles = 
true)
+    val filesCountWithNoSkipping = fileIndex.listFiles(Seq(), 
Seq(dataFilter)).flatMap(s => s.files).size
+    assertTrue(filesCountWithNoSkipping == getLatestDataFilesCount(opts))
+  }
+
+  private def getLatestDataFilesCount(opts: Map[String, String], 
includeLogFiles: Boolean = true) = {
+    var totalLatestDataFiles = 0L
+    
getTableFileSystemView(opts).getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().getTimestamp)
+      .values()
+      .forEach(JFunction.toJavaConsumer[java.util.stream.Stream[FileSlice]]
+        (slices => slices.forEach(JFunction.toJavaConsumer[FileSlice](
+          slice => totalLatestDataFiles += (if (includeLogFiles) 
slice.getLogFiles.count() else 0)
+            + (if (slice.getBaseFile.isPresent) 1 else 0)))))
+    totalLatestDataFiles
+  }
+
+  private def getTableFileSystemView(opts: Map[String, String]): 
HoodieMetadataFileSystemView = {
+    new HoodieMetadataFileSystemView(metaClient, metaClient.getActiveTimeline, 
metadataWriter(getWriteConfig(opts)).getTableMetadata)
+  }
+
+  private def doWriteAndValidateBloomFilters(hudiOpts: Map[String, String],
+                                             operation: String,
+                                             saveMode: SaveMode,
+                                             shouldValidate: Boolean = true): 
Unit = {
+    var latestBatch: mutable.Buffer[String] = null
+    if (operation == UPSERT_OPERATION_OPT_VAL) {
+      val instantTime = getInstantTime()
+      val records = 
recordsToStrings(dataGen.generateUniqueUpdates(instantTime, 1))
+      records.addAll(recordsToStrings(dataGen.generateInserts(instantTime, 1)))
+      latestBatch = records.asScala
+    } else if (operation == INSERT_OVERWRITE_OPERATION_OPT_VAL) {
+      latestBatch = recordsToStrings(dataGen.generateInsertsForPartition(
+        getInstantTime(), 5, dataGen.getPartitionPaths.last)).asScala
+    } else {
+      latestBatch = recordsToStrings(dataGen.generateInserts(getInstantTime(), 
5)).asScala
+    }
+    val latestBatchDf = 
spark.read.json(spark.sparkContext.parallelize(latestBatch.toSeq, 2))
+    latestBatchDf.cache()
+    latestBatchDf.write.format("org.apache.hudi")
+      .options(hudiOpts)
+      .option(DataSourceWriteOptions.OPERATION.key, operation)
+      .mode(saveMode)
+      .save(basePath)
+
+    metaClient = HoodieTableMetaClient.reload(metaClient)
+    calculateMergedDf(latestBatchDf, operation)
+    if (shouldValidate) {
+      validateBloomFiltersIndex(hudiOpts)
+    }
+  }
+
+  private def getInstantTime(): String = {
+    String.format("%03d", new Integer(instantTime.incrementAndGet()))
+  }
+
+  private def validateBloomFiltersIndex(hudiOpts: Map[String, String]): Unit = 
{
+    metaClient = HoodieTableMetaClient.reload(metaClient)
+    val writeConfig = getWriteConfig(hudiOpts)
+    val metadata = metadataWriter(writeConfig).getTableMetadata
+    val readDf = spark.read.format("hudi").load(basePath)
+    val rowArr = readDf.collect()
+
+    assertTrue(rowArr.length > 0)
+
+    for (row <- rowArr) {
+      val recordKey: String = row.getAs("_hoodie_record_key")
+      val partitionPath: String = row.getAs("_hoodie_partition_path")
+      val fileName: String = row.getAs("_hoodie_file_name")
+      val bloomFilter = metadata.getBloomFilter(partitionPath, fileName)
+      assertTrue(bloomFilter.isPresent, "BloomFilter should be present for " + 
fileName)
+      assertTrue(bloomFilter.get().mightContain(recordKey))
+    }
+  }
+
+  private def getWriteConfig(hudiOpts: Map[String, String]): HoodieWriteConfig 
= {
+    val props = 
TypedProperties.fromMap(JavaConverters.mapAsJavaMapConverter(hudiOpts).asJava)
+    HoodieWriteConfig.newBuilder()
+      .withProps(props)
+      .withPath(basePath)
+      .build()
+  }
+
+  private def calculateMergedDf(latestBatchDf: DataFrame, operation: String): 
DataFrame = {
+    val prevDfOpt = mergedDfList.lastOption
+    if (prevDfOpt.isEmpty) {
+      mergedDfList = mergedDfList :+ latestBatchDf
+      sparkSession.emptyDataFrame
+    } else {
+      if (operation == INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL) {
+        mergedDfList = mergedDfList :+ latestBatchDf
+        // after insert_overwrite_table, all previous snapshot's records 
should be deleted from RLI

Review Comment:
   Not sure why the bloom_filter index is related with RLI?



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