CodeStyle formatting to conform to basic Checkstyle rules.
The code-style rules follow google style with some changes: 1. Increase line length from 100 to 120 2. Disable JavaDoc related checkstyles as this needs more manual work. Both source and test code are checked for code-style
This commit is contained in:
committed by
vinoth chandar
parent
987f5d6b96
commit
788e4f2d2e
@@ -50,17 +50,17 @@ import scala.Tuple2;
|
||||
*/
|
||||
public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieReadClient.class);
|
||||
private static final Logger logger = LogManager.getLogger(HoodieReadClient.class);
|
||||
|
||||
private transient final JavaSparkContext jsc;
|
||||
private final transient JavaSparkContext jsc;
|
||||
|
||||
private transient final FileSystem fs;
|
||||
private final transient FileSystem fs;
|
||||
/**
|
||||
* TODO: We need to persist the index type into hoodie.properties and be able to access the index
|
||||
* just with a simple basepath pointing to the dataset. Until, then just always assume a
|
||||
* BloomIndex
|
||||
*/
|
||||
private transient final HoodieIndex<T> index;
|
||||
private final transient HoodieIndex<T> index;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTable hoodieTable;
|
||||
private transient Optional<SQLContext> sqlContextOpt;
|
||||
@@ -69,8 +69,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
|
||||
* @param basePath path to Hoodie dataset
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
|
||||
this(jsc, HoodieWriteConfig.newBuilder()
|
||||
.withPath(basePath)
|
||||
this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
// by default we use HoodieBloomIndex
|
||||
.withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
@@ -78,7 +77,6 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param jsc
|
||||
* @param basePath
|
||||
* @param sqlContext
|
||||
@@ -96,9 +94,9 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
|
||||
this.jsc = jsc;
|
||||
this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
this.hoodieTable = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true),
|
||||
clientConfig);
|
||||
this.hoodieTable = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true),
|
||||
clientConfig);
|
||||
this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants();
|
||||
this.index = HoodieIndex.createIndex(clientConfig, jsc);
|
||||
this.sqlContextOpt = Optional.absent();
|
||||
@@ -126,33 +124,27 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
|
||||
*
|
||||
* @return a dataframe
|
||||
*/
|
||||
public Dataset<Row> read(JavaRDD<HoodieKey> hoodieKeys, int parallelism)
|
||||
throws Exception {
|
||||
public Dataset<Row> read(JavaRDD<HoodieKey> hoodieKeys, int parallelism) throws Exception {
|
||||
|
||||
assertSqlContext();
|
||||
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
|
||||
index.fetchRecordLocation(hoodieKeys, hoodieTable);
|
||||
List<String> paths = keyToFileRDD
|
||||
.filter(keyFileTuple -> keyFileTuple._2().isPresent())
|
||||
.map(keyFileTuple -> keyFileTuple._2().get())
|
||||
.collect();
|
||||
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD = index
|
||||
.fetchRecordLocation(hoodieKeys, hoodieTable);
|
||||
List<String> paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent())
|
||||
.map(keyFileTuple -> keyFileTuple._2().get()).collect();
|
||||
|
||||
// record locations might be same for multiple keys, so need a unique list
|
||||
Set<String> uniquePaths = new HashSet<>(paths);
|
||||
Dataset<Row> originalDF = sqlContextOpt.get().read()
|
||||
.parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
|
||||
StructType schema = originalDF.schema();
|
||||
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD()
|
||||
.mapToPair(row -> {
|
||||
HoodieKey key = new HoodieKey(
|
||||
row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
|
||||
row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
|
||||
return new Tuple2<>(key, row);
|
||||
});
|
||||
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD().mapToPair(row -> {
|
||||
HoodieKey key = new HoodieKey(row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
|
||||
row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
|
||||
return new Tuple2<>(key, row);
|
||||
});
|
||||
|
||||
// Now, we need to further filter out, for only rows that match the supplied hoodie keys
|
||||
JavaRDD<Row> rowRDD = keyRowRDD.join(keyToFileRDD, parallelism)
|
||||
.map(tuple -> tuple._2()._1());
|
||||
JavaRDD<Row> rowRDD = keyRowRDD.join(keyToFileRDD, parallelism).map(tuple -> tuple._2()._1());
|
||||
|
||||
return sqlContextOpt.get().createDataFrame(rowRDD, schema);
|
||||
}
|
||||
|
||||
@@ -81,18 +81,18 @@ import scala.Tuple2;
|
||||
/**
|
||||
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient
|
||||
* mutations on a HDFS dataset [upsert()]
|
||||
*
|
||||
* <p>
|
||||
* Note that, at any given time, there can only be one Spark job performing these operatons on a
|
||||
* Hoodie dataset.
|
||||
*/
|
||||
public class HoodieWriteClient<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieWriteClient.class);
|
||||
private transient final FileSystem fs;
|
||||
private transient final JavaSparkContext jsc;
|
||||
private final transient FileSystem fs;
|
||||
private final transient JavaSparkContext jsc;
|
||||
private final HoodieWriteConfig config;
|
||||
private transient final HoodieMetrics metrics;
|
||||
private transient final HoodieIndex<T> index;
|
||||
private final transient HoodieMetrics metrics;
|
||||
private final transient HoodieIndex<T> index;
|
||||
private transient Timer.Context writeContext = null;
|
||||
|
||||
/**
|
||||
@@ -100,8 +100,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @param clientConfig
|
||||
* @throws Exception
|
||||
*/
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig)
|
||||
throws Exception {
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) throws Exception {
|
||||
this(jsc, clientConfig, false);
|
||||
}
|
||||
|
||||
@@ -129,6 +128,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
}
|
||||
|
||||
public static SparkConf registerClasses(SparkConf conf) {
|
||||
conf.registerKryoClasses(
|
||||
new Class[] {HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
|
||||
return conf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Filter out HoodieRecords that already exists in the output folder. This is useful in
|
||||
* deduplication.
|
||||
@@ -139,8 +144,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, table);
|
||||
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
||||
@@ -153,9 +157,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
HoodieTable<T> table = getTableAndInitCtx();
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
combineOnCondition(config.shouldCombineBeforeUpsert(), records,
|
||||
config.getUpsertShuffleParallelism());
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords = combineOnCondition(
|
||||
config.shouldCombineBeforeUpsert(), records, config.getUpsertShuffleParallelism());
|
||||
|
||||
// perform index loop up to get existing location of records
|
||||
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, table);
|
||||
@@ -170,12 +173,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
/**
|
||||
* Upserts the given prepared records into the Hoodie table, at the supplied commitTime.
|
||||
*
|
||||
* <p>
|
||||
* This implementation requires that the input records are already tagged, and de-duped if
|
||||
* needed.
|
||||
*
|
||||
* @param preppedRecords Prepared HoodieRecords to upsert
|
||||
* @param commitTime Commit Time handle
|
||||
* @param commitTime Commit Time handle
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords,
|
||||
@@ -187,19 +190,19 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
if (e instanceof HoodieUpsertException) {
|
||||
throw (HoodieUpsertException) e;
|
||||
}
|
||||
throw new HoodieUpsertException("Failed to upsert prepared records for commit time " +
|
||||
commitTime, e);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to upsert prepared records for commit time " + commitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Inserts the given HoodieRecords, into the table. This API is intended to be used for normal
|
||||
* writes.
|
||||
*
|
||||
* <p>
|
||||
* This implementation skips the index check and is able to leverage benefits such as small file
|
||||
* handling/blocking alignment, as with upsert(), by profiling the workload
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
@@ -207,9 +210,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
HoodieTable<T> table = getTableAndInitCtx();
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
combineOnCondition(config.shouldCombineBeforeInsert(), records,
|
||||
config.getInsertShuffleParallelism());
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords = combineOnCondition(
|
||||
config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism());
|
||||
|
||||
return upsertRecordsInternal(dedupedRecords, commitTime, table, false);
|
||||
} catch (Throwable e) {
|
||||
@@ -222,13 +224,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
/**
|
||||
* Inserts the given prepared records into the Hoodie table, at the supplied commitTime.
|
||||
*
|
||||
* <p>
|
||||
* This implementation skips the index check, skips de-duping and is able to leverage benefits
|
||||
* such as small file handling/blocking alignment, as with insert(), by profiling the workload.
|
||||
* The prepared HoodieRecords should be de-duped if needed.
|
||||
*
|
||||
* @param preppedRecords HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @param commitTime Commit Time handle
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> insertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords,
|
||||
@@ -240,8 +242,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to insert prepared records for commit time " +
|
||||
commitTime, e);
|
||||
throw new HoodieInsertException(
|
||||
"Failed to insert prepared records for commit time " + commitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -249,12 +251,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk
|
||||
* loads into a Hoodie table for the very first time (e.g: converting an existing dataset to
|
||||
* Hoodie).
|
||||
*
|
||||
* <p>
|
||||
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and
|
||||
* attempts to control the numbers of files with less memory compared to the {@link
|
||||
* HoodieWriteClient#insert(JavaRDD, String)}
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
@@ -267,36 +269,33 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk
|
||||
* loads into a Hoodie table for the very first time (e.g: converting an existing dataset to
|
||||
* Hoodie).
|
||||
*
|
||||
* <p>
|
||||
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and
|
||||
* attempts to control the numbers of files with less memory compared to the {@link
|
||||
* HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own
|
||||
* partitioner. If specified then it will be used for repartitioning records. See {@link
|
||||
* UserDefinedBulkInsertPartitioner}.
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @param bulkInsertPartitioner If specified then it will be used to partition input records
|
||||
* before they are inserted into hoodie.
|
||||
* before they are inserted into hoodie.
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records,
|
||||
final String commitTime,
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime,
|
||||
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
HoodieTable<T> table = getTableAndInitCtx();
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
combineOnCondition(config.shouldCombineBeforeInsert(), records,
|
||||
config.getInsertShuffleParallelism());
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords = combineOnCondition(
|
||||
config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism());
|
||||
|
||||
return bulkInsertInternal(dedupedRecords, commitTime, table, bulkInsertPartitioner);
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime,
|
||||
e);
|
||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -304,22 +303,21 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk
|
||||
* loads into a Hoodie table for the very first time (e.g: converting an existing dataset to
|
||||
* Hoodie). The input records should contain no duplicates if needed.
|
||||
*
|
||||
* <p>
|
||||
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and
|
||||
* attempts to control the numbers of files with less memory compared to the {@link
|
||||
* HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own
|
||||
* partitioner. If specified then it will be used for repartitioning records. See {@link
|
||||
* UserDefinedBulkInsertPartitioner}.
|
||||
*
|
||||
* @param preppedRecords HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @param preppedRecords HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @param bulkInsertPartitioner If specified then it will be used to partition input records
|
||||
* before they are inserted into hoodie.
|
||||
* before they are inserted into hoodie.
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> bulkInsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords,
|
||||
final String commitTime,
|
||||
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
final String commitTime, Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
HoodieTable<T> table = getTableAndInitCtx();
|
||||
try {
|
||||
return bulkInsertInternal(preppedRecords, commitTime, table, bulkInsertPartitioner);
|
||||
@@ -327,35 +325,29 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to bulk insert prepared records for commit time " +
|
||||
commitTime, e);
|
||||
throw new HoodieInsertException(
|
||||
"Failed to bulk insert prepared records for commit time " + commitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
private JavaRDD<WriteStatus> bulkInsertInternal(
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords,
|
||||
String commitTime,
|
||||
HoodieTable<T> table,
|
||||
private JavaRDD<WriteStatus> bulkInsertInternal(JavaRDD<HoodieRecord<T>> dedupedRecords,
|
||||
String commitTime, HoodieTable<T> table,
|
||||
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
final JavaRDD<HoodieRecord<T>> repartitionedRecords;
|
||||
if (bulkInsertPartitioner.isDefined()) {
|
||||
repartitionedRecords =
|
||||
bulkInsertPartitioner.get().repartitionRecords(dedupedRecords,
|
||||
config.getBulkInsertShuffleParallelism());
|
||||
repartitionedRecords = bulkInsertPartitioner.get()
|
||||
.repartitionRecords(dedupedRecords, config.getBulkInsertShuffleParallelism());
|
||||
} else {
|
||||
// Now, sort the records and line them up nicely for loading.
|
||||
repartitionedRecords = dedupedRecords
|
||||
.sortBy(record -> {
|
||||
// Let's use "partitionPath + key" as the sort key. Spark, will ensure
|
||||
// the records split evenly across RDD partitions, such that small partitions fit
|
||||
// into 1 RDD partition, while big ones spread evenly across multiple RDD partitions
|
||||
return String
|
||||
.format("%s+%s", record.getPartitionPath(), record.getRecordKey());
|
||||
}, true, config.getBulkInsertShuffleParallelism());
|
||||
repartitionedRecords = dedupedRecords.sortBy(record -> {
|
||||
// Let's use "partitionPath + key" as the sort key. Spark, will ensure
|
||||
// the records split evenly across RDD partitions, such that small partitions fit
|
||||
// into 1 RDD partition, while big ones spread evenly across multiple RDD partitions
|
||||
return String.format("%s+%s", record.getPartitionPath(), record.getRecordKey());
|
||||
}, true, config.getBulkInsertShuffleParallelism());
|
||||
}
|
||||
JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
|
||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table),
|
||||
true)
|
||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table), true)
|
||||
.flatMap(writeStatuses -> writeStatuses.iterator());
|
||||
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime);
|
||||
@@ -375,8 +367,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
private JavaRDD<HoodieRecord<T>> combineOnCondition(boolean condition,
|
||||
JavaRDD<HoodieRecord<T>> records,
|
||||
int parallelism) {
|
||||
JavaRDD<HoodieRecord<T>> records, int parallelism) {
|
||||
if (condition) {
|
||||
return deduplicateRecords(records, parallelism);
|
||||
}
|
||||
@@ -390,8 +381,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* files) are rolled back based on commit time. // TODO : Create a new WorkloadProfile metadata
|
||||
* file instead of using HoodieCommitMetadata
|
||||
*/
|
||||
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile,
|
||||
HoodieTable<T> table, String commitTime) throws HoodieCommitException {
|
||||
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, HoodieTable<T> table,
|
||||
String commitTime) throws HoodieCommitException {
|
||||
try {
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
profile.getPartitionPaths().stream().forEach(path -> {
|
||||
@@ -416,9 +407,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
private JavaRDD<WriteStatus> upsertRecordsInternal(JavaRDD<HoodieRecord<T>> preppedRecords,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
final boolean isUpsert) {
|
||||
String commitTime, HoodieTable<T> hoodieTable, final boolean isUpsert) {
|
||||
|
||||
// Cache the tagged records, so we don't end up computing both
|
||||
// TODO: Consistent contract in HoodieWriteClient regarding preppedRecord storage level handling
|
||||
@@ -441,20 +430,16 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords
|
||||
.mapPartitionsWithIndex((partition, recordItr) -> {
|
||||
if (isUpsert) {
|
||||
return hoodieTable
|
||||
.handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
return hoodieTable.handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
} else {
|
||||
return hoodieTable
|
||||
.handleInsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
return hoodieTable.handleInsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
}
|
||||
}, true)
|
||||
.flatMap(writeStatuses -> writeStatuses.iterator());
|
||||
}, true).flatMap(writeStatuses -> writeStatuses.iterator());
|
||||
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime);
|
||||
}
|
||||
|
||||
private Partitioner getPartitioner(HoodieTable table, boolean isUpsert,
|
||||
WorkloadProfile profile) {
|
||||
private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, WorkloadProfile profile) {
|
||||
if (isUpsert) {
|
||||
return table.getUpsertPartitioner(profile);
|
||||
} else {
|
||||
@@ -474,13 +459,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords,
|
||||
Partitioner partitioner) {
|
||||
return dedupedRecords
|
||||
.mapToPair(record ->
|
||||
new Tuple2<>(
|
||||
new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())),
|
||||
record))
|
||||
.partitionBy(partitioner)
|
||||
.map(tuple -> tuple._2());
|
||||
return dedupedRecords.mapToPair(record -> new Tuple2<>(
|
||||
new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record))
|
||||
.partitionBy(partitioner).map(tuple -> tuple._2());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -493,12 +474,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
/**
|
||||
* Commit changes performed at the given commitTime marker
|
||||
*/
|
||||
public boolean commit(String commitTime,
|
||||
JavaRDD<WriteStatus> writeStatuses,
|
||||
public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses,
|
||||
Optional<HashMap<String, String>> extraMetadata) {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
return commit(commitTime, writeStatuses, extraMetadata, table.getCommitActionType());
|
||||
}
|
||||
|
||||
@@ -508,15 +487,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
logger.info("Commiting " + commitTime);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
|
||||
List<Tuple2<String, HoodieWriteStat>> stats = writeStatuses
|
||||
.mapToPair((PairFunction<WriteStatus, String, HoodieWriteStat>) writeStatus ->
|
||||
new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat()))
|
||||
.collect();
|
||||
List<Tuple2<String, HoodieWriteStat>> stats = writeStatuses.mapToPair(
|
||||
(PairFunction<WriteStatus, String, HoodieWriteStat>) writeStatus -> new Tuple2<>(
|
||||
writeStatus.getPartitionPath(), writeStatus.getStat())).collect();
|
||||
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
for (Tuple2<String, HoodieWriteStat> stat : stats) {
|
||||
@@ -529,10 +506,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
if (finalizeCtx != null && result.isPresent()) {
|
||||
Optional<Long> durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop()));
|
||||
durationInMs.ifPresent(duration -> {
|
||||
logger.info("Finalize write elapsed time (milliseconds): " + duration);
|
||||
metrics.updateFinalizeWriteMetrics(duration, result.get());
|
||||
}
|
||||
);
|
||||
logger.info("Finalize write elapsed time (milliseconds): " + duration);
|
||||
metrics.updateFinalizeWriteMetrics(duration, result.get());
|
||||
});
|
||||
}
|
||||
|
||||
// add in extra metadata
|
||||
@@ -541,8 +517,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
try {
|
||||
activeTimeline.saveAsComplete(
|
||||
new HoodieInstant(true, actionType, commitTime),
|
||||
activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, commitTime),
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Save was a success
|
||||
// Do a inline compaction if enabled
|
||||
@@ -566,9 +541,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
if (writeContext != null) {
|
||||
long durationInMs = metrics.getDurationInMs(writeContext.stop());
|
||||
metrics.updateCommitMetrics(
|
||||
HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), durationInMs,
|
||||
metadata);
|
||||
metrics
|
||||
.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(),
|
||||
durationInMs, metadata);
|
||||
writeContext = null;
|
||||
}
|
||||
logger.info("Committed " + commitTime);
|
||||
@@ -587,20 +562,19 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime will
|
||||
* be referenced in the savepoint and will never be cleaned. The savepointed commit will never be
|
||||
* rolledback or archived.
|
||||
*
|
||||
* <p>
|
||||
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be
|
||||
* manually created and deleted.
|
||||
*
|
||||
* <p>
|
||||
* Savepoint should be on a commit that could not have been cleaned.
|
||||
*
|
||||
* @param user - User creating the savepoint
|
||||
* @param user - User creating the savepoint
|
||||
* @param comment - Comment for the savepoint
|
||||
* @return true if the savepoint was created successfully
|
||||
*/
|
||||
public boolean savepoint(String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
if (table.getCompletedCommitTimeline().empty()) {
|
||||
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
|
||||
}
|
||||
@@ -614,21 +588,20 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime will
|
||||
* be referenced in the savepoint and will never be cleaned. The savepointed commit will never be
|
||||
* rolledback or archived.
|
||||
*
|
||||
* <p>
|
||||
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be
|
||||
* manually created and deleted.
|
||||
*
|
||||
* <p>
|
||||
* Savepoint should be on a commit that could not have been cleaned.
|
||||
*
|
||||
* @param commitTime - commit that should be savepointed
|
||||
* @param user - User creating the savepoint
|
||||
* @param comment - Comment for the savepoint
|
||||
* @param user - User creating the savepoint
|
||||
* @param comment - Comment for the savepoint
|
||||
* @return true if the savepoint was created successfully
|
||||
*/
|
||||
public boolean savepoint(String commitTime, String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
|
||||
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
|
||||
@@ -646,8 +619,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
table.getActiveTimeline().getInstantDetails(cleanInstant.get()).get());
|
||||
lastCommitRetained = cleanMetadata.getEarliestCommitToRetain();
|
||||
} else {
|
||||
lastCommitRetained =
|
||||
table.getCompletedCommitTimeline().firstInstant().get().getTimestamp();
|
||||
lastCommitRetained = table.getCompletedCommitTimeline().firstInstant().get().getTimestamp();
|
||||
}
|
||||
|
||||
// Cannot allow savepoint time on a commit that could have been cleaned
|
||||
@@ -656,25 +628,24 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
"Could not savepoint commit " + commitTime + " as this is beyond the lookup window "
|
||||
+ lastCommitRetained);
|
||||
|
||||
Map<String, List<String>> latestFilesMap = jsc.parallelize(
|
||||
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(),
|
||||
Map<String, List<String>> latestFilesMap = jsc.parallelize(FSUtils
|
||||
.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()))
|
||||
.mapToPair((PairFunction<String, String, List<String>>) partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
logger.info("Collecting latest files in partition path " + partitionPath);
|
||||
TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView();
|
||||
List<String> latestFiles =
|
||||
view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime)
|
||||
.map(HoodieDataFile::getFileName).collect(Collectors.toList());
|
||||
List<String> latestFiles = view.getLatestDataFilesBeforeOrOn(partitionPath, commitTime)
|
||||
.map(HoodieDataFile::getFileName).collect(Collectors.toList());
|
||||
return new Tuple2<>(partitionPath, latestFiles);
|
||||
}).collectAsMap();
|
||||
|
||||
HoodieSavepointMetadata metadata =
|
||||
AvroUtils.convertSavepointMetadata(user, comment, latestFilesMap);
|
||||
HoodieSavepointMetadata metadata = AvroUtils
|
||||
.convertSavepointMetadata(user, comment, latestFilesMap);
|
||||
// Nothing to save in the savepoint
|
||||
table.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime),
|
||||
AvroUtils.serializeSavepointMetadata(metadata));
|
||||
table.getActiveTimeline()
|
||||
.saveAsComplete(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime),
|
||||
AvroUtils.serializeSavepointMetadata(metadata));
|
||||
logger.info("Savepoint " + commitTime + " created");
|
||||
return true;
|
||||
} catch (IOException e) {
|
||||
@@ -691,22 +662,20 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public void deleteSavepoint(String savepointTime) {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
|
||||
HoodieInstant savePoint =
|
||||
new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
|
||||
boolean isSavepointPresent =
|
||||
table.getCompletedSavepointTimeline().containsInstant(savePoint);
|
||||
HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION,
|
||||
savepointTime);
|
||||
boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint);
|
||||
if (!isSavepointPresent) {
|
||||
logger.warn("No savepoint present " + savepointTime);
|
||||
return;
|
||||
}
|
||||
|
||||
activeTimeline.revertToInflight(savePoint);
|
||||
activeTimeline.deleteInflight(
|
||||
new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, savepointTime));
|
||||
activeTimeline
|
||||
.deleteInflight(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, savepointTime));
|
||||
logger.info("Savepoint " + savepointTime + " deleted");
|
||||
}
|
||||
|
||||
@@ -719,30 +688,27 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public boolean rollbackToSavepoint(String savepointTime) {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieTimeline commitTimeline = table.getCommitsTimeline();
|
||||
|
||||
HoodieInstant savePoint =
|
||||
new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
|
||||
boolean isSavepointPresent =
|
||||
table.getCompletedSavepointTimeline().containsInstant(savePoint);
|
||||
HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION,
|
||||
savepointTime);
|
||||
boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint);
|
||||
if (!isSavepointPresent) {
|
||||
throw new HoodieRollbackException("No savepoint for commitTime " + savepointTime);
|
||||
}
|
||||
|
||||
List<String> commitsToRollback =
|
||||
commitTimeline.findInstantsAfter(savepointTime, Integer.MAX_VALUE).getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
List<String> commitsToRollback = commitTimeline
|
||||
.findInstantsAfter(savepointTime, Integer.MAX_VALUE).getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
logger.info("Rolling back commits " + commitsToRollback);
|
||||
|
||||
rollback(commitsToRollback);
|
||||
|
||||
// Make sure the rollback was successful
|
||||
Optional<HoodieInstant> lastInstant =
|
||||
activeTimeline.reload().getCommitsTimeline().filterCompletedInstants()
|
||||
.lastInstant();
|
||||
Optional<HoodieInstant> lastInstant = activeTimeline.reload().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
Preconditions.checkArgument(lastInstant.isPresent());
|
||||
Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime),
|
||||
savepointTime + "is not the last commit after rolling back " + commitsToRollback
|
||||
@@ -771,16 +737,14 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieTimeline inflightTimeline = table.getInflightCommitTimeline();
|
||||
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
|
||||
|
||||
// Check if any of the commits is a savepoint - do not allow rollback on those commits
|
||||
List<String> savepoints =
|
||||
table.getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
List<String> savepoints = table.getCompletedSavepointTimeline().getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
commits.forEach(s -> {
|
||||
if (savepoints.contains(s)) {
|
||||
throw new HoodieRollbackException(
|
||||
@@ -800,16 +764,15 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
String lastCommit = commits.get(commits.size() - 1);
|
||||
if (!commitTimeline.empty() && !commitTimeline
|
||||
.findInstantsAfter(lastCommit, Integer.MAX_VALUE).empty()) {
|
||||
throw new HoodieRollbackException("Found commits after time :" + lastCommit +
|
||||
", please rollback greater commits first");
|
||||
throw new HoodieRollbackException(
|
||||
"Found commits after time :" + lastCommit + ", please rollback greater commits first");
|
||||
}
|
||||
|
||||
List<String> inflights = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
if (!inflights.isEmpty() && inflights.indexOf(lastCommit) != inflights.size() - 1) {
|
||||
throw new HoodieRollbackException(
|
||||
"Found in-flight commits after time :" + lastCommit +
|
||||
", please rollback greater commits first");
|
||||
throw new HoodieRollbackException("Found in-flight commits after time :" + lastCommit
|
||||
+ ", please rollback greater commits first");
|
||||
}
|
||||
|
||||
List<HoodieRollbackStat> stats = table.rollback(jsc, commits);
|
||||
@@ -817,8 +780,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
// cleanup index entries
|
||||
commits.stream().forEach(s -> {
|
||||
if (!index.rollbackCommit(s)) {
|
||||
throw new HoodieRollbackException(
|
||||
"Rollback index changes failed, for time :" + s);
|
||||
throw new HoodieRollbackException("Rollback index changes failed, for time :" + s);
|
||||
}
|
||||
});
|
||||
logger.info("Index rolled back for commits " + commits);
|
||||
@@ -826,13 +788,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
Optional<Long> durationInMs = Optional.empty();
|
||||
if (context != null) {
|
||||
durationInMs = Optional.of(metrics.getDurationInMs(context.stop()));
|
||||
Long numFilesDeleted = stats.stream()
|
||||
.mapToLong(stat -> stat.getSuccessDeleteFiles().size())
|
||||
Long numFilesDeleted = stats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size())
|
||||
.sum();
|
||||
metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted);
|
||||
}
|
||||
HoodieRollbackMetadata rollbackMetadata =
|
||||
AvroUtils.convertRollbackMetadata(startRollbackTime, durationInMs, commits, stats);
|
||||
HoodieRollbackMetadata rollbackMetadata = AvroUtils
|
||||
.convertRollbackMetadata(startRollbackTime, durationInMs, commits, stats);
|
||||
table.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime),
|
||||
AvroUtils.serializeRollbackMetadata(rollbackMetadata));
|
||||
@@ -846,8 +807,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
table.getActiveTimeline().getRollbackTimeline().getInstants());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRollbackException("Failed to rollback " +
|
||||
config.getBasePath() + " commits " + commits, e);
|
||||
throw new HoodieRollbackException(
|
||||
"Failed to rollback " + config.getBasePath() + " commits " + commits, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -880,8 +841,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(),
|
||||
true), config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
|
||||
List<HoodieCleanStat> cleanStats = table.clean(jsc);
|
||||
if (cleanStats.isEmpty()) {
|
||||
@@ -896,15 +856,15 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
// Create the metadata and save it
|
||||
HoodieCleanMetadata metadata =
|
||||
AvroUtils.convertCleanMetadata(startCleanTime, durationInMs, cleanStats);
|
||||
HoodieCleanMetadata metadata = AvroUtils
|
||||
.convertCleanMetadata(startCleanTime, durationInMs, cleanStats);
|
||||
logger.info("Cleaned " + metadata.getTotalFilesDeleted() + " files");
|
||||
metrics.updateCleanMetrics(durationInMs.orElseGet(() -> -1L),
|
||||
metadata.getTotalFilesDeleted());
|
||||
metrics
|
||||
.updateCleanMetrics(durationInMs.orElseGet(() -> -1L), metadata.getTotalFilesDeleted());
|
||||
|
||||
table.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, startCleanTime),
|
||||
AvroUtils.serializeCleanMetadata(metadata));
|
||||
table.getActiveTimeline()
|
||||
.saveAsComplete(new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, startCleanTime),
|
||||
AvroUtils.serializeCleanMetadata(metadata));
|
||||
logger.info("Marked clean started on " + startCleanTime + " as complete");
|
||||
|
||||
if (!table.getActiveTimeline().getCleanerTimeline().empty()) {
|
||||
@@ -930,12 +890,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
public void startCommitWithTime(String commitTime) {
|
||||
logger.info("Generate a new commit time " + commitTime);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
String commitActionType = table.getCommitActionType();
|
||||
activeTimeline.createInflight(
|
||||
new HoodieInstant(true, commitActionType, commitTime));
|
||||
activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -948,17 +906,16 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
return commitTime;
|
||||
}
|
||||
|
||||
/** Since MOR tableType default to {@link HoodieTimeline#DELTA_COMMIT_ACTION},
|
||||
* we need to explicitly set to {@link HoodieTimeline#COMMIT_ACTION} for compaction
|
||||
/**
|
||||
* Since MOR tableType default to {@link HoodieTimeline#DELTA_COMMIT_ACTION}, we need to
|
||||
* explicitly set to {@link HoodieTimeline#COMMIT_ACTION} for compaction
|
||||
*/
|
||||
public void startCompactionWithTime(String commitTime) {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
String commitActionType = HoodieTimeline.COMMIT_ACTION;
|
||||
activeTimeline.createInflight(
|
||||
new HoodieInstant(true, commitActionType, commitTime));
|
||||
String commitActionType = HoodieTimeline.COMMIT_ACTION;
|
||||
activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -968,8 +925,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
public JavaRDD<WriteStatus> compact(String commitTime) throws IOException {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
JavaRDD<WriteStatus> statuses = table.compact(jsc, commitTime);
|
||||
// Trigger the insert and collect statuses
|
||||
statuses = statuses.persist(config.getWriteStatusStorageLevel());
|
||||
@@ -980,9 +936,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
/**
|
||||
* Commit a compaction operation
|
||||
* @param commitTime
|
||||
* @param writeStatuses
|
||||
* @param extraMetadata
|
||||
*/
|
||||
public void commitCompaction(String commitTime, JavaRDD<WriteStatus> writeStatuses,
|
||||
Optional<HashMap<String, String>> extraMetadata) {
|
||||
@@ -992,8 +945,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
/**
|
||||
* Commit a compaction operation
|
||||
* @param commitTime
|
||||
* @param writeStatuses
|
||||
*/
|
||||
public void commitCompaction(String commitTime, JavaRDD<WriteStatus> writeStatuses) {
|
||||
String commitCompactionActionType = HoodieActiveTimeline.COMMIT_ACTION;
|
||||
@@ -1006,8 +957,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
private void forceCompact(String compactionCommitTime) throws IOException {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
config.getBasePath(), true);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
JavaRDD<WriteStatus> compactedStatuses = table.compact(jsc, compactionCommitTime);
|
||||
if (!compactedStatuses.isEmpty()) {
|
||||
@@ -1029,8 +980,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
private void commitForceCompaction(JavaRDD<WriteStatus> writeStatuses,
|
||||
HoodieTableMetaClient metaClient,
|
||||
String compactionCommitTime) {
|
||||
HoodieTableMetaClient metaClient, String compactionCommitTime) {
|
||||
List<HoodieWriteStat> updateStatusMap = writeStatuses.map(writeStatus -> writeStatus.getStat())
|
||||
.collect();
|
||||
|
||||
@@ -1054,12 +1004,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
}
|
||||
|
||||
public static SparkConf registerClasses(SparkConf conf) {
|
||||
conf.registerKryoClasses(
|
||||
new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
|
||||
return conf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Deduplicate Hoodie records, using the given deduplication funciton.
|
||||
*/
|
||||
@@ -1074,13 +1018,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
return new Tuple2<>(key, record);
|
||||
})
|
||||
.reduceByKey((rec1, rec2) -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
|
||||
// we cannot allow the user to change the key or partitionPath, since that will affect everything
|
||||
@SuppressWarnings("unchecked") T reducedData = (T) rec1.getData()
|
||||
.preCombine(rec2.getData());
|
||||
// we cannot allow the user to change the key or partitionPath, since that will affect
|
||||
// everything
|
||||
// so pick it from one of the records.
|
||||
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
||||
}, parallelism)
|
||||
.map(recordTuple -> recordTuple._2());
|
||||
}, parallelism).map(recordTuple -> recordTuple._2());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -1088,8 +1032,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
private void rollbackInflightCommits() {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights();
|
||||
List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
@@ -1103,7 +1046,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
writeContext = metrics.getCommitCtx();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
return HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -53,10 +53,10 @@ public class WriteStatus implements Serializable {
|
||||
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
|
||||
* objects are collected in Spark Driver.
|
||||
*
|
||||
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
|
||||
* it.
|
||||
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
|
||||
* it.
|
||||
* @param optionalRecordMetadata optional metadata related to data contained in {@link
|
||||
* HoodieRecord} before deflation.
|
||||
* HoodieRecord} before deflation.
|
||||
*/
|
||||
public void markSuccess(HoodieRecord record,
|
||||
Optional<Map<String, String>> optionalRecordMetadata) {
|
||||
@@ -69,10 +69,10 @@ public class WriteStatus implements Serializable {
|
||||
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
|
||||
* objects are collected in Spark Driver.
|
||||
*
|
||||
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
|
||||
* it.
|
||||
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
|
||||
* it.
|
||||
* @param optionalRecordMetadata optional metadata related to data contained in {@link
|
||||
* HoodieRecord} before deflation.
|
||||
* HoodieRecord} before deflation.
|
||||
*/
|
||||
public void markFailure(HoodieRecord record, Throwable t,
|
||||
Optional<Map<String, String>> optionalRecordMetadata) {
|
||||
@@ -106,14 +106,14 @@ public class WriteStatus implements Serializable {
|
||||
return globalError != null;
|
||||
}
|
||||
|
||||
public void setGlobalError(Throwable t) {
|
||||
this.globalError = t;
|
||||
}
|
||||
|
||||
public Throwable getGlobalError() {
|
||||
return this.globalError;
|
||||
}
|
||||
|
||||
public void setGlobalError(Throwable t) {
|
||||
this.globalError = t;
|
||||
}
|
||||
|
||||
public List<HoodieRecord> getWrittenRecords() {
|
||||
return writtenRecords;
|
||||
}
|
||||
|
||||
@@ -30,10 +30,6 @@ public class DefaultHoodieConfig implements Serializable {
|
||||
this.props = props;
|
||||
}
|
||||
|
||||
public Properties getProps() {
|
||||
return props;
|
||||
}
|
||||
|
||||
public static void setDefaultOnCondition(Properties props, boolean condition, String propName,
|
||||
String defaultValue) {
|
||||
if (condition) {
|
||||
@@ -48,4 +44,8 @@ public class DefaultHoodieConfig implements Serializable {
|
||||
}
|
||||
}
|
||||
|
||||
public Properties getProps() {
|
||||
return props;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -34,81 +34,76 @@ import javax.annotation.concurrent.Immutable;
|
||||
public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
|
||||
private static final String DEFAULT_CLEANER_POLICY =
|
||||
HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name();
|
||||
|
||||
public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
|
||||
private static final String DEFAULT_AUTO_CLEAN = "true";
|
||||
|
||||
// Turn on inline compaction - after fw delta commits a inline compaction will be run
|
||||
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
|
||||
private static final String DEFAULT_INLINE_COMPACT = "false";
|
||||
|
||||
// Run a compaction every N delta commits
|
||||
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
|
||||
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10";
|
||||
|
||||
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP =
|
||||
"hoodie.compact.inline.max" + ".delta.commits";
|
||||
public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP =
|
||||
"hoodie.cleaner.fileversions.retained";
|
||||
private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3";
|
||||
|
||||
"hoodie.cleaner.fileversions" + ".retained";
|
||||
public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained";
|
||||
private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24";
|
||||
|
||||
public static final String MAX_COMMITS_TO_KEEP = "hoodie.keep.max.commits";
|
||||
private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128);
|
||||
public static final String MIN_COMMITS_TO_KEEP = "hoodie.keep.min.commits";
|
||||
private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96);
|
||||
// Upsert uses this file size to compact new data onto existing files..
|
||||
public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit";
|
||||
// Turned off by default
|
||||
public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(0);
|
||||
|
||||
/**
|
||||
* Configs related to specific table types
|
||||
**/
|
||||
// Number of inserts, that will be put each partition/bucket for writing
|
||||
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = "hoodie.copyonwrite.insert.split.size";
|
||||
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE =
|
||||
"hoodie.copyonwrite.insert" + ".split.size";
|
||||
// The rationale to pick the insert parallelism is the following. Writing out 100MB files,
|
||||
// with atleast 1kb records, means 100K records per file. we just overprovision to 500K
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000);
|
||||
|
||||
// Config to control whether we control insert split sizes automatically based on average record sizes
|
||||
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert.auto.split";
|
||||
// Config to control whether we control insert split sizes automatically based on average
|
||||
// record sizes
|
||||
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS =
|
||||
"hoodie.copyonwrite.insert" + ".auto.split";
|
||||
// its off by default
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(false);
|
||||
|
||||
|
||||
// This value is used as a guessimate for the record size, if we can't determine this from previous commits
|
||||
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = "hoodie.copyonwrite.record.size.estimate";
|
||||
// Used to determine how much more can be packed into a small file, before it exceeds the size limit.
|
||||
// This value is used as a guessimate for the record size, if we can't determine this from
|
||||
// previous commits
|
||||
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE =
|
||||
"hoodie.copyonwrite" + ".record.size.estimate";
|
||||
// Used to determine how much more can be packed into a small file, before it exceeds the size
|
||||
// limit.
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String
|
||||
.valueOf(1024);
|
||||
|
||||
public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
|
||||
public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
|
||||
|
||||
public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
|
||||
// 500GB of target IO per compaction (both read and write)
|
||||
public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
|
||||
|
||||
public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
|
||||
// 200GB of target IO per compaction
|
||||
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class
|
||||
.getName();
|
||||
|
||||
// used to merge records written to log file
|
||||
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
|
||||
public static final String PAYLOAD_CLASS_PROP = "hoodie.compaction.payload.class";
|
||||
|
||||
// used to choose a trade off between IO vs Memory when performing compaction process
|
||||
// Depending on outputfile_size and memory provided, choose true to avoid OOM for large file size + small memory
|
||||
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "hoodie.compaction.lazy.block.read";
|
||||
// Depending on outputfile_size and memory provided, choose true to avoid OOM for large file
|
||||
// size + small memory
|
||||
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP =
|
||||
"hoodie.compaction.lazy" + ".block.read";
|
||||
public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "false";
|
||||
|
||||
// used to choose whether to enable reverse log reading (reverse log traversal)
|
||||
public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = "hoodie.compaction.reverse.log.read";
|
||||
public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP =
|
||||
"hoodie.compaction" + ".reverse.log.read";
|
||||
public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false";
|
||||
private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
.name();
|
||||
private static final String DEFAULT_AUTO_CLEAN = "true";
|
||||
private static final String DEFAULT_INLINE_COMPACT = "false";
|
||||
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10";
|
||||
private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3";
|
||||
private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24";
|
||||
private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128);
|
||||
private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96);
|
||||
|
||||
private HoodieCompactionConfig(Properties props) {
|
||||
super(props);
|
||||
@@ -159,8 +154,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
}
|
||||
|
||||
public Builder retainFileVersions(int fileVersionsRetained) {
|
||||
props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP,
|
||||
String.valueOf(fileVersionsRetained));
|
||||
props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP, String.valueOf(fileVersionsRetained));
|
||||
return this;
|
||||
}
|
||||
|
||||
@@ -238,22 +232,22 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
|
||||
public HoodieCompactionConfig build() {
|
||||
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP),
|
||||
AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP),
|
||||
INLINE_COMPACT_PROP, DEFAULT_INLINE_COMPACT);
|
||||
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP,
|
||||
DEFAULT_AUTO_CLEAN);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), INLINE_COMPACT_PROP,
|
||||
DEFAULT_INLINE_COMPACT);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP),
|
||||
INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP),
|
||||
CLEANER_POLICY_PROP, DEFAULT_CLEANER_POLICY);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP), CLEANER_POLICY_PROP,
|
||||
DEFAULT_CLEANER_POLICY);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP),
|
||||
CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP),
|
||||
CLEANER_COMMITS_RETAINED_PROP, DEFAULT_CLEANER_COMMITS_RETAINED);
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP),
|
||||
MAX_COMMITS_TO_KEEP, DEFAULT_MAX_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP),
|
||||
MIN_COMMITS_TO_KEEP, DEFAULT_MIN_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP), MAX_COMMITS_TO_KEEP,
|
||||
DEFAULT_MAX_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP), MIN_COMMITS_TO_KEEP,
|
||||
DEFAULT_MIN_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES),
|
||||
PARQUET_SMALL_FILE_LIMIT_BYTES, DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
|
||||
@@ -263,8 +257,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE),
|
||||
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
|
||||
DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM),
|
||||
CLEANER_PARALLELISM, DEFAULT_CLEANER_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM), CLEANER_PARALLELISM,
|
||||
DEFAULT_CLEANER_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP),
|
||||
COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY);
|
||||
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP),
|
||||
@@ -277,9 +271,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED);
|
||||
|
||||
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
|
||||
Preconditions.checkArgument(
|
||||
Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer
|
||||
.parseInt(props.getProperty(MIN_COMMITS_TO_KEEP)));
|
||||
Preconditions.checkArgument(Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer
|
||||
.parseInt(props.getProperty(MIN_COMMITS_TO_KEEP)));
|
||||
return config;
|
||||
}
|
||||
|
||||
|
||||
@@ -40,23 +40,25 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism";
|
||||
// Disable explicit bloom index parallelism setting by default - hoodie auto computes
|
||||
public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0";
|
||||
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = "hoodie.bloom.index.prune.by.ranges";
|
||||
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP =
|
||||
"hoodie.bloom.index.prune.by" + ".ranges";
|
||||
public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true";
|
||||
public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching";
|
||||
public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true";
|
||||
public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = "hoodie.bloom.index.input.storage.level";
|
||||
public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL =
|
||||
"hoodie.bloom.index.input.storage" + ".level";
|
||||
public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
|
||||
// ***** HBase Index Configs *****
|
||||
public final static String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
|
||||
public final static String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
|
||||
public final static String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
|
||||
public final static String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
|
||||
public final static String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
|
||||
public final static String DEFAULT_HBASE_BATCH_SIZE = "100";
|
||||
public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
|
||||
public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
|
||||
public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
|
||||
public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
|
||||
public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
|
||||
public static final String DEFAULT_HBASE_BATCH_SIZE = "100";
|
||||
|
||||
// ***** Bucketed Index Configs *****
|
||||
public final static String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets";
|
||||
public static final String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets";
|
||||
|
||||
private HoodieIndexConfig(Properties props) {
|
||||
super(props);
|
||||
@@ -152,12 +154,12 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
|
||||
public HoodieIndexConfig build() {
|
||||
HoodieIndexConfig config = new HoodieIndexConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP),
|
||||
INDEX_TYPE_PROP, DEFAULT_INDEX_TYPE);
|
||||
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP,
|
||||
DEFAULT_INDEX_TYPE);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES),
|
||||
BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP),
|
||||
BLOOM_FILTER_FPP, DEFAULT_BLOOM_FILTER_FPP);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), BLOOM_FILTER_FPP,
|
||||
DEFAULT_BLOOM_FILTER_FPP);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP),
|
||||
BLOOM_INDEX_PARALLELISM_PROP, DEFAULT_BLOOM_INDEX_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP),
|
||||
|
||||
@@ -30,8 +30,8 @@ import org.apache.spark.util.Utils;
|
||||
@Immutable
|
||||
public class HoodieMemoryConfig extends DefaultHoodieConfig {
|
||||
|
||||
// This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use during merge
|
||||
// This makes it easier to scale this value as one increases the spark.executor.memory
|
||||
// This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use
|
||||
// during merge. This makes it easier to scale this value as one increases the spark.executor.memory
|
||||
public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = "hoodie.memory.merge.fraction";
|
||||
// Default max memory fraction during hash-merge, excess spills to disk
|
||||
public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE = String.valueOf(0.6);
|
||||
@@ -87,19 +87,21 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
|
||||
}
|
||||
|
||||
/**
|
||||
* Dynamic calculation of max memory to use for for spillable map. user.available.memory =
|
||||
* spark.executor.memory * (1 - spark.memory.fraction) spillable.available.memory =
|
||||
* user.available.memory * hoodie.memory.fraction. Anytime the spark.executor.memory or the
|
||||
* spark.memory.fraction is changed, the memory used for spillable map changes accordingly
|
||||
* Dynamic calculation of max memory to use for for spillable map. user.available.memory = spark.executor.memory *
|
||||
* (1 - spark.memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime
|
||||
* the spark.executor.memory or the spark.memory.fraction is changed, the memory used for spillable map changes
|
||||
* accordingly
|
||||
*/
|
||||
private long getMaxMemoryAllowedForMerge(String maxMemoryFraction) {
|
||||
final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory";
|
||||
final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction";
|
||||
// This is hard-coded in spark code {@link https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala#L231}
|
||||
// so have to re-define this here
|
||||
// This is hard-coded in spark code {@link
|
||||
// https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
|
||||
// spark/memory/UnifiedMemoryManager.scala#L231} so have to re-define this here
|
||||
final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6";
|
||||
// This is hard-coded in spark code {@link https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/spark/SparkContext.scala#L471}
|
||||
// so have to re-define this here
|
||||
// This is hard-coded in spark code {@link
|
||||
// https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
|
||||
// spark/SparkContext.scala#L471} so have to re-define this here
|
||||
final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB
|
||||
|
||||
if (SparkEnv.get() != null) {
|
||||
@@ -109,7 +111,8 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
|
||||
DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024
|
||||
* 1024L);
|
||||
// 0.6 is the default value used by Spark,
|
||||
// look at {@link https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
|
||||
// look at {@link
|
||||
// https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
|
||||
double memoryFraction = Double
|
||||
.valueOf(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP,
|
||||
DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION));
|
||||
@@ -143,5 +146,4 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -29,22 +29,22 @@ import javax.annotation.concurrent.Immutable;
|
||||
@Immutable
|
||||
public class HoodieMetricsConfig extends DefaultHoodieConfig {
|
||||
|
||||
public final static String METRIC_PREFIX = "hoodie.metrics";
|
||||
public final static String METRICS_ON = METRIC_PREFIX + ".on";
|
||||
public final static boolean DEFAULT_METRICS_ON = false;
|
||||
public final static String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
|
||||
public final static MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE =
|
||||
MetricsReporterType.GRAPHITE;
|
||||
public static final String METRIC_PREFIX = "hoodie.metrics";
|
||||
public static final String METRICS_ON = METRIC_PREFIX + ".on";
|
||||
public static final boolean DEFAULT_METRICS_ON = false;
|
||||
public static final String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
|
||||
public static final MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = MetricsReporterType
|
||||
.GRAPHITE;
|
||||
|
||||
// Graphite
|
||||
public final static String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
|
||||
public final static String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host";
|
||||
public final static String DEFAULT_GRAPHITE_SERVER_HOST = "localhost";
|
||||
public static final String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
|
||||
public static final String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host";
|
||||
public static final String DEFAULT_GRAPHITE_SERVER_HOST = "localhost";
|
||||
|
||||
public final static String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port";
|
||||
public final static int DEFAULT_GRAPHITE_SERVER_PORT = 4756;
|
||||
public static final String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port";
|
||||
public static final int DEFAULT_GRAPHITE_SERVER_PORT = 4756;
|
||||
|
||||
public final static String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
|
||||
public static final String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
|
||||
|
||||
private HoodieMetricsConfig(Properties props) {
|
||||
super(props);
|
||||
@@ -103,14 +103,14 @@ public class HoodieMetricsConfig extends DefaultHoodieConfig {
|
||||
HoodieMetricsConfig config = new HoodieMetricsConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON,
|
||||
String.valueOf(DEFAULT_METRICS_ON));
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE),
|
||||
METRICS_REPORTER_TYPE, DEFAULT_METRICS_REPORTER_TYPE.name());
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST),
|
||||
GRAPHITE_SERVER_HOST, DEFAULT_GRAPHITE_SERVER_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT),
|
||||
GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT),
|
||||
GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE), METRICS_REPORTER_TYPE,
|
||||
DEFAULT_METRICS_REPORTER_TYPE.name());
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST), GRAPHITE_SERVER_HOST,
|
||||
DEFAULT_GRAPHITE_SERVER_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT,
|
||||
String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT,
|
||||
String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -36,10 +36,11 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
|
||||
public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
|
||||
// used to size log files
|
||||
public static final String LOGFILE_SIZE_MAX_BYTES = "hoodie.logfile.max.size";
|
||||
public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String.valueOf(1024*1024*1024); // 1 GB
|
||||
public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String
|
||||
.valueOf(1024 * 1024 * 1024); // 1 GB
|
||||
// used to size data blocks in log file
|
||||
public static final String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = "hoodie.logfile.data.block.max.size";
|
||||
public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256*1024*1024); // 256 MB
|
||||
public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256 * 1024 * 1024); // 256 MB
|
||||
public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio";
|
||||
// Default compression ratio for parquet
|
||||
public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1);
|
||||
|
||||
@@ -16,7 +16,6 @@
|
||||
|
||||
package com.uber.hoodie.config;
|
||||
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
|
||||
@@ -24,15 +23,14 @@ import com.uber.hoodie.common.util.ReflectionUtils;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
|
||||
import com.uber.hoodie.metrics.MetricsReporterType;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
/**
|
||||
* Class storing configs for the {@link com.uber.hoodie.HoodieWriteClient}
|
||||
@@ -40,9 +38,9 @@ import java.util.Properties;
|
||||
@Immutable
|
||||
public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String TABLE_NAME = "hoodie.table.name";
|
||||
private static final String BASE_PATH_PROP = "hoodie.base.path";
|
||||
private static final String AVRO_SCHEMA = "hoodie.avro.schema";
|
||||
public static final String TABLE_NAME = "hoodie.table.name";
|
||||
private static final String DEFAULT_PARALLELISM = "200";
|
||||
private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism";
|
||||
private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism";
|
||||
@@ -57,13 +55,16 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
private static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
private static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit";
|
||||
private static final String DEFAULT_HOODIE_AUTO_COMMIT = "true";
|
||||
private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning";
|
||||
private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP =
|
||||
"hoodie.assume.date" + ".partitioning";
|
||||
private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
|
||||
private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
|
||||
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
|
||||
private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = "hoodie.copyonwrite.use.temp.folder.for.create";
|
||||
private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE =
|
||||
"hoodie.copyonwrite.use" + ".temp.folder.for.create";
|
||||
private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE = "false";
|
||||
private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "hoodie.copyonwrite.use.temp.folder.for.merge";
|
||||
private static final String HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE =
|
||||
"hoodie.copyonwrite.use" + ".temp.folder.for.merge";
|
||||
private static final String DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE = "false";
|
||||
private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism";
|
||||
private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM;
|
||||
@@ -72,6 +73,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieWriteConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
/**
|
||||
* base properties
|
||||
**/
|
||||
@@ -137,8 +142,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
}
|
||||
|
||||
public boolean shouldUseTempFolderForCopyOnWrite() {
|
||||
return shouldUseTempFolderForCopyOnWriteForCreate() ||
|
||||
shouldUseTempFolderForCopyOnWriteForMerge();
|
||||
return shouldUseTempFolderForCopyOnWriteForCreate()
|
||||
|| shouldUseTempFolderForCopyOnWriteForMerge();
|
||||
}
|
||||
|
||||
public int getFinalizeWriteParallelism() {
|
||||
@@ -154,8 +159,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
}
|
||||
|
||||
public int getCleanerFileVersionsRetained() {
|
||||
return Integer.parseInt(
|
||||
props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
|
||||
}
|
||||
|
||||
public int getCleanerCommitsRetained() {
|
||||
@@ -177,8 +182,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
}
|
||||
|
||||
public int getCopyOnWriteInsertSplitSize() {
|
||||
return Integer.parseInt(
|
||||
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
|
||||
}
|
||||
|
||||
public int getCopyOnWriteRecordSizeEstimate() {
|
||||
@@ -204,8 +209,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
}
|
||||
|
||||
public int getInlineCompactDeltaCommitMax() {
|
||||
return Integer.parseInt(
|
||||
props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
|
||||
}
|
||||
|
||||
public CompactionStrategy getCompactionStrategy() {
|
||||
@@ -340,11 +345,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
public String getGraphiteMetricPrefix() {
|
||||
return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
|
||||
}
|
||||
|
||||
public static HoodieWriteConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* memory configs
|
||||
*/
|
||||
@@ -486,15 +487,15 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
|
||||
public Builder withUseTempFolderCopyOnWriteForCreate(
|
||||
boolean shouldUseTempFolderCopyOnWriteForCreate) {
|
||||
props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, String.valueOf
|
||||
(shouldUseTempFolderCopyOnWriteForCreate));
|
||||
props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE,
|
||||
String.valueOf(shouldUseTempFolderCopyOnWriteForCreate));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withUseTempFolderCopyOnWriteForMerge(
|
||||
boolean shouldUseTempFolderCopyOnWriteForMerge) {
|
||||
props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, String.valueOf
|
||||
(shouldUseTempFolderCopyOnWriteForMerge));
|
||||
props.setProperty(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE,
|
||||
String.valueOf(shouldUseTempFolderCopyOnWriteForMerge));
|
||||
return this;
|
||||
}
|
||||
|
||||
@@ -510,8 +511,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM,
|
||||
DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM),
|
||||
BULKINSERT_PARALLELISM,
|
||||
DEFAULT_PARALLELISM);
|
||||
BULKINSERT_PARALLELISM, DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM,
|
||||
DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP),
|
||||
|
||||
@@ -21,12 +21,6 @@ import com.google.common.base.Preconditions;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.util.SizeEstimator;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
@@ -35,53 +29,63 @@ import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.util.SizeEstimator;
|
||||
|
||||
/**
|
||||
* Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It internally samples
|
||||
* every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in buffer accordingly. This is done to
|
||||
* ensure that we don't OOM.
|
||||
* Used for buffering input records. Buffer limit is controlled by {@link #bufferMemoryLimit}. It
|
||||
* internally samples every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in
|
||||
* buffer accordingly. This is done to ensure that we don't OOM.
|
||||
*/
|
||||
public class BufferedIterator<K extends HoodieRecordPayload, T extends HoodieRecord<K>>
|
||||
implements Iterator<BufferedIterator.BufferedIteratorPayload<T>> {
|
||||
public class BufferedIterator<K extends HoodieRecordPayload, T extends HoodieRecord<K>> implements
|
||||
Iterator<BufferedIterator.BufferedIteratorPayload<T>> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(BufferedIterator.class);
|
||||
// interval used for polling records in the queue.
|
||||
public static final int RECORD_POLL_INTERVAL_SEC = 5;
|
||||
// rate used for sampling records to determine avg record size in bytes.
|
||||
public static final int RECORD_SAMPLING_RATE = 64;
|
||||
// maximum records that will be cached
|
||||
private static final int RECORD_CACHING_LIMIT = 128 * 1024;
|
||||
// It indicates number of records to cache. We will be using sampled record's average size to determine how many
|
||||
private static Logger logger = LogManager.getLogger(BufferedIterator.class);
|
||||
// It indicates number of records to cache. We will be using sampled record's average size to
|
||||
// determine how many
|
||||
// records we should cache and will change (increase/decrease) permits accordingly.
|
||||
@VisibleForTesting
|
||||
public final Semaphore rateLimiter = new Semaphore(1);
|
||||
// used for sampling records with "RECORD_SAMPLING_RATE" frequency.
|
||||
public final AtomicLong samplingRecordCounter = new AtomicLong(-1);
|
||||
// indicates rate limit (number of records to cache). it is updated whenever there is a change in avg record size.
|
||||
@VisibleForTesting
|
||||
public int currentRateLimit = 1;
|
||||
// internal buffer to cache buffered records.
|
||||
private final LinkedBlockingQueue<Optional<BufferedIteratorPayload<T>>> buffer = new LinkedBlockingQueue<>();
|
||||
private final LinkedBlockingQueue<Optional<BufferedIteratorPayload<T>>> buffer = new
|
||||
LinkedBlockingQueue<>();
|
||||
// maximum amount of memory to be used for buffering records.
|
||||
private final long bufferMemoryLimit;
|
||||
// original iterator from where records are read for buffering.
|
||||
private final Iterator<T> inputIterator;
|
||||
// it holds the root cause of the exception in case either buffering records (reading from
|
||||
// inputIterator) fails or
|
||||
// thread reading records from buffer fails.
|
||||
private final AtomicReference<Exception> hasFailed = new AtomicReference(null);
|
||||
// used for indicating that all the records from buffer are read successfully.
|
||||
private final AtomicBoolean isDone = new AtomicBoolean(false);
|
||||
// schema used for fetching insertValue from HoodieRecord.
|
||||
private final Schema schema;
|
||||
// indicates rate limit (number of records to cache). it is updated whenever there is a change
|
||||
// in avg record size.
|
||||
@VisibleForTesting
|
||||
public int currentRateLimit = 1;
|
||||
// indicates avg record size in bytes. It is updated whenever a new record is sampled.
|
||||
@VisibleForTesting
|
||||
public long avgRecordSizeInBytes = 0;
|
||||
// indicates number of samples collected so far.
|
||||
private long numSamples = 0;
|
||||
// original iterator from where records are read for buffering.
|
||||
private final Iterator<T> inputIterator;
|
||||
// it holds the root cause of the exception in case either buffering records (reading from inputIterator) fails or
|
||||
// thread reading records from buffer fails.
|
||||
private final AtomicReference<Exception> hasFailed = new AtomicReference(null);
|
||||
// used for indicating that all the records from buffer are read successfully.
|
||||
private final AtomicBoolean isDone = new AtomicBoolean(false);
|
||||
// next record to be read from buffer.
|
||||
private BufferedIteratorPayload<T> nextRecord;
|
||||
// schema used for fetching insertValue from HoodieRecord.
|
||||
private final Schema schema;
|
||||
|
||||
public BufferedIterator(final Iterator<T> iterator, final long bufferMemoryLimit, final Schema schema) {
|
||||
public BufferedIterator(final Iterator<T> iterator, final long bufferMemoryLimit,
|
||||
final Schema schema) {
|
||||
this.inputIterator = iterator;
|
||||
this.bufferMemoryLimit = bufferMemoryLimit;
|
||||
this.schema = schema;
|
||||
@@ -92,23 +96,28 @@ public class BufferedIterator<K extends HoodieRecordPayload, T extends HoodieRec
|
||||
return this.buffer.size();
|
||||
}
|
||||
|
||||
// It samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in bytes. It is used
|
||||
// for determining how many maximum records to buffer. Based on change in avg size it may increase or decrease
|
||||
// It samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in
|
||||
// bytes. It is used
|
||||
// for determining how many maximum records to buffer. Based on change in avg size it may
|
||||
// increase or decrease
|
||||
// available permits.
|
||||
private void adjustBufferSizeIfNeeded(final T record) throws InterruptedException {
|
||||
if (this.samplingRecordCounter.incrementAndGet() % RECORD_SAMPLING_RATE != 0) {
|
||||
return;
|
||||
}
|
||||
final long recordSizeInBytes = SizeEstimator.estimate(record);
|
||||
final long newAvgRecordSizeInBytes =
|
||||
Math.max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1));
|
||||
final int newRateLimit =
|
||||
(int) Math.min(RECORD_CACHING_LIMIT, Math.max(1, this.bufferMemoryLimit / newAvgRecordSizeInBytes));
|
||||
// System.out.println("recordSizeInBytes:" + recordSizeInBytes + ":newAvgRecordSizeInBytes:" + newAvgRecordSizeInBytes
|
||||
// + ":newRateLimit:" + newRateLimit + ":currentRateLimit:" + currentRateLimit + ":numSamples:" + numSamples
|
||||
// + ":avgRecordSizeInBytes:" + avgRecordSizeInBytes);
|
||||
final long newAvgRecordSizeInBytes = Math
|
||||
.max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1));
|
||||
final int newRateLimit = (int) Math
|
||||
.min(RECORD_CACHING_LIMIT, Math.max(1, this.bufferMemoryLimit / newAvgRecordSizeInBytes));
|
||||
// System.out.println("recordSizeInBytes:" + recordSizeInBytes + ":newAvgRecordSizeInBytes:" +
|
||||
// newAvgRecordSizeInBytes
|
||||
// + ":newRateLimit:" + newRateLimit + ":currentRateLimit:" + currentRateLimit +
|
||||
// ":numSamples:" + numSamples
|
||||
// + ":avgRecordSizeInBytes:" + avgRecordSizeInBytes);
|
||||
|
||||
// If there is any change in number of records to cache then we will either release (if it increased) or acquire
|
||||
// If there is any change in number of records to cache then we will either release (if it
|
||||
// increased) or acquire
|
||||
// (if it decreased) to adjust rate limiting to newly computed value.
|
||||
if (newRateLimit > currentRateLimit) {
|
||||
rateLimiter.release(newRateLimit - currentRateLimit);
|
||||
@@ -120,12 +129,14 @@ public class BufferedIterator<K extends HoodieRecordPayload, T extends HoodieRec
|
||||
numSamples++;
|
||||
}
|
||||
|
||||
// inserts record into internal buffer. It also fetches insert value from the record to offload computation work on to
|
||||
// inserts record into internal buffer. It also fetches insert value from the record to offload
|
||||
// computation work on to
|
||||
// buffering thread.
|
||||
private void insertRecord(T t) throws Exception {
|
||||
rateLimiter.acquire();
|
||||
adjustBufferSizeIfNeeded(t);
|
||||
// We are retrieving insert value in the record buffering thread to offload computation around schema validation
|
||||
// We are retrieving insert value in the record buffering thread to offload computation
|
||||
// around schema validation
|
||||
// and record creation to it.
|
||||
final BufferedIteratorPayload<T> payload = new BufferedIteratorPayload<>(t, this.schema);
|
||||
buffer.put(Optional.of(payload));
|
||||
@@ -198,12 +209,15 @@ public class BufferedIterator<K extends HoodieRecordPayload, T extends HoodieRec
|
||||
|
||||
public void markAsFailed(Exception e) {
|
||||
this.hasFailed.set(e);
|
||||
// release the permits so that if the buffering thread is waiting for permits then it will get it.
|
||||
// release the permits so that if the buffering thread is waiting for permits then it will
|
||||
// get it.
|
||||
this.rateLimiter.release(RECORD_CACHING_LIMIT + 1);
|
||||
}
|
||||
|
||||
// Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread.
|
||||
// Used for caching HoodieRecord along with insertValue. We need this to offload computation
|
||||
// work to buffering thread.
|
||||
static class BufferedIteratorPayload<T extends HoodieRecord> {
|
||||
|
||||
public T record;
|
||||
public Optional<IndexedRecord> insertValue;
|
||||
// It caches the exception seen while fetching insert value.
|
||||
|
||||
@@ -29,8 +29,8 @@ import org.apache.spark.api.java.function.Function2;
|
||||
/**
|
||||
* Map function that handles a sorted stream of HoodieRecords
|
||||
*/
|
||||
public class BulkInsertMapFunction<T extends HoodieRecordPayload>
|
||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
|
||||
public class BulkInsertMapFunction<T extends HoodieRecordPayload> implements
|
||||
Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
|
||||
|
||||
private String commitTime;
|
||||
private HoodieWriteConfig config;
|
||||
@@ -45,8 +45,7 @@ public class BulkInsertMapFunction<T extends HoodieRecordPayload>
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> call(Integer partition,
|
||||
Iterator<HoodieRecord<T>> sortedRecordItr)
|
||||
throws Exception {
|
||||
Iterator<HoodieRecord<T>> sortedRecordItr) throws Exception {
|
||||
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -24,11 +24,6 @@ import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.io.HoodieCreateHandle;
|
||||
import com.uber.hoodie.io.HoodieIOHandle;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.TaskContext$;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
@@ -38,6 +33,10 @@ import java.util.Set;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.TaskContext$;
|
||||
|
||||
/**
|
||||
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
|
||||
@@ -68,35 +67,37 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
protected List<WriteStatus> computeNext() {
|
||||
// Need to set current spark thread's TaskContext into newly launched thread so that new thread can access
|
||||
// Need to set current spark thread's TaskContext into newly launched thread so that new
|
||||
// thread can access
|
||||
// TaskContext properties.
|
||||
final TaskContext sparkThreadTaskContext = TaskContext.get();
|
||||
// Executor service used for launching writer thread.
|
||||
final ExecutorService writerService = Executors.newFixedThreadPool(1);
|
||||
try {
|
||||
// Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES.
|
||||
final BufferedIterator<T, HoodieRecord<T>> bufferedIterator =
|
||||
new BufferedIterator<>(inputItr, hoodieConfig.getWriteBufferLimitBytes(),
|
||||
HoodieIOHandle.createHoodieWriteSchema(hoodieConfig));
|
||||
Future<List<WriteStatus>> writerResult =
|
||||
writerService.submit(
|
||||
() -> {
|
||||
logger.info("starting hoodie writer thread");
|
||||
// Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext
|
||||
// properties.
|
||||
TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext);
|
||||
List<WriteStatus> statuses = new LinkedList<>();
|
||||
try {
|
||||
statuses.addAll(handleWrite(bufferedIterator));
|
||||
logger.info("hoodie write is done; notifying reader thread");
|
||||
return statuses;
|
||||
} catch (Exception e) {
|
||||
logger.error("error writing hoodie records", e);
|
||||
bufferedIterator.markAsFailed(e);
|
||||
throw e;
|
||||
}
|
||||
});
|
||||
// Buffering records into internal buffer. This can throw exception either if reading records from spark fails or
|
||||
// Used for buffering records which is controlled by
|
||||
// HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES.
|
||||
final BufferedIterator<T, HoodieRecord<T>> bufferedIterator = new BufferedIterator<>(inputItr,
|
||||
hoodieConfig.getWriteBufferLimitBytes(),
|
||||
HoodieIOHandle.createHoodieWriteSchema(hoodieConfig));
|
||||
Future<List<WriteStatus>> writerResult = writerService.submit(() -> {
|
||||
logger.info("starting hoodie writer thread");
|
||||
// Passing parent thread's TaskContext to newly launched thread for it to access original
|
||||
// TaskContext
|
||||
// properties.
|
||||
TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext);
|
||||
List<WriteStatus> statuses = new LinkedList<>();
|
||||
try {
|
||||
statuses.addAll(handleWrite(bufferedIterator));
|
||||
logger.info("hoodie write is done; notifying reader thread");
|
||||
return statuses;
|
||||
} catch (Exception e) {
|
||||
logger.error("error writing hoodie records", e);
|
||||
bufferedIterator.markAsFailed(e);
|
||||
throw e;
|
||||
}
|
||||
});
|
||||
// Buffering records into internal buffer. This can throw exception either if reading
|
||||
// records from spark fails or
|
||||
// if writing buffered records into parquet file fails.
|
||||
bufferedIterator.startBuffering();
|
||||
logger.info("waiting for hoodie write to finish");
|
||||
@@ -110,28 +111,27 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
}
|
||||
|
||||
private List<WriteStatus> handleWrite(final BufferedIterator<T, HoodieRecord<T>> bufferedIterator) {
|
||||
private List<WriteStatus> handleWrite(
|
||||
final BufferedIterator<T, HoodieRecord<T>> bufferedIterator) {
|
||||
List<WriteStatus> statuses = new ArrayList<>();
|
||||
while (bufferedIterator.hasNext()) {
|
||||
final BufferedIterator.BufferedIteratorPayload<HoodieRecord<T>> payload = bufferedIterator.next();
|
||||
final BufferedIterator.BufferedIteratorPayload<HoodieRecord<T>> payload = bufferedIterator
|
||||
.next();
|
||||
|
||||
// clean up any partial failures
|
||||
if (!partitionsCleaned.contains(payload.record.getPartitionPath())) {
|
||||
// This insert task could fail multiple times, but Spark will faithfully retry with
|
||||
// the same data again. Thus, before we open any files under a given partition, we
|
||||
// first delete any files in the same partitionPath written by same Spark partition
|
||||
HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig,
|
||||
commitTime,
|
||||
payload.record.getPartitionPath(),
|
||||
TaskContext.getPartitionId(),
|
||||
hoodieTable);
|
||||
HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, commitTime,
|
||||
payload.record.getPartitionPath(), TaskContext.getPartitionId(), hoodieTable);
|
||||
partitionsCleaned.add(payload.record.getPartitionPath());
|
||||
}
|
||||
|
||||
// lazily initialize the handle, for the first time
|
||||
if (handle == null) {
|
||||
handle =
|
||||
new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, payload.record.getPartitionPath());
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable,
|
||||
payload.record.getPartitionPath());
|
||||
}
|
||||
|
||||
if (handle.canWrite(payload.record)) {
|
||||
@@ -141,9 +141,10 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
|
||||
// handle is full.
|
||||
statuses.add(handle.close());
|
||||
// Need to handle the rejected payload & open new handle
|
||||
handle =
|
||||
new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, payload.record.getPartitionPath());
|
||||
handle.write(payload.record, payload.insertValue, payload.exception); // we should be able to write 1 payload.
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable,
|
||||
payload.record.getPartitionPath());
|
||||
handle.write(payload.record, payload.insertValue,
|
||||
payload.exception); // we should be able to write 1 payload.
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -23,9 +23,9 @@ import java.util.Iterator;
|
||||
* inputItr classes in order to simplify the implementation of lazy iterators for mapPartitions use
|
||||
* cases. Note [SPARK-3369], which gives the reasons for backwards compatibility with regard to the
|
||||
* iterable API despite Spark's single pass nature.
|
||||
*
|
||||
* <p>
|
||||
* Provide a way to obtain a inputItr of type O (output), out of an inputItr of type I (input)
|
||||
*
|
||||
* <p>
|
||||
* Things to remember: - Assumes Spark calls hasNext() to check for elements, before calling next()
|
||||
* to obtain them - Assumes hasNext() gets called atleast once. - Concrete Implementation is
|
||||
* responsible for calling inputIterator.next() and doing the processing in computeNext()
|
||||
|
||||
@@ -37,22 +37,30 @@ import org.apache.spark.api.java.JavaSparkContext;
|
||||
*/
|
||||
public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
|
||||
public enum IndexType {
|
||||
HBASE,
|
||||
INMEMORY,
|
||||
BLOOM,
|
||||
BUCKETED
|
||||
}
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
|
||||
protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
this.config = config;
|
||||
this.jsc = jsc;
|
||||
}
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(HoodieWriteConfig config,
|
||||
JavaSparkContext jsc) throws HoodieIndexException {
|
||||
switch (config.getIndexType()) {
|
||||
case HBASE:
|
||||
return new HBaseIndex<>(config, jsc);
|
||||
case INMEMORY:
|
||||
return new InMemoryHashIndex<>(config, jsc);
|
||||
case BLOOM:
|
||||
return new HoodieBloomIndex<>(config, jsc);
|
||||
case BUCKETED:
|
||||
return new BucketedIndex<>(config, jsc);
|
||||
default:
|
||||
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]]
|
||||
* If the optional FullFilePath value is not present, then the key is not found. If the
|
||||
@@ -71,7 +79,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
|
||||
/**
|
||||
* Extracts the location of written records, and updates the index.
|
||||
*
|
||||
* <p>
|
||||
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
@@ -107,18 +115,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
public abstract boolean isImplicitWithStorage();
|
||||
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(
|
||||
HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException {
|
||||
switch (config.getIndexType()) {
|
||||
case HBASE:
|
||||
return new HBaseIndex<>(config, jsc);
|
||||
case INMEMORY:
|
||||
return new InMemoryHashIndex<>(config, jsc);
|
||||
case BLOOM:
|
||||
return new HoodieBloomIndex<>(config, jsc);
|
||||
case BUCKETED:
|
||||
return new BucketedIndex<>(config, jsc);
|
||||
}
|
||||
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
|
||||
public enum IndexType {
|
||||
HBASE, INMEMORY, BLOOM, BUCKETED
|
||||
}
|
||||
}
|
||||
|
||||
@@ -49,32 +49,11 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
final HoodieTable<T> table) {
|
||||
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
|
||||
}
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
class LocationTagFunction
|
||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
|
||||
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord<T> rec = hoodieRecordIterator.next();
|
||||
if (recordLocationMap.containsKey(rec.getKey())) {
|
||||
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
|
||||
}
|
||||
taggedRecords.add(rec);
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
@@ -132,4 +111,25 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
class LocationTagFunction implements
|
||||
Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
|
||||
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord<T> rec = hoodieRecordIterator.next();
|
||||
if (recordLocationMap.containsKey(rec.getKey())) {
|
||||
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
|
||||
}
|
||||
taggedRecords.add(rec);
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -64,8 +64,7 @@ public class BloomIndexFileInfo implements Serializable {
|
||||
* Does the given key fall within the range (inclusive)
|
||||
*/
|
||||
public boolean isKeyInRange(String recordKey) {
|
||||
return minRecordKey.compareTo(recordKey) <= 0 &&
|
||||
maxRecordKey.compareTo(recordKey) >= 0;
|
||||
return minRecordKey.compareTo(recordKey) <= 0 && maxRecordKey.compareTo(recordKey) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -78,9 +77,8 @@ public class BloomIndexFileInfo implements Serializable {
|
||||
}
|
||||
|
||||
BloomIndexFileInfo that = (BloomIndexFileInfo) o;
|
||||
return Objects.equal(that.fileName, fileName) &&
|
||||
Objects.equal(that.minRecordKey, minRecordKey) &&
|
||||
Objects.equal(that.maxRecordKey, maxRecordKey);
|
||||
return Objects.equal(that.fileName, fileName) && Objects.equal(that.minRecordKey, minRecordKey)
|
||||
&& Objects.equal(that.maxRecordKey, maxRecordKey);
|
||||
|
||||
}
|
||||
|
||||
|
||||
@@ -56,12 +56,11 @@ import scala.Tuple2;
|
||||
*/
|
||||
public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
|
||||
|
||||
// we need to limit the join such that it stays within 1.5GB per Spark partition. (SPARK-1476)
|
||||
private static final int SPARK_MAXIMUM_BYTES_PER_PARTITION = 1500 * 1024 * 1024;
|
||||
// this is how much a triplet of (partitionPath, fileId, recordKey) costs.
|
||||
private static final int BYTES_PER_PARTITION_FILE_KEY_TRIPLET = 300;
|
||||
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
|
||||
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION =
|
||||
SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
|
||||
|
||||
@@ -108,32 +107,31 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
return taggedRecordRDD;
|
||||
}
|
||||
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
||||
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
final HoodieTable<T> table) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = hoodieKeys
|
||||
.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
||||
lookupIndex(partitionRecordKeyPairRDD, table);
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD,
|
||||
table);
|
||||
|
||||
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
|
||||
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
|
||||
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD = hoodieKeys
|
||||
.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
|
||||
|
||||
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD)
|
||||
.mapToPair(keyPathTuple -> {
|
||||
Optional<String> recordLocationPath;
|
||||
if (keyPathTuple._2._2.isPresent()) {
|
||||
String fileName = keyPathTuple._2._2.get();
|
||||
String partitionPath = keyPathTuple._2._1.getPartitionPath();
|
||||
recordLocationPath = Optional.of(new Path(
|
||||
new Path(table.getMetaClient().getBasePath(), partitionPath),
|
||||
fileName).toUri().getPath());
|
||||
} else {
|
||||
recordLocationPath = Optional.absent();
|
||||
}
|
||||
return new Tuple2<>(keyPathTuple._2._1, recordLocationPath);
|
||||
});
|
||||
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).mapToPair(keyPathTuple -> {
|
||||
Optional<String> recordLocationPath;
|
||||
if (keyPathTuple._2._2.isPresent()) {
|
||||
String fileName = keyPathTuple._2._2.get();
|
||||
String partitionPath = keyPathTuple._2._1.getPartitionPath();
|
||||
recordLocationPath = Optional
|
||||
.of(new Path(new Path(table.getMetaClient().getBasePath(), partitionPath), fileName)
|
||||
.toUri().getPath());
|
||||
} else {
|
||||
recordLocationPath = Optional.absent();
|
||||
}
|
||||
return new Tuple2<>(keyPathTuple._2._1, recordLocationPath);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -152,21 +150,21 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
|
||||
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
|
||||
|
||||
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it.
|
||||
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id,
|
||||
// that contains it.
|
||||
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo,
|
||||
partitionRecordKeyPairRDD);
|
||||
return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo,
|
||||
partitionRecordKeyPairRDD,
|
||||
parallelism);
|
||||
partitionRecordKeyPairRDD, parallelism);
|
||||
}
|
||||
|
||||
/**
|
||||
* The index lookup can be skewed in three dimensions : #files, #partitions, #records
|
||||
*
|
||||
* <p>
|
||||
* To be able to smoothly handle skews, we need to compute how to split each partitions into
|
||||
* subpartitions. We do it here, in a way that keeps the amount of each Spark join partition to <
|
||||
* 2GB.
|
||||
*
|
||||
* <p>
|
||||
* If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified
|
||||
* as a NON-zero number, then that is used explicitly.
|
||||
*/
|
||||
@@ -184,7 +182,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
// records for a partition.
|
||||
Map<String, Long> filesPerPartition = partitionToFileInfo.entrySet().stream()
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, e -> Long.valueOf(e.getValue().size())));
|
||||
long totalFiles = 0, totalRecords = 0;
|
||||
long totalFiles = 0;
|
||||
long totalRecords = 0;
|
||||
for (String partitionPath : recordsPerPartition.keySet()) {
|
||||
long numRecords = recordsPerPartition.get(partitionPath);
|
||||
long numFiles =
|
||||
@@ -210,22 +209,22 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
|
||||
/**
|
||||
* Its crucial to pick the right parallelism.
|
||||
*
|
||||
* <p>
|
||||
* totalSubPartitions : this is deemed safe limit, to be nice with Spark. inputParallelism :
|
||||
* typically number of input file splits
|
||||
*
|
||||
* <p>
|
||||
* We pick the max such that, we are always safe, but go higher if say a there are a lot of input
|
||||
* files. (otherwise, we will fallback to number of partitions in input and end up with slow
|
||||
* performance)
|
||||
*/
|
||||
private int determineParallelism(int inputParallelism, int totalSubPartitions) {
|
||||
// If bloom index parallelism is set, use it to to check against the input parallelism and take the max
|
||||
// If bloom index parallelism is set, use it to to check against the input parallelism and
|
||||
// take the max
|
||||
int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
|
||||
int joinParallelism = Math.max(totalSubPartitions, indexParallelism);
|
||||
logger.info("InputParallelism: ${" + inputParallelism + "}, " +
|
||||
"IndexParallelism: ${" + config.getBloomIndexParallelism() + "}, " +
|
||||
"TotalSubParts: ${" + totalSubPartitions + "}, " +
|
||||
"Join Parallelism set to : " + joinParallelism);
|
||||
logger.info("InputParallelism: ${" + inputParallelism + "}, " + "IndexParallelism: ${" + config
|
||||
.getBloomIndexParallelism() + "}, " + "TotalSubParts: ${" + totalSubPartitions + "}, "
|
||||
+ "Join Parallelism set to : " + joinParallelism);
|
||||
return joinParallelism;
|
||||
}
|
||||
|
||||
@@ -237,36 +236,31 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
final HoodieTable<T> hoodieTable) {
|
||||
// Obtain the latest data files from all the partitions.
|
||||
List<Tuple2<String, HoodieDataFile>> dataFilesList = jsc
|
||||
.parallelize(partitions, Math.max(partitions.size(), 1))
|
||||
.flatMapToPair(partitionPath -> {
|
||||
java.util.Optional<HoodieInstant> latestCommitTime =
|
||||
hoodieTable.getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
.parallelize(partitions, Math.max(partitions.size(), 1)).flatMapToPair(partitionPath -> {
|
||||
java.util.Optional<HoodieInstant> latestCommitTime = hoodieTable.getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>();
|
||||
if (latestCommitTime.isPresent()) {
|
||||
filteredFiles =
|
||||
hoodieTable.getROFileSystemView().getLatestDataFilesBeforeOrOn(partitionPath,
|
||||
latestCommitTime.get().getTimestamp())
|
||||
.map(f -> new Tuple2<>(partitionPath, f))
|
||||
.collect(toList());
|
||||
filteredFiles = hoodieTable.getROFileSystemView()
|
||||
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
|
||||
.map(f -> new Tuple2<>(partitionPath, f)).collect(toList());
|
||||
}
|
||||
return filteredFiles.iterator();
|
||||
}).collect();
|
||||
|
||||
if (config.getBloomIndexPruneByRanges()) {
|
||||
// also obtain file ranges, if range pruning is enabled
|
||||
return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1))
|
||||
.mapToPair(ft -> {
|
||||
try {
|
||||
String[] minMaxKeys = ParquetUtils
|
||||
.readMinMaxRecordKeys(hoodieTable.getHadoopConf(),
|
||||
ft._2().getFileStatus().getPath());
|
||||
return new Tuple2<>(ft._1(),
|
||||
new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
|
||||
} catch (MetadataNotFoundException me) {
|
||||
logger.warn("Unable to find range metadata in file :" + ft._2());
|
||||
return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName()));
|
||||
}
|
||||
}).collect();
|
||||
return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1)).mapToPair(ft -> {
|
||||
try {
|
||||
String[] minMaxKeys = ParquetUtils
|
||||
.readMinMaxRecordKeys(hoodieTable.getHadoopConf(), ft._2().getFileStatus().getPath());
|
||||
return new Tuple2<>(ft._1(),
|
||||
new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
|
||||
} catch (MetadataNotFoundException me) {
|
||||
logger.warn("Unable to find range metadata in file :" + ft._2());
|
||||
return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName()));
|
||||
}
|
||||
}).collect();
|
||||
} else {
|
||||
return dataFilesList.stream()
|
||||
.map(ft -> new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName())))
|
||||
@@ -320,62 +314,60 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* (e.g: timestamp as prefix), the number of files to be compared gets cut down a lot from range
|
||||
* pruning.
|
||||
*/
|
||||
// sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on recordKey
|
||||
// sub-partition to ensure the records can be looked up against files & also prune
|
||||
// file<=>record comparisons based on recordKey
|
||||
// ranges in the index info.
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<String, Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
return partitionRecordKeyPairRDD
|
||||
.map(partitionRecordKeyPair -> {
|
||||
String recordKey = partitionRecordKeyPair._2();
|
||||
String partitionPath = partitionRecordKeyPair._1();
|
||||
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
|
||||
String recordKey = partitionRecordKeyPair._2();
|
||||
String partitionPath = partitionRecordKeyPair._1();
|
||||
|
||||
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
|
||||
List<Tuple2<String, Tuple2<String, HoodieKey>>> recordComparisons = new ArrayList<>();
|
||||
if (indexInfos
|
||||
!= null) { // could be null, if there are no files in a given partition yet.
|
||||
// for each candidate file in partition, that needs to be compared.
|
||||
for (BloomIndexFileInfo indexInfo : indexInfos) {
|
||||
if (shouldCompareWithFile(indexInfo, recordKey)) {
|
||||
recordComparisons.add(
|
||||
new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey),
|
||||
new Tuple2<>(indexInfo.getFileName(),
|
||||
new HoodieKey(recordKey, partitionPath))));
|
||||
}
|
||||
}
|
||||
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
|
||||
List<Tuple2<String, Tuple2<String, HoodieKey>>> recordComparisons = new ArrayList<>();
|
||||
if (indexInfos != null) { // could be null, if there are no files in a given partition yet.
|
||||
// for each candidate file in partition, that needs to be compared.
|
||||
for (BloomIndexFileInfo indexInfo : indexInfos) {
|
||||
if (shouldCompareWithFile(indexInfo, recordKey)) {
|
||||
recordComparisons.add(
|
||||
new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey),
|
||||
new Tuple2<>(indexInfo.getFileName(),
|
||||
new HoodieKey(recordKey, partitionPath))));
|
||||
}
|
||||
return recordComparisons;
|
||||
})
|
||||
.flatMapToPair(t -> t.iterator());
|
||||
}
|
||||
}
|
||||
return recordComparisons;
|
||||
}).flatMapToPair(t -> t.iterator());
|
||||
}
|
||||
|
||||
/**
|
||||
* Find out <RowKey, filename> pair. All workload grouped by file-level.
|
||||
*
|
||||
* <p>
|
||||
* Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such
|
||||
* that each RDD partition is a file, then for each file, we do (1) load bloom filter, (2) load
|
||||
* rowKeys, (3) Tag rowKey
|
||||
*
|
||||
* <p>
|
||||
* Make sure the parallelism is atleast the groupby parallelism for tagging location
|
||||
*/
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(HoodieTable hoodieTable,
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD,
|
||||
int totalSubpartitions) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int totalSubpartitions) {
|
||||
|
||||
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(),
|
||||
totalSubpartitions);
|
||||
|
||||
JavaPairRDD<String, Tuple2<String, HoodieKey>> fileSortedTripletRDD = explodeRecordRDDWithFileComparisons(
|
||||
partitionToFileIndexInfo, partitionRecordKeyPairRDD)
|
||||
// sort further based on filename, such that all checking for the file can happen within a single partition, on-the-fly
|
||||
.sortByKey(true, joinParallelism);
|
||||
JavaPairRDD<String, Tuple2<String, HoodieKey>> fileSortedTripletRDD =
|
||||
explodeRecordRDDWithFileComparisons(
|
||||
partitionToFileIndexInfo, partitionRecordKeyPairRDD)
|
||||
// sort further based on filename, such that all checking for the file can happen within
|
||||
// a single partition, on-the-fly
|
||||
.sortByKey(true, joinParallelism);
|
||||
|
||||
return fileSortedTripletRDD
|
||||
.mapPartitionsWithIndex(
|
||||
new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true)
|
||||
return fileSortedTripletRDD.mapPartitionsWithIndex(
|
||||
new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true)
|
||||
.flatMap(indexLookupResults -> indexLookupResults.iterator())
|
||||
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
|
||||
.flatMapToPair(lookupResult -> {
|
||||
@@ -391,25 +383,23 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* Tag the <rowKey, filename> back to the original HoodieRecord RDD.
|
||||
*/
|
||||
private JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD,
|
||||
JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||
JavaPairRDD<String, HoodieRecord<T>> rowKeyRecordPairRDD = recordRDD
|
||||
.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
|
||||
|
||||
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join.
|
||||
return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(
|
||||
v1 -> {
|
||||
HoodieRecord<T> record = v1._1();
|
||||
if (v1._2().isPresent()) {
|
||||
String filename = v1._2().get();
|
||||
if (filename != null && !filename.isEmpty()) {
|
||||
record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename),
|
||||
FSUtils.getFileId(filename)));
|
||||
}
|
||||
}
|
||||
return record;
|
||||
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null),
|
||||
// so we do left outer join.
|
||||
return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(v1 -> {
|
||||
HoodieRecord<T> record = v1._1();
|
||||
if (v1._2().isPresent()) {
|
||||
String filename = v1._2().get();
|
||||
if (filename != null && !filename.isEmpty()) {
|
||||
record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename),
|
||||
FSUtils.getFileId(filename)));
|
||||
}
|
||||
);
|
||||
}
|
||||
return record;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -41,7 +41,8 @@ import scala.Tuple2;
|
||||
* actual files
|
||||
*/
|
||||
public class HoodieBloomIndexCheckFunction implements
|
||||
Function2<Integer, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>>, Iterator<List<IndexLookupResult>>> {
|
||||
Function2<Integer, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>>,
|
||||
Iterator<List<IndexLookupResult>>> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieBloomIndexCheckFunction.class);
|
||||
|
||||
@@ -58,8 +59,7 @@ public class HoodieBloomIndexCheckFunction implements
|
||||
* Given a list of row keys and one file, return only row keys existing in that file.
|
||||
*/
|
||||
public static List<String> checkCandidatesAgainstFile(Configuration configuration,
|
||||
List<String> candidateRecordKeys,
|
||||
Path filePath) throws HoodieIndexException {
|
||||
List<String> candidateRecordKeys, Path filePath) throws HoodieIndexException {
|
||||
List<String> foundRecordKeys = new ArrayList<>();
|
||||
try {
|
||||
// Load all rowKeys from the file, to double-confirm
|
||||
@@ -86,6 +86,13 @@ public class HoodieBloomIndexCheckFunction implements
|
||||
return foundRecordKeys;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<IndexLookupResult>> call(Integer partition,
|
||||
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr)
|
||||
throws Exception {
|
||||
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
|
||||
}
|
||||
|
||||
class LazyKeyCheckIterator extends
|
||||
LazyIterableIterator<Tuple2<String, Tuple2<String, HoodieKey>>, List<IndexLookupResult>> {
|
||||
|
||||
@@ -143,7 +150,8 @@ public class HoodieBloomIndexCheckFunction implements
|
||||
|
||||
// if continue on current file)
|
||||
if (fileName.equals(currentFile)) {
|
||||
// check record key against bloom filter of current file & add to possible keys if needed
|
||||
// check record key against bloom filter of current file & add to possible keys if
|
||||
// needed
|
||||
if (bloomFilter.mightContain(recordKey)) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("#1 Adding " + recordKey + " as candidate for file " + fileName);
|
||||
@@ -201,12 +209,4 @@ public class HoodieBloomIndexCheckFunction implements
|
||||
protected void end() {
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Iterator<List<IndexLookupResult>> call(Integer partition,
|
||||
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr)
|
||||
throws Exception {
|
||||
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -38,9 +38,9 @@ import scala.Tuple2;
|
||||
/**
|
||||
* An `stateless` index implementation that will using a deterministic mapping function to determine
|
||||
* the fileID for a given record.
|
||||
*
|
||||
* <p>
|
||||
* Pros: - Fast
|
||||
*
|
||||
* <p>
|
||||
* Cons : - Need to tune the number of buckets per partition path manually (FIXME: Need to autotune
|
||||
* this) - Could increase write amplification on copy-on-write storage since inserts always rewrite
|
||||
* files - Not global.
|
||||
|
||||
@@ -27,12 +27,16 @@ import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException;
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
@@ -51,23 +55,18 @@ import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Hoodie Index implementation backed by HBase
|
||||
*/
|
||||
public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
private final static byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
|
||||
private final static byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
|
||||
private final static byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
|
||||
private final static byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
|
||||
|
||||
private static final byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
|
||||
private static final byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
|
||||
private static final byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
|
||||
private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HBaseIndex.class);
|
||||
|
||||
private static Connection hbaseConnection = null;
|
||||
private final String tableName;
|
||||
|
||||
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
@@ -77,14 +76,12 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
HoodieTable<T> table) {
|
||||
//TODO : Change/Remove filterExists in HoodieReadClient() and revisit
|
||||
throw new UnsupportedOperationException("HBase index does not implement check exist");
|
||||
}
|
||||
|
||||
private static Connection hbaseConnection = null;
|
||||
|
||||
private Connection getHBaseConnection() {
|
||||
Configuration hbaseConfig = HBaseConfiguration.create();
|
||||
String quorum = config.getHbaseZkQuorum();
|
||||
@@ -100,15 +97,15 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
}
|
||||
|
||||
/**
|
||||
* Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is closed when
|
||||
* JVM exits
|
||||
* Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is
|
||||
* closed when JVM exits
|
||||
*/
|
||||
private void addShutDownHook() {
|
||||
Runtime.getRuntime().addShutdownHook(new Thread() {
|
||||
public void run() {
|
||||
try {
|
||||
hbaseConnection.close();
|
||||
} catch(Exception e) {
|
||||
} catch (Exception e) {
|
||||
// fail silently for any sort of exception
|
||||
}
|
||||
}
|
||||
@@ -126,101 +123,103 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
// Check if the last commit ts for this row is 1) present in the timeline or
|
||||
// 2) is less than the first commit ts in the timeline
|
||||
return !commitTimeline.empty() && (commitTimeline.containsInstant(
|
||||
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs)) ||
|
||||
HoodieTimeline.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(),
|
||||
commitTs, HoodieTimeline.GREATER));
|
||||
return !commitTimeline.empty() && (commitTimeline
|
||||
.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))
|
||||
|| HoodieTimeline
|
||||
.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs,
|
||||
HoodieTimeline.GREATER));
|
||||
}
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
private Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>
|
||||
locationTagFunction(HoodieTable<T> hoodieTable) {
|
||||
locationTagFunction(HoodieTable<T> hoodieTable) {
|
||||
|
||||
return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>)
|
||||
(partitionNum, hoodieRecordIterator) -> {
|
||||
|
||||
Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize();
|
||||
Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize();
|
||||
|
||||
// Grab the global HBase connection
|
||||
synchronized (HBaseIndex.class) {
|
||||
if (hbaseConnection == null || hbaseConnection.isClosed()) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
HTable hTable = null;
|
||||
try {
|
||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||
List<Get> statements = new ArrayList<>();
|
||||
List<HoodieRecord> currentBatchOfRecords = new LinkedList<>();
|
||||
// Do the tagging.
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord rec = hoodieRecordIterator.next();
|
||||
statements.add(generateStatement(rec.getRecordKey()));
|
||||
currentBatchOfRecords.add(rec);
|
||||
// iterator till we reach batch size
|
||||
if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) {
|
||||
// get results for batch from Hbase
|
||||
Result[] results = hTable.get(statements);
|
||||
// clear statements to be GC'd
|
||||
statements.clear();
|
||||
for (Result result : results) {
|
||||
// first, attempt to grab location from HBase
|
||||
HoodieRecord currentRecord = currentBatchOfRecords.remove(0);
|
||||
if (result.getRow() != null) {
|
||||
String keyFromResult = Bytes.toString(result.getRow());
|
||||
String commitTs =
|
||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
|
||||
String fileId =
|
||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
||||
String partitionPath =
|
||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
|
||||
|
||||
if (checkIfValidCommit(hoodieTable, commitTs)) {
|
||||
currentRecord = new HoodieRecord(new HoodieKey(currentRecord.getRecordKey(),
|
||||
partitionPath), currentRecord.getData());
|
||||
currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
|
||||
taggedRecords.add(currentRecord);
|
||||
// the key from Result and the key being processed should be same
|
||||
assert (currentRecord.getRecordKey().contentEquals(keyFromResult));
|
||||
} else { //if commit is invalid, treat this as a new taggedRecord
|
||||
taggedRecords.add(currentRecord);
|
||||
}
|
||||
} else {
|
||||
taggedRecords.add(currentRecord);
|
||||
}
|
||||
// Grab the global HBase connection
|
||||
synchronized (HBaseIndex.class) {
|
||||
if (hbaseConnection == null || hbaseConnection.isClosed()) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException(
|
||||
"Failed to Tag indexed locations because of exception with HBase Client", e);
|
||||
} finally {
|
||||
if (hTable != null) {
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
HTable hTable = null;
|
||||
try {
|
||||
hTable.close();
|
||||
} catch (IOException e) {
|
||||
// Ignore
|
||||
}
|
||||
}
|
||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||
List<Get> statements = new ArrayList<>();
|
||||
List<HoodieRecord> currentBatchOfRecords = new LinkedList<>();
|
||||
// Do the tagging.
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord rec = hoodieRecordIterator.next();
|
||||
statements.add(generateStatement(rec.getRecordKey()));
|
||||
currentBatchOfRecords.add(rec);
|
||||
// iterator till we reach batch size
|
||||
if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) {
|
||||
// get results for batch from Hbase
|
||||
Result[] results = hTable.get(statements);
|
||||
// clear statements to be GC'd
|
||||
statements.clear();
|
||||
for (Result result : results) {
|
||||
// first, attempt to grab location from HBase
|
||||
HoodieRecord currentRecord = currentBatchOfRecords.remove(0);
|
||||
if (result.getRow() != null) {
|
||||
String keyFromResult = Bytes.toString(result.getRow());
|
||||
String commitTs = Bytes
|
||||
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
|
||||
String fileId = Bytes
|
||||
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
||||
String partitionPath = Bytes
|
||||
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
|
||||
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
};
|
||||
if (checkIfValidCommit(hoodieTable, commitTs)) {
|
||||
currentRecord = new HoodieRecord(
|
||||
new HoodieKey(currentRecord.getRecordKey(), partitionPath),
|
||||
currentRecord.getData());
|
||||
currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
|
||||
taggedRecords.add(currentRecord);
|
||||
// the key from Result and the key being processed should be same
|
||||
assert (currentRecord.getRecordKey().contentEquals(keyFromResult));
|
||||
} else { //if commit is invalid, treat this as a new taggedRecord
|
||||
taggedRecords.add(currentRecord);
|
||||
}
|
||||
} else {
|
||||
taggedRecords.add(currentRecord);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException(
|
||||
"Failed to Tag indexed locations because of exception with HBase Client", e);
|
||||
} finally {
|
||||
if (hTable != null) {
|
||||
try {
|
||||
hTable.close();
|
||||
} catch (IOException e) {
|
||||
// Ignore
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable), true);
|
||||
}
|
||||
|
||||
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
|
||||
|
||||
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition, statusIterator) -> {
|
||||
|
||||
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>
|
||||
updateLocationFunction() {
|
||||
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition,
|
||||
statusIterator) -> {
|
||||
Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize();
|
||||
|
||||
List<WriteStatus> writeStatusList = new ArrayList<>();
|
||||
@@ -292,16 +291,13 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
/**
|
||||
* Helper method to facilitate performing puts and deletes in Hbase
|
||||
* @param hTable
|
||||
* @param puts
|
||||
* @param deletes
|
||||
* @throws IOException
|
||||
*/
|
||||
private void doPutsAndDeletes(HTable hTable, List<Put> puts, List<Delete> deletes) throws IOException {
|
||||
if(puts.size() > 0) {
|
||||
private void doPutsAndDeletes(HTable hTable, List<Put> puts, List<Delete> deletes)
|
||||
throws IOException {
|
||||
if (puts.size() > 0) {
|
||||
hTable.put(puts);
|
||||
}
|
||||
if(deletes.size() > 0) {
|
||||
if (deletes.size() > 0) {
|
||||
hTable.delete(deletes);
|
||||
}
|
||||
hTable.flushCommits();
|
||||
@@ -311,7 +307,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true);
|
||||
}
|
||||
|
||||
@@ -323,7 +319,6 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
/**
|
||||
* Only looks up by recordKey
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
@@ -332,7 +327,6 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
/**
|
||||
* Mapping is available in HBase already.
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
@@ -341,7 +335,6 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
/**
|
||||
* Index needs to be explicitly updated after storage write.
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
|
||||
@@ -37,14 +37,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieAppendException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.util.SizeEstimator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
@@ -53,6 +45,13 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.util.SizeEstimator;
|
||||
|
||||
/**
|
||||
* IO Operation to append data onto an existing file.
|
||||
@@ -61,14 +60,13 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class);
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
private TableFileSystemView.RealtimeView fileSystemView;
|
||||
private final WriteStatus writeStatus;
|
||||
private final String fileId;
|
||||
private String partitionPath;
|
||||
private Iterator<HoodieRecord<T>> recordItr;
|
||||
List<IndexedRecord> recordList = new ArrayList<>();
|
||||
List<String> keysToDelete = new ArrayList<>();
|
||||
private TableFileSystemView.RealtimeView fileSystemView;
|
||||
private String partitionPath;
|
||||
private Iterator<HoodieRecord<T>> recordItr;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private long averageRecordSize = 0;
|
||||
@@ -76,11 +74,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private Writer writer;
|
||||
private boolean doInit = true;
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
writeStatus.setStat(new HoodieDeltaWriteStat());
|
||||
@@ -93,42 +88,40 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private void init(String partitionPath) {
|
||||
|
||||
// extract some information from the first record
|
||||
FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath)
|
||||
.filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId))
|
||||
.findFirst().get();
|
||||
// HACK(vc) This also assumes a base file. It will break, if appending without one.
|
||||
String latestValidFilePath = fileSlice.getDataFile().get().getFileName();
|
||||
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
|
||||
writeStatus.getStat().setPrevCommit(baseCommitTime);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
this.partitionPath = partitionPath;
|
||||
FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath)
|
||||
.filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)).findFirst()
|
||||
.get();
|
||||
// HACK(vc) This also assumes a base file. It will break, if appending without one.
|
||||
String latestValidFilePath = fileSlice.getDataFile().get().getFileName();
|
||||
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
|
||||
writeStatus.getStat().setPrevCommit(baseCommitTime);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
this.partitionPath = partitionPath;
|
||||
|
||||
try {
|
||||
this.writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(fileSlice.getLogFiles()
|
||||
.map(logFile -> logFile.getLogVersion())
|
||||
.max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
|
||||
.withSizeThreshold(config.getLogFileMaxSize())
|
||||
.withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
this.currentLogFile = writer.getLogFile();
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||
.setLogVersion(currentLogFile.getLogVersion());
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||
.setLogOffset(writer.getCurrentSize());
|
||||
} catch (Exception e) {
|
||||
logger.error("Error in update task at commit " + commitTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId
|
||||
+ " on commit " + commitTime + " on HDFS path " + hoodieTable
|
||||
.getMetaClient().getBasePath() + partitionPath, e);
|
||||
}
|
||||
Path path = new Path(partitionPath,
|
||||
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
|
||||
writeStatus.getStat().setPath(path.toString());
|
||||
try {
|
||||
this.writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(
|
||||
fileSlice.getLogFiles().map(logFile -> logFile.getLogVersion())
|
||||
.max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
|
||||
.withSizeThreshold(config.getLogFileMaxSize()).withFs(fs)
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
this.currentLogFile = writer.getLogFile();
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion());
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize());
|
||||
} catch (Exception e) {
|
||||
logger.error("Error in update task at commit " + commitTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
|
||||
+ commitTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath()
|
||||
+ partitionPath, e);
|
||||
}
|
||||
Path path = new Path(partitionPath,
|
||||
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
|
||||
writeStatus.getStat().setPath(path.toString());
|
||||
}
|
||||
|
||||
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
@@ -150,7 +143,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
}
|
||||
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a part of marking
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
hoodieRecord.deflate();
|
||||
return avroRecord;
|
||||
@@ -165,7 +159,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
// to make sure we don't append records with older (shorter) schema than already appended
|
||||
public void doAppend() {
|
||||
|
||||
int maxBlockSize = config.getLogFileDataBlockMaxSize(); int numberOfRecords = 0;
|
||||
int maxBlockSize = config.getLogFileDataBlockMaxSize();
|
||||
int numberOfRecords = 0;
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
|
||||
@@ -173,16 +168,17 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
HoodieRecord record = recordItr.next();
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
if(doInit) {
|
||||
if (doInit) {
|
||||
init(record.getPartitionPath());
|
||||
averageRecordSize = SizeEstimator.estimate(record);
|
||||
doInit = false;
|
||||
}
|
||||
// Append if max number of records reached to achieve block size
|
||||
if(numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) {
|
||||
// Recompute averageRecordSize before writing a new block and update existing value with avg of new and old
|
||||
if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) {
|
||||
// Recompute averageRecordSize before writing a new block and update existing value with
|
||||
// avg of new and old
|
||||
logger.info("AvgRecordSize => " + averageRecordSize);
|
||||
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record))/2;
|
||||
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2;
|
||||
doAppend(header);
|
||||
numberOfRecords = 0;
|
||||
}
|
||||
|
||||
@@ -68,9 +68,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
throws IOException {
|
||||
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
|
||||
.getCleanerFileVersionsRetained() + " file versions. ");
|
||||
List<HoodieFileGroup> fileGroups =
|
||||
fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
List<HoodieFileGroup> fileGroups = fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||
@@ -94,11 +93,9 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||
deletePaths.add(dataFile.getFileStatus().getPath().toString());
|
||||
if (hoodieTable.getMetaClient().getTableType()
|
||||
== HoodieTableType.MERGE_ON_READ) {
|
||||
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths.addAll(nextSlice.getLogFiles()
|
||||
.map(file -> file.getPath().toString())
|
||||
deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
@@ -121,8 +118,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath)
|
||||
throws IOException {
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
logger.info(
|
||||
"Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
||||
logger
|
||||
.info("Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
@@ -132,15 +129,14 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
// determine if we have enough commits, to start cleaning.
|
||||
if (commitTimeline.countInstants() > commitsRetained) {
|
||||
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
|
||||
List<HoodieFileGroup> fileGroups =
|
||||
fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
List<HoodieFileGroup> fileGroups = fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
List<FileSlice> fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList());
|
||||
HoodieDataFile dataFile = fileSliceList.get(0).getDataFile().get();
|
||||
String lastVersion = dataFile.getCommitTime();
|
||||
String lastVersionBeforeEarliestCommitToRetain =
|
||||
getLatestVersionBeforeCommit(fileSliceList, earliestCommitToRetain);
|
||||
String lastVersionBeforeEarliestCommitToRetain = getLatestVersionBeforeCommit(fileSliceList,
|
||||
earliestCommitToRetain);
|
||||
|
||||
// Ensure there are more than 1 version of the file (we only clean old files from updates)
|
||||
// i.e always spare the last commit.
|
||||
@@ -151,28 +147,26 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
// do not clean up a savepoint data file
|
||||
continue;
|
||||
}
|
||||
// Dont delete the latest commit and also the last commit before the earliest commit we are retaining
|
||||
// The window of commit retain == max query run time. So a query could be running which still
|
||||
// Dont delete the latest commit and also the last commit before the earliest commit we
|
||||
// are retaining
|
||||
// The window of commit retain == max query run time. So a query could be running which
|
||||
// still
|
||||
// uses this file.
|
||||
if (fileCommitTime.equals(lastVersion) || (
|
||||
lastVersionBeforeEarliestCommitToRetain != null && fileCommitTime
|
||||
.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||
if (fileCommitTime.equals(lastVersion) || (lastVersionBeforeEarliestCommitToRetain != null
|
||||
&& fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||
// move on to the next file
|
||||
continue;
|
||||
}
|
||||
|
||||
// Always keep the last commit
|
||||
if (HoodieTimeline.compareTimestamps(
|
||||
earliestCommitToRetain.getTimestamp(),
|
||||
fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
if (HoodieTimeline
|
||||
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this is a commit, that should be cleaned.
|
||||
deletePaths.add(aFile.getFileStatus().getPath().toString());
|
||||
if (hoodieTable.getMetaClient().getTableType()
|
||||
== HoodieTableType.MERGE_ON_READ) {
|
||||
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths.addAll(aSlice.getLogFiles()
|
||||
.map(file -> file.getPath().toString())
|
||||
deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
@@ -190,9 +184,10 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
HoodieInstant commitTime) {
|
||||
for (FileSlice file : fileSliceList) {
|
||||
String fileCommitTime = file.getDataFile().get().getCommitTime();
|
||||
if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
|
||||
if (HoodieTimeline
|
||||
.compareTimestamps(commitTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
|
||||
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the
|
||||
// one we want
|
||||
return fileCommitTime;
|
||||
}
|
||||
}
|
||||
@@ -213,8 +208,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
|
||||
}
|
||||
logger.info(
|
||||
deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
||||
logger.info(deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
||||
|
||||
return deletePaths;
|
||||
}
|
||||
@@ -227,8 +221,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
&& commitTimeline.countInstants() > commitsRetained) {
|
||||
earliestCommitToRetain =
|
||||
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
||||
earliestCommitToRetain = commitTimeline
|
||||
.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
||||
}
|
||||
return earliestCommitToRetain;
|
||||
}
|
||||
|
||||
@@ -40,13 +40,6 @@ import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.file.DataFileStream;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
@@ -54,6 +47,11 @@ import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Archiver to bound the growth of <action>.commit files
|
||||
@@ -76,11 +74,9 @@ public class HoodieCommitArchiveLog {
|
||||
private HoodieLogFormat.Writer openWriter() {
|
||||
try {
|
||||
if (this.writer == null) {
|
||||
return HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(archiveFilePath.getParent())
|
||||
return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent())
|
||||
.withFileId(archiveFilePath.getName())
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
|
||||
.withFs(metaClient.getFs())
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFs(metaClient.getFs())
|
||||
.overBaseCommit("").build();
|
||||
} else {
|
||||
return this.writer;
|
||||
@@ -136,21 +132,19 @@ public class HoodieCommitArchiveLog {
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION))
|
||||
.filterCompletedInstants();
|
||||
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
|
||||
.collect(Collectors.groupingBy(s -> s.getAction()))
|
||||
.entrySet()
|
||||
.stream()
|
||||
.map(i -> {
|
||||
.collect(Collectors.groupingBy(s -> s.getAction())).entrySet().stream().map(i -> {
|
||||
if (i.getValue().size() > maxCommitsToKeep) {
|
||||
return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep);
|
||||
} else {
|
||||
return new ArrayList<HoodieInstant>();
|
||||
}
|
||||
})
|
||||
.flatMap(i -> i.stream());
|
||||
}).flatMap(i -> i.stream());
|
||||
|
||||
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats
|
||||
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify
|
||||
// with logic above to avoid Stream.concats
|
||||
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
|
||||
// We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present.
|
||||
// We cannot have any holes in the commit timeline. We cannot archive any commits which are
|
||||
// made after the first savepoint present.
|
||||
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
|
||||
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
|
||||
// Actually do the commits
|
||||
@@ -169,16 +163,14 @@ public class HoodieCommitArchiveLog {
|
||||
log.info("Deleting instants " + archivedInstants);
|
||||
boolean success = true;
|
||||
for (HoodieInstant archivedInstant : archivedInstants) {
|
||||
Path commitFile =
|
||||
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||
Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||
try {
|
||||
if (metaClient.getFs().exists(commitFile)) {
|
||||
success &= metaClient.getFs().delete(commitFile, false);
|
||||
log.info("Archived and deleted instant file " + commitFile);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant,
|
||||
e);
|
||||
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e);
|
||||
}
|
||||
}
|
||||
return success;
|
||||
@@ -186,8 +178,8 @@ public class HoodieCommitArchiveLog {
|
||||
|
||||
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
|
||||
try {
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
|
||||
log.info("Wrapper schema " + wrapperSchema.toString());
|
||||
List<IndexedRecord> records = new ArrayList<>();
|
||||
@@ -247,6 +239,8 @@ public class HoodieCommitArchiveLog {
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw new UnsupportedOperationException("Action not fully supported yet");
|
||||
}
|
||||
return archivedMetaWrapper;
|
||||
}
|
||||
@@ -256,9 +250,8 @@ public class HoodieCommitArchiveLog {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
//Need this to ignore other public get() methods
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData =
|
||||
mapper.convertValue(hoodieCommitMetadata,
|
||||
com.uber.hoodie.avro.model.HoodieCommitMetadata.class);
|
||||
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData = mapper
|
||||
.convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class);
|
||||
return avroMetaData;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -49,8 +49,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTable<T> hoodieTable, String partitionPath) {
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
status.setFileId(UUID.randomUUID().toString());
|
||||
@@ -64,14 +64,11 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
}
|
||||
|
||||
try {
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
||||
commitTime,
|
||||
new Path(config.getBasePath()),
|
||||
new Path(config.getBasePath(), partitionPath));
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime,
|
||||
new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
this.storageWriter =
|
||||
HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema);
|
||||
this.storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, getStorageWriterPath(), hoodieTable, config, schema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to initialize HoodieStorageWriter for path " + getStorageWriterPath(), e);
|
||||
@@ -81,13 +78,12 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
|
||||
/**
|
||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||
*
|
||||
* <p>
|
||||
* - Whether it belongs to the same partitionPath as existing records - Whether the current file
|
||||
* written bytes lt max file size
|
||||
*/
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return storageWriter.canWrite() && record.getPartitionPath()
|
||||
.equals(status.getPartitionPath());
|
||||
return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -111,7 +107,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
recordsDeleted++;
|
||||
}
|
||||
status.markSuccess(record, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a part of marking
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
record.deflate();
|
||||
} catch (Throwable t) {
|
||||
@@ -126,9 +123,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
* Performs actions to durably, persist the current changes and returns a WriteStatus object
|
||||
*/
|
||||
public WriteStatus close() {
|
||||
logger.info(
|
||||
"Closing the file " + status.getFileId() + " as we are done with all the records "
|
||||
+ recordsWritten);
|
||||
logger.info("Closing the file " + status.getFileId() + " as we are done with all the records "
|
||||
+ recordsWritten);
|
||||
try {
|
||||
storageWriter.close();
|
||||
|
||||
@@ -144,8 +140,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
|
||||
return status;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path,
|
||||
e);
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -39,11 +39,10 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
protected HoodieTimeline hoodieTimeline;
|
||||
protected final Schema schema;
|
||||
protected HoodieTimeline hoodieTimeline;
|
||||
|
||||
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.fs = hoodieTable.getMetaClient().getFs();
|
||||
@@ -52,6 +51,32 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
this.schema = createHoodieWriteSchema(config);
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes any new tmp files written during the current commit, into the partition
|
||||
*/
|
||||
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config, String commitTime,
|
||||
String partitionPath, int taskPartitionId, HoodieTable hoodieTable) {
|
||||
FileSystem fs = hoodieTable.getMetaClient().getFs();
|
||||
try {
|
||||
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
FSUtils.maskWithoutFileId(commitTime, taskPartitionId))));
|
||||
if (prevFailedFiles != null) {
|
||||
logger.info(
|
||||
"Deleting " + prevFailedFiles.length + " files generated by previous failed attempts.");
|
||||
for (FileStatus status : prevFailedFiles) {
|
||||
fs.delete(status.getPath(), false);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
public static Schema createHoodieWriteSchema(HoodieWriteConfig config) {
|
||||
return HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
}
|
||||
|
||||
public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) {
|
||||
Path path = new Path(config.getBasePath(), partitionPath);
|
||||
try {
|
||||
@@ -72,37 +97,7 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
taskAttemptId));
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes any new tmp files written during the current commit, into the partition
|
||||
*/
|
||||
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
String partitionPath,
|
||||
int taskPartitionId,
|
||||
HoodieTable hoodieTable) {
|
||||
FileSystem fs = hoodieTable.getMetaClient().getFs();
|
||||
try {
|
||||
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
FSUtils.maskWithoutFileId(commitTime, taskPartitionId))));
|
||||
if (prevFailedFiles != null) {
|
||||
logger.info("Deleting " + prevFailedFiles.length
|
||||
+ " files generated by previous failed attempts.");
|
||||
for (FileStatus status : prevFailedFiles) {
|
||||
fs.delete(status.getPath(), false);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
public Schema getSchema() {
|
||||
return schema;
|
||||
}
|
||||
|
||||
public static Schema createHoodieWriteSchema(HoodieWriteConfig config) {
|
||||
return HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -26,14 +26,18 @@ import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ReflectionUtils;
|
||||
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
|
||||
import com.uber.hoodie.common.util.collection.converter.StringConverter;
|
||||
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
|
||||
import com.uber.hoodie.common.util.collection.converter.StringConverter;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -41,11 +45,6 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
@SuppressWarnings("Duplicates")
|
||||
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
|
||||
@@ -62,59 +61,46 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
private long recordsDeleted = 0;
|
||||
private long updatedRecordsWritten = 0;
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr,
|
||||
String fileId) {
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String fileId) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
this.fileSystemView = hoodieTable.getROFileSystemView();
|
||||
init(fileId, init(fileId, recordItr));
|
||||
}
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords,
|
||||
String fileId) {
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, String fileId) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
this.fileSystemView = hoodieTable.getROFileSystemView();
|
||||
this.keyToNewRecords = keyToNewRecords;
|
||||
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get()).getPartitionPath());
|
||||
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get())
|
||||
.getPartitionPath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Extract old file path, initialize StorageWriter and WriteStatus
|
||||
* @param fileId
|
||||
* @param partitionPath
|
||||
*/
|
||||
private void init(String fileId, String partitionPath) {
|
||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
try {
|
||||
String latestValidFilePath = fileSystemView
|
||||
.getLatestDataFiles(partitionPath)
|
||||
.filter(dataFile -> dataFile.getFileId().equals(fileId))
|
||||
.findFirst()
|
||||
.get().getFileName();
|
||||
String latestValidFilePath = fileSystemView.getLatestDataFiles(partitionPath)
|
||||
.filter(dataFile -> dataFile.getFileId().equals(fileId)).findFirst().get().getFileName();
|
||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
||||
commitTime,
|
||||
new Path(config.getBasePath()),
|
||||
new Path(config.getBasePath(), partitionPath));
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime,
|
||||
new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
|
||||
oldFilePath = new Path(
|
||||
config.getBasePath() + "/" + partitionPath + "/"
|
||||
+ latestValidFilePath);
|
||||
config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath);
|
||||
String relativePath = new Path(partitionPath + "/" + FSUtils
|
||||
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
|
||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||
if (config.shouldUseTempFolderForCopyOnWriteForMerge()) {
|
||||
this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(),
|
||||
fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId());
|
||||
this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(), fileId,
|
||||
TaskContext.get().stageId(), TaskContext.get().taskAttemptId());
|
||||
}
|
||||
|
||||
// handle cases of partial failures, for update task
|
||||
@@ -122,8 +108,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
fs.delete(newFilePath, false);
|
||||
}
|
||||
|
||||
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
|
||||
oldFilePath.toString(), getStorageWriterPath().toString()));
|
||||
logger.info(String
|
||||
.format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(),
|
||||
getStorageWriterPath().toString()));
|
||||
// file name is same for all records, in this bunch
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
@@ -143,9 +130,6 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
|
||||
/**
|
||||
* Load the new incoming records in a map and return partitionPath
|
||||
* @param fileId
|
||||
* @param newRecordsItr
|
||||
* @return
|
||||
*/
|
||||
private String init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
try {
|
||||
@@ -153,7 +137,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge());
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(),
|
||||
Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, config.getPayloadClass()));
|
||||
} catch(IOException io) {
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
||||
}
|
||||
String partitionPath = null;
|
||||
@@ -164,14 +148,14 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
}
|
||||
logger.debug("Number of entries in MemoryBasedMap => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries()
|
||||
+ "Total size in bytes of MemoryBasedMap => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize()
|
||||
+ "Number of entries in DiskBasedMap => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries()
|
||||
+ "Size of file spilled to disk => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
|
||||
logger.debug("Number of entries in MemoryBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries()
|
||||
+ "Total size in bytes of MemoryBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize()
|
||||
+ "Number of entries in DiskBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries()
|
||||
+ "Size of file spilled to disk => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
|
||||
|
||||
return partitionPath;
|
||||
}
|
||||
@@ -189,7 +173,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
}
|
||||
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a part of marking
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
hoodieRecord.deflate();
|
||||
return true;
|
||||
@@ -201,8 +186,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
}
|
||||
|
||||
/**
|
||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to
|
||||
* the file.
|
||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
|
||||
*/
|
||||
public void write(GenericRecord oldRecord) {
|
||||
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
@@ -213,12 +197,12 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
|
||||
.combineAndGetUpdateValue(oldRecord, schema);
|
||||
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
||||
/* ONLY WHEN
|
||||
* 1) we have an update for this key AND
|
||||
* 2) We are able to successfully write the the combined new value
|
||||
*
|
||||
* We no longer need to copy the old record over.
|
||||
*/
|
||||
/* ONLY WHEN
|
||||
* 1) we have an update for this key AND
|
||||
* 2) We are able to successfully write the the combined new value
|
||||
*
|
||||
* We no longer need to copy the old record over.
|
||||
*/
|
||||
copyOldRecord = false;
|
||||
}
|
||||
keyToNewRecords.remove(key);
|
||||
@@ -236,10 +220,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
try {
|
||||
storageWriter.writeAvro(key, oldRecord);
|
||||
} catch (ClassCastException e) {
|
||||
logger.error(
|
||||
"Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||
+ getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema
|
||||
.toString(true));
|
||||
logger.error("Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||
+ getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema
|
||||
.toString(true));
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
} catch (IOException e) {
|
||||
logger.error("Failed to merge old record into new file for key " + key + " from old file "
|
||||
|
||||
@@ -53,8 +53,8 @@ public class CompactionOperation implements Serializable {
|
||||
this.partitionPath = partitionPath;
|
||||
this.dataFileCommitTime = dataFile.getCommitTime();
|
||||
this.dataFileSize = dataFile.getFileSize();
|
||||
this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()).collect(
|
||||
Collectors.toList());
|
||||
this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString())
|
||||
.collect(Collectors.toList());
|
||||
this.metrics = writeConfig.getCompactionStrategy()
|
||||
.captureMetrics(dataFile, partitionPath, logFiles);
|
||||
}
|
||||
|
||||
@@ -17,17 +17,15 @@
|
||||
package com.uber.hoodie.io.compact;
|
||||
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Date;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* A HoodieCompactor runs compaction on a hoodie table
|
||||
@@ -38,7 +36,7 @@ public interface HoodieCompactor extends Serializable {
|
||||
* Compact the delta files with the data files
|
||||
*/
|
||||
JavaRDD<WriteStatus> compact(JavaSparkContext jsc, final HoodieWriteConfig config,
|
||||
HoodieTable hoodieTable, String compactionCommitTime) throws Exception;
|
||||
HoodieTable hoodieTable, String compactionCommitTime) throws Exception;
|
||||
|
||||
|
||||
// Helper methods
|
||||
|
||||
@@ -17,6 +17,7 @@
|
||||
package com.uber.hoodie.io.compact;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
@@ -70,9 +71,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
}
|
||||
|
||||
private JavaRDD<WriteStatus> executeCompaction(JavaSparkContext jsc,
|
||||
List<CompactionOperation> operations,
|
||||
HoodieTable hoodieTable,
|
||||
HoodieWriteConfig config, String compactionCommitTime) throws IOException {
|
||||
List<CompactionOperation> operations, HoodieTable hoodieTable, HoodieWriteConfig config,
|
||||
String compactionCommitTime) throws IOException {
|
||||
|
||||
log.info("After filtering, Compacting " + operations + " files");
|
||||
return jsc.parallelize(operations, operations.size())
|
||||
@@ -80,18 +80,19 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
.flatMap(writeStatusesItr -> writeStatusesItr.iterator());
|
||||
}
|
||||
|
||||
private List<WriteStatus> compact(HoodieTable hoodieTable,
|
||||
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
|
||||
throws IOException {
|
||||
private List<WriteStatus> compact(HoodieTable hoodieTable, HoodieWriteConfig config,
|
||||
CompactionOperation operation, String commitTime) throws IOException {
|
||||
FileSystem fs = hoodieTable.getMetaClient().getFs();
|
||||
Schema readerSchema =
|
||||
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
Schema readerSchema = HoodieAvroUtils
|
||||
.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
|
||||
log.info("Compacting base " + operation.getDataFilePath() + " with delta files " + operation
|
||||
.getDeltaFilePaths() + " for commit " + commitTime);
|
||||
// TODO - FIX THIS
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file (failure recover).
|
||||
// Load all the delta commits since the last compaction commit and get all the blocks to be loaded and load it using CompositeAvroLogReader
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file
|
||||
// (failure recover).
|
||||
// Load all the delta commits since the last compaction commit and get all the blocks to be
|
||||
// loaded and load it using CompositeAvroLogReader
|
||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
String maxInstantTime = metaClient.getActiveTimeline()
|
||||
@@ -114,52 +115,47 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
Iterator<List<WriteStatus>> result = table
|
||||
.handleUpdate(commitTime, operation.getFileId(), scanner.getRecords());
|
||||
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
||||
return StreamSupport.stream(resultIterable.spliterator(), false)
|
||||
.flatMap(Collection::stream)
|
||||
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
|
||||
.map(s -> {
|
||||
s.getStat().setTotalRecordsToBeUpdate(scanner.getTotalRecordsToUpdate());
|
||||
s.getStat().setTotalLogFiles(scanner.getTotalLogFiles());
|
||||
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
|
||||
s.getStat().setPartitionPath(operation.getPartitionPath());
|
||||
return s;
|
||||
})
|
||||
.collect(toList());
|
||||
}).collect(toList());
|
||||
}
|
||||
|
||||
private List<CompactionOperation> getCompactionWorkload(JavaSparkContext jsc,
|
||||
HoodieTable hoodieTable,
|
||||
HoodieWriteConfig config, String compactionCommitTime)
|
||||
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime)
|
||||
throws IOException {
|
||||
|
||||
Preconditions.checkArgument(
|
||||
hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
|
||||
"HoodieRealtimeTableCompactor can only compact table of type "
|
||||
+ HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient()
|
||||
.getTableType().name());
|
||||
Preconditions
|
||||
.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
|
||||
"HoodieRealtimeTableCompactor can only compact table of type "
|
||||
+ HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient()
|
||||
.getTableType().name());
|
||||
|
||||
//TODO : check if maxMemory is not greater than JVM or spark.executor memory
|
||||
// TODO - rollback any compactions in flight
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
|
||||
List<String> partitionPaths =
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
List<String> partitionPaths = FSUtils
|
||||
.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
|
||||
TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView();
|
||||
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
List<CompactionOperation> operations =
|
||||
jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.map(s -> new CompactionOperation(s.getDataFile().get(),
|
||||
partitionPath,
|
||||
List<CompactionOperation> operations = jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath).map(
|
||||
s -> new CompactionOperation(s.getDataFile().get(), partitionPath,
|
||||
s.getLogFiles().sorted(HoodieLogFile.getLogVersionComparator().reversed())
|
||||
.collect(Collectors.toList()), config))
|
||||
.filter(c -> !c.getDeltaFilePaths().isEmpty())
|
||||
.collect(toList()).iterator()).collect();
|
||||
.filter(c -> !c.getDeltaFilePaths().isEmpty()).collect(toList()).iterator()).collect();
|
||||
log.info("Total of " + operations.size() + " compactions are retrieved");
|
||||
|
||||
// Filter the compactions with the passed in filter. This lets us choose most effective compactions only
|
||||
// Filter the compactions with the passed in filter. This lets us choose most effective
|
||||
// compactions only
|
||||
operations = config.getCompactionStrategy().orderAndFilter(config, operations);
|
||||
if (operations.isEmpty()) {
|
||||
log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
|
||||
|
||||
@@ -44,9 +44,9 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy {
|
||||
List<HoodieLogFile> logFiles) {
|
||||
Map<String, Object> metrics = Maps.newHashMap();
|
||||
// Total size of all the log files
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(
|
||||
Optional::isPresent).map(Optional::get).reduce(
|
||||
(size1, size2) -> size1 + size2).orElse(0L);
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize)
|
||||
.filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2)
|
||||
.orElse(0L);
|
||||
// Total read will be the base file + all the log files
|
||||
Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize);
|
||||
// Total write will be similar to the size of the base file
|
||||
@@ -64,7 +64,8 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy {
|
||||
@Override
|
||||
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<CompactionOperation> operations) {
|
||||
// Iterate through the operations in order and accept operations as long as we are within the IO limit
|
||||
// Iterate through the operations in order and accept operations as long as we are within the
|
||||
// IO limit
|
||||
// Preserves the original ordering of compactions
|
||||
List<CompactionOperation> finalOperations = Lists.newArrayList();
|
||||
long targetIORemaining = writeConfig.getTargetIOPerCompactionInMB();
|
||||
|
||||
@@ -28,7 +28,7 @@ import java.util.Map;
|
||||
* Strategy for compaction. Pluggable implementation of define how compaction should be done. The
|
||||
* implementations of this interface can capture the relevant metrics to order and filter the final
|
||||
* list of compaction operation to run in a single compaction.
|
||||
*
|
||||
* <p>
|
||||
* Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be
|
||||
* passed in every time
|
||||
*
|
||||
@@ -41,9 +41,9 @@ public interface CompactionStrategy extends Serializable {
|
||||
* Callback hook when a CompactionOperation is created. Individual strategies can capture the
|
||||
* metrics they need to decide on the priority.
|
||||
*
|
||||
* @param dataFile - Base file to compact
|
||||
* @param dataFile - Base file to compact
|
||||
* @param partitionPath - Partition path
|
||||
* @param logFiles - List of log files to compact with the base file
|
||||
* @param logFiles - List of log files to compact with the base file
|
||||
* @return Map[String, Object] - metrics captured
|
||||
*/
|
||||
Map<String, Object> captureMetrics(HoodieDataFile dataFile, String partitionPath,
|
||||
@@ -54,7 +54,7 @@ public interface CompactionStrategy extends Serializable {
|
||||
* order and filter out compactions
|
||||
*
|
||||
* @param writeConfig - HoodieWriteConfig - config for this compaction is passed in
|
||||
* @param operations - list of compactions collected
|
||||
* @param operations - list of compactions collected
|
||||
* @return list of compactions to perform in this run
|
||||
*/
|
||||
List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
|
||||
@@ -20,7 +20,6 @@ package com.uber.hoodie.io.compact.strategy;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.io.compact.CompactionOperation;
|
||||
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Comparator;
|
||||
@@ -30,20 +29,21 @@ import java.util.Locale;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* This strategy orders compactions in reverse order of creation of Hive Partitions.
|
||||
* It helps to compact data in latest partitions first and then older capped at the Total_IO allowed.
|
||||
* This strategy orders compactions in reverse order of creation of Hive Partitions. It helps to
|
||||
* compact data in latest partitions first and then older capped at the Total_IO allowed.
|
||||
*/
|
||||
public class DayBasedCompactionStrategy extends BoundedIOCompactionStrategy {
|
||||
|
||||
// For now, use SimpleDateFormat as default partition format
|
||||
private static String datePartitionFormat = "yyyy/MM/dd";
|
||||
// Sorts compaction in LastInFirstCompacted order
|
||||
private static Comparator<CompactionOperation> comparator = (CompactionOperation leftC, CompactionOperation rightC) -> {
|
||||
private static Comparator<CompactionOperation> comparator = (CompactionOperation leftC,
|
||||
CompactionOperation rightC) -> {
|
||||
try {
|
||||
Date left = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH)
|
||||
.parse(leftC.getPartitionPath());
|
||||
.parse(leftC.getPartitionPath());
|
||||
Date right = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH)
|
||||
.parse(rightC.getPartitionPath());
|
||||
.parse(rightC.getPartitionPath());
|
||||
return left.after(right) ? -1 : right.after(left) ? 1 : 0;
|
||||
} catch (ParseException e) {
|
||||
throw new HoodieException("Invalid Partition Date Format", e);
|
||||
@@ -55,8 +55,10 @@ public class DayBasedCompactionStrategy extends BoundedIOCompactionStrategy {
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig, List<CompactionOperation> operations) {
|
||||
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<CompactionOperation> operations) {
|
||||
// Iterate through the operations and accept operations as long as we are within the IO limit
|
||||
return super.orderAndFilter(writeConfig, operations.stream().sorted(comparator).collect(Collectors.toList()));
|
||||
return super.orderAndFilter(writeConfig,
|
||||
operations.stream().sorted(comparator).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
@@ -44,9 +44,9 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat
|
||||
|
||||
Map<String, Object> metrics = super.captureMetrics(dataFile, partitionPath, logFiles);
|
||||
// Total size of all the log files
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(
|
||||
Optional::isPresent).map(Optional::get).reduce(
|
||||
(size1, size2) -> size1 + size2).orElse(0L);
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize)
|
||||
.filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2)
|
||||
.orElse(0L);
|
||||
// save the metrics needed during the order
|
||||
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize);
|
||||
return metrics;
|
||||
|
||||
@@ -36,8 +36,8 @@ import org.apache.spark.TaskContext;
|
||||
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides
|
||||
* a way to check if the current file can take more records with the <code>canWrite()</code>
|
||||
*/
|
||||
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord>
|
||||
extends ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
|
||||
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord> extends
|
||||
ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
|
||||
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
@@ -49,6 +49,29 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
private final Schema schema;
|
||||
|
||||
|
||||
public HoodieParquetWriter(String commitTime, Path file, HoodieParquetConfig parquetConfig,
|
||||
Schema schema) throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(),
|
||||
parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(),
|
||||
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED,
|
||||
ParquetWriter.DEFAULT_WRITER_VERSION,
|
||||
registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file
|
||||
.getFileSystem(registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
// We cannot accurately measure the snappy compressed output file size. We are choosing a
|
||||
// conservative 10%
|
||||
// TODO - compute this compression ratio dynamically by looking at the bytes written to the
|
||||
// stream and the actual file size reported by HDFS
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
|
||||
.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.commitTime = commitTime;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
private static Configuration registerFileSystem(Path file, Configuration conf) {
|
||||
Configuration returnConf = new Configuration(conf);
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
@@ -57,37 +80,12 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
return returnConf;
|
||||
}
|
||||
|
||||
public HoodieParquetWriter(String commitTime, Path file,
|
||||
HoodieParquetConfig parquetConfig, Schema schema) throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(),
|
||||
parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(),
|
||||
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
|
||||
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION,
|
||||
registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
this.file =
|
||||
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file
|
||||
.getFileSystem(registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
|
||||
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
|
||||
.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.commitTime = commitTime;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord,
|
||||
record.getRecordKey(),
|
||||
record.getPartitionPath(),
|
||||
file.getName());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(),
|
||||
record.getPartitionPath(), file.getName());
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
|
||||
@@ -30,22 +30,22 @@ import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
public class HoodieStorageWriterFactory {
|
||||
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
||||
String commitTime, Path path, HoodieTable<T> hoodieTable, HoodieWriteConfig config,
|
||||
Schema schema)
|
||||
throws IOException {
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R>
|
||||
getStorageWriter(String commitTime, Path path, HoodieTable<T> hoodieTable,
|
||||
HoodieWriteConfig config, Schema schema) throws IOException {
|
||||
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
|
||||
// Currently only parquet is supported
|
||||
return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
|
||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R>
|
||||
newParquetStorageWriter(
|
||||
String commitTime, Path path, HoodieWriteConfig config, Schema schema,
|
||||
HoodieTable hoodieTable) throws IOException {
|
||||
BloomFilter filter =
|
||||
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(),
|
||||
config.getBloomFilterFPP());
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
|
||||
new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
|
||||
HoodieParquetConfig parquetConfig =
|
||||
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
|
||||
@@ -59,8 +59,8 @@ import org.apache.hadoop.util.Progressable;
|
||||
*/
|
||||
public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
private static final Set<String> SUPPORT_SCHEMES;
|
||||
public static final String HOODIE_SCHEME_PREFIX = "hoodie-";
|
||||
private static final Set<String> SUPPORT_SCHEMES;
|
||||
|
||||
static {
|
||||
SUPPORT_SCHEMES = new HashSet<>();
|
||||
@@ -69,18 +69,50 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
SUPPORT_SCHEMES.add("s3");
|
||||
SUPPORT_SCHEMES.add("s3a");
|
||||
|
||||
|
||||
// Hoodie currently relies on underlying object store being fully
|
||||
// consistent so only regional buckets should be used.
|
||||
SUPPORT_SCHEMES.add("gs");
|
||||
SUPPORT_SCHEMES.add("viewfs");
|
||||
}
|
||||
|
||||
private ConcurrentMap<String, SizeAwareFSDataOutputStream> openStreams =
|
||||
new ConcurrentHashMap<>();
|
||||
private ConcurrentMap<String, SizeAwareFSDataOutputStream> openStreams = new
|
||||
ConcurrentHashMap<>();
|
||||
private FileSystem fileSystem;
|
||||
private URI uri;
|
||||
|
||||
public static Path convertToHoodiePath(Path file, Configuration conf) {
|
||||
try {
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
return convertPathWithScheme(file, getHoodieScheme(scheme));
|
||||
} catch (HoodieIOException e) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private static Path convertPathWithScheme(Path oldPath, String newScheme) {
|
||||
URI oldURI = oldPath.toUri();
|
||||
URI newURI;
|
||||
try {
|
||||
newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(),
|
||||
oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment());
|
||||
return new Path(newURI);
|
||||
} catch (URISyntaxException e) {
|
||||
// TODO - Better Exception handling
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static String getHoodieScheme(String scheme) {
|
||||
String newScheme;
|
||||
if (SUPPORT_SCHEMES.contains(scheme)) {
|
||||
newScheme = HOODIE_SCHEME_PREFIX + scheme;
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"BlockAlignedAvroParquetWriter does not support scheme " + scheme);
|
||||
}
|
||||
return newScheme;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(URI uri, Configuration conf) throws IOException {
|
||||
// Get the default filesystem to decorate
|
||||
@@ -90,7 +122,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
path = new Path(path.toString().replace(HOODIE_SCHEME_PREFIX, ""));
|
||||
}
|
||||
this.fileSystem = FSUtils.getFs(path.toString(), conf);
|
||||
// Do not need to explicitly initialize the default filesystem, its done already in the above FileSystem.get
|
||||
// Do not need to explicitly initialize the default filesystem, its done already in the above
|
||||
// FileSystem.get
|
||||
// fileSystem.initialize(FileSystem.getDefaultUri(conf), conf);
|
||||
// fileSystem.setConf(conf);
|
||||
this.uri = uri;
|
||||
@@ -108,8 +141,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress)
|
||||
throws IOException {
|
||||
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
|
||||
final Path translatedPath = convertToDefaultPath(f);
|
||||
return wrapOutputStream(f, fileSystem
|
||||
.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize,
|
||||
@@ -122,8 +154,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
return fsDataOutputStream;
|
||||
}
|
||||
|
||||
SizeAwareFSDataOutputStream os =
|
||||
new SizeAwareFSDataOutputStream(fsDataOutputStream, new Runnable() {
|
||||
SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(fsDataOutputStream,
|
||||
new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
openStreams.remove(path.getName());
|
||||
@@ -160,14 +192,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize)
|
||||
throws IOException {
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException {
|
||||
return fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
|
||||
Progressable progress) throws IOException {
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress)
|
||||
throws IOException {
|
||||
return fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress);
|
||||
}
|
||||
|
||||
@@ -175,14 +206,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
|
||||
long blockSize, Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize,
|
||||
progress);
|
||||
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress)
|
||||
throws IOException {
|
||||
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize,
|
||||
progress);
|
||||
@@ -197,7 +226,6 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
progress, checksumOpt);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication,
|
||||
long blockSize) throws IOException {
|
||||
@@ -205,7 +233,6 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream append(Path f, int bufferSize, Progressable progress)
|
||||
throws IOException {
|
||||
@@ -228,13 +255,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWorkingDirectory(Path new_dir) {
|
||||
fileSystem.setWorkingDirectory(convertToDefaultPath(new_dir));
|
||||
public Path getWorkingDirectory() {
|
||||
return convertToHoodiePath(fileSystem.getWorkingDirectory());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getWorkingDirectory() {
|
||||
return convertToHoodiePath(fileSystem.getWorkingDirectory());
|
||||
public void setWorkingDirectory(Path newDir) {
|
||||
fileSystem.setWorkingDirectory(convertToDefaultPath(newDir));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -290,8 +317,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public BlockLocation[] getFileBlockLocations(Path p, long start, long len)
|
||||
throws IOException {
|
||||
public BlockLocation[] getFileBlockLocations(Path p, long start, long len) throws IOException {
|
||||
return fileSystem.getFileBlockLocations(convertToDefaultPath(p), start, len);
|
||||
}
|
||||
|
||||
@@ -319,17 +345,16 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize,
|
||||
short replication, long blockSize, Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.createNonRecursive(convertToDefaultPath(f), overwrite, bufferSize, replication,
|
||||
blockSize, progress);
|
||||
.createNonRecursive(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize,
|
||||
progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress)
|
||||
throws IOException {
|
||||
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
|
||||
return fileSystem
|
||||
.createNonRecursive(convertToDefaultPath(f), permission, overwrite, bufferSize,
|
||||
replication, blockSize, progress);
|
||||
.createNonRecursive(convertToDefaultPath(f), permission, overwrite, bufferSize, replication,
|
||||
blockSize, progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -418,20 +443,17 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path f, PathFilter filter)
|
||||
throws IOException {
|
||||
public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException {
|
||||
return fileSystem.listStatus(convertToDefaultPath(f), filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path[] files)
|
||||
throws IOException {
|
||||
public FileStatus[] listStatus(Path[] files) throws IOException {
|
||||
return fileSystem.listStatus(convertDefaults(files));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path[] files, PathFilter filter)
|
||||
throws IOException {
|
||||
public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException {
|
||||
return fileSystem.listStatus(convertDefaults(files), filter);
|
||||
}
|
||||
|
||||
@@ -441,20 +463,17 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
|
||||
throws IOException {
|
||||
public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException {
|
||||
return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
|
||||
throws IOException {
|
||||
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f) throws IOException {
|
||||
return fileSystem.listLocatedStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive)
|
||||
throws IOException {
|
||||
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive) throws IOException {
|
||||
return fileSystem.listFiles(convertToDefaultPath(f), recursive);
|
||||
}
|
||||
|
||||
@@ -498,8 +517,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
@Override
|
||||
public void copyFromLocalFile(boolean delSrc, boolean overwrite, Path src, Path dst)
|
||||
throws IOException {
|
||||
fileSystem.copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src),
|
||||
convertToDefaultPath(dst));
|
||||
fileSystem
|
||||
.copyFromLocalFile(delSrc, overwrite, convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -525,15 +544,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile)
|
||||
throws IOException {
|
||||
return convertToHoodiePath(fileSystem.startLocalOutput(convertToDefaultPath(fsOutputFile),
|
||||
convertToDefaultPath(tmpLocalFile)));
|
||||
public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException {
|
||||
return convertToHoodiePath(fileSystem
|
||||
.startLocalOutput(convertToDefaultPath(fsOutputFile), convertToDefaultPath(tmpLocalFile)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile)
|
||||
throws IOException {
|
||||
public void completeLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException {
|
||||
fileSystem.completeLocalOutput(convertToDefaultPath(fsOutputFile),
|
||||
convertToDefaultPath(tmpLocalFile));
|
||||
}
|
||||
@@ -574,23 +591,18 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void access(Path path, FsAction mode)
|
||||
throws IOException {
|
||||
public void access(Path path, FsAction mode) throws IOException {
|
||||
fileSystem.access(convertToDefaultPath(path), mode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createSymlink(Path target, Path link, boolean createParent)
|
||||
throws
|
||||
IOException {
|
||||
public void createSymlink(Path target, Path link, boolean createParent) throws IOException {
|
||||
fileSystem
|
||||
.createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus getFileLinkStatus(Path f)
|
||||
throws
|
||||
IOException {
|
||||
public FileStatus getFileLinkStatus(Path f) throws IOException {
|
||||
return fileSystem.getFileLinkStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@@ -651,8 +663,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public Path createSnapshot(Path path, String snapshotName) throws IOException {
|
||||
return convertToHoodiePath(
|
||||
fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName));
|
||||
return convertToHoodiePath(fileSystem.createSnapshot(convertToDefaultPath(path), snapshotName));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -718,8 +729,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, byte[]> getXAttrs(Path path, List<String> names)
|
||||
throws IOException {
|
||||
public Map<String, byte[]> getXAttrs(Path path, List<String> names) throws IOException {
|
||||
return fileSystem.getXAttrs(convertToDefaultPath(path), names);
|
||||
}
|
||||
|
||||
@@ -734,13 +744,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration conf) {
|
||||
// ignore this. we will set conf on init
|
||||
public Configuration getConf() {
|
||||
return fileSystem.getConf();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConf() {
|
||||
return fileSystem.getConf();
|
||||
public void setConf(Configuration conf) {
|
||||
// ignore this. we will set conf on init
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -762,15 +772,6 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
return convertPathWithScheme(oldPath, getHoodieScheme(fileSystem.getScheme()));
|
||||
}
|
||||
|
||||
public static Path convertToHoodiePath(Path file, Configuration conf) {
|
||||
try {
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
return convertPathWithScheme(file, getHoodieScheme(scheme));
|
||||
} catch (HoodieIOException e) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private Path convertToDefaultPath(Path oldPath) {
|
||||
return convertPathWithScheme(oldPath, fileSystem.getScheme());
|
||||
}
|
||||
@@ -783,30 +784,6 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
return psrcsNew;
|
||||
}
|
||||
|
||||
private static Path convertPathWithScheme(Path oldPath, String newScheme) {
|
||||
URI oldURI = oldPath.toUri();
|
||||
URI newURI;
|
||||
try {
|
||||
newURI = new URI(newScheme, oldURI.getUserInfo(), oldURI.getHost(), oldURI.getPort(),
|
||||
oldURI.getPath(), oldURI.getQuery(), oldURI.getFragment());
|
||||
return new Path(newURI);
|
||||
} catch (URISyntaxException e) {
|
||||
// TODO - Better Exception handling
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static String getHoodieScheme(String scheme) {
|
||||
String newScheme;
|
||||
if (SUPPORT_SCHEMES.contains(scheme)) {
|
||||
newScheme = HOODIE_SCHEME_PREFIX + scheme;
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"BlockAlignedAvroParquetWriter does not support scheme " + scheme);
|
||||
}
|
||||
return newScheme;
|
||||
}
|
||||
|
||||
public long getBytesWritten(Path file) {
|
||||
if (openStreams.containsKey(file.getName())) {
|
||||
return openStreams.get(file.getName()).getBytesWritten();
|
||||
|
||||
@@ -30,14 +30,14 @@ import org.apache.log4j.Logger;
|
||||
*/
|
||||
public class HoodieMetrics {
|
||||
|
||||
private HoodieWriteConfig config = null;
|
||||
private String tableName = null;
|
||||
private static Logger logger = LogManager.getLogger(HoodieMetrics.class);
|
||||
// Some timers
|
||||
public String rollbackTimerName = null;
|
||||
public String cleanTimerName = null;
|
||||
public String commitTimerName = null;
|
||||
public String finalizeTimerName = null;
|
||||
private HoodieWriteConfig config = null;
|
||||
private String tableName = null;
|
||||
private Timer rollbackTimer = null;
|
||||
private Timer cleanTimer = null;
|
||||
private Timer commitTimer = null;
|
||||
@@ -113,8 +113,9 @@ public class HoodieMetrics {
|
||||
|
||||
public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
|
||||
if (config.isMetricsOn()) {
|
||||
logger.info(String.format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)",
|
||||
durationInMs, numFilesDeleted));
|
||||
logger.info(String
|
||||
.format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)", durationInMs,
|
||||
numFilesDeleted));
|
||||
registerGauge(getMetricsName("rollback", "duration"), durationInMs);
|
||||
registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
|
||||
}
|
||||
@@ -122,8 +123,9 @@ public class HoodieMetrics {
|
||||
|
||||
public void updateCleanMetrics(long durationInMs, int numFilesDeleted) {
|
||||
if (config.isMetricsOn()) {
|
||||
logger.info(String.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)",
|
||||
durationInMs, numFilesDeleted));
|
||||
logger.info(String
|
||||
.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)", durationInMs,
|
||||
numFilesDeleted));
|
||||
registerGauge(getMetricsName("clean", "duration"), durationInMs);
|
||||
registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted);
|
||||
}
|
||||
@@ -131,8 +133,9 @@ public class HoodieMetrics {
|
||||
|
||||
public void updateFinalizeWriteMetrics(long durationInMs, int numFilesFinalized) {
|
||||
if (config.isMetricsOn()) {
|
||||
logger.info(String.format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)",
|
||||
durationInMs, numFilesFinalized));
|
||||
logger.info(String
|
||||
.format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)",
|
||||
durationInMs, numFilesFinalized));
|
||||
registerGauge(getMetricsName("finalize", "duration"), durationInMs);
|
||||
registerGauge(getMetricsName("finalize", "numFilesFinalized"), numFilesFinalized);
|
||||
}
|
||||
@@ -140,8 +143,7 @@ public class HoodieMetrics {
|
||||
|
||||
@VisibleForTesting
|
||||
String getMetricsName(String action, String metric) {
|
||||
return config == null ? null :
|
||||
String.format("%s.%s.%s", tableName, action, metric);
|
||||
return config == null ? null : String.format("%s.%s.%s", tableName, action, metric);
|
||||
}
|
||||
|
||||
void registerGauge(String metricName, final long value) {
|
||||
@@ -154,7 +156,8 @@ public class HoodieMetrics {
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
// Here we catch all exception, so the major upsert pipeline will not be affected if the metrics system
|
||||
// Here we catch all exception, so the major upsert pipeline will not be affected if the
|
||||
// metrics system
|
||||
// has some issues.
|
||||
logger.error("Failed to send metrics: ", e);
|
||||
}
|
||||
|
||||
@@ -40,7 +40,7 @@ public class Metrics {
|
||||
if (reporter == null) {
|
||||
throw new RuntimeException("Cannot initialize Reporter.");
|
||||
}
|
||||
// reporter.start();
|
||||
// reporter.start();
|
||||
|
||||
Runtime.getRuntime().addShutdownHook(new Thread() {
|
||||
@Override
|
||||
|
||||
@@ -33,14 +33,13 @@ import org.apache.log4j.Logger;
|
||||
*/
|
||||
public class MetricsGraphiteReporter extends MetricsReporter {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class);
|
||||
private final MetricRegistry registry;
|
||||
private final GraphiteReporter graphiteReporter;
|
||||
private final HoodieWriteConfig config;
|
||||
private String serverHost;
|
||||
private int serverPort;
|
||||
|
||||
private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class);
|
||||
|
||||
public MetricsGraphiteReporter(HoodieWriteConfig config, MetricRegistry registry) {
|
||||
this.registry = registry;
|
||||
this.config = config;
|
||||
@@ -49,8 +48,8 @@ public class MetricsGraphiteReporter extends MetricsReporter {
|
||||
this.serverHost = config.getGraphiteServerHost();
|
||||
this.serverPort = config.getGraphiteServerPort();
|
||||
if (serverHost == null || serverPort == 0) {
|
||||
throw new RuntimeException(
|
||||
String.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].",
|
||||
throw new RuntimeException(String
|
||||
.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].",
|
||||
serverHost, serverPort));
|
||||
}
|
||||
|
||||
@@ -81,14 +80,10 @@ public class MetricsGraphiteReporter extends MetricsReporter {
|
||||
}
|
||||
|
||||
private GraphiteReporter createGraphiteReport() {
|
||||
Graphite graphite = new Graphite(
|
||||
new InetSocketAddress(serverHost, serverPort));
|
||||
Graphite graphite = new Graphite(new InetSocketAddress(serverHost, serverPort));
|
||||
String reporterPrefix = config.getGraphiteMetricPrefix();
|
||||
return GraphiteReporter.forRegistry(registry)
|
||||
.prefixedWith(reporterPrefix)
|
||||
.convertRatesTo(TimeUnit.SECONDS)
|
||||
.convertDurationsTo(TimeUnit.MILLISECONDS)
|
||||
.filter(MetricFilter.ALL)
|
||||
.build(graphite);
|
||||
return GraphiteReporter.forRegistry(registry).prefixedWith(reporterPrefix)
|
||||
.convertRatesTo(TimeUnit.SECONDS).convertDurationsTo(TimeUnit.MILLISECONDS)
|
||||
.filter(MetricFilter.ALL).build(graphite);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -28,8 +28,7 @@ public class MetricsReporterFactory {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(MetricsReporterFactory.class);
|
||||
|
||||
public static MetricsReporter createReporter(HoodieWriteConfig config,
|
||||
MetricRegistry registry) {
|
||||
public static MetricsReporter createReporter(HoodieWriteConfig config, MetricRegistry registry) {
|
||||
MetricsReporterType type = config.getMetricsReporterType();
|
||||
MetricsReporter reporter = null;
|
||||
switch (type) {
|
||||
|
||||
@@ -21,6 +21,5 @@ package com.uber.hoodie.metrics;
|
||||
* future.
|
||||
*/
|
||||
public enum MetricsReporterType {
|
||||
GRAPHITE,
|
||||
INMEMORY
|
||||
GRAPHITE, INMEMORY
|
||||
}
|
||||
|
||||
@@ -75,23 +75,425 @@ import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Implementation of a very heavily read-optimized Hoodie Table where
|
||||
*
|
||||
* <p>
|
||||
* INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing
|
||||
* file, to expand it
|
||||
*
|
||||
* <p>
|
||||
* UPDATES - Produce a new version of the file, just replacing the updated records with new values
|
||||
*/
|
||||
public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieTable<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class);
|
||||
|
||||
public HoodieCopyOnWriteTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
super(config, metaClient);
|
||||
}
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class);
|
||||
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String,
|
||||
PartitionCleanStat> deleteFilesFunc(
|
||||
HoodieTable table) {
|
||||
return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>)
|
||||
iter -> {
|
||||
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
|
||||
|
||||
FileSystem fs = table.getMetaClient().getFs();
|
||||
while (iter.hasNext()) {
|
||||
Tuple2<String, String> partitionDelFileTuple = iter.next();
|
||||
String partitionPath = partitionDelFileTuple._1();
|
||||
String deletePathStr = partitionDelFileTuple._2();
|
||||
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
|
||||
if (!partitionCleanStatMap.containsKey(partitionPath)) {
|
||||
partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
|
||||
}
|
||||
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePathStr);
|
||||
partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult);
|
||||
}
|
||||
|
||||
return partitionCleanStatMap.entrySet().stream()
|
||||
.map(e -> new Tuple2<>(e.getKey(), e.getValue()))
|
||||
.collect(Collectors.toList()).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private static PairFlatMapFunction<String, String, String> getFilesToDeleteFunc(HoodieTable table,
|
||||
HoodieWriteConfig config) {
|
||||
return (PairFlatMapFunction<String, String, String>) partitionPathToClean -> {
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config);
|
||||
return cleaner.getDeletePaths(partitionPathToClean).stream()
|
||||
.map(deleteFile -> new Tuple2<>(partitionPathToClean, deleteFile.toString())).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr)
|
||||
throws IOException {
|
||||
Path deletePath = new Path(deletePathStr);
|
||||
logger.debug("Working on delete path :" + deletePath);
|
||||
boolean deleteResult = fs.delete(deletePath, false);
|
||||
if (deleteResult) {
|
||||
logger.debug("Cleaned file at path :" + deletePath);
|
||||
}
|
||||
return deleteResult;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
|
||||
if (profile == null) {
|
||||
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
|
||||
}
|
||||
return new UpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
|
||||
return getUpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isWorkloadProfileNeeded() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String commitTime) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileLoc);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords) throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, keyToNewRecords);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileLoc);
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle,
|
||||
String commitTime, String fileLoc) throws IOException {
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException(
|
||||
"Error in finding the old file path at commit " + commitTime + " at fileLoc: " + fileLoc);
|
||||
} else {
|
||||
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
|
||||
ParquetReader<IndexedRecord> reader = AvroParquetReader.builder(upsertHandle.getOldFilePath())
|
||||
.withConf(getHadoopConf()).build();
|
||||
try {
|
||||
IndexedRecord record;
|
||||
while ((record = reader.read()) != null) {
|
||||
// Two types of writes here (new record, and old record).
|
||||
// We have already catch the exception during writing new records.
|
||||
// But for old records, we should fail if any exception happens.
|
||||
upsertHandle.write((GenericRecord) record);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to read record from " + upsertHandle.getOldFilePath() + " with new Schema "
|
||||
+ upsertHandle.getSchema(), e);
|
||||
} finally {
|
||||
reader.close();
|
||||
upsertHandle.close();
|
||||
}
|
||||
}
|
||||
//TODO(vc): This needs to be revisited
|
||||
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
|
||||
logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
|
||||
+ upsertHandle.getWriteStatus());
|
||||
}
|
||||
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus()))
|
||||
.iterator();
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc);
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileLoc);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
return new LazyInsertIterable<>(recordItr, config, commitTime, this);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime, Integer partition,
|
||||
Iterator recordItr, Partitioner partitioner) {
|
||||
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
|
||||
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
|
||||
BucketType btype = binfo.bucketType;
|
||||
try {
|
||||
if (btype.equals(BucketType.INSERT)) {
|
||||
return handleInsert(commitTime, recordItr);
|
||||
} else if (btype.equals(BucketType.UPDATE)) {
|
||||
return handleUpdate(commitTime, binfo.fileLoc, recordItr);
|
||||
} else {
|
||||
throw new HoodieUpsertException(
|
||||
"Unknown bucketType " + btype + " for partition :" + partition);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
|
||||
logger.error(msg, t);
|
||||
throw new HoodieUpsertException(msg, t);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsertPartition(String commitTime, Integer partition,
|
||||
Iterator recordItr, Partitioner partitioner) {
|
||||
return handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs cleaning of partition paths according to cleaning policy and returns the number of
|
||||
* files cleaned. Handles skews in partitions to clean by making files to clean as the unit of
|
||||
* task distribution.
|
||||
*
|
||||
* @throws IllegalArgumentException if unknown cleaning policy is provided
|
||||
*/
|
||||
@Override
|
||||
public List<HoodieCleanStat> clean(JavaSparkContext jsc) {
|
||||
try {
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
List<String> partitionsToClean = FSUtils
|
||||
.getAllPartitionPaths(fs, getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config
|
||||
.getCleanerPolicy());
|
||||
if (partitionsToClean.isEmpty()) {
|
||||
logger.info("Nothing to clean here mom. It is already clean");
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return cleanPartitionPaths(partitionsToClean, jsc);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to clean up after commit", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Common method used for cleaning out parquet files under a partition path during rollback of a
|
||||
* set of commits
|
||||
*/
|
||||
protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits)
|
||||
throws IOException {
|
||||
logger.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
FileStatus[] toBeDeleted = fs
|
||||
.listStatus(new Path(config.getBasePath(), partitionPath), path -> {
|
||||
if (!path.toString().contains(".parquet")) {
|
||||
return false;
|
||||
}
|
||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||
return commits.contains(fileCommitTime);
|
||||
});
|
||||
Map<FileStatus, Boolean> results = Maps.newHashMap();
|
||||
for (FileStatus file : toBeDeleted) {
|
||||
boolean success = fs.delete(file.getPath(), false);
|
||||
results.put(file, success);
|
||||
logger.info("Delete file " + file.getPath() + "\t" + success);
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
|
||||
throws IOException {
|
||||
String actionType = this.getCommitActionType();
|
||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
|
||||
List<String> inflights = this.getInflightCommitTimeline().getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
|
||||
// Atomically unpublish all the commits
|
||||
commits.stream().filter(s -> !inflights.contains(s))
|
||||
.map(s -> new HoodieInstant(false, actionType, s))
|
||||
.forEach(activeTimeline::revertToInflight);
|
||||
logger.info("Unpublished " + commits);
|
||||
|
||||
// delete all the data files for all these commits
|
||||
logger.info("Clean out all parquet files generated for commits: " + commits);
|
||||
List<HoodieRollbackStat> stats = jsc.parallelize(FSUtils
|
||||
.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
Map<FileStatus, Boolean> results = deleteCleanedFiles(partitionPath, commits);
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(results).build();
|
||||
}).collect();
|
||||
|
||||
// clean temporary data files
|
||||
cleanTemporaryDataFiles(jsc);
|
||||
|
||||
// Remove the rolled back inflight commits
|
||||
commits.stream().map(s -> new HoodieInstant(true, actionType, s))
|
||||
.forEach(activeTimeline::deleteInflight);
|
||||
logger.info("Deleted inflight commits " + commits);
|
||||
return stats;
|
||||
}
|
||||
|
||||
/**
|
||||
* Finalize the written data files
|
||||
*
|
||||
* @param writeStatuses List of WriteStatus
|
||||
* @return number of files finalized
|
||||
*/
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Optional<Integer> finalizeWrite(JavaSparkContext jsc, List writeStatuses) {
|
||||
if (!config.shouldUseTempFolderForCopyOnWrite()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
// This is to rename each data file from temporary path to its final location
|
||||
List<Tuple2<String, Boolean>> results = jsc
|
||||
.parallelize(writeStatuses, config.getFinalizeWriteParallelism()).map(writeStatus -> {
|
||||
Tuple2<String, HoodieWriteStat> writeStatTuple2 = (Tuple2<String, HoodieWriteStat>)
|
||||
writeStatus;
|
||||
HoodieWriteStat writeStat = writeStatTuple2._2();
|
||||
final FileSystem fs = getMetaClient().getFs();
|
||||
final Path finalPath = new Path(config.getBasePath(), writeStat.getPath());
|
||||
|
||||
if (writeStat.getTempPath() != null) {
|
||||
final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath());
|
||||
boolean success;
|
||||
try {
|
||||
logger.info("Renaming temporary file: " + tempPath + " to " + finalPath);
|
||||
success = fs.rename(tempPath, finalPath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to rename file: " + tempPath + " to " + finalPath);
|
||||
}
|
||||
|
||||
if (!success) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to rename file: " + tempPath + " to " + finalPath);
|
||||
}
|
||||
}
|
||||
|
||||
return new Tuple2<>(writeStat.getPath(), true);
|
||||
}).collect();
|
||||
|
||||
// clean temporary data files
|
||||
cleanTemporaryDataFiles(jsc);
|
||||
|
||||
return Optional.of(results.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean temporary data files that are produced from previous failed commit or retried spark
|
||||
* stages.
|
||||
*/
|
||||
private void cleanTemporaryDataFiles(JavaSparkContext jsc) {
|
||||
if (!config.shouldUseTempFolderForCopyOnWrite()) {
|
||||
return;
|
||||
}
|
||||
|
||||
final FileSystem fs = getMetaClient().getFs();
|
||||
final Path temporaryFolder = new Path(config.getBasePath(),
|
||||
HoodieTableMetaClient.TEMPFOLDER_NAME);
|
||||
try {
|
||||
if (!fs.exists(temporaryFolder)) {
|
||||
logger.info("Temporary folder does not exist: " + temporaryFolder);
|
||||
return;
|
||||
}
|
||||
List<FileStatus> fileStatusesList = Arrays.asList(fs.listStatus(temporaryFolder));
|
||||
List<Tuple2<String, Boolean>> results = jsc
|
||||
.parallelize(fileStatusesList, config.getFinalizeWriteParallelism()).map(fileStatus -> {
|
||||
FileSystem fs1 = getMetaClient().getFs();
|
||||
boolean success = fs1.delete(fileStatus.getPath(), false);
|
||||
logger
|
||||
.info("Deleting file in temporary folder" + fileStatus.getPath() + "\t" + success);
|
||||
return new Tuple2<>(fileStatus.getPath().toString(), success);
|
||||
}).collect();
|
||||
|
||||
for (Tuple2<String, Boolean> result : results) {
|
||||
if (!result._2()) {
|
||||
logger.info("Failed to delete file: " + result._1());
|
||||
throw new HoodieIOException("Failed to delete file in temporary folder: " + result._1());
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to clean data files in temporary folder: " + temporaryFolder);
|
||||
}
|
||||
}
|
||||
|
||||
private List<HoodieCleanStat> cleanPartitionPaths(List<String> partitionsToClean,
|
||||
JavaSparkContext jsc) {
|
||||
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
|
||||
logger.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
|
||||
.parallelize(partitionsToClean, cleanerParallelism)
|
||||
.flatMapToPair(getFilesToDeleteFunc(this, config))
|
||||
.repartition(cleanerParallelism) // repartition to remove skews
|
||||
.mapPartitionsToPair(deleteFilesFunc(this)).reduceByKey(
|
||||
// merge partition level clean stats below
|
||||
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1
|
||||
.merge(e2)).collect();
|
||||
|
||||
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
|
||||
.collect(Collectors.toMap(e -> e._1(), e -> e._2()));
|
||||
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
|
||||
// Return PartitionCleanStat for each partition passed.
|
||||
return partitionsToClean.stream().map(partitionPath -> {
|
||||
PartitionCleanStat partitionCleanStat =
|
||||
(partitionCleanStatsMap.containsKey(partitionPath)) ? partitionCleanStatsMap
|
||||
.get(partitionPath) : new PartitionCleanStat(partitionPath);
|
||||
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy())
|
||||
.withPartitionPath(partitionPath)
|
||||
.withEarliestCommitRetained(cleaner.getEarliestCommitToRetain())
|
||||
.withDeletePathPattern(partitionCleanStat.deletePathPatterns)
|
||||
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles)
|
||||
.withFailedDeletes(partitionCleanStat.failedDeleteFiles).build();
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
enum BucketType {
|
||||
UPDATE,
|
||||
INSERT
|
||||
UPDATE, INSERT
|
||||
}
|
||||
|
||||
private static class PartitionCleanStat implements Serializable {
|
||||
|
||||
private final String partitionPath;
|
||||
private final List<String> deletePathPatterns = new ArrayList<>();
|
||||
private final List<String> successDeleteFiles = new ArrayList<>();
|
||||
private final List<String> failedDeleteFiles = new ArrayList<>();
|
||||
|
||||
private PartitionCleanStat(String partitionPath) {
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
private void addDeletedFileResult(String deletePathStr, Boolean deletedFileResult) {
|
||||
if (deletedFileResult) {
|
||||
successDeleteFiles.add(deletePathStr);
|
||||
} else {
|
||||
failedDeleteFiles.add(deletePathStr);
|
||||
}
|
||||
}
|
||||
|
||||
private void addDeleteFilePatterns(String deletePathStr) {
|
||||
deletePathPatterns.add(deletePathStr);
|
||||
}
|
||||
|
||||
private PartitionCleanStat merge(PartitionCleanStat other) {
|
||||
if (!this.partitionPath.equals(other.partitionPath)) {
|
||||
throw new RuntimeException(String
|
||||
.format("partitionPath is not a match: (%s, %s)", partitionPath, other.partitionPath));
|
||||
}
|
||||
successDeleteFiles.addAll(other.successDeleteFiles);
|
||||
deletePathPatterns.addAll(other.deletePathPatterns);
|
||||
failedDeleteFiles.addAll(other.failedDeleteFiles);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -150,45 +552,37 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition)
|
||||
*/
|
||||
class UpsertPartitioner extends Partitioner {
|
||||
|
||||
/**
|
||||
* List of all small files to be corrected
|
||||
*/
|
||||
List<SmallFile> smallFiles = new ArrayList<SmallFile>();
|
||||
/**
|
||||
* Total number of RDD partitions, is determined by total buckets we want to pack the incoming
|
||||
* workload into
|
||||
*/
|
||||
private int totalBuckets = 0;
|
||||
|
||||
/**
|
||||
* Stat for the current workload. Helps in determining total inserts, upserts etc.
|
||||
*/
|
||||
private WorkloadStat globalStat;
|
||||
|
||||
/**
|
||||
* Helps decide which bucket an incoming update should go to.
|
||||
*/
|
||||
private HashMap<String, Integer> updateLocationToBucket;
|
||||
|
||||
|
||||
/**
|
||||
* Helps us pack inserts into 1 or more buckets depending on number of incoming records.
|
||||
*/
|
||||
private HashMap<String, List<InsertBucket>> partitionPathToInsertBuckets;
|
||||
|
||||
|
||||
/**
|
||||
* Remembers what type each bucket is for later.
|
||||
*/
|
||||
private HashMap<Integer, BucketInfo> bucketInfoMap;
|
||||
|
||||
/**
|
||||
* List of all small files to be corrected
|
||||
*/
|
||||
List<SmallFile> smallFiles = new ArrayList<SmallFile>();
|
||||
|
||||
UpsertPartitioner(WorkloadProfile profile) {
|
||||
updateLocationToBucket = new HashMap<>();
|
||||
partitionPathToInsertBuckets = new HashMap<>();
|
||||
@@ -198,16 +592,17 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
assignUpdates(profile);
|
||||
assignInserts(profile);
|
||||
|
||||
logger.info("Total Buckets :" + totalBuckets + ", " +
|
||||
"buckets info => " + bucketInfoMap + ", \n" +
|
||||
"Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n" +
|
||||
"UpdateLocations mapped to buckets =>" + updateLocationToBucket);
|
||||
logger.info(
|
||||
"Total Buckets :" + totalBuckets + ", " + "buckets info => " + bucketInfoMap + ", \n"
|
||||
+ "Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n"
|
||||
+ "UpdateLocations mapped to buckets =>" + updateLocationToBucket);
|
||||
}
|
||||
|
||||
private void assignUpdates(WorkloadProfile profile) {
|
||||
// each update location gets a partition
|
||||
WorkloadStat gStat = profile.getGlobalStat();
|
||||
for (Map.Entry<String, Pair<String, Long>> updateLocEntry : gStat.getUpdateLocationToCount().entrySet()) {
|
||||
for (Map.Entry<String, Pair<String, Long>> updateLocEntry : gStat.getUpdateLocationToCount()
|
||||
.entrySet()) {
|
||||
addUpdateBucket(updateLocEntry.getKey());
|
||||
}
|
||||
}
|
||||
@@ -270,10 +665,10 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
int insertBuckets = (int) Math.max(totalUnassignedInserts / insertRecordsPerBucket, 1L);
|
||||
logger
|
||||
.info("After small file assignment: unassignedInserts => " + totalUnassignedInserts
|
||||
+ ", totalInsertBuckets => " + insertBuckets
|
||||
+ ", recordsPerBucket => " + insertRecordsPerBucket);
|
||||
logger.info(
|
||||
"After small file assignment: unassignedInserts => " + totalUnassignedInserts
|
||||
+ ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => "
|
||||
+ insertRecordsPerBucket);
|
||||
for (int b = 0; b < insertBuckets; b++) {
|
||||
bucketNumbers.add(totalBuckets);
|
||||
recordsPerBucket.add(totalUnassignedInserts / insertBuckets);
|
||||
@@ -339,8 +734,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
*/
|
||||
private long averageBytesPerRecord() {
|
||||
long avgSize = 0L;
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants();
|
||||
try {
|
||||
if (!commitTimeline.empty()) {
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
@@ -372,7 +767,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
|
||||
@Override
|
||||
public int getPartition(Object key) {
|
||||
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation = (Tuple2<HoodieKey, Option<HoodieRecordLocation>>) key;
|
||||
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation = (Tuple2<HoodieKey,
|
||||
Option<HoodieRecordLocation>>) key;
|
||||
if (keyLocation._2().isDefined()) {
|
||||
HoodieRecordLocation location = keyLocation._2().get();
|
||||
return updateLocationToBucket.get(location.getFileId());
|
||||
@@ -396,420 +792,4 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
|
||||
if (profile == null) {
|
||||
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
|
||||
}
|
||||
return new UpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
|
||||
return getUpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isWorkloadProfileNeeded() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String commitTime) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
|
||||
}
|
||||
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc,
|
||||
Iterator<HoodieRecord<T>> recordItr)
|
||||
throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileLoc);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords)
|
||||
throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, keyToNewRecords);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileLoc);
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String commitTime, String fileLoc)
|
||||
throws IOException {
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException("Error in finding the old file path at commit " +
|
||||
commitTime + " at fileLoc: " + fileLoc);
|
||||
} else {
|
||||
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
|
||||
ParquetReader<IndexedRecord> reader =
|
||||
AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(getHadoopConf())
|
||||
.build();
|
||||
try {
|
||||
IndexedRecord record;
|
||||
while ((record = reader.read()) != null) {
|
||||
// Two types of writes here (new record, and old record).
|
||||
// We have already catch the exception during writing new records.
|
||||
// But for old records, we should fail if any exception happens.
|
||||
upsertHandle.write((GenericRecord) record);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to read record from " + upsertHandle.getOldFilePath()
|
||||
+ " with new Schema " + upsertHandle.getSchema(), e);
|
||||
} finally {
|
||||
reader.close();
|
||||
upsertHandle.close();
|
||||
}
|
||||
}
|
||||
//TODO(vc): This needs to be revisited
|
||||
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
|
||||
logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath()
|
||||
+ ", " + upsertHandle.getWriteStatus());
|
||||
}
|
||||
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus()))
|
||||
.iterator();
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileLoc);
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileLoc,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileLoc);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
return new LazyInsertIterable<>(recordItr, config, commitTime, this);
|
||||
}
|
||||
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime, Integer partition,
|
||||
Iterator recordItr, Partitioner partitioner) {
|
||||
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
|
||||
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
|
||||
BucketType btype = binfo.bucketType;
|
||||
try {
|
||||
if (btype.equals(BucketType.INSERT)) {
|
||||
return handleInsert(commitTime, recordItr);
|
||||
} else if (btype.equals(BucketType.UPDATE)) {
|
||||
return handleUpdate(commitTime, binfo.fileLoc, recordItr);
|
||||
} else {
|
||||
throw new HoodieUpsertException(
|
||||
"Unknown bucketType " + btype + " for partition :" + partition);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
|
||||
logger.error(msg, t);
|
||||
throw new HoodieUpsertException(msg, t);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsertPartition(String commitTime, Integer partition,
|
||||
Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
return handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs cleaning of partition paths according to cleaning policy and returns the number of
|
||||
* files cleaned. Handles skews in partitions to clean by making files to clean as the unit of
|
||||
* task distribution.
|
||||
*
|
||||
* @throws IllegalArgumentException if unknown cleaning policy is provided
|
||||
*/
|
||||
@Override
|
||||
public List<HoodieCleanStat> clean(JavaSparkContext jsc) {
|
||||
try {
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
List<String> partitionsToClean =
|
||||
FSUtils.getAllPartitionPaths(fs, getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config
|
||||
.getCleanerPolicy());
|
||||
if (partitionsToClean.isEmpty()) {
|
||||
logger.info("Nothing to clean here mom. It is already clean");
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return cleanPartitionPaths(partitionsToClean, jsc);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to clean up after commit", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Common method used for cleaning out parquet files under a partition path during rollback of a
|
||||
* set of commits
|
||||
*/
|
||||
protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits)
|
||||
throws IOException {
|
||||
logger.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
FileStatus[] toBeDeleted =
|
||||
fs.listStatus(new Path(config.getBasePath(), partitionPath), path -> {
|
||||
if (!path.toString().contains(".parquet")) {
|
||||
return false;
|
||||
}
|
||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||
return commits.contains(fileCommitTime);
|
||||
});
|
||||
Map<FileStatus, Boolean> results = Maps.newHashMap();
|
||||
for (FileStatus file : toBeDeleted) {
|
||||
boolean success = fs.delete(file.getPath(), false);
|
||||
results.put(file, success);
|
||||
logger.info("Delete file " + file.getPath() + "\t" + success);
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
|
||||
throws IOException {
|
||||
String actionType = this.getCommitActionType();
|
||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
|
||||
List<String> inflights = this.getInflightCommitTimeline().getInstants()
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// Atomically unpublish all the commits
|
||||
commits.stream().filter(s -> !inflights.contains(s))
|
||||
.map(s -> new HoodieInstant(false, actionType, s))
|
||||
.forEach(activeTimeline::revertToInflight);
|
||||
logger.info("Unpublished " + commits);
|
||||
|
||||
// delete all the data files for all these commits
|
||||
logger.info("Clean out all parquet files generated for commits: " + commits);
|
||||
List<HoodieRollbackStat> stats = jsc.parallelize(
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
Map<FileStatus, Boolean> results = deleteCleanedFiles(partitionPath, commits);
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(results).build();
|
||||
}).collect();
|
||||
|
||||
// clean temporary data files
|
||||
cleanTemporaryDataFiles(jsc);
|
||||
|
||||
// Remove the rolled back inflight commits
|
||||
commits.stream().map(s -> new HoodieInstant(true, actionType, s))
|
||||
.forEach(activeTimeline::deleteInflight);
|
||||
logger.info("Deleted inflight commits " + commits);
|
||||
return stats;
|
||||
}
|
||||
|
||||
/**
|
||||
* Finalize the written data files
|
||||
*
|
||||
* @param writeStatuses List of WriteStatus
|
||||
* @return number of files finalized
|
||||
*/
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Optional<Integer> finalizeWrite(JavaSparkContext jsc, List writeStatuses) {
|
||||
if (!config.shouldUseTempFolderForCopyOnWrite()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
// This is to rename each data file from temporary path to its final location
|
||||
List<Tuple2<String, Boolean>> results = jsc.parallelize(writeStatuses, config.getFinalizeWriteParallelism())
|
||||
.map(writeStatus -> {
|
||||
Tuple2<String, HoodieWriteStat> writeStatTuple2 = (Tuple2<String, HoodieWriteStat>) writeStatus;
|
||||
HoodieWriteStat writeStat = writeStatTuple2._2();
|
||||
final FileSystem fs = getMetaClient().getFs();
|
||||
final Path finalPath = new Path(config.getBasePath(), writeStat.getPath());
|
||||
|
||||
if (writeStat.getTempPath() != null) {
|
||||
final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath());
|
||||
boolean success;
|
||||
try {
|
||||
logger.info("Renaming temporary file: " + tempPath + " to " + finalPath);
|
||||
success = fs.rename(tempPath, finalPath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to rename file: " + tempPath + " to " + finalPath);
|
||||
}
|
||||
|
||||
if (!success) {
|
||||
throw new HoodieIOException("Failed to rename file: " + tempPath + " to " + finalPath);
|
||||
}
|
||||
}
|
||||
|
||||
return new Tuple2<>(writeStat.getPath(), true);
|
||||
}).collect();
|
||||
|
||||
// clean temporary data files
|
||||
cleanTemporaryDataFiles(jsc);
|
||||
|
||||
return Optional.of(results.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean temporary data files that are produced from previous failed commit or retried spark
|
||||
* stages.
|
||||
*/
|
||||
private void cleanTemporaryDataFiles(JavaSparkContext jsc) {
|
||||
if (!config.shouldUseTempFolderForCopyOnWrite()) {
|
||||
return;
|
||||
}
|
||||
|
||||
final FileSystem fs = getMetaClient().getFs();
|
||||
final Path temporaryFolder = new Path(config.getBasePath(),
|
||||
HoodieTableMetaClient.TEMPFOLDER_NAME);
|
||||
try {
|
||||
if (!fs.exists(temporaryFolder)) {
|
||||
logger.info("Temporary folder does not exist: " + temporaryFolder);
|
||||
return;
|
||||
}
|
||||
List<FileStatus> fileStatusesList = Arrays.asList(fs.listStatus(temporaryFolder));
|
||||
List<Tuple2<String, Boolean>> results = jsc
|
||||
.parallelize(fileStatusesList, config.getFinalizeWriteParallelism())
|
||||
.map(fileStatus -> {
|
||||
FileSystem fs1 = getMetaClient().getFs();
|
||||
boolean success = fs1.delete(fileStatus.getPath(), false);
|
||||
logger.info("Deleting file in temporary folder" + fileStatus.getPath() + "\t"
|
||||
+ success);
|
||||
return new Tuple2<>(fileStatus.getPath().toString(), success);
|
||||
}).collect();
|
||||
|
||||
for (Tuple2<String, Boolean> result : results) {
|
||||
if (!result._2()) {
|
||||
logger.info("Failed to delete file: " + result._1());
|
||||
throw new HoodieIOException(
|
||||
"Failed to delete file in temporary folder: " + result._1());
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to clean data files in temporary folder: " + temporaryFolder);
|
||||
}
|
||||
}
|
||||
|
||||
private static class PartitionCleanStat implements Serializable {
|
||||
|
||||
private final String partitionPath;
|
||||
private final List<String> deletePathPatterns = new ArrayList<>();
|
||||
private final List<String> successDeleteFiles = new ArrayList<>();
|
||||
private final List<String> failedDeleteFiles = new ArrayList<>();
|
||||
|
||||
private PartitionCleanStat(String partitionPath) {
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
private void addDeletedFileResult(String deletePathStr, Boolean deletedFileResult) {
|
||||
if (deletedFileResult) {
|
||||
successDeleteFiles.add(deletePathStr);
|
||||
} else {
|
||||
failedDeleteFiles.add(deletePathStr);
|
||||
}
|
||||
}
|
||||
|
||||
private void addDeleteFilePatterns(String deletePathStr) {
|
||||
deletePathPatterns.add(deletePathStr);
|
||||
}
|
||||
|
||||
private PartitionCleanStat merge(PartitionCleanStat other) {
|
||||
if (!this.partitionPath.equals(other.partitionPath)) {
|
||||
throw new RuntimeException(String.format(
|
||||
"partitionPath is not a match: (%s, %s)",
|
||||
partitionPath, other.partitionPath));
|
||||
}
|
||||
successDeleteFiles.addAll(other.successDeleteFiles);
|
||||
deletePathPatterns.addAll(other.deletePathPatterns);
|
||||
failedDeleteFiles.addAll(other.failedDeleteFiles);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
private List<HoodieCleanStat> cleanPartitionPaths(List<String> partitionsToClean,
|
||||
JavaSparkContext jsc) {
|
||||
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
|
||||
logger.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
|
||||
.parallelize(partitionsToClean, cleanerParallelism)
|
||||
.flatMapToPair(getFilesToDeleteFunc(this, config))
|
||||
.repartition(cleanerParallelism) // repartition to remove skews
|
||||
.mapPartitionsToPair(deleteFilesFunc(this))
|
||||
.reduceByKey(
|
||||
// merge partition level clean stats below
|
||||
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1
|
||||
.merge(e2))
|
||||
.collect();
|
||||
|
||||
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats
|
||||
.stream().collect(Collectors.toMap(e -> e._1(), e -> e._2()));
|
||||
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
|
||||
// Return PartitionCleanStat for each partition passed.
|
||||
return partitionsToClean.stream().map(partitionPath -> {
|
||||
PartitionCleanStat partitionCleanStat =
|
||||
(partitionCleanStatsMap.containsKey(partitionPath)) ?
|
||||
partitionCleanStatsMap.get(partitionPath)
|
||||
: new PartitionCleanStat(partitionPath);
|
||||
return HoodieCleanStat.newBuilder()
|
||||
.withPolicy(config.getCleanerPolicy())
|
||||
.withPartitionPath(partitionPath)
|
||||
.withEarliestCommitRetained(cleaner.getEarliestCommitToRetain())
|
||||
.withDeletePathPattern(partitionCleanStat.deletePathPatterns)
|
||||
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles)
|
||||
.withFailedDeletes(partitionCleanStat.failedDeleteFiles)
|
||||
.build();
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat> deleteFilesFunc(
|
||||
HoodieTable table) {
|
||||
return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>) iter -> {
|
||||
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
|
||||
|
||||
FileSystem fs = table.getMetaClient().getFs();
|
||||
while (iter.hasNext()) {
|
||||
Tuple2<String, String> partitionDelFileTuple = iter.next();
|
||||
String partitionPath = partitionDelFileTuple._1();
|
||||
String deletePathStr = partitionDelFileTuple._2();
|
||||
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
|
||||
if (!partitionCleanStatMap.containsKey(partitionPath)) {
|
||||
partitionCleanStatMap.put(partitionPath,
|
||||
new PartitionCleanStat(partitionPath));
|
||||
}
|
||||
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePathStr);
|
||||
partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult);
|
||||
}
|
||||
|
||||
return partitionCleanStatMap.entrySet().stream()
|
||||
.map(e -> new Tuple2<>(e.getKey(), e.getValue()))
|
||||
.collect(Collectors.toList()).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private static PairFlatMapFunction<String, String, String> getFilesToDeleteFunc(
|
||||
HoodieTable table, HoodieWriteConfig config) {
|
||||
return (PairFlatMapFunction<String, String, String>) partitionPathToClean -> {
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config);
|
||||
return cleaner.getDeletePaths(partitionPathToClean).stream()
|
||||
.map(deleteFile -> new Tuple2<>(partitionPathToClean, deleteFile.toString()))
|
||||
.iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr)
|
||||
throws IOException {
|
||||
Path deletePath = new Path(deletePathStr);
|
||||
logger.debug("Working on delete path :" + deletePath);
|
||||
boolean deleteResult = fs.delete(deletePath, false);
|
||||
if (deleteResult) {
|
||||
logger.debug("Cleaned file at path :" + deletePath);
|
||||
}
|
||||
return deleteResult;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -62,19 +62,12 @@ import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
|
||||
/**
|
||||
* Implementation of a more real-time read-optimized Hoodie Table where
|
||||
* <p>
|
||||
* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or)
|
||||
* Merge with the smallest existing file, to expand it
|
||||
* </p>
|
||||
* <p>
|
||||
* UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the
|
||||
* log file into the base file.
|
||||
* </p>
|
||||
* <p>
|
||||
* WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an
|
||||
* attempted commit action
|
||||
* </p>
|
||||
* Implementation of a more real-time read-optimized Hoodie Table where <p> INSERTS - Same as
|
||||
* HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the
|
||||
* smallest existing file, to expand it </p> <p> UPDATES - Appends the changes to a rolling log file
|
||||
* maintained per file Id. Compaction merges the log file into the base file. </p> <p> WARNING - MOR
|
||||
* table type does not support nested rollbacks, every rollback must be followed by an attempted
|
||||
* commit action </p>
|
||||
*/
|
||||
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
HoodieCopyOnWriteTable<T> {
|
||||
@@ -88,57 +81,6 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
super(config, metaClient);
|
||||
}
|
||||
|
||||
/**
|
||||
* UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet
|
||||
* files to larger ones without the need for an index in the logFile.
|
||||
*/
|
||||
class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner {
|
||||
|
||||
MergeOnReadUpsertPartitioner(WorkloadProfile profile) {
|
||||
super(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
// Init here since this class (and member variables) might not have been initialized
|
||||
HoodieTimeline commitTimeline = getCompletedCommitTimeline();
|
||||
|
||||
if (!commitTimeline.empty()) {
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
// find smallest file in partition and append to it
|
||||
Optional<FileSlice> smallFileSlice = getRTFileSystemView()
|
||||
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp())
|
||||
.filter(fileSlice -> fileSlice.getLogFiles().count() < 1 &&
|
||||
fileSlice.getDataFile().get().getFileSize() < config.getParquetSmallFileLimit())
|
||||
.sorted((FileSlice left, FileSlice right) ->
|
||||
left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() ? -1 : 1)
|
||||
.findFirst();
|
||||
|
||||
if(smallFileSlice.isPresent()) {
|
||||
String filename = smallFileSlice.get().getDataFile().get().getFileName();
|
||||
SmallFile sf = new SmallFile();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename),
|
||||
FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = smallFileSlice.get().getDataFile().get().getFileSize();
|
||||
smallFileLocations.add(sf);
|
||||
// Update the global small files list
|
||||
smallFiles.add(sf);
|
||||
}
|
||||
}
|
||||
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
public List<String> getSmallFileIds() {
|
||||
return (List<String>) smallFiles.stream().map(smallFile -> ((SmallFile) smallFile).location.getFileId())
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
|
||||
if (profile == null) {
|
||||
@@ -150,15 +92,16 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
|
||||
|
||||
if(mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
|
||||
logger.info("Small file corrections for updates for commit " + commitTime + " for file " + fileId);
|
||||
if (mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
|
||||
logger.info(
|
||||
"Small file corrections for updates for commit " + commitTime + " for file " + fileId);
|
||||
return super.handleUpdate(commitTime, fileId, recordItr);
|
||||
} else {
|
||||
HoodieAppendHandle<T> appendHandle =
|
||||
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
|
||||
HoodieAppendHandle<T> appendHandle = new HoodieAppendHandle<>(config, commitTime, this,
|
||||
fileId, recordItr);
|
||||
appendHandle.doAppend();
|
||||
appendHandle.close();
|
||||
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
|
||||
@@ -202,13 +145,11 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
if (commits.size() > 1) {
|
||||
throw new UnsupportedOperationException("Nested Rollbacks are not supported");
|
||||
}
|
||||
Map<String, HoodieInstant> commitsAndCompactions =
|
||||
this.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
||||
.getInstants()
|
||||
.filter(i -> commits.contains(i.getTimestamp()))
|
||||
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
||||
Map<String, HoodieInstant> commitsAndCompactions = this.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION)).getInstants()
|
||||
.filter(i -> commits.contains(i.getTimestamp()))
|
||||
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
||||
|
||||
// Atomically un-publish all non-inflight commits
|
||||
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
|
||||
@@ -218,9 +159,9 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
Long startTime = System.currentTimeMillis();
|
||||
|
||||
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize
|
||||
(FSUtils.getAllPartitionPaths(this.metaClient.getFs(),
|
||||
this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()))
|
||||
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize(FSUtils
|
||||
.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, List<HoodieRollbackStat>>) partitionPath -> {
|
||||
return commits.stream().map(commit -> {
|
||||
HoodieInstant instant = commitsAndCompactions.get(commit);
|
||||
@@ -228,56 +169,63 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
switch (instant.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
try {
|
||||
Map<FileStatus, Boolean> results = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(results).build();
|
||||
Map<FileStatus, Boolean> results = super
|
||||
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath).withDeletedFileResults(results).build();
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
try {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
this.getCommitTimeline().getInstantDetails(
|
||||
new HoodieInstant(true, instant.getAction(), instant.getTimestamp()))
|
||||
.get());
|
||||
|
||||
// read commit file and (either append delete blocks or delete file)
|
||||
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
|
||||
|
||||
// we do not know fileIds for inserts (first inserts are parquet files), delete all parquet files for the corresponding failed commit, if present (same as COW)
|
||||
// we do not know fileIds for inserts (first inserts are parquet files), delete
|
||||
// all parquet files for the corresponding failed commit, if present (same as COW)
|
||||
filesToDeletedStatus = super
|
||||
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
|
||||
// append rollback blocks for updates
|
||||
if(commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
|
||||
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
|
||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
||||
.filter(wStat -> {
|
||||
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT
|
||||
return wStat != null
|
||||
&& wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT
|
||||
&& wStat.getPrevCommit() != null;
|
||||
})
|
||||
.forEach(wStat -> {
|
||||
}).forEach(wStat -> {
|
||||
HoodieLogFormat.Writer writer = null;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(this.getMetaClient().getBasePath(), partitionPath))
|
||||
writer = HoodieLogFormat.newWriterBuilder().onParentPath(
|
||||
new Path(this.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
|
||||
.withFs(this.metaClient.getFs())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
Long numRollbackBlocks = 0L;
|
||||
// generate metadata
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> header =
|
||||
Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME,
|
||||
metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, commit);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE,
|
||||
String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME,
|
||||
commit);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String
|
||||
.valueOf(
|
||||
HoodieCommandBlock.HoodieCommandBlockTypeEnum
|
||||
.ROLLBACK_PREVIOUS_BLOCK
|
||||
.ordinal()));
|
||||
// if update belongs to an existing log file
|
||||
writer = writer.appendBlock(new HoodieCommandBlock(
|
||||
header));
|
||||
writer = writer.appendBlock(new HoodieCommandBlock(header));
|
||||
numRollbackBlocks++;
|
||||
filesToNumBlocksRollback
|
||||
.put(this.getMetaClient().getFs().getFileStatus(writer.getLogFile().getPath()),
|
||||
numRollbackBlocks);
|
||||
filesToNumBlocksRollback.put(this.getMetaClient().getFs()
|
||||
.getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks);
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException(
|
||||
"Failed to rollback for commit " + commit, io);
|
||||
@@ -289,7 +237,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
}
|
||||
});
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(filesToDeletedStatus)
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
|
||||
}
|
||||
@@ -297,17 +246,19 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return hoodieRollbackStats;
|
||||
}).collect(Collectors.toList());
|
||||
}).flatMap(x -> x.iterator()).filter(x -> x != null).collect();
|
||||
|
||||
commitsAndCompactions.entrySet().stream()
|
||||
.map(entry -> new HoodieInstant(true, entry.getValue().getAction(),
|
||||
entry.getValue().getTimestamp()))
|
||||
.forEach(this.getActiveTimeline()::deleteInflight);
|
||||
commitsAndCompactions.entrySet().stream().map(
|
||||
entry -> new HoodieInstant(true, entry.getValue().getAction(),
|
||||
entry.getValue().getTimestamp())).forEach(this.getActiveTimeline()::deleteInflight);
|
||||
|
||||
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
logger
|
||||
.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
|
||||
return allRollbackStats;
|
||||
}
|
||||
@@ -317,4 +268,56 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
// do nothing for MOR tables
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet
|
||||
* files to larger ones without the need for an index in the logFile.
|
||||
*/
|
||||
class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner {
|
||||
|
||||
MergeOnReadUpsertPartitioner(WorkloadProfile profile) {
|
||||
super(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
// Init here since this class (and member variables) might not have been initialized
|
||||
HoodieTimeline commitTimeline = getCompletedCommitTimeline();
|
||||
|
||||
if (!commitTimeline.empty()) {
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
// find smallest file in partition and append to it
|
||||
Optional<FileSlice> smallFileSlice = getRTFileSystemView()
|
||||
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter(
|
||||
fileSlice -> fileSlice.getLogFiles().count() < 1
|
||||
&& fileSlice.getDataFile().get().getFileSize() < config
|
||||
.getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) ->
|
||||
left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize()
|
||||
? -1 : 1).findFirst();
|
||||
|
||||
if (smallFileSlice.isPresent()) {
|
||||
String filename = smallFileSlice.get().getDataFile().get().getFileName();
|
||||
SmallFile sf = new SmallFile();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename),
|
||||
FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = smallFileSlice.get().getDataFile().get().getFileSize();
|
||||
smallFileLocations.add(sf);
|
||||
// Update the global small files list
|
||||
smallFiles.add(sf);
|
||||
}
|
||||
}
|
||||
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
public List<String> getSmallFileIds() {
|
||||
return (List<String>) smallFiles.stream()
|
||||
.map(smallFile -> ((SmallFile) smallFile).location.getFileId())
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -60,18 +60,28 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
this.metaClient = metaClient;
|
||||
}
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
||||
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return new HoodieCopyOnWriteTable<>(config, metaClient);
|
||||
case MERGE_ON_READ:
|
||||
return new HoodieMergeOnReadTable<>(config, metaClient);
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a partitioner to perform the upsert operation, based on the workload profile
|
||||
*/
|
||||
public abstract Partitioner getUpsertPartitioner(WorkloadProfile profile);
|
||||
|
||||
|
||||
/**
|
||||
* Provides a partitioner to perform the insert operation, based on the workload profile
|
||||
*/
|
||||
public abstract Partitioner getInsertPartitioner(WorkloadProfile profile);
|
||||
|
||||
|
||||
/**
|
||||
* Return whether this HoodieTable implementation can benefit from workload profiling
|
||||
*/
|
||||
@@ -131,7 +141,6 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
return getCommitsTimeline().filterInflights();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) clean timeline
|
||||
*/
|
||||
@@ -162,12 +171,12 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
throw new HoodieSavepointException(
|
||||
"Could not get data files for savepoint " + savepointTime + ". No such savepoint.");
|
||||
}
|
||||
HoodieInstant instant =
|
||||
new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
|
||||
HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION,
|
||||
savepointTime);
|
||||
HoodieSavepointMetadata metadata = null;
|
||||
try {
|
||||
metadata = AvroUtils.deserializeHoodieSavepointMetadata(
|
||||
getActiveTimeline().getInstantDetails(instant).get());
|
||||
metadata = AvroUtils
|
||||
.deserializeHoodieSavepointMetadata(getActiveTimeline().getInstantDetails(instant).get());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieSavepointException(
|
||||
"Could not get savepointed data files for savepoint " + savepointTime, e);
|
||||
@@ -189,7 +198,8 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
return getActiveTimeline().getCommitTimeline();
|
||||
case MERGE_ON_READ:
|
||||
// We need to include the parquet files written out in delta commits
|
||||
// Include commit action to be able to start doing a MOR over a COW dataset - no migration required
|
||||
// Include commit action to be able to start doing a MOR over a COW dataset - no
|
||||
// migration required
|
||||
return getActiveTimeline().getCommitsTimeline();
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
@@ -219,9 +229,10 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
return HoodieActiveTimeline.COMMIT_ACTION;
|
||||
case MERGE_ON_READ:
|
||||
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
|
||||
default:
|
||||
throw new HoodieCommitException(
|
||||
"Could not commit on unknown storage type " + metaClient.getTableType());
|
||||
}
|
||||
throw new HoodieCommitException(
|
||||
"Could not commit on unknown storage type " + metaClient.getTableType());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -236,21 +247,9 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime,
|
||||
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
||||
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return new HoodieCopyOnWriteTable<>(config, metaClient);
|
||||
case MERGE_ON_READ:
|
||||
return new HoodieMergeOnReadTable<>(config, metaClient);
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Run Compaction on the table.
|
||||
* Compaction arranges the data so that it is optimized for data access
|
||||
* Run Compaction on the table. Compaction arranges the data so that it is optimized for data
|
||||
* access
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String commitTime);
|
||||
|
||||
|
||||
@@ -13,6 +13,7 @@
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
@@ -16,7 +16,6 @@
|
||||
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
@@ -31,7 +30,7 @@ import scala.Tuple2;
|
||||
/**
|
||||
* Information about incoming records for upsert/insert obtained either via sampling or
|
||||
* introspecting the data fully
|
||||
*
|
||||
* <p>
|
||||
* TODO(vc): Think about obtaining this directly from index.tagLocation
|
||||
*/
|
||||
public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializable {
|
||||
@@ -60,11 +59,9 @@ public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializa
|
||||
private void buildProfile() {
|
||||
|
||||
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = taggedRecords
|
||||
.mapToPair(record ->
|
||||
new Tuple2<>(
|
||||
new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())),
|
||||
record))
|
||||
.countByKey();
|
||||
.mapToPair(record -> new Tuple2<>(
|
||||
new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())),
|
||||
record)).countByKey();
|
||||
|
||||
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts
|
||||
.entrySet()) {
|
||||
|
||||
@@ -17,10 +17,9 @@
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
|
||||
/**
|
||||
* Wraps stats about a single partition path.
|
||||
|
||||
Reference in New Issue
Block a user