Adding range based pruning to bloom index
- keys compared lexicographically using String::compareTo - Range metadata additionally written into parquet file footers - Trim fat & few optimizations to speed up indexing - Add param to control whether input shall be cached, to speed up lookup - Add param to turn on/off range pruning - Auto compute of parallelism now simply factors in amount of comparisons done - More accurate parallelism computation when range pruning is on - tests added & hardened, docs updated
This commit is contained in:
@@ -21,9 +21,10 @@ import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Comparator;
|
||||
|
||||
public class HoodieDataFile {
|
||||
public class HoodieDataFile implements Serializable {
|
||||
private FileStatus fileStatus;
|
||||
|
||||
public HoodieDataFile(FileStatus fileStatus) {
|
||||
|
||||
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Comparator;
|
||||
import java.util.Optional;
|
||||
|
||||
@@ -32,7 +33,7 @@ import java.util.Optional;
|
||||
*
|
||||
* Also contains logic to roll-over the log file
|
||||
*/
|
||||
public class HoodieLogFile {
|
||||
public class HoodieLogFile implements Serializable {
|
||||
public static final String DELTA_EXTENSION = ".log";
|
||||
|
||||
private final Path path;
|
||||
|
||||
@@ -20,8 +20,10 @@ import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
import com.uber.hoodie.exception.MetadataNotFoundException;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@@ -120,22 +122,40 @@ public class ParquetUtils {
|
||||
}
|
||||
|
||||
|
||||
private static List<String> readParquetFooter(Path parquetFilePath, String... footerNames) {
|
||||
List<String> footerVals = new ArrayList<>();
|
||||
ParquetMetadata footer = readMetadata(parquetFilePath);
|
||||
Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData();
|
||||
for (String footerName: footerNames) {
|
||||
if (metadata.containsKey(footerName)) {
|
||||
footerVals.add(metadata.get(footerName));
|
||||
} else {
|
||||
throw new MetadataNotFoundException("Could not find index in Parquet footer. " +
|
||||
"Looked for key " + footerName + " in " + parquetFilePath);
|
||||
}
|
||||
}
|
||||
return footerVals;
|
||||
}
|
||||
|
||||
/**
|
||||
* Read out the bloom filter from the parquet file meta data.
|
||||
*/
|
||||
public static BloomFilter readBloomFilterFromParquetMetadata(Path parquetFilePath) {
|
||||
ParquetMetadata footer = readMetadata(parquetFilePath);
|
||||
Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData();
|
||||
if (metadata.containsKey(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY)) {
|
||||
return new BloomFilter(metadata.get(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY));
|
||||
} else {
|
||||
throw new HoodieIndexException("Could not find index in Parquet footer. Looked for key "
|
||||
+ HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY + " in "
|
||||
+ parquetFilePath);
|
||||
}
|
||||
String footerVal = readParquetFooter(parquetFilePath,
|
||||
HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY).get(0);
|
||||
return new BloomFilter(footerVal);
|
||||
}
|
||||
|
||||
public static String[] readMinMaxRecordKeys(Path parquetFilePath) {
|
||||
List<String> minMaxKeys = readParquetFooter(parquetFilePath, HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER,
|
||||
HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER);
|
||||
if (minMaxKeys.size() != 2) {
|
||||
throw new HoodieException(String.format(
|
||||
"Could not read min/max record key out of footer correctly from %s. read) : %s",
|
||||
parquetFilePath, minMaxKeys));
|
||||
}
|
||||
return new String[]{minMaxKeys.get(0), minMaxKeys.get(1)};
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
|
||||
Reference in New Issue
Block a user