1
0

CodeStyle formatting to conform to basic Checkstyle rules.

The code-style rules follow google style with some changes:

1. Increase line length from 100 to 120
2. Disable JavaDoc related checkstyles as this needs more manual work.

Both source and test code are checked for code-style
This commit is contained in:
Balaji Varadarajan
2018-03-20 16:29:20 -07:00
committed by vinoth chandar
parent 987f5d6b96
commit 788e4f2d2e
200 changed files with 6209 additions and 5975 deletions

View File

@@ -37,22 +37,30 @@ import org.apache.spark.api.java.JavaSparkContext;
*/
public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
protected transient JavaSparkContext jsc = null;
public enum IndexType {
HBASE,
INMEMORY,
BLOOM,
BUCKETED
}
protected final HoodieWriteConfig config;
protected transient JavaSparkContext jsc = null;
protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
this.config = config;
this.jsc = jsc;
}
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(HoodieWriteConfig config,
JavaSparkContext jsc) throws HoodieIndexException {
switch (config.getIndexType()) {
case HBASE:
return new HBaseIndex<>(config, jsc);
case INMEMORY:
return new InMemoryHashIndex<>(config, jsc);
case BLOOM:
return new HoodieBloomIndex<>(config, jsc);
case BUCKETED:
return new BucketedIndex<>(config, jsc);
default:
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
}
}
/**
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]]
* If the optional FullFilePath value is not present, then the key is not found. If the
@@ -71,7 +79,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
/**
* Extracts the location of written records, and updates the index.
*
* <p>
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
*/
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
@@ -107,18 +115,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
public abstract boolean isImplicitWithStorage();
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(
HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException {
switch (config.getIndexType()) {
case HBASE:
return new HBaseIndex<>(config, jsc);
case INMEMORY:
return new InMemoryHashIndex<>(config, jsc);
case BLOOM:
return new HoodieBloomIndex<>(config, jsc);
case BUCKETED:
return new BucketedIndex<>(config, jsc);
}
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
public enum IndexType {
HBASE, INMEMORY, BLOOM, BUCKETED
}
}

View File

@@ -49,32 +49,11 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
final HoodieTable<T> table) {
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
}
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
class LocationTagFunction
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
@Override
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
while (hoodieRecordIterator.hasNext()) {
HoodieRecord<T> rec = hoodieRecordIterator.next();
if (recordLocationMap.containsKey(rec.getKey())) {
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
}
taggedRecords.add(rec);
}
return taggedRecords.iterator();
}
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) {
@@ -132,4 +111,25 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
public boolean isImplicitWithStorage() {
return false;
}
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
class LocationTagFunction implements
Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
@Override
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
while (hoodieRecordIterator.hasNext()) {
HoodieRecord<T> rec = hoodieRecordIterator.next();
if (recordLocationMap.containsKey(rec.getKey())) {
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
}
taggedRecords.add(rec);
}
return taggedRecords.iterator();
}
}
}

View File

@@ -64,8 +64,7 @@ public class BloomIndexFileInfo implements Serializable {
* Does the given key fall within the range (inclusive)
*/
public boolean isKeyInRange(String recordKey) {
return minRecordKey.compareTo(recordKey) <= 0 &&
maxRecordKey.compareTo(recordKey) >= 0;
return minRecordKey.compareTo(recordKey) <= 0 && maxRecordKey.compareTo(recordKey) >= 0;
}
@Override
@@ -78,9 +77,8 @@ public class BloomIndexFileInfo implements Serializable {
}
BloomIndexFileInfo that = (BloomIndexFileInfo) o;
return Objects.equal(that.fileName, fileName) &&
Objects.equal(that.minRecordKey, minRecordKey) &&
Objects.equal(that.maxRecordKey, maxRecordKey);
return Objects.equal(that.fileName, fileName) && Objects.equal(that.minRecordKey, minRecordKey)
&& Objects.equal(that.maxRecordKey, maxRecordKey);
}

View File

@@ -56,12 +56,11 @@ import scala.Tuple2;
*/
public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
// we need to limit the join such that it stays within 1.5GB per Spark partition. (SPARK-1476)
private static final int SPARK_MAXIMUM_BYTES_PER_PARTITION = 1500 * 1024 * 1024;
// this is how much a triplet of (partitionPath, fileId, recordKey) costs.
private static final int BYTES_PER_PARTITION_FILE_KEY_TRIPLET = 300;
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION =
SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
@@ -108,32 +107,31 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
return taggedRecordRDD;
}
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
final HoodieTable<T> table) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD = hoodieKeys
.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
lookupIndex(partitionRecordKeyPairRDD, table);
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD,
table);
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD = hoodieKeys
.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD)
.mapToPair(keyPathTuple -> {
Optional<String> recordLocationPath;
if (keyPathTuple._2._2.isPresent()) {
String fileName = keyPathTuple._2._2.get();
String partitionPath = keyPathTuple._2._1.getPartitionPath();
recordLocationPath = Optional.of(new Path(
new Path(table.getMetaClient().getBasePath(), partitionPath),
fileName).toUri().getPath());
} else {
recordLocationPath = Optional.absent();
}
return new Tuple2<>(keyPathTuple._2._1, recordLocationPath);
});
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).mapToPair(keyPathTuple -> {
Optional<String> recordLocationPath;
if (keyPathTuple._2._2.isPresent()) {
String fileName = keyPathTuple._2._2.get();
String partitionPath = keyPathTuple._2._1.getPartitionPath();
recordLocationPath = Optional
.of(new Path(new Path(table.getMetaClient().getBasePath(), partitionPath), fileName)
.toUri().getPath());
} else {
recordLocationPath = Optional.absent();
}
return new Tuple2<>(keyPathTuple._2._1, recordLocationPath);
});
}
/**
@@ -152,21 +150,21 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
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.
// 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(hoodieTable, partitionToFileInfo,
partitionRecordKeyPairRDD,
parallelism);
partitionRecordKeyPairRDD, parallelism);
}
/**
* The index lookup can be skewed in three dimensions : #files, #partitions, #records
*
* <p>
* To be able to smoothly handle skews, we need to compute how to split each partitions into
* subpartitions. We do it here, in a way that keeps the amount of each Spark join partition to <
* 2GB.
*
* <p>
* If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified
* as a NON-zero number, then that is used explicitly.
*/
@@ -184,7 +182,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// records for a partition.
Map<String, Long> filesPerPartition = partitionToFileInfo.entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> Long.valueOf(e.getValue().size())));
long totalFiles = 0, totalRecords = 0;
long totalFiles = 0;
long totalRecords = 0;
for (String partitionPath : recordsPerPartition.keySet()) {
long numRecords = recordsPerPartition.get(partitionPath);
long numFiles =
@@ -210,22 +209,22 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
/**
* Its crucial to pick the right parallelism.
*
* <p>
* totalSubPartitions : this is deemed safe limit, to be nice with Spark. inputParallelism :
* typically number of input file splits
*
* <p>
* We pick the max such that, we are always safe, but go higher if say a there are a lot of input
* files. (otherwise, we will fallback to number of partitions in input and end up with slow
* performance)
*/
private int determineParallelism(int inputParallelism, int totalSubPartitions) {
// If bloom index parallelism is set, use it to to check against the input parallelism and take the max
// If bloom index parallelism is set, use it to to check against the input parallelism and
// take the max
int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
int joinParallelism = Math.max(totalSubPartitions, indexParallelism);
logger.info("InputParallelism: ${" + inputParallelism + "}, " +
"IndexParallelism: ${" + config.getBloomIndexParallelism() + "}, " +
"TotalSubParts: ${" + totalSubPartitions + "}, " +
"Join Parallelism set to : " + joinParallelism);
logger.info("InputParallelism: ${" + inputParallelism + "}, " + "IndexParallelism: ${" + config
.getBloomIndexParallelism() + "}, " + "TotalSubParts: ${" + totalSubPartitions + "}, "
+ "Join Parallelism set to : " + joinParallelism);
return joinParallelism;
}
@@ -237,36 +236,31 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
final HoodieTable<T> hoodieTable) {
// Obtain the latest data files from all the partitions.
List<Tuple2<String, HoodieDataFile>> dataFilesList = jsc
.parallelize(partitions, Math.max(partitions.size(), 1))
.flatMapToPair(partitionPath -> {
java.util.Optional<HoodieInstant> latestCommitTime =
hoodieTable.getCommitsTimeline().filterCompletedInstants().lastInstant();
.parallelize(partitions, Math.max(partitions.size(), 1)).flatMapToPair(partitionPath -> {
java.util.Optional<HoodieInstant> latestCommitTime = hoodieTable.getCommitsTimeline()
.filterCompletedInstants().lastInstant();
List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>();
if (latestCommitTime.isPresent()) {
filteredFiles =
hoodieTable.getROFileSystemView().getLatestDataFilesBeforeOrOn(partitionPath,
latestCommitTime.get().getTimestamp())
.map(f -> new Tuple2<>(partitionPath, f))
.collect(toList());
filteredFiles = hoodieTable.getROFileSystemView()
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
.map(f -> new Tuple2<>(partitionPath, f)).collect(toList());
}
return filteredFiles.iterator();
}).collect();
if (config.getBloomIndexPruneByRanges()) {
// also obtain file ranges, if range pruning is enabled
return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1))
.mapToPair(ft -> {
try {
String[] minMaxKeys = ParquetUtils
.readMinMaxRecordKeys(hoodieTable.getHadoopConf(),
ft._2().getFileStatus().getPath());
return new Tuple2<>(ft._1(),
new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
} catch (MetadataNotFoundException me) {
logger.warn("Unable to find range metadata in file :" + ft._2());
return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName()));
}
}).collect();
return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1)).mapToPair(ft -> {
try {
String[] minMaxKeys = ParquetUtils
.readMinMaxRecordKeys(hoodieTable.getHadoopConf(), ft._2().getFileStatus().getPath());
return new Tuple2<>(ft._1(),
new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
} catch (MetadataNotFoundException me) {
logger.warn("Unable to find range metadata in file :" + ft._2());
return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName()));
}
}).collect();
} else {
return dataFilesList.stream()
.map(ft -> new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName())))
@@ -320,62 +314,60 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* (e.g: timestamp as prefix), the number of files to be compared gets cut down a lot from range
* pruning.
*/
// sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on recordKey
// sub-partition to ensure the records can be looked up against files & also prune
// file<=>record comparisons based on recordKey
// ranges in the index info.
@VisibleForTesting
JavaPairRDD<String, Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
return partitionRecordKeyPairRDD
.map(partitionRecordKeyPair -> {
String recordKey = partitionRecordKeyPair._2();
String partitionPath = partitionRecordKeyPair._1();
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
String recordKey = partitionRecordKeyPair._2();
String partitionPath = partitionRecordKeyPair._1();
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
List<Tuple2<String, Tuple2<String, HoodieKey>>> recordComparisons = new ArrayList<>();
if (indexInfos
!= null) { // could be null, if there are no files in a given partition yet.
// for each candidate file in partition, that needs to be compared.
for (BloomIndexFileInfo indexInfo : indexInfos) {
if (shouldCompareWithFile(indexInfo, recordKey)) {
recordComparisons.add(
new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey),
new Tuple2<>(indexInfo.getFileName(),
new HoodieKey(recordKey, partitionPath))));
}
}
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
List<Tuple2<String, Tuple2<String, HoodieKey>>> recordComparisons = new ArrayList<>();
if (indexInfos != null) { // could be null, if there are no files in a given partition yet.
// for each candidate file in partition, that needs to be compared.
for (BloomIndexFileInfo indexInfo : indexInfos) {
if (shouldCompareWithFile(indexInfo, recordKey)) {
recordComparisons.add(
new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey),
new Tuple2<>(indexInfo.getFileName(),
new HoodieKey(recordKey, partitionPath))));
}
return recordComparisons;
})
.flatMapToPair(t -> t.iterator());
}
}
return recordComparisons;
}).flatMapToPair(t -> t.iterator());
}
/**
* Find out <RowKey, filename> pair. All workload grouped by file-level.
*
* <p>
* Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such
* that each RDD partition is a file, then for each file, we do (1) load bloom filter, (2) load
* rowKeys, (3) Tag rowKey
*
* <p>
* Make sure the parallelism is atleast the groupby parallelism for tagging location
*/
@VisibleForTesting
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(HoodieTable hoodieTable,
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD,
int totalSubpartitions) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int totalSubpartitions) {
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(),
totalSubpartitions);
JavaPairRDD<String, Tuple2<String, HoodieKey>> fileSortedTripletRDD = explodeRecordRDDWithFileComparisons(
partitionToFileIndexInfo, partitionRecordKeyPairRDD)
// sort further based on filename, such that all checking for the file can happen within a single partition, on-the-fly
.sortByKey(true, joinParallelism);
JavaPairRDD<String, Tuple2<String, HoodieKey>> fileSortedTripletRDD =
explodeRecordRDDWithFileComparisons(
partitionToFileIndexInfo, partitionRecordKeyPairRDD)
// sort further based on filename, such that all checking for the file can happen within
// a single partition, on-the-fly
.sortByKey(true, joinParallelism);
return fileSortedTripletRDD
.mapPartitionsWithIndex(
new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true)
return fileSortedTripletRDD.mapPartitionsWithIndex(
new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true)
.flatMap(indexLookupResults -> indexLookupResults.iterator())
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
.flatMapToPair(lookupResult -> {
@@ -391,25 +383,23 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Tag the <rowKey, filename> back to the original HoodieRecord RDD.
*/
private JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
JavaPairRDD<String, String> rowKeyFilenamePairRDD,
JavaRDD<HoodieRecord<T>> recordRDD) {
JavaPairRDD<String, String> rowKeyFilenamePairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
JavaPairRDD<String, HoodieRecord<T>> rowKeyRecordPairRDD = recordRDD
.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join.
return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(
v1 -> {
HoodieRecord<T> record = v1._1();
if (v1._2().isPresent()) {
String filename = v1._2().get();
if (filename != null && !filename.isEmpty()) {
record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename),
FSUtils.getFileId(filename)));
}
}
return record;
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null),
// so we do left outer join.
return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(v1 -> {
HoodieRecord<T> record = v1._1();
if (v1._2().isPresent()) {
String filename = v1._2().get();
if (filename != null && !filename.isEmpty()) {
record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename),
FSUtils.getFileId(filename)));
}
);
}
return record;
});
}
@Override

View File

@@ -41,7 +41,8 @@ import scala.Tuple2;
* actual files
*/
public class HoodieBloomIndexCheckFunction implements
Function2<Integer, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>>, Iterator<List<IndexLookupResult>>> {
Function2<Integer, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>>,
Iterator<List<IndexLookupResult>>> {
private static Logger logger = LogManager.getLogger(HoodieBloomIndexCheckFunction.class);
@@ -58,8 +59,7 @@ public class HoodieBloomIndexCheckFunction implements
* Given a list of row keys and one file, return only row keys existing in that file.
*/
public static List<String> checkCandidatesAgainstFile(Configuration configuration,
List<String> candidateRecordKeys,
Path filePath) throws HoodieIndexException {
List<String> candidateRecordKeys, Path filePath) throws HoodieIndexException {
List<String> foundRecordKeys = new ArrayList<>();
try {
// Load all rowKeys from the file, to double-confirm
@@ -86,6 +86,13 @@ public class HoodieBloomIndexCheckFunction implements
return foundRecordKeys;
}
@Override
public Iterator<List<IndexLookupResult>> call(Integer partition,
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr)
throws Exception {
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
}
class LazyKeyCheckIterator extends
LazyIterableIterator<Tuple2<String, Tuple2<String, HoodieKey>>, List<IndexLookupResult>> {
@@ -143,7 +150,8 @@ public class HoodieBloomIndexCheckFunction implements
// if continue on current file)
if (fileName.equals(currentFile)) {
// check record key against bloom filter of current file & add to possible keys if needed
// check record key against bloom filter of current file & add to possible keys if
// needed
if (bloomFilter.mightContain(recordKey)) {
if (logger.isDebugEnabled()) {
logger.debug("#1 Adding " + recordKey + " as candidate for file " + fileName);
@@ -201,12 +209,4 @@ public class HoodieBloomIndexCheckFunction implements
protected void end() {
}
}
@Override
public Iterator<List<IndexLookupResult>> call(Integer partition,
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr)
throws Exception {
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
}
}

View File

@@ -38,9 +38,9 @@ import scala.Tuple2;
/**
* An `stateless` index implementation that will using a deterministic mapping function to determine
* the fileID for a given record.
*
* <p>
* Pros: - Fast
*
* <p>
* Cons : - Need to tune the number of buckets per partition path manually (FIXME: Need to autotune
* this) - Could increase write amplification on copy-on-write storage since inserts always rewrite
* files - Not global.

View File

@@ -27,12 +27,16 @@ import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
@@ -51,23 +55,18 @@ import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function2;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
/**
* Hoodie Index implementation backed by HBase
*/
public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
private final static byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
private final static byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
private final static byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
private final static byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
private static final byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
private static final byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
private static final byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
private static Logger logger = LogManager.getLogger(HBaseIndex.class);
private static Connection hbaseConnection = null;
private final String tableName;
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
@@ -77,14 +76,12 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
HoodieTable<T> table) {
//TODO : Change/Remove filterExists in HoodieReadClient() and revisit
throw new UnsupportedOperationException("HBase index does not implement check exist");
}
private static Connection hbaseConnection = null;
private Connection getHBaseConnection() {
Configuration hbaseConfig = HBaseConfiguration.create();
String quorum = config.getHbaseZkQuorum();
@@ -100,15 +97,15 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
}
/**
* Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is closed when
* JVM exits
* Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is
* closed when JVM exits
*/
private void addShutDownHook() {
Runtime.getRuntime().addShutdownHook(new Thread() {
public void run() {
try {
hbaseConnection.close();
} catch(Exception e) {
} catch (Exception e) {
// fail silently for any sort of exception
}
}
@@ -126,101 +123,103 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
// Check if the last commit ts for this row is 1) present in the timeline or
// 2) is less than the first commit ts in the timeline
return !commitTimeline.empty() && (commitTimeline.containsInstant(
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs)) ||
HoodieTimeline.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(),
commitTs, HoodieTimeline.GREATER));
return !commitTimeline.empty() && (commitTimeline
.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))
|| HoodieTimeline
.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs,
HoodieTimeline.GREATER));
}
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
private Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>
locationTagFunction(HoodieTable<T> hoodieTable) {
locationTagFunction(HoodieTable<T> hoodieTable) {
return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>)
(partitionNum, hoodieRecordIterator) -> {
Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize();
Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize();
// Grab the global HBase connection
synchronized (HBaseIndex.class) {
if (hbaseConnection == null || hbaseConnection.isClosed()) {
hbaseConnection = getHBaseConnection();
}
}
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
HTable hTable = null;
try {
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
List<Get> statements = new ArrayList<>();
List<HoodieRecord> currentBatchOfRecords = new LinkedList<>();
// Do the tagging.
while (hoodieRecordIterator.hasNext()) {
HoodieRecord rec = hoodieRecordIterator.next();
statements.add(generateStatement(rec.getRecordKey()));
currentBatchOfRecords.add(rec);
// iterator till we reach batch size
if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) {
// get results for batch from Hbase
Result[] results = hTable.get(statements);
// clear statements to be GC'd
statements.clear();
for (Result result : results) {
// first, attempt to grab location from HBase
HoodieRecord currentRecord = currentBatchOfRecords.remove(0);
if (result.getRow() != null) {
String keyFromResult = Bytes.toString(result.getRow());
String commitTs =
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
String fileId =
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
String partitionPath =
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
if (checkIfValidCommit(hoodieTable, commitTs)) {
currentRecord = new HoodieRecord(new HoodieKey(currentRecord.getRecordKey(),
partitionPath), currentRecord.getData());
currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
taggedRecords.add(currentRecord);
// the key from Result and the key being processed should be same
assert (currentRecord.getRecordKey().contentEquals(keyFromResult));
} else { //if commit is invalid, treat this as a new taggedRecord
taggedRecords.add(currentRecord);
}
} else {
taggedRecords.add(currentRecord);
}
// Grab the global HBase connection
synchronized (HBaseIndex.class) {
if (hbaseConnection == null || hbaseConnection.isClosed()) {
hbaseConnection = getHBaseConnection();
}
}
}
} catch (IOException e) {
throw new HoodieIndexException(
"Failed to Tag indexed locations because of exception with HBase Client", e);
} finally {
if (hTable != null) {
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
HTable hTable = null;
try {
hTable.close();
} catch (IOException e) {
// Ignore
}
}
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
List<Get> statements = new ArrayList<>();
List<HoodieRecord> currentBatchOfRecords = new LinkedList<>();
// Do the tagging.
while (hoodieRecordIterator.hasNext()) {
HoodieRecord rec = hoodieRecordIterator.next();
statements.add(generateStatement(rec.getRecordKey()));
currentBatchOfRecords.add(rec);
// iterator till we reach batch size
if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) {
// get results for batch from Hbase
Result[] results = hTable.get(statements);
// clear statements to be GC'd
statements.clear();
for (Result result : results) {
// first, attempt to grab location from HBase
HoodieRecord currentRecord = currentBatchOfRecords.remove(0);
if (result.getRow() != null) {
String keyFromResult = Bytes.toString(result.getRow());
String commitTs = Bytes
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
String fileId = Bytes
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
String partitionPath = Bytes
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
}
return taggedRecords.iterator();
};
if (checkIfValidCommit(hoodieTable, commitTs)) {
currentRecord = new HoodieRecord(
new HoodieKey(currentRecord.getRecordKey(), partitionPath),
currentRecord.getData());
currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
taggedRecords.add(currentRecord);
// the key from Result and the key being processed should be same
assert (currentRecord.getRecordKey().contentEquals(keyFromResult));
} else { //if commit is invalid, treat this as a new taggedRecord
taggedRecords.add(currentRecord);
}
} else {
taggedRecords.add(currentRecord);
}
}
}
}
} catch (IOException e) {
throw new HoodieIndexException(
"Failed to Tag indexed locations because of exception with HBase Client", e);
} finally {
if (hTable != null) {
try {
hTable.close();
} catch (IOException e) {
// Ignore
}
}
}
return taggedRecords.iterator();
};
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) {
HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable), true);
}
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition, statusIterator) -> {
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>
updateLocationFunction() {
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition,
statusIterator) -> {
Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize();
List<WriteStatus> writeStatusList = new ArrayList<>();
@@ -292,16 +291,13 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
/**
* Helper method to facilitate performing puts and deletes in Hbase
* @param hTable
* @param puts
* @param deletes
* @throws IOException
*/
private void doPutsAndDeletes(HTable hTable, List<Put> puts, List<Delete> deletes) throws IOException {
if(puts.size() > 0) {
private void doPutsAndDeletes(HTable hTable, List<Put> puts, List<Delete> deletes)
throws IOException {
if (puts.size() > 0) {
hTable.put(puts);
}
if(deletes.size() > 0) {
if (deletes.size() > 0) {
hTable.delete(deletes);
}
hTable.flushCommits();
@@ -311,7 +307,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) {
HoodieTable<T> hoodieTable) {
return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true);
}
@@ -323,7 +319,6 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
/**
* Only looks up by recordKey
* @return
*/
@Override
public boolean isGlobal() {
@@ -332,7 +327,6 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
/**
* Mapping is available in HBase already.
* @return
*/
@Override
public boolean canIndexLogFiles() {
@@ -341,7 +335,6 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
/**
* Index needs to be explicitly updated after storage write.
* @return
*/
@Override
public boolean isImplicitWithStorage() {