lokeshj1703 commented on code in PR #9308:
URL: https://github.com/apache/hudi/pull/9308#discussion_r1278900198


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala:
##########
@@ -0,0 +1,108 @@
+package org.apache.hudi
+
+import org.apache.hadoop.fs.FileStatus
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.metadata.{HoodieTableMetadata, HoodieTableMetadataUtil}
+import org.apache.hudi.util.JFunction
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, 
Expression, Literal}
+
+import scala.collection.{JavaConverters, mutable}
+
+class RecordLevelIndexSupport(spark: SparkSession,
+                              metadataConfig: HoodieMetadataConfig,
+                              metaClient: HoodieTableMetaClient) {
+
+  @transient private lazy val engineCtx = new HoodieSparkEngineContext(new 
JavaSparkContext(spark.sparkContext))
+  @transient private lazy val metadataTable: HoodieTableMetadata =
+    HoodieTableMetadata.create(engineCtx, metadataConfig, 
metaClient.getBasePathV2.toString)
+
+  def getCandidateFiles(allFiles: Seq[FileStatus], queryFilters: 
Seq[Expression]): Set[String] = {
+    val (_, recordKeys) = filterQueryFiltersWithRecordKey(queryFilters)
+    val recordKeyLocationsMap = 
metadataTable.readRecordIndex(JavaConverters.seqAsJavaListConverter(recordKeys).asJava)
+    val fileIdToPartitionMap: mutable.Map[String, String] = mutable.Map.empty
+    val candidateFiles: mutable.Set[String] = mutable.Set.empty
+    for (location <- 
JavaConverters.collectionAsScalaIterableConverter(recordKeyLocationsMap.values()).asScala)
 {
+      fileIdToPartitionMap.put(location.getFileId, location.getPartitionPath)
+    }
+    for (file <- allFiles) {
+      val fileId = FSUtils.getFileIdFromFilePath(file.getPath)
+      val partitionOpt = fileIdToPartitionMap.get(fileId)
+      if (partitionOpt.isDefined) {
+        candidateFiles += file.getPath.getName
+      }
+    }
+    candidateFiles.toSet
+  }
+
+  def getRecordKey: Option[String] = {
+    val recordKeysOpt: org.apache.hudi.common.util.Option[Array[String]] = 
metaClient.getTableConfig.getRecordKeyFields
+    val recordKeyOpt = 
recordKeysOpt.map[String](JFunction.toJavaFunction[Array[String], String](arr =>
+      if (arr.length == 1) {
+        arr(0)
+      } else {

Review Comment:
   I am keeping the scope of this PR limited to simple record key. Complex key 
support can be added in a separate PR.



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