lokeshj1703 commented on code in PR #9373:
URL: https://github.com/apache/hudi/pull/9373#discussion_r1285243723
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala:
##########
@@ -123,28 +123,47 @@ class RecordLevelIndexSupport(spark: SparkSession,
* @param queryFilters The queries that need to be filtered.
* @return Tuple of List of filtered queries and list of record key literals
that need to be matched
*/
- private def filterQueryFiltersWithRecordKey(queryFilters: Seq[Expression]):
(List[Expression], List[String]) = {
+ private def filterQueriesWithRecordKey(queryFilters: Seq[Expression]):
(List[Expression], List[String]) = {
var recordKeyQueries: List[Expression] = List.empty
var recordKeys: List[String] = List.empty
for (query <- queryFilters) {
- query match {
- case equalToQuery: EqualTo =>
- val (attribute, literal) =
getAttributeLiteralTuple(equalToQuery.left, equalToQuery.right).orNull
- if (attribute != null && attribute.name != null &&
attributeMatchesRecordKey(attribute.name)) {
- recordKeys = recordKeys :+ literal.value.toString
- recordKeyQueries = recordKeyQueries :+ equalToQuery
- }
- case _ =>
- }
+ filterQueryWithRecordKey(query).foreach({
+ case (exp: Expression, recKeys: List[String]) =>
+ recordKeys = recordKeys ++ recKeys
+ recordKeyQueries = recordKeyQueries :+ exp
+ })
}
+
Tuple2.apply(recordKeyQueries, recordKeys)
}
- /**
+ private def filterQueryWithRecordKey(queryFilter: Expression):
Option[(Expression, List[String])] = {
Review Comment:
Addressed
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala:
##########
@@ -45,12 +52,71 @@ class TestRecordLevelIndexWithSQL extends
RecordLevelIndexTestBase {
validate = false)
createTempTable(hudiOpts)
- val reckey =
mergedDfList.last.limit(1).collect()(0).getAs("_row_key").toString
- spark.sql("select * from " + sqlTempTable + " where '" + reckey + "' =
_row_key").show(false)
+ testInQuery(hudiOpts)
+ testEqualToQuery(hudiOpts)
+ }
+
+ def testEqualToQuery(hudiOpts: Map[String, String]): Unit = {
+ val reckey = mergedDfList.last.limit(1).collect().map(row =>
row.getAs("_row_key").toString)
+ val dataFilter = EqualTo(attribute("_row_key"), Literal(reckey(0)))
+ assertEquals(1, spark.sql("select * from " + sqlTempTable + " where " +
dataFilter.sql).count())
+ verifyPruningFileCount(hudiOpts, dataFilter, 1)
+ }
+
+ def testInQuery(hudiOpts: Map[String, String]): Unit = {
+ var reckey = mergedDfList.last.limit(1).collect().map(row =>
row.getAs("_row_key").toString)
+ var dataFilter = In(attribute("_row_key"), reckey.map(l =>
literal(l)).toList)
+ assertEquals(1, spark.sql("select * from " + sqlTempTable + " where " +
dataFilter.sql).count())
+ var numFiles = if (isTableMOR()) 2 else 1
+ verifyPruningFileCount(hudiOpts, dataFilter, numFiles)
+
+ reckey = mergedDfList.last.limit(2).collect().map(row =>
row.getAs("_row_key").toString)
+ dataFilter = In(attribute("_row_key"), reckey.map(l => literal(l)).toList)
+ assertEquals(2, spark.sql("select * from " + sqlTempTable + " where " +
dataFilter.sql).count())
+ numFiles = if (isTableMOR()) 2 else 2
+ verifyPruningFileCount(hudiOpts, dataFilter, numFiles)
+ }
+
+ private def attribute(partition: String): AttributeReference = {
+ AttributeReference(partition, StringType, true)()
+ }
+
+ private def literal(value: String): Literal = {
+ Literal.create(value)
+ }
+
+ private def verifyPruningFileCount(opts: Map[String, String], dataFilter:
Expression, numFiles: Int): Unit = {
+ metaClient = HoodieTableMetaClient.reload(metaClient)
+ val fileIndex = HoodieFileIndex(spark, metaClient, None, opts + ("path" ->
basePath))
+ fileIndex.setIncludeLogFiles(isTableMOR())
+ val filteredPartitionDirectories = fileIndex.listFiles(Seq(),
Seq(dataFilter))
+ val filteredFilesCount = filteredPartitionDirectories.flatMap(s =>
s.files).size
+ assertTrue(filteredFilesCount < getLatestDataFilesCount(opts))
+ assertEquals(filteredFilesCount, numFiles)
+ }
+
+ private def isTableMOR(): Boolean = {
+ metaClient.getTableType == HoodieTableType.MERGE_ON_READ
+ }
+
+ private def getLatestDataFilesCount(opts: Map[String, String],
includeLogFiles: Boolean = true) = {
+ var totalLatestDataFiles = 0L
+
getTableFileSystenView(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 getTableFileSystenView(opts: Map[String, String]):
HoodieMetadataFileSystemView = {
+ new HoodieMetadataFileSystemView(metaClient, metaClient.getActiveTimeline,
metadataWriter(getWriteConfig(opts)).getTableMetadata)
}
private def createTempTable(hudiOpts: Map[String, String]): Unit = {
val readDf = spark.read.format("hudi").options(hudiOpts).load(basePath)
+ readDf.printSchema()
Review Comment:
Addressed
--
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]