/** * if we dont have key ranges, then also we need to compare against the file. no other choice if * we do, then only compare the file if the record key falls in range. */ boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) { return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey); }
@Override public Iterator<List<IndexLookupResult>> call(Integer partition, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> filePartitionRecordKeyTripletItr) throws Exception { return new LazyKeyCheckIterator(filePartitionRecordKeyTripletItr); }
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int totalSubpartitions, HoodieTableMetaClient metaClient) { int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), totalSubpartitions); explodeRecordRDDWithFileComparisons( partitionToFileIndexInfo, partitionRecordKeyPairRDD) new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true) .flatMap(indexLookupResults -> indexLookupResults.iterator()) .filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0) .flatMapToPair(lookupResult -> { List<Tuple2<String, String>> vals = new ArrayList<>(); for (String recordKey : lookupResult.getMatchingRecordKeys()) { vals.add(new Tuple2<>(recordKey, lookupResult.getFileName()));
@Test public void testRangePruning() { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieBloomIndex index = new HoodieBloomIndex(config); final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>(); partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"), new BloomIndexFileInfo("f2", "000", "000"), new BloomIndexFileInfo("f3", "001", "003"), new BloomIndexFileInfo("f4", "002", "007"), new BloomIndexFileInfo("f5", "009", "010"))); JavaPairRDD<String, String> partitionRecordKeyPairRDD = jsc.parallelize(Arrays.asList( new Tuple2<>("2017/10/22", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/22", "004"))).mapToPair(t -> t); List<Tuple2<String, Tuple2<String, HoodieKey>>> comparisonKeyList = index.explodeRecordRDDWithFileComparisons( partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect(); assertEquals(10, comparisonKeyList.size()); Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream().collect(Collectors.groupingBy( t -> t._2()._2().getRecordKey(), Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList()))); assertEquals(4, recordKeyToFileComps.size()); assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("002")); assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("003")); assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("004")); assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("005")); }
initState(fileName, partitionPath); .debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); ret.add(new IndexLookupResult(currentFile, checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath))); initState(fileName, partitionPath); if (bloomFilter.mightContain(recordKey)) { if (logger.isDebugEnabled()) { logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); ret.add(new IndexLookupResult(currentFile, checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath)));
/** * Lookup the location for each record key and return the pair<record_key,location> for all record * keys already present and drop the record keys if not present */ private JavaPairRDD<String, String> lookupIndex( JavaPairRDD<String, String> partitionRecordKeyPairRDD, final JavaSparkContext jsc, final HoodieTable hoodieTable) { // Obtain records per partition, in the incoming records Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey(); List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet()); // Step 2: Load all involved files as <Partition, filename> pairs List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, jsc, hoodieTable); final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream() .collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList()))); // Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, // that contains it. int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD); return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, parallelism, hoodieTable.getMetaClient()); }
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException { switch (config.getIndexType()) { case HBASE: return new HBaseIndex<>(config); case INMEMORY: return new InMemoryHashIndex<>(config); case BLOOM: return new HoodieBloomIndex<>(config); case GLOBAL_BLOOM: return new HoodieGlobalBloomIndex<>(config); case BUCKETED: return new BucketedIndex<>(config); default: throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); } }
if (shouldCompareWithFile(indexInfo, recordKey)) { recordComparisons.add( new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey), new Tuple2<>(indexInfo.getFileName(), new HoodieKey(recordKey, partitionPath))));
@Test public void testTagLocationWithEmptyRDD() throws Exception { // We have some records to be tagged (two different partitions) JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD(); // Also create the metadata and config HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc); // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config); try { bloomIndex.tagLocation(recordRDD, jsc, table); } catch (IllegalArgumentException e) { fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices " + "required"); } }
@Override public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc, HoodieTable<T> hoodieTable) { // Step 0: cache the input record RDD if (config.getBloomIndexUseCaching()) { recordRDD.persist(config.getBloomIndexInputStorageLevel()); } // Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey) JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD .mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())); // Lookup indexes for all the partition/recordkey pair JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable); // Cache the result, for subsequent stages. if (config.getBloomIndexUseCaching()) { rowKeyFilenamePairRDD.persist(StorageLevel.MEMORY_AND_DISK_SER()); } if (logger.isDebugEnabled()) { long totalTaggedRecords = rowKeyFilenamePairRDD.count(); logger.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords); } // Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys // Cost: 4 sec. JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(rowKeyFilenamePairRDD, recordRDD); if (config.getBloomIndexUseCaching()) { recordRDD.unpersist(); // unpersist the input Record RDD rowKeyFilenamePairRDD.unpersist(); } return taggedRecordRDD; }
private Map<String, BloomIndexFileInfo> toFileMap(List<Tuple2<String, BloomIndexFileInfo>> filesList) { Map<String, BloomIndexFileInfo> filesMap = new HashMap<>(); for (Tuple2<String, BloomIndexFileInfo> t : filesList) { filesMap.put(t._1() + "/" + t._2().getFileName(), t._2()); } return filesMap; }
/** * Load all involved files as <Partition, filename> pair RDD from all partitions in the table. */ @Override @VisibleForTesting List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc, final HoodieTable hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); try { List<String> allPartitionPaths = FSUtils .getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning()); return super.loadInvolvedFiles(allPartitionPaths, jsc, hoodieTable); } catch (IOException e) { throw new HoodieIOException("Failed to load all partitions", e); } }
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException { switch (config.getIndexType()) { case HBASE: return new HBaseIndex<>(config); case INMEMORY: return new InMemoryHashIndex<>(config); case BLOOM: return new HoodieBloomIndex<>(config); case BUCKETED: return new BucketedIndex<>(config); default: throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); } }
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int totalSubpartitions, HoodieTableMetaClient metaClient) { int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), totalSubpartitions); explodeRecordRDDWithFileComparisons( partitionToFileIndexInfo, partitionRecordKeyPairRDD) new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true) .flatMap(List::iterator) .filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0) .flatMapToPair(lookupResult -> { List<Tuple2<String, String>> vals = new ArrayList<>(); for (String recordKey : lookupResult.getMatchingRecordKeys()) { vals.add(new Tuple2<>(recordKey, lookupResult.getFileName()));
initState(fileName, partitionPath); .debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); ret.add(new IndexLookupResult(currentFile, checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath))); initState(fileName, partitionPath); if (bloomFilter.mightContain(recordKey)) { if (logger.isDebugEnabled()) { logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); ret.add(new IndexLookupResult(currentFile, checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath)));
/** * Lookup the location for each record key and return the pair<record_key,location> for all record * keys already present and drop the record keys if not present */ private JavaPairRDD<String, String> lookupIndex( JavaPairRDD<String, String> partitionRecordKeyPairRDD, final JavaSparkContext jsc, final HoodieTable hoodieTable) { // Obtain records per partition, in the incoming records Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey(); List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet()); // Step 2: Load all involved files as <Partition, filename> pairs List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, jsc, hoodieTable); final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream() .collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList()))); // Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, // that contains it. int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD); return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, parallelism, hoodieTable.getMetaClient()); }
/** * if we dont have key ranges, then also we need to compare against the file. no other choice if * we do, then only compare the file if the record key falls in range. */ private boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) { return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey); }
if (shouldCompareWithFile(indexInfo, recordKey)) { recordComparisons.add( new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey), new Tuple2<>(indexInfo.getFileName(), new HoodieKey(recordKey, partitionPath))));
@Override public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc, HoodieTable<T> hoodieTable) { // Step 0: cache the input record RDD if (config.getBloomIndexUseCaching()) { recordRDD.persist(config.getBloomIndexInputStorageLevel()); } // Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey) JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD .mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())); // Lookup indexes for all the partition/recordkey pair JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable); // Cache the result, for subsequent stages. if (config.getBloomIndexUseCaching()) { rowKeyFilenamePairRDD.persist(StorageLevel.MEMORY_AND_DISK_SER()); } if (logger.isDebugEnabled()) { long totalTaggedRecords = rowKeyFilenamePairRDD.count(); logger.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords); } // Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys // Cost: 4 sec. JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(rowKeyFilenamePairRDD, recordRDD); if (config.getBloomIndexUseCaching()) { recordRDD.unpersist(); // unpersist the input Record RDD rowKeyFilenamePairRDD.unpersist(); } return taggedRecordRDD; }
@Override public Iterator<List<IndexLookupResult>> call(Integer partition, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr) throws Exception { return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr); }