Reformatting code per Google Code Style all over
This commit is contained in:
committed by
vinoth chandar
parent
5a62480a92
commit
e45679f5e2
@@ -17,118 +17,108 @@
|
||||
package com.uber.hoodie.index;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
||||
import com.uber.hoodie.index.bucketed.BucketedIndex;
|
||||
import com.uber.hoodie.index.hbase.HBaseIndex;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.Serializable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Base class for different types of indexes to determine the mapping from uuid
|
||||
*
|
||||
*/
|
||||
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 HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
this.config = config;
|
||||
this.jsc = jsc;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 FullFilePath
|
||||
* value is present, it is the path component (without scheme) of the URI underlying file
|
||||
*
|
||||
* @param hoodieKeys
|
||||
* @param table
|
||||
* @return
|
||||
*/
|
||||
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table);
|
||||
|
||||
/**
|
||||
* Looks up the index and tags each incoming record with a location of a file that contains the
|
||||
* row (if it is actually present)
|
||||
*/
|
||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Extracts the location of written records, and updates the index.
|
||||
*
|
||||
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Rollback the efffects of the commit made at commitTime.
|
||||
*/
|
||||
public abstract boolean rollbackCommit(String commitTime);
|
||||
|
||||
/**
|
||||
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the `partitionPath`.
|
||||
* Such an implementation is able to obtain the same mapping, for two hoodie keys with same `recordKey`
|
||||
* but different `partitionPath`
|
||||
*
|
||||
* @return whether or not, the index implementation is global in nature
|
||||
*/
|
||||
public abstract boolean isGlobal();
|
||||
|
||||
/**
|
||||
* This is used by storage to determine, if its safe to send inserts, straight to the log,
|
||||
* i.e having a {@link com.uber.hoodie.common.model.FileSlice}, with no data file.
|
||||
*
|
||||
* @return Returns true/false depending on whether the impl has this capability
|
||||
*/
|
||||
public abstract boolean canIndexLogFiles();
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* An index is "implicit" with respect to storage, if just writing new data to a file slice,
|
||||
* updates the index as well. This is used by storage, to save memory footprint in
|
||||
* certain cases.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
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());
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
|
||||
public enum IndexType {
|
||||
HBASE,
|
||||
INMEMORY,
|
||||
BLOOM,
|
||||
BUCKETED
|
||||
}
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
this.config = config;
|
||||
this.jsc = jsc;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
* FullFilePath value is present, it is the path component (without scheme) of the URI underlying
|
||||
* file
|
||||
*/
|
||||
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table);
|
||||
|
||||
/**
|
||||
* Looks up the index and tags each incoming record with a location of a file that contains the
|
||||
* row (if it is actually present)
|
||||
*/
|
||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Extracts the location of written records, and updates the index.
|
||||
*
|
||||
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Rollback the efffects of the commit made at commitTime.
|
||||
*/
|
||||
public abstract boolean rollbackCommit(String commitTime);
|
||||
|
||||
/**
|
||||
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the
|
||||
* `partitionPath`. Such an implementation is able to obtain the same mapping, for two hoodie keys
|
||||
* with same `recordKey` but different `partitionPath`
|
||||
*
|
||||
* @return whether or not, the index implementation is global in nature
|
||||
*/
|
||||
public abstract boolean isGlobal();
|
||||
|
||||
/**
|
||||
* This is used by storage to determine, if its safe to send inserts, straight to the log, i.e
|
||||
* having a {@link com.uber.hoodie.common.model.FileSlice}, with no data file.
|
||||
*
|
||||
* @return Returns true/false depending on whether the impl has this capability
|
||||
*/
|
||||
public abstract boolean canIndexLogFiles();
|
||||
|
||||
|
||||
/**
|
||||
* An index is "implicit" with respect to storage, if just writing new data to a file slice,
|
||||
* updates the index as well. This is used by storage, to save memory footprint in certain cases.
|
||||
*/
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -17,129 +17,119 @@
|
||||
package com.uber.hoodie.index;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
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.Function;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
|
||||
/**
|
||||
* Hoodie Index implementation backed by an in-memory Hash map.
|
||||
* <p>
|
||||
* ONLY USE FOR LOCAL TESTING
|
||||
* Hoodie Index implementation backed by an in-memory Hash map. <p> ONLY USE FOR LOCAL TESTING
|
||||
*/
|
||||
public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
|
||||
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
|
||||
|
||||
public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
recordLocationMap = new ConcurrentHashMap<>();
|
||||
}
|
||||
public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
recordLocationMap = new ConcurrentHashMap<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
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 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();
|
||||
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) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
|
||||
}
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
|
||||
@Override
|
||||
public WriteStatus call(WriteStatus writeStatus) {
|
||||
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(record.getKey())) {
|
||||
HoodieKey key = record.getKey();
|
||||
java.util.Optional<HoodieRecordLocation> newLocation = record.getNewLocation();
|
||||
if (newLocation.isPresent()) {
|
||||
recordLocationMap.put(key, newLocation.get());
|
||||
} else {
|
||||
//Delete existing index for a deleted record
|
||||
recordLocationMap.remove(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
return writeStatus;
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
|
||||
@Override
|
||||
public WriteStatus call(WriteStatus writeStatus) {
|
||||
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(record.getKey())) {
|
||||
HoodieKey key = record.getKey();
|
||||
java.util.Optional<HoodieRecordLocation> newLocation = record.getNewLocation();
|
||||
if (newLocation.isPresent()) {
|
||||
recordLocationMap.put(key, newLocation.get());
|
||||
} else {
|
||||
//Delete existing index for a deleted record
|
||||
recordLocationMap.remove(key);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
return writeStatus;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
return true;
|
||||
}
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Only looks up by recordKey
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
/**
|
||||
* Only looks up by recordKey
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping is available in HBase already.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
/**
|
||||
* Mapping is available in HBase already.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Index needs to be explicitly updated after storage write.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
/**
|
||||
* Index needs to be explicitly updated after storage write.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,7 +19,6 @@
|
||||
package com.uber.hoodie.index.bloom;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
@@ -27,73 +26,75 @@ import java.io.Serializable;
|
||||
*/
|
||||
public class BloomIndexFileInfo implements Serializable {
|
||||
|
||||
private final String fileName;
|
||||
private final String fileName;
|
||||
|
||||
private final String minRecordKey;
|
||||
private final String minRecordKey;
|
||||
|
||||
private final String maxRecordKey;
|
||||
private final String maxRecordKey;
|
||||
|
||||
public BloomIndexFileInfo(String fileName, String minRecordKey, String maxRecordKey) {
|
||||
this.fileName = fileName;
|
||||
this.minRecordKey = minRecordKey;
|
||||
this.maxRecordKey = maxRecordKey;
|
||||
public BloomIndexFileInfo(String fileName, String minRecordKey, String maxRecordKey) {
|
||||
this.fileName = fileName;
|
||||
this.minRecordKey = minRecordKey;
|
||||
this.maxRecordKey = maxRecordKey;
|
||||
}
|
||||
|
||||
public BloomIndexFileInfo(String fileName) {
|
||||
this.fileName = fileName;
|
||||
this.minRecordKey = null;
|
||||
this.maxRecordKey = null;
|
||||
}
|
||||
|
||||
public String getFileName() {
|
||||
return fileName;
|
||||
}
|
||||
|
||||
public String getMinRecordKey() {
|
||||
return minRecordKey;
|
||||
}
|
||||
|
||||
public String getMaxRecordKey() {
|
||||
return maxRecordKey;
|
||||
}
|
||||
|
||||
public boolean hasKeyRanges() {
|
||||
return minRecordKey != null && maxRecordKey != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Does the given key fall within the range (inclusive)
|
||||
*/
|
||||
public boolean isKeyInRange(String recordKey) {
|
||||
return minRecordKey.compareTo(recordKey) <= 0 &&
|
||||
maxRecordKey.compareTo(recordKey) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
public BloomIndexFileInfo(String fileName) {
|
||||
this.fileName = fileName;
|
||||
this.minRecordKey = null;
|
||||
this.maxRecordKey = null;
|
||||
}
|
||||
BloomIndexFileInfo that = (BloomIndexFileInfo) o;
|
||||
return Objects.equal(that.fileName, fileName) &&
|
||||
Objects.equal(that.minRecordKey, minRecordKey) &&
|
||||
Objects.equal(that.maxRecordKey, maxRecordKey);
|
||||
|
||||
public String getFileName() {
|
||||
return fileName;
|
||||
}
|
||||
}
|
||||
|
||||
public String getMinRecordKey() {
|
||||
return minRecordKey;
|
||||
}
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(fileName, minRecordKey, maxRecordKey);
|
||||
}
|
||||
|
||||
public String getMaxRecordKey() {
|
||||
return maxRecordKey;
|
||||
}
|
||||
|
||||
public boolean hasKeyRanges() {
|
||||
return minRecordKey != null && maxRecordKey != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Does the given key fall within the range (inclusive)
|
||||
* @param recordKey
|
||||
* @return
|
||||
*/
|
||||
public boolean isKeyInRange(String recordKey) {
|
||||
return minRecordKey.compareTo(recordKey) <= 0 &&
|
||||
maxRecordKey.compareTo(recordKey) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
BloomIndexFileInfo that = (BloomIndexFileInfo) o;
|
||||
return Objects.equal(that.fileName, fileName) &&
|
||||
Objects.equal(that.minRecordKey, minRecordKey) &&
|
||||
Objects.equal(that.maxRecordKey, maxRecordKey);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(fileName, minRecordKey, maxRecordKey);
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("BloomIndexFileInfo {");
|
||||
sb.append(" fileName=").append(fileName);
|
||||
sb.append(" minRecordKey=").append(minRecordKey);
|
||||
sb.append(" maxRecordKey=").append(maxRecordKey);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("BloomIndexFileInfo {");
|
||||
sb.append(" fileName=").append(fileName);
|
||||
sb.append(" minRecordKey=").append(minRecordKey);
|
||||
sb.append(" maxRecordKey=").append(maxRecordKey);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,9 +18,12 @@
|
||||
|
||||
package com.uber.hoodie.index.bloom;
|
||||
|
||||
import static java.util.stream.Collectors.groupingBy;
|
||||
import static java.util.stream.Collectors.mapping;
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Optional;
|
||||
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
@@ -34,7 +37,10 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.MetadataNotFoundException;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -42,369 +48,370 @@ 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.storage.StorageLevel;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static java.util.stream.Collectors.*;
|
||||
|
||||
/**
|
||||
* Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in
|
||||
* its metadata.
|
||||
*/
|
||||
public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
|
||||
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 int MAX_ITEMS_PER_SHUFFLE_PARTITION = SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
|
||||
// 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 int MAX_ITEMS_PER_SHUFFLE_PARTITION =
|
||||
SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
|
||||
|
||||
public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
final HoodieTable<T> hoodieTable) {
|
||||
|
||||
// Step 0: cache the input record RDD
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
recordRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTable<T> hoodieTable) {
|
||||
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
|
||||
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
|
||||
|
||||
// Step 0: cache the input record RDD
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
recordRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||
}
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD,
|
||||
hoodieTable);
|
||||
|
||||
// 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, 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;
|
||||
// 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);
|
||||
}
|
||||
|
||||
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()));
|
||||
// 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);
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
||||
lookupIndex(partitionRecordKeyPairRDD, table);
|
||||
|
||||
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);
|
||||
});
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
recordRDD.unpersist(); // unpersist the input Record RDD
|
||||
rowKeyFilenamePairRDD.unpersist();
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 HoodieTable<T> hoodieTable) {
|
||||
// Obtain records per partition, in the incoming records
|
||||
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||
return taggedRecordRDD;
|
||||
}
|
||||
|
||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, hoodieTable);
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
|
||||
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
|
||||
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()));
|
||||
|
||||
// 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);
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
||||
lookupIndex(partitionRecordKeyPairRDD, table);
|
||||
|
||||
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);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 HoodieTable<T> 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, 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);
|
||||
}
|
||||
|
||||
/**
|
||||
* The index lookup can be skewed in three dimensions : #files, #partitions, #records
|
||||
*
|
||||
* 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.
|
||||
*
|
||||
* If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified
|
||||
* as a NON-zero number, then that is used explicitly.
|
||||
*/
|
||||
private int autoComputeParallelism(final Map<String, Long> recordsPerPartition,
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
|
||||
long totalComparisons = 0;
|
||||
if (config.getBloomIndexPruneByRanges()) {
|
||||
// we will just try exploding the input and then count to determine comparisons
|
||||
totalComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo,
|
||||
partitionRecordKeyPairRDD).count();
|
||||
} else {
|
||||
// if not pruning by ranges, then each file in a partition needs to compared against all
|
||||
// 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;
|
||||
for (String partitionPath : recordsPerPartition.keySet()) {
|
||||
long numRecords = recordsPerPartition.get(partitionPath);
|
||||
long numFiles =
|
||||
filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath)
|
||||
: 1L;
|
||||
|
||||
totalComparisons += numFiles * numRecords;
|
||||
totalFiles +=
|
||||
filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath)
|
||||
: 0L;
|
||||
totalRecords += numRecords;
|
||||
}
|
||||
logger.info("TotalRecords: " + totalRecords + ", TotalFiles: " + totalFiles
|
||||
+ ", TotalAffectedPartitions:" + recordsPerPartition.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* The index lookup can be skewed in three dimensions : #files, #partitions, #records
|
||||
*
|
||||
* 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.
|
||||
*
|
||||
* If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified as a NON-zero number,
|
||||
* then that is used explicitly.
|
||||
*
|
||||
*/
|
||||
private int autoComputeParallelism(final Map<String, Long> recordsPerPartition,
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
// each partition will have an item per comparison.
|
||||
int parallelism = (int) (totalComparisons / MAX_ITEMS_PER_SHUFFLE_PARTITION + 1);
|
||||
logger.info(
|
||||
"Auto computed parallelism :" + parallelism + ", totalComparisons: " + totalComparisons);
|
||||
return parallelism;
|
||||
}
|
||||
|
||||
long totalComparisons = 0;
|
||||
if (config.getBloomIndexPruneByRanges()) {
|
||||
// we will just try exploding the input and then count to determine comparisons
|
||||
totalComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairRDD).count();
|
||||
} else {
|
||||
// if not pruning by ranges, then each file in a partition needs to compared against all
|
||||
// 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;
|
||||
for (String partitionPath : recordsPerPartition.keySet()) {
|
||||
long numRecords = recordsPerPartition.get(partitionPath);
|
||||
long numFiles = filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath) : 1L;
|
||||
/**
|
||||
* Its crucial to pick the right parallelism.
|
||||
*
|
||||
* totalSubPartitions : this is deemed safe limit, to be nice with Spark. inputParallelism :
|
||||
* typically number of input file splits
|
||||
*
|
||||
* 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
|
||||
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);
|
||||
return joinParallelism;
|
||||
}
|
||||
|
||||
totalComparisons += numFiles * numRecords;
|
||||
totalFiles += filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath) : 0L;
|
||||
totalRecords += numRecords;
|
||||
}
|
||||
logger.info("TotalRecords: " + totalRecords + ", TotalFiles: " + totalFiles + ", TotalAffectedPartitions:" + recordsPerPartition.size());
|
||||
}
|
||||
|
||||
// each partition will have an item per comparison.
|
||||
int parallelism = (int) (totalComparisons/ MAX_ITEMS_PER_SHUFFLE_PARTITION + 1);
|
||||
logger.info("Auto computed parallelism :" + parallelism + ", totalComparisons: " + totalComparisons);
|
||||
return parallelism;
|
||||
}
|
||||
|
||||
/**
|
||||
* Its crucial to pick the right parallelism.
|
||||
*
|
||||
* totalSubPartitions : this is deemed safe limit, to be nice with Spark.
|
||||
* inputParallelism : typically number of input file splits
|
||||
*
|
||||
* 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
|
||||
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);
|
||||
return joinParallelism;
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair RDD.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, 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.getCommitTimeline().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());
|
||||
}
|
||||
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(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())))
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair RDD.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions,
|
||||
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.getCommitTimeline().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());
|
||||
}
|
||||
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(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())))
|
||||
.collect(toList());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// Nope, don't need to do anything.
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* This is not global, since we depend on the partitionPath to do the lookup
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* No indexes into log files yet.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Bloom filters are stored, into the same data files.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* For each incoming record, produce N output records, 1 each for each file against which the
|
||||
* record's key needs to be checked. For datasets, where the keys have a definite insert order
|
||||
* (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
|
||||
// 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();
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
/**
|
||||
* Find out <RowKey, filename> pair. All workload grouped by file-level.
|
||||
*
|
||||
* 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
|
||||
*
|
||||
* Make sure the parallelism is atleast the groupby parallelism for tagging location
|
||||
*/
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
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);
|
||||
|
||||
return fileSortedTripletRDD
|
||||
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(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()));
|
||||
}
|
||||
return vals.iterator();
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* 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, 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;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// Nope, don't need to do anything.
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* This is not global, since we depend on the partitionPath to do the lookup
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* No indexes into log files yet.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Bloom filters are stored, into the same data files.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
|
||||
* @param indexInfo
|
||||
* @param recordKey
|
||||
* @return
|
||||
*/
|
||||
private boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) {
|
||||
return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* For each incoming record, produce N output records, 1 each for each file against which the record's key
|
||||
* needs to be checked. For datasets, where the keys have a definite insert order (e.g: timestamp as prefix),
|
||||
* the number of files to be compared gets cut down a lot from range pruning.
|
||||
*
|
||||
*
|
||||
* @param partitionToFileIndexInfo
|
||||
* @param partitionRecordKeyPairRDD
|
||||
* @return
|
||||
*/
|
||||
// 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();
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
/**
|
||||
* Find out <RowKey, filename> pair. All workload grouped by file-level.
|
||||
*
|
||||
* 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
|
||||
*
|
||||
* Make sure the parallelism is atleast the groupby parallelism for tagging location
|
||||
*/
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
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);
|
||||
|
||||
return fileSortedTripletRDD
|
||||
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(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()));
|
||||
}
|
||||
return vals.iterator();
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* 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, 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;
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -24,172 +24,182 @@ import com.uber.hoodie.common.util.ParquetUtils;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
import com.uber.hoodie.func.LazyIterableIterator;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Function performing actual checking of RDD parition containing (fileId, hoodieKeys) against the
|
||||
* actual files
|
||||
*/
|
||||
public class HoodieBloomIndexCheckFunction implements Function2<Integer, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>>, Iterator<List<IndexLookupResult>>> {
|
||||
public class HoodieBloomIndexCheckFunction implements
|
||||
Function2<Integer, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>>, Iterator<List<IndexLookupResult>>> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieBloomIndexCheckFunction.class);
|
||||
private static Logger logger = LogManager.getLogger(HoodieBloomIndexCheckFunction.class);
|
||||
|
||||
private final String basePath;
|
||||
private final String basePath;
|
||||
|
||||
public HoodieBloomIndexCheckFunction(String basePath) {
|
||||
this.basePath = basePath;
|
||||
public HoodieBloomIndexCheckFunction(String basePath) {
|
||||
this.basePath = basePath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a list of row keys and one file, return only row keys existing in that file.
|
||||
*/
|
||||
public static List<String> checkCandidatesAgainstFile(List<String> candidateRecordKeys,
|
||||
Path filePath) throws HoodieIndexException {
|
||||
List<String> foundRecordKeys = new ArrayList<>();
|
||||
try {
|
||||
// Load all rowKeys from the file, to double-confirm
|
||||
if (!candidateRecordKeys.isEmpty()) {
|
||||
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath);
|
||||
logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Keys from " + filePath + " => " + fileRowKeys);
|
||||
}
|
||||
for (String rowKey : candidateRecordKeys) {
|
||||
if (fileRowKeys.contains(rowKey)) {
|
||||
foundRecordKeys.add(rowKey);
|
||||
}
|
||||
}
|
||||
logger.info("After checking with row keys, we have " + foundRecordKeys.size()
|
||||
+ " results, for file " + filePath + " => " + foundRecordKeys);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Keys matching for file " + filePath + " => " + foundRecordKeys);
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieIndexException("Error checking candidate keys against file.", e);
|
||||
}
|
||||
return foundRecordKeys;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a list of row keys and one file, return only row keys existing in that file.
|
||||
*/
|
||||
public static List<String> checkCandidatesAgainstFile(List<String> candidateRecordKeys, Path filePath) throws HoodieIndexException {
|
||||
List<String> foundRecordKeys = new ArrayList<>();
|
||||
try {
|
||||
// Load all rowKeys from the file, to double-confirm
|
||||
if (!candidateRecordKeys.isEmpty()) {
|
||||
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath);
|
||||
logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Keys from " + filePath + " => " + fileRowKeys);
|
||||
}
|
||||
for (String rowKey : candidateRecordKeys) {
|
||||
if (fileRowKeys.contains(rowKey)) {
|
||||
foundRecordKeys.add(rowKey);
|
||||
}
|
||||
}
|
||||
logger.info("After checking with row keys, we have " + foundRecordKeys.size() + " results, for file " + filePath + " => " + foundRecordKeys);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Keys matching for file " + filePath + " => " + foundRecordKeys);
|
||||
}
|
||||
}
|
||||
} catch (Exception e){
|
||||
throw new HoodieIndexException("Error checking candidate keys against file.", e);
|
||||
}
|
||||
return foundRecordKeys;
|
||||
class LazyKeyCheckIterator extends
|
||||
LazyIterableIterator<Tuple2<String, Tuple2<String, HoodieKey>>, List<IndexLookupResult>> {
|
||||
|
||||
private List<String> candidateRecordKeys;
|
||||
|
||||
private BloomFilter bloomFilter;
|
||||
|
||||
private String currentFile;
|
||||
|
||||
private String currentParitionPath;
|
||||
|
||||
LazyKeyCheckIterator(
|
||||
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr) {
|
||||
super(fileParitionRecordKeyTripletItr);
|
||||
currentFile = null;
|
||||
candidateRecordKeys = new ArrayList<>();
|
||||
bloomFilter = null;
|
||||
currentParitionPath = null;
|
||||
}
|
||||
|
||||
class LazyKeyCheckIterator extends LazyIterableIterator<Tuple2<String, Tuple2<String, HoodieKey>>, List<IndexLookupResult>> {
|
||||
|
||||
private List<String> candidateRecordKeys;
|
||||
|
||||
private BloomFilter bloomFilter;
|
||||
|
||||
private String currentFile;
|
||||
|
||||
private String currentParitionPath;
|
||||
|
||||
LazyKeyCheckIterator(Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr) {
|
||||
super(fileParitionRecordKeyTripletItr);
|
||||
currentFile = null;
|
||||
candidateRecordKeys = new ArrayList<>();
|
||||
bloomFilter = null;
|
||||
currentParitionPath = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void start() {
|
||||
}
|
||||
|
||||
private void initState(String fileName, String partitionPath) throws HoodieIndexException {
|
||||
try {
|
||||
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
|
||||
bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(filePath);
|
||||
candidateRecordKeys = new ArrayList<>();
|
||||
currentFile = fileName;
|
||||
currentParitionPath = partitionPath;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieIndexException("Error checking candidate keys against file.", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<IndexLookupResult> computeNext() {
|
||||
|
||||
List<IndexLookupResult> ret = new ArrayList<>();
|
||||
try {
|
||||
// process one file in each go.
|
||||
while (inputItr.hasNext()) {
|
||||
|
||||
Tuple2<String, Tuple2<String, HoodieKey>> currentTuple = inputItr.next();
|
||||
String fileName = currentTuple._2._1;
|
||||
String partitionPath = currentTuple._2._2.getPartitionPath();
|
||||
String recordKey = currentTuple._2._2.getRecordKey();
|
||||
|
||||
// lazily init state
|
||||
if (currentFile == null) {
|
||||
initState(fileName, partitionPath);
|
||||
}
|
||||
|
||||
// if continue on current file)
|
||||
if (fileName.equals(currentFile)) {
|
||||
// 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);
|
||||
}
|
||||
candidateRecordKeys.add(recordKey);
|
||||
}
|
||||
} else {
|
||||
// do the actual checking of file & break out
|
||||
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
|
||||
logger.info("#1 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys.size() + " for " + filePath);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
|
||||
}
|
||||
ret.add(new IndexLookupResult(currentFile, checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
|
||||
|
||||
initState(fileName, partitionPath);
|
||||
if (bloomFilter.mightContain(recordKey)) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("#2 Adding " + recordKey + " as candidate for file " + fileName);
|
||||
}
|
||||
candidateRecordKeys.add(recordKey);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// handle case, where we ran out of input, finish pending work, update return val
|
||||
if (!inputItr.hasNext()) {
|
||||
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
|
||||
logger.info("#2 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys.size() + " for " + filePath);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
|
||||
}
|
||||
ret.add(new IndexLookupResult(currentFile, checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
|
||||
}
|
||||
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieIndexException("Error checking bloom filter index. ", e);
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void end() {
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Iterator<List<IndexLookupResult>> call(Integer partition,
|
||||
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr) throws Exception {
|
||||
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
|
||||
protected void start() {
|
||||
}
|
||||
|
||||
private void initState(String fileName, String partitionPath) throws HoodieIndexException {
|
||||
try {
|
||||
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
|
||||
bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(filePath);
|
||||
candidateRecordKeys = new ArrayList<>();
|
||||
currentFile = fileName;
|
||||
currentParitionPath = partitionPath;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieIndexException("Error checking candidate keys against file.", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<IndexLookupResult> computeNext() {
|
||||
|
||||
List<IndexLookupResult> ret = new ArrayList<>();
|
||||
try {
|
||||
// process one file in each go.
|
||||
while (inputItr.hasNext()) {
|
||||
|
||||
Tuple2<String, Tuple2<String, HoodieKey>> currentTuple = inputItr.next();
|
||||
String fileName = currentTuple._2._1;
|
||||
String partitionPath = currentTuple._2._2.getPartitionPath();
|
||||
String recordKey = currentTuple._2._2.getRecordKey();
|
||||
|
||||
// lazily init state
|
||||
if (currentFile == null) {
|
||||
initState(fileName, partitionPath);
|
||||
}
|
||||
|
||||
// if continue on current file)
|
||||
if (fileName.equals(currentFile)) {
|
||||
// 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);
|
||||
}
|
||||
candidateRecordKeys.add(recordKey);
|
||||
}
|
||||
} else {
|
||||
// do the actual checking of file & break out
|
||||
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
|
||||
logger.info(
|
||||
"#1 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys
|
||||
.size() + " for " + filePath);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger
|
||||
.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
|
||||
}
|
||||
ret.add(new IndexLookupResult(currentFile,
|
||||
checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
|
||||
|
||||
initState(fileName, partitionPath);
|
||||
if (bloomFilter.mightContain(recordKey)) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("#2 Adding " + recordKey + " as candidate for file " + fileName);
|
||||
}
|
||||
candidateRecordKeys.add(recordKey);
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// handle case, where we ran out of input, finish pending work, update return val
|
||||
if (!inputItr.hasNext()) {
|
||||
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
|
||||
logger.info(
|
||||
"#2 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys
|
||||
.size() + " for " + filePath);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
|
||||
}
|
||||
ret.add(new IndexLookupResult(currentFile,
|
||||
checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
|
||||
}
|
||||
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieIndexException("Error checking bloom filter index. ", e);
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void end() {
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Iterator<List<IndexLookupResult>> call(Integer partition,
|
||||
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr)
|
||||
throws Exception {
|
||||
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -25,21 +25,21 @@ import java.util.List;
|
||||
*/
|
||||
public class IndexLookupResult {
|
||||
|
||||
private String fileName;
|
||||
private String fileName;
|
||||
|
||||
|
||||
private List<String> matchingRecordKeys;
|
||||
private List<String> matchingRecordKeys;
|
||||
|
||||
public IndexLookupResult(String fileName, List<String> matchingRecordKeys) {
|
||||
this.fileName = fileName;
|
||||
this.matchingRecordKeys = matchingRecordKeys;
|
||||
}
|
||||
public IndexLookupResult(String fileName, List<String> matchingRecordKeys) {
|
||||
this.fileName = fileName;
|
||||
this.matchingRecordKeys = matchingRecordKeys;
|
||||
}
|
||||
|
||||
public String getFileName() {
|
||||
return fileName;
|
||||
}
|
||||
public String getFileName() {
|
||||
return fileName;
|
||||
}
|
||||
|
||||
public List<String> getMatchingRecordKeys() {
|
||||
return matchingRecordKeys;
|
||||
}
|
||||
public List<String> getMatchingRecordKeys() {
|
||||
return matchingRecordKeys;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,7 +19,6 @@
|
||||
package com.uber.hoodie.index.bucketed;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
@@ -29,96 +28,86 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* An `stateless` index implementation that will using a deterministic mapping function to
|
||||
* determine the fileID for a given record.
|
||||
*
|
||||
* Pros:
|
||||
* - Fast
|
||||
*
|
||||
* 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.
|
||||
*
|
||||
* An `stateless` index implementation that will using a deterministic mapping function to determine
|
||||
* the fileID for a given record.
|
||||
*
|
||||
* Pros: - Fast
|
||||
*
|
||||
* 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.
|
||||
*/
|
||||
public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(BucketedIndex.class);
|
||||
private static Logger logger = LogManager.getLogger(BucketedIndex.class);
|
||||
|
||||
public BucketedIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
}
|
||||
public BucketedIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
}
|
||||
|
||||
private String getBucket(String recordKey) {
|
||||
return String.valueOf(recordKey.hashCode() % config.getNumBucketsPerPartition());
|
||||
}
|
||||
private String getBucket(String recordKey) {
|
||||
return String.valueOf(recordKey.hashCode() % config.getNumBucketsPerPartition());
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
|
||||
return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey()))));
|
||||
}
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
HoodieTable<T> table) {
|
||||
return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey()))));
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieTable<T> hoodieTable) throws HoodieIndexException {
|
||||
return recordRDD.map(record -> {
|
||||
String bucket = getBucket(record.getRecordKey());
|
||||
//HACK(vc) a non-existent commit is provided here.
|
||||
record.setCurrentLocation(new HoodieRecordLocation("000", bucket));
|
||||
return record;
|
||||
});
|
||||
}
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException {
|
||||
return recordRDD.map(record -> {
|
||||
String bucket = getBucket(record.getRecordKey());
|
||||
//HACK(vc) a non-existent commit is provided here.
|
||||
record.setCurrentLocation(new HoodieRecordLocation("000", bucket));
|
||||
return record;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> hoodieTable) throws HoodieIndexException {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// nothing to rollback in the index.
|
||||
return true;
|
||||
}
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// nothing to rollback in the index.
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Bucketing is still done within each partition.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return false;
|
||||
}
|
||||
/**
|
||||
* Bucketing is still done within each partition.
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Since indexing is just a deterministic hash, we can identify file group correctly even without an index
|
||||
* on the actual log file.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
/**
|
||||
* Since indexing is just a deterministic hash, we can identify file group correctly even without
|
||||
* an index on the actual log file.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Indexing is just a hash function.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return true;
|
||||
}
|
||||
/**
|
||||
* Indexing is just a hash function.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,24 +19,33 @@
|
||||
package com.uber.hoodie.index.hbase;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
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.List;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.*;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -45,230 +54,221 @@ 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.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 Logger logger = LogManager.getLogger(HBaseIndex.class);
|
||||
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 final String tableName;
|
||||
private static Logger logger = LogManager.getLogger(HBaseIndex.class);
|
||||
|
||||
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
this.tableName = config.getProps().getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP);
|
||||
private final String tableName;
|
||||
|
||||
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
this.tableName = config.getProps().getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
|
||||
throw new UnsupportedOperationException("HBase index does not implement check exist yet");
|
||||
}
|
||||
|
||||
private static Connection hbaseConnection = null;
|
||||
|
||||
private Connection getHBaseConnection() {
|
||||
Configuration hbaseConfig = HBaseConfiguration.create();
|
||||
String quorum = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP);
|
||||
hbaseConfig.set("hbase.zookeeper.quorum", quorum);
|
||||
String port = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP);
|
||||
hbaseConfig.set("hbase.zookeeper.property.clientPort", port);
|
||||
try {
|
||||
return ConnectionFactory.createConnection(hbaseConfig);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieDependentSystemUnavailableException(
|
||||
HoodieDependentSystemUnavailableException.HBASE, quorum + ":" + port);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
class LocationTagFunction
|
||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||
|
||||
private final HoodieTable<T> hoodieTable;
|
||||
|
||||
LocationTagFunction(HoodieTable<T> hoodieTable) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
|
||||
throw new UnsupportedOperationException("HBase index does not implement check exist yet");
|
||||
}
|
||||
|
||||
private static Connection hbaseConnection = null;
|
||||
|
||||
private Connection getHBaseConnection() {
|
||||
Configuration hbaseConfig = HBaseConfiguration.create();
|
||||
String quorum = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP);
|
||||
hbaseConfig.set("hbase.zookeeper.quorum", quorum);
|
||||
String port = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP);
|
||||
hbaseConfig.set("hbase.zookeeper.property.clientPort", port);
|
||||
try {
|
||||
return ConnectionFactory.createConnection(hbaseConfig);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieDependentSystemUnavailableException(
|
||||
HoodieDependentSystemUnavailableException.HBASE, quorum + ":" + port);
|
||||
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
|
||||
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
||||
// Grab the global HBase connection
|
||||
synchronized (HBaseIndex.class) {
|
||||
if (hbaseConnection == null) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
}
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
HTable hTable = null;
|
||||
try {
|
||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||
// Do the tagging.
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord rec = hoodieRecordIterator.next();
|
||||
// TODO(vc): This may need to be a multi get.
|
||||
Result result = hTable.get(
|
||||
new Get(Bytes.toBytes(rec.getRecordKey())).setMaxVersions(1)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
class LocationTagFunction
|
||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||
// first, attempt to grab location from HBase
|
||||
if (result.getRow() != null) {
|
||||
String commitTs =
|
||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
|
||||
String fileId =
|
||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
||||
|
||||
private final HoodieTable<T> hoodieTable;
|
||||
|
||||
LocationTagFunction(HoodieTable<T> hoodieTable) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
// if the last commit ts for this row is less than the system commit ts
|
||||
if (!commitTimeline.empty() && commitTimeline.containsInstant(
|
||||
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))) {
|
||||
rec.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
|
||||
}
|
||||
}
|
||||
taggedRecords.add(rec);
|
||||
}
|
||||
} 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
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
|
||||
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
||||
// Grab the global HBase connection
|
||||
synchronized (HBaseIndex.class) {
|
||||
if (hbaseConnection == null) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
HTable hTable = null;
|
||||
try {
|
||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||
// Do the tagging.
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord rec = hoodieRecordIterator.next();
|
||||
// TODO(vc): This may need to be a multi get.
|
||||
Result result = hTable.get(
|
||||
new Get(Bytes.toBytes(rec.getRecordKey())).setMaxVersions(1)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
}
|
||||
}
|
||||
|
||||
// first, attempt to grab location from HBase
|
||||
if (result.getRow() != null) {
|
||||
String commitTs =
|
||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
|
||||
String fileId =
|
||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(hoodieTable), true);
|
||||
}
|
||||
|
||||
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
// if the last commit ts for this row is less than the system commit ts
|
||||
if (!commitTimeline.empty() && commitTimeline.containsInstant(
|
||||
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))) {
|
||||
rec.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
|
||||
}
|
||||
}
|
||||
taggedRecords.add(rec);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException(
|
||||
"Failed to Tag indexed locations because of exception with HBase Client", e);
|
||||
}
|
||||
class UpdateLocationTask implements
|
||||
Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> {
|
||||
|
||||
finally {
|
||||
if (hTable != null) {
|
||||
try {
|
||||
hTable.close();
|
||||
} catch (IOException e) {
|
||||
// Ignore
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public Iterator<WriteStatus> call(Integer partition, Iterator<WriteStatus> statusIterator) {
|
||||
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
List<WriteStatus> writeStatusList = new ArrayList<>();
|
||||
// Grab the global HBase connection
|
||||
synchronized (HBaseIndex.class) {
|
||||
if (hbaseConnection == null) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieTable<T> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(hoodieTable), true);
|
||||
}
|
||||
|
||||
class UpdateLocationTask implements Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> {
|
||||
@Override
|
||||
public Iterator<WriteStatus> call(Integer partition, Iterator<WriteStatus> statusIterator) {
|
||||
|
||||
List<WriteStatus> writeStatusList = new ArrayList<>();
|
||||
// Grab the global HBase connection
|
||||
synchronized (HBaseIndex.class) {
|
||||
if (hbaseConnection == null) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
HTable hTable = null;
|
||||
try {
|
||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||
while (statusIterator.hasNext()) {
|
||||
WriteStatus writeStatus = statusIterator.next();
|
||||
List<Put> puts = new ArrayList<>();
|
||||
List<Delete> deletes = new ArrayList<>();
|
||||
try {
|
||||
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(rec.getKey())) {
|
||||
java.util.Optional<HoodieRecordLocation> loc = rec.getNewLocation();
|
||||
if (loc.isPresent()) {
|
||||
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
|
||||
Bytes.toBytes(loc.get().getCommitTime()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
|
||||
Bytes.toBytes(loc.get().getFileId()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
|
||||
Bytes.toBytes(rec.getPartitionPath()));
|
||||
puts.add(put);
|
||||
} else {
|
||||
//Delete existing index for a deleted record
|
||||
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
|
||||
deletes.add(delete);
|
||||
}
|
||||
}
|
||||
}
|
||||
HTable hTable = null;
|
||||
try {
|
||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||
while (statusIterator.hasNext()) {
|
||||
WriteStatus writeStatus = statusIterator.next();
|
||||
List<Put> puts = new ArrayList<>();
|
||||
List<Delete> deletes = new ArrayList<>();
|
||||
try {
|
||||
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(rec.getKey())) {
|
||||
java.util.Optional<HoodieRecordLocation> loc = rec.getNewLocation();
|
||||
if(loc.isPresent()) {
|
||||
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
|
||||
Bytes.toBytes(loc.get().getCommitTime()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
|
||||
Bytes.toBytes(loc.get().getFileId()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
|
||||
Bytes.toBytes(rec.getPartitionPath()));
|
||||
puts.add(put);
|
||||
} else {
|
||||
//Delete existing index for a deleted record
|
||||
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
|
||||
deletes.add(delete);
|
||||
}
|
||||
}
|
||||
}
|
||||
hTable.put(puts);
|
||||
hTable.delete(deletes);
|
||||
hTable.flushCommits();
|
||||
} catch (Exception e) {
|
||||
Exception we = new Exception("Error updating index for " + writeStatus, e);
|
||||
logger.error(we);
|
||||
writeStatus.setGlobalError(we);
|
||||
}
|
||||
writeStatusList.add(writeStatus);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException(
|
||||
"Failed to Update Index locations because of exception with HBase Client", e);
|
||||
} finally {
|
||||
if (hTable != null) {
|
||||
try {
|
||||
hTable.close();
|
||||
} catch (IOException e) {
|
||||
// Ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
return writeStatusList.iterator();
|
||||
hTable.put(puts);
|
||||
hTable.delete(deletes);
|
||||
hTable.flushCommits();
|
||||
} catch (Exception e) {
|
||||
Exception we = new Exception("Error updating index for " + writeStatus, e);
|
||||
logger.error(we);
|
||||
writeStatus.setGlobalError(we);
|
||||
}
|
||||
writeStatusList.add(writeStatus);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException(
|
||||
"Failed to Update Index locations because of exception with HBase Client", e);
|
||||
} finally {
|
||||
if (hTable != null) {
|
||||
try {
|
||||
hTable.close();
|
||||
} catch (IOException e) {
|
||||
// Ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
return writeStatusList.iterator();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true);
|
||||
}
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// Can't really rollback here. HBase only can let you go from recordKey to fileID,
|
||||
// not the other way around
|
||||
return true;
|
||||
}
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// Can't really rollback here. HBase only can let you go from recordKey to fileID,
|
||||
// not the other way around
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Only looks up by recordKey
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
/**
|
||||
* Only looks up by recordKey
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping is available in HBase already.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
/**
|
||||
* Mapping is available in HBase already.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Index needs to be explicitly updated after storage write.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
/**
|
||||
* Index needs to be explicitly updated after storage write.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user