vinothchandar commented on a change in pull request #1402: [WIP][HUDI-407] Adding Simple Index URL: https://github.com/apache/incubator-hudi/pull/1402#discussion_r407235240
########## File path: hudi-client/src/main/java/org/apache/hudi/index/HoodieSimpleIndex.java ########## @@ -0,0 +1,202 @@ +/* + * 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.index; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.utils.SparkConfigUtils; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ParquetUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.PairFlatMapFunction; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import scala.Tuple2; + +import static java.util.stream.Collectors.toList; +import static org.apache.hudi.index.HoodieIndexUtils.loadLatestDataFilesForAllPartitions; + +/** + * A simple index which reads interested fields(record key and partition path) from base files and + * joins with incoming records to find the tagged location. + * + * @param <T> + */ +public class HoodieSimpleIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> { + + public HoodieSimpleIndex(HoodieWriteConfig config) { + super(config); + } + + @Override + public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc, + HoodieTable<T> hoodieTable) { + return writeStatusRDD; + } + + @Override + public boolean rollbackCommit(String commitTime) { + return true; + } + + @Override + public boolean isGlobal() { + return false; + } + + @Override + public boolean canIndexLogFiles() { + return false; + } + + @Override + public boolean isImplicitWithStorage() { + return true; + } + + @Override + public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc, + HoodieTable<T> hoodieTable) { + if (config.getSimpleIndexUseCaching()) { + recordRDD.persist(SparkConfigUtils.getBloomIndexInputStorageLevel(config.getProps())); + } + + JavaPairRDD<HoodieKey, HoodieRecord> incomingRecords = recordRDD.mapToPair(entry -> new Tuple2<>(entry.getKey(), entry)); + + JavaPairRDD<HoodieKey, HoodieRecordLocation> existingRecords = fetchRecordLocations(incomingRecords.keys(), jsc, hoodieTable); + + jsc.setJobGroup(this.getClass().getSimpleName(), "Tagging incoming records with record location"); + JavaRDD<Tuple2<HoodieKey, Tuple2<HoodieRecord, Option<HoodieRecordLocation>>>> untaggedRecordsRDD = incomingRecords.leftOuterJoin(existingRecords) + .map(entry -> new Tuple2(entry._1, new Tuple2(entry._2._1, Option.ofNullable(entry._2._2.orNull())))); + + JavaRDD<HoodieRecord<T>> taggedRecordRDD = untaggedRecordsRDD.map(entry -> getTaggedRecord(entry._2._1, entry._2._2)); + + if (config.getSimpleIndexUseCaching()) { + recordRDD.unpersist(); // unpersist the input Record RDD + } + return taggedRecordRDD; + } + + /** + * Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option.Empty if the key is not. + * found. + * + * @param hoodieKeys keys to lookup + * @param jsc spark context + * @param hoodieTable hoodie table object + */ + @Override + public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, + JavaSparkContext jsc, HoodieTable<T> hoodieTable) { + JavaPairRDD<HoodieKey, Option<HoodieRecordLocation>> incomingRecords = + hoodieKeys.mapToPair(entry -> new Tuple2<>(entry, Option.empty())); + + JavaPairRDD<HoodieKey, HoodieRecordLocation> existingRecords = fetchRecordLocations(hoodieKeys, jsc, hoodieTable); + + jsc.setJobGroup(this.getClass().getSimpleName(), "Joining existing records with incoming keys to find record locations"); + return incomingRecords.leftOuterJoin(existingRecords) + .mapToPair(entry -> new Tuple2(entry._1, Option.ofNullable(entry._2._2.orNull()))); + } + + /** + * Fetch record locations for passed in {@link HoodieKey}s. + * + * @param hoodieKeys {@link JavaRDD} of {@link HoodieKey}s for which locations are fetched + * @param jsc instance of {@link JavaSparkContext} to use + * @param hoodieTable instance of {@link HoodieTable} of interest + * @return {@link JavaPairRDD} of {@link HoodieKey} and {@link HoodieRecordLocation} + */ + private JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchRecordLocations(JavaRDD<HoodieKey> hoodieKeys, JavaSparkContext jsc, HoodieTable hoodieTable) { + + JavaPairRDD<String, String> partitionRecordKeyPairRDD = hoodieKeys.mapToPair(entry -> new Tuple2(entry.getPartitionPath(), entry.getRecordKey())); + jsc.setJobGroup(this.getClass().getSimpleName(), "Fetching fileInfos for partitions of interest"); + List<String> affectedPartitionPathList = partitionRecordKeyPairRDD.map(tuple -> tuple._1).distinct().collect(); + JavaRDD<Tuple2<String, String>> fileInfoList = jsc.parallelize( + loadAllFilesForPartitions(affectedPartitionPathList, jsc, hoodieTable)).sortBy(Tuple2::_1, true, config.getSimpleIndexParallelism()); + + JavaPairRDD<String, String> partitionFileIndexInfoPairRDD = fileInfoList.mapToPair(entry -> new Tuple2<>(entry._1, entry._2)); + + jsc.setJobGroup(this.getClass().getSimpleName(), "Fetching records from all files of interest"); + return partitionFileIndexInfoPairRDD.flatMapToPair( + (PairFlatMapFunction<Tuple2<String, String>, HoodieKey, HoodieRecordLocation>) partitionPathFileId -> + new RecordFetcher(partitionPathFileId, hoodieTable).getResultSet() + ); + } + + /** + * Load all involved files as <Partition, filename> pair RDD. + */ + private List<Tuple2<String, String>> loadAllFilesForPartitions(List<String> partitions, final JavaSparkContext jsc, + final HoodieTable hoodieTable) { + + // Obtain the latest data files from all the partitions. + List<Pair<String, String>> partitionPathFileIDList = loadLatestDataFilesForAllPartitions(partitions, + jsc, hoodieTable); + return partitionPathFileIDList.stream() + .map(pf -> new Tuple2<>(pf.getKey(), pf.getValue())).collect(toList()); + } + + /** + * Fetch the latest base file for the given partition and fileId. + * + * @param hoodieTable instance of {@link HoodieTable} in which the partition exists + * @param partitionPathFilePair Partition path fileId pair + * @return the latest data file for the given partition and fileId + */ + private HoodieBaseFile getLatestDataFile(HoodieTable hoodieTable, Pair<String, String> partitionPathFilePair) { + return hoodieTable.getBaseFileOnlyView() + .getLatestBaseFile(partitionPathFilePair.getLeft(), partitionPathFilePair.getRight()).get(); + } + + /** + * Record Fetcher for a given partitionPath, fileId pair. + */ + class RecordFetcher { + private HoodieTable<T> table; + private Tuple2<String, String> partitionPathFileIdPair; + + RecordFetcher(Tuple2<String, String> partitionPathFileIdPair, HoodieTable<T> table) { + this.partitionPathFileIdPair = partitionPathFileIdPair; + this.table = table; + } + + Iterator<Tuple2<HoodieKey, HoodieRecordLocation>> getResultSet() throws Exception { + HoodieBaseFile baseFile = getLatestDataFile(table, Pair.of(partitionPathFileIdPair._1, partitionPathFileIdPair._2)); + List<Pair<HoodieKey, HoodieRecordLocation>> records = ParquetUtils.fetchRecordKeyPartitionPathFromParquet(table.getHadoopConf(), new Path(baseFile.getPath()), Review comment: we can't call `ParquetUtils` from here.. that breaks abstraction. Lets design a new handle ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
