1
0

[HUDI-3776] Fix BloomIndex incorrectly using ColStats to lookup records locations (#5213)

This commit is contained in:
Sagar Sumit
2022-04-03 03:52:57 +05:30
committed by GitHub
parent 20964df770
commit 74eb09be9b
3 changed files with 98 additions and 16 deletions

View File

@@ -18,9 +18,6 @@
package org.apache.hudi.metadata;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieMetadataRecord;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
@@ -52,6 +49,10 @@ import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -67,6 +68,8 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import static org.apache.hudi.common.util.CollectionUtils.isNullOrEmpty;
/**
* Table metadata provided by an internal DFS backed Hudi metadata table.
*/
@@ -164,7 +167,6 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
}
});
ValidationUtils.checkState(keys.size() == fileSlicesKeysCount.get());
return result;
}
@@ -263,10 +265,12 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
Map<Pair<String, FileSlice>, List<String>> partitionFileSliceToKeysMap = new HashMap<>();
for (String key : keys) {
final FileSlice slice = latestFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key,
latestFileSlices.size()));
final Pair<String, FileSlice> partitionNameFileSlicePair = Pair.of(partitionName, slice);
partitionFileSliceToKeysMap.computeIfAbsent(partitionNameFileSlicePair, k -> new ArrayList<>()).add(key);
if (!isNullOrEmpty(latestFileSlices)) {
final FileSlice slice = latestFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(key,
latestFileSlices.size()));
final Pair<String, FileSlice> partitionNameFileSlicePair = Pair.of(partitionName, slice);
partitionFileSliceToKeysMap.computeIfAbsent(partitionNameFileSlicePair, k -> new ArrayList<>()).add(key);
}
}
return partitionFileSliceToKeysMap;
}