KnightChess commented on code in PR #11043: URL: https://github.com/apache/hudi/pull/11043#discussion_r1608419895
########## hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BloomFiltersIndexSupport.scala: ########## @@ -0,0 +1,87 @@ +/* + * 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 + +import org.apache.hudi.RecordLevelIndexSupport.filterQueryWithRecordKey +import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.metadata.HoodieTableMetadataUtil +import org.apache.hudi.storage.StoragePath +import org.apache.hudi.util.JFunction +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.Expression + +class BloomFiltersIndexSupport(spark: SparkSession, + metadataConfig: HoodieMetadataConfig, + metaClient: HoodieTableMetaClient) extends RecordLevelIndexSupport(spark, metadataConfig, metaClient) { + + override def getCandidateFiles(allFiles: Seq[StoragePath], recordKeys: List[String]): Set[String] = { + val fileToBloomFilterMap = allFiles.map { file => + val relativePartitionPath = FSUtils.getRelativePartitionPath(metaClient.getBasePathV2, file) + val fileName = FSUtils.getFileName(file.getName, relativePartitionPath) + val bloomFilter = metadataTable.getBloomFilter(relativePartitionPath, fileName) + file -> bloomFilter + }.toMap + + recordKeys.flatMap { recordKey => + fileToBloomFilterMap.filter { case (_, bloomFilter) => Review Comment: Read all the bloom filter at once, which may cause too much memory temporary use. I wonder if this is evaluated? May cause driver full gc. How about early the logic of judgment to before cycle in `allFiles.map`, merge the iterative compute. -- 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]
