1
0

Adding ability for inserts to be written to log files

This commit is contained in:
Nishith Agarwal
2018-05-13 16:25:11 -07:00
committed by vinoth chandar
parent 34827d50e1
commit 3da063f83b
52 changed files with 1061 additions and 519 deletions

View File

@@ -94,10 +94,11 @@ 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
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true),
clientConfig);
this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants();
.getHoodieTable(metaClient,
clientConfig, jsc);
this.commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
this.index = HoodieIndex.createIndex(clientConfig, jsc);
this.sqlContextOpt = Optional.absent();
}
@@ -128,7 +129,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
assertSqlContext();
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD = index
.fetchRecordLocation(hoodieKeys, hoodieTable);
.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
List<String> paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent())
.map(keyFileTuple -> keyFileTuple._2().get()).collect();
@@ -156,7 +157,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
* file
*/
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
return index.fetchRecordLocation(hoodieKeys, hoodieTable);
return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
}
/**
@@ -180,6 +181,6 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
*/
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords)
throws HoodieIndexException {
return index.tagLocation(hoodieRecords, hoodieTable);
return index.tagLocation(hoodieRecords, jsc, hoodieTable);
}
}

View File

@@ -130,7 +130,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public static SparkConf registerClasses(SparkConf conf) {
conf.registerKryoClasses(
new Class[] {HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
return conf;
}
@@ -144,9 +144,9 @@ 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, jsc);
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, table);
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, jsc, table);
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
}
@@ -161,7 +161,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
config.shouldCombineBeforeUpsert(), records, config.getUpsertShuffleParallelism());
// perform index loop up to get existing location of records
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, table);
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, jsc, table);
return upsertRecordsInternal(taggedRecords, commitTime, table, true);
} catch (Throwable e) {
if (e instanceof HoodieUpsertException) {
@@ -178,7 +178,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* 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,
@@ -202,7 +202,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* 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
*/
@@ -230,7 +230,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* 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,
@@ -256,7 +256,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* 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
*/
@@ -276,10 +276,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* 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,
@@ -310,10 +310,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* 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,
@@ -390,6 +390,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
partitionStat.getUpdateLocationToCount().entrySet().stream().forEach(entry -> {
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId(entry.getKey());
// TODO : Write baseCommitTime is possible here ?
writeStat.setPrevCommit(entry.getValue().getKey());
writeStat.setNumUpdateWrites(entry.getValue().getValue());
metadata.addWriteStat(path.toString(), writeStat);
@@ -450,10 +451,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> table, String commitTime) {
// Update the index back
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, table);
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, jsc, table);
// Trigger the insert and collect statuses
statuses = statuses.persist(config.getWriteStatusStorageLevel());
commitOnAutoCommit(commitTime, statuses, table.getCommitActionType());
commitOnAutoCommit(commitTime, statuses,
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)
.getCommitActionType());
return statuses;
}
@@ -476,9 +479,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/
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);
return commit(commitTime, writeStatuses, extraMetadata, table.getCommitActionType());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true);
return commit(commitTime, writeStatuses, extraMetadata, metaClient.getCommitActionType());
}
private boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses,
@@ -487,7 +489,7 @@ 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, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
@@ -531,7 +533,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// We cannot have unbounded commit files. Archive commits if we have to archive
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config,
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true));
archiveLog.archiveIfRequired();
archiveLog.archiveIfRequired(jsc);
if (config.isAutoClean()) {
// Call clean to cleanup if there is anything to cleanup after the commit,
logger.info("Auto cleaning is enabled. Running cleaner now");
@@ -568,13 +570,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* <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, jsc);
if (table.getCompletedCommitTimeline().empty()) {
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
}
@@ -595,13 +597,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* 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, jsc);
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
@@ -662,7 +664,7 @@ 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, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION,
@@ -688,9 +690,9 @@ 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, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieTimeline commitTimeline = table.getCommitsTimeline();
HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline();
HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION,
savepointTime);
@@ -737,7 +739,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, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieTimeline inflightTimeline = table.getInflightCommitTimeline();
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
@@ -841,7 +843,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, jsc);
List<HoodieCleanStat> cleanStats = table.clean(jsc);
if (cleanStats.isEmpty()) {
@@ -890,9 +892,9 @@ 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, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
String commitActionType = table.getCommitActionType();
String commitActionType = table.getMetaClient().getCommitActionType();
activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime));
}
@@ -912,7 +914,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/
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, jsc);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
String commitActionType = HoodieTimeline.COMMIT_ACTION;
activeTimeline.createInflight(new HoodieInstant(true, commitActionType, commitTime));
@@ -925,7 +927,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, jsc);
// TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction
writeContext = metrics.getCommitCtx();
JavaRDD<WriteStatus> statuses = table.compact(jsc, commitTime);
@@ -961,7 +963,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
config.getBasePath(), true);
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config);
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);
// TODO : Fix table.getActionType for MOR table type to return different actions based on delta or compaction and
// then use getTableAndInitCtx
Timer.Context writeContext = metrics.getCommitCtx();
@@ -1048,8 +1050,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/
private void rollbackInflightCommits() {
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights();
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterInflights();
List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
Collections.reverse(commits);
@@ -1061,8 +1063,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
private HoodieTable getTableAndInitCtx() {
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
if (table.getCommitActionType() == HoodieTimeline.COMMIT_ACTION) {
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
if (table.getMetaClient().getCommitActionType() == HoodieTimeline.COMMIT_ACTION) {
writeContext = metrics.getCommitCtx();
} else {
writeContext = metrics.getDeltaCommitCtx();

View File

@@ -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) {

View File

@@ -44,6 +44,9 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
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);
public static final String LOGFILE_TO_PARQUET_COMPRESSION_RATIO = "hoodie.logfile.to.parquet.compression.ratio";
// Default compression ratio for log file to parquet, general 3x
public static final String DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO = String.valueOf(0.35);
private HoodieStorageConfig(Properties props) {
super(props);
@@ -102,6 +105,11 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
return this;
}
public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressionRatio) {
props.setProperty(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio));
return this;
}
public HoodieStorageConfig build() {
HoodieStorageConfig config = new HoodieStorageConfig(props);
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES),
@@ -116,6 +124,8 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
LOGFILE_SIZE_MAX_BYTES, DEFAULT_LOGFILE_SIZE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO),
PARQUET_COMPRESSION_RATIO, DEFAULT_STREAM_COMPRESSION_RATIO);
setDefaultOnCondition(props, !props.containsKey(LOGFILE_TO_PARQUET_COMPRESSION_RATIO),
LOGFILE_TO_PARQUET_COMPRESSION_RATIO, DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO);
return config;
}
}

View File

@@ -322,6 +322,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return Double.valueOf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO));
}
public double getLogFileToParquetCompressionRatio() {
return Double.valueOf(props.getProperty(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO));
}
/**
* metrics properties
**/
@@ -345,7 +349,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
public String getGraphiteMetricPrefix() {
return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
}
/**
* memory configs
*/

View File

@@ -46,6 +46,6 @@ public class BulkInsertMapFunction<T extends HoodieRecordPayload> implements
@Override
public Iterator<List<WriteStatus>> call(Integer partition,
Iterator<HoodieRecord<T>> sortedRecordItr) throws Exception {
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable);
return new CopyOnWriteLazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable);
}
}

View File

@@ -33,6 +33,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.function.Function;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
@@ -43,15 +44,15 @@ import scala.Tuple2;
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
* files.
*/
public class LazyInsertIterable<T extends HoodieRecordPayload> extends
public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extends
LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
private final HoodieWriteConfig hoodieConfig;
private final String commitTime;
private final HoodieTable<T> hoodieTable;
private Set<String> partitionsCleaned;
protected final HoodieWriteConfig hoodieConfig;
protected final String commitTime;
protected final HoodieTable<T> hoodieTable;
protected Set<String> partitionsCleaned;
public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
public CopyOnWriteLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
String commitTime, HoodieTable<T> hoodieTable) {
super(sortedRecordItr);
this.partitionsCleaned = new HashSet<>();
@@ -89,7 +90,7 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
final Schema schema = HoodieIOHandle.createHoodieWriteSchema(hoodieConfig);
bufferedIteratorExecutor =
new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr,
new InsertHandler(), getTransformFunction(schema));
getInsertHandler(), getTransformFunction(schema));
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
return result;
@@ -107,15 +108,19 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
}
protected CopyOnWriteInsertHandler getInsertHandler() {
return new CopyOnWriteInsertHandler();
}
/**
* Consumes stream of hoodie records from in-memory queue and
* writes to one or more create-handles
*/
private class InsertHandler extends
protected class CopyOnWriteInsertHandler extends
BoundedInMemoryQueueConsumer<Tuple2<HoodieRecord<T>, Optional<IndexedRecord>>, List<WriteStatus>> {
private final List<WriteStatus> statuses = new ArrayList<>();
private HoodieCreateHandle handle;
protected final List<WriteStatus> statuses = new ArrayList<>();
protected HoodieIOHandle handle;
@Override
protected void consumeOneRecord(Tuple2<HoodieRecord<T>, Optional<IndexedRecord>> payload) {
@@ -132,7 +137,8 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
// lazily initialize the handle, for the first time
if (handle == null) {
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath());
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), UUID
.randomUUID().toString());
}
if (handle.canWrite(payload._1())) {
@@ -142,7 +148,8 @@ 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, insertPayload.getPartitionPath());
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), UUID
.randomUUID().toString());
handle.write(insertPayload, payload._2()); // we should be able to write 1 payload.
}
}
@@ -161,4 +168,4 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
return statuses;
}
}
}
}

View File

@@ -0,0 +1,73 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.func;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import org.apache.avro.generic.IndexedRecord;
import scala.Tuple2;
/**
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
* log files.
*/
public class MergeOnReadLazyInsertIterable<T extends HoodieRecordPayload> extends
CopyOnWriteLazyInsertIterable<T> {
public MergeOnReadLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
String commitTime, HoodieTable<T> hoodieTable) {
super(sortedRecordItr, config, commitTime, hoodieTable);
}
@Override
protected CopyOnWriteInsertHandler getInsertHandler() {
return new MergeOnReadInsertHandler();
}
protected class MergeOnReadInsertHandler extends CopyOnWriteInsertHandler {
@Override
protected void consumeOneRecord(Tuple2<HoodieRecord<T>, Optional<IndexedRecord>> payload) {
final HoodieRecord insertPayload = payload._1();
List<WriteStatus> statuses = new ArrayList<>();
// lazily initialize the handle, for the first time
if (handle == null) {
handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable);
}
if (handle.canWrite(insertPayload)) {
// write the payload, if the handle has capacity
handle.write(insertPayload, payload._2);
} else {
// handle is full.
handle.close();
statuses.add(handle.getWriteStatus());
// Need to handle the rejected payload & open new handle
handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable);
handle.write(insertPayload, payload._2); // we should be able to write 1 payload.
}
}
}
}

View File

@@ -38,24 +38,23 @@ import org.apache.spark.api.java.JavaSparkContext;
public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
protected final HoodieWriteConfig config;
protected transient JavaSparkContext jsc = null;
protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
protected HoodieIndex(HoodieWriteConfig config) {
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);
return new HBaseIndex<>(config);
case INMEMORY:
return new InMemoryHashIndex<>(config, jsc);
return new InMemoryHashIndex<>(config);
case BLOOM:
return new HoodieBloomIndex<>(config, jsc);
return new HoodieBloomIndex<>(config);
case BUCKETED:
return new BucketedIndex<>(config, jsc);
return new BucketedIndex<>(config);
default:
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
}
@@ -68,22 +67,23 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
* file
*/
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table);
JavaRDD<HoodieKey> hoodieKeys, final JavaSparkContext jsc, HoodieTable<T> hoodieTable);
/**
* Looks up the index and tags each incoming record with a location of a file that contains the
* row (if it is actually present)
*/
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException;
JavaSparkContext jsc, HoodieTable<T> hoodieTable) throws HoodieIndexException;
/**
* 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,
HoodieTable<T> hoodieTable) throws HoodieIndexException;
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable)
throws HoodieIndexException;
/**
* Rollback the efffects of the commit made at commitTime.

View File

@@ -43,25 +43,25 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
public InMemoryHashIndex(HoodieWriteConfig config) {
super(config);
recordLocationMap = new ConcurrentHashMap<>();
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
final HoodieTable<T> table) {
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
@Override

View File

@@ -30,6 +30,7 @@ import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils;
@@ -64,13 +65,13 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION =
SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
public HoodieBloomIndex(HoodieWriteConfig config) {
super(config);
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
final HoodieTable<T> hoodieTable) {
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
// Step 0: cache the input record RDD
if (config.getBloomIndexUseCaching()) {
@@ -82,8 +83,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD,
hoodieTable);
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable);
// Cache the result, for subsequent stages.
if (config.getBloomIndexUseCaching()) {
@@ -108,13 +108,12 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
final HoodieTable<T> table) {
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
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, jsc, hoodieTable);
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD = hoodieKeys
.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
@@ -125,7 +124,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
String fileName = keyPathTuple._2._2.get();
String partitionPath = keyPathTuple._2._1.getPartitionPath();
recordLocationPath = Optional
.of(new Path(new Path(table.getMetaClient().getBasePath(), partitionPath), fileName)
.of(new Path(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath), fileName)
.toUri().getPath());
} else {
recordLocationPath = Optional.absent();
@@ -139,14 +138,15 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* keys already present and drop the record keys if not present
*/
private JavaPairRDD<String, String> lookupIndex(
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTable<T> hoodieTable) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final JavaSparkContext
jsc, final HoodieTable hoodieTable) {
// Obtain records per partition, in the incoming records
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
// Step 2: Load all involved files as <Partition, filename> pairs
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(
affectedPartitionPathList, hoodieTable);
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, jsc,
hoodieTable);
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
@@ -154,8 +154,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// that contains it.
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo,
partitionRecordKeyPairRDD);
return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo,
partitionRecordKeyPairRDD, parallelism);
return findMatchingFilesForRecordKeys(partitionToFileInfo,
partitionRecordKeyPairRDD, parallelism, hoodieTable.getMetaClient());
}
/**
@@ -232,12 +232,12 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Load all involved files as <Partition, filename> pair RDD.
*/
@VisibleForTesting
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions,
final HoodieTable<T> hoodieTable) {
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
final HoodieTable 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()
java.util.Optional<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
.filterCompletedInstants().lastInstant();
List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>();
if (latestCommitTime.isPresent()) {
@@ -352,9 +352,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Make sure the parallelism is atleast the groupby parallelism for tagging location
*/
@VisibleForTesting
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(HoodieTable hoodieTable,
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int totalSubpartitions) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int totalSubpartitions, HoodieTableMetaClient metaClient) {
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(),
totalSubpartitions);
@@ -367,7 +367,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.sortByKey(true, joinParallelism);
return fileSortedTripletRDD.mapPartitionsWithIndex(
new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true)
new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true)
.flatMap(indexLookupResults -> indexLookupResults.iterator())
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
.flatMapToPair(lookupResult -> {
@@ -410,7 +410,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
return writeStatusRDD;
}

View File

@@ -20,11 +20,11 @@ package com.uber.hoodie.index.bloom;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.func.LazyIterableIterator;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
@@ -49,10 +49,10 @@ public class HoodieBloomIndexCheckFunction implements
private final String basePath;
private final HoodieTable table;
private final HoodieTableMetaClient metaClient;
public HoodieBloomIndexCheckFunction(HoodieTable table, String basePath) {
this.table = table;
public HoodieBloomIndexCheckFunction(HoodieTableMetaClient metaClient, String basePath) {
this.metaClient = metaClient;
this.basePath = basePath;
}
@@ -115,7 +115,7 @@ public class HoodieBloomIndexCheckFunction implements
try {
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
bloomFilter = ParquetUtils
.readBloomFilterFromParquetMetadata(table.getHadoopConf(), filePath);
.readBloomFilterFromParquetMetadata(metaClient.getHadoopConf(), filePath);
candidateRecordKeys = new ArrayList<>();
currentFile = fileName;
currentParitionPath = partitionPath;
@@ -163,7 +163,7 @@ public class HoodieBloomIndexCheckFunction implements
.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
}
ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath)));
checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath)));
initState(fileName, partitionPath);
if (bloomFilter.mightContain(recordKey)) {
@@ -176,7 +176,7 @@ public class HoodieBloomIndexCheckFunction implements
}
}
// handle case, where we ran out of input, finish pending work, update return val
// handle case, where we ran out of input, close pending work, update return val
if (!inputItr.hasNext()) {
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
logger.info(
@@ -186,7 +186,7 @@ public class HoodieBloomIndexCheckFunction implements
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
}
ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath)));
checkCandidatesAgainstFile(metaClient.getHadoopConf(), candidateRecordKeys, filePath)));
}
} catch (Throwable e) {

View File

@@ -49,8 +49,8 @@ public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T>
private static Logger logger = LogManager.getLogger(BucketedIndex.class);
public BucketedIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
public BucketedIndex(HoodieWriteConfig config) {
super(config);
}
private String getBucket(String recordKey) {
@@ -59,13 +59,14 @@ public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T>
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
HoodieTable<T> table) {
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey()))));
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException {
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable)
throws HoodieIndexException {
return recordRDD.map(record -> {
String bucket = getBucket(record.getRecordKey());
//HACK(vc) a non-existent commit is provided here.
@@ -75,8 +76,9 @@ public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T>
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTable<T> hoodieTable) throws HoodieIndexException {
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable)
throws HoodieIndexException {
return writeStatusRDD;
}

View File

@@ -25,6 +25,7 @@ import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -69,15 +70,15 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
private static Connection hbaseConnection = null;
private final String tableName;
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
public HBaseIndex(HoodieWriteConfig config) {
super(config);
this.tableName = config.getHbaseTableName();
addShutDownHook();
}
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
HoodieTable<T> table) {
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
//TODO : Change/Remove filterExists in HoodieReadClient() and revisit
throw new UnsupportedOperationException("HBase index does not implement check exist");
}
@@ -119,8 +120,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN);
}
private boolean checkIfValidCommit(HoodieTable<T> hoodieTable, String commitTs) {
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) {
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().filterCompletedInstants();
// 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
@@ -133,8 +134,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
private Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>
locationTagFunction(HoodieTable<T> hoodieTable) {
private Function2<Integer, Iterator<HoodieRecord<T>>,
Iterator<HoodieRecord<T>>> locationTagFunction(HoodieTableMetaClient metaClient) {
return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>)
(partitionNum, hoodieRecordIterator) -> {
@@ -176,7 +177,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
String partitionPath = Bytes
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
if (checkIfValidCommit(hoodieTable, commitTs)) {
if (checkIfValidCommit(metaClient, commitTs)) {
currentRecord = new HoodieRecord(
new HoodieKey(currentRecord.getRecordKey(), partitionPath),
currentRecord.getData());
@@ -211,13 +212,12 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
}
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable), true);
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true);
}
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>
updateLocationFunction() {
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition,
statusIterator) -> {
Integer multiPutBatchSize = config.getHbaseIndexPutBatchSize();
@@ -306,7 +306,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) {
return writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true);
}

View File

@@ -31,7 +31,6 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -45,6 +44,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -60,21 +60,35 @@ import org.apache.spark.util.SizeEstimator;
public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class);
// This acts as the sequenceID for records written
private static AtomicLong recordIndex = new AtomicLong(1);
private final WriteStatus writeStatus;
private final String fileId;
// Buffer for holding records in memory before they are flushed to disk
List<IndexedRecord> recordList = new ArrayList<>();
// Buffer for holding records (to be deleted) in memory before they are flushed to disk
List<String> keysToDelete = new ArrayList<>();
private TableFileSystemView.RealtimeView fileSystemView;
private String partitionPath;
private Iterator<HoodieRecord<T>> recordItr;
// Total number of records written during an append
private long recordsWritten = 0;
// Total number of records deleted during an append
private long recordsDeleted = 0;
// Average record size for a HoodieRecord. This size is updated at the end of every log block flushed to disk
private long averageRecordSize = 0;
private HoodieLogFile currentLogFile;
private Writer writer;
// Flag used to initialize some metadata
private boolean doInit = true;
// Total number of bytes written during this append phase (an estimation)
private long estimatedNumberOfBytesWritten;
// Number of records that must be written to meet the max block size for a log block
private int numberOfRecords = 0;
// Max block size to limit to for a log block
private int maxBlockSize = config.getLogFileDataBlockMaxSize();
// Header metadata for a log block
private Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
String fileId, Iterator<HoodieRecord<T>> recordItr) {
@@ -87,43 +101,46 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
this.recordItr = recordItr;
}
private void init(String partitionPath) {
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable) {
this(config, commitTime, hoodieTable, UUID.randomUUID().toString(), null);
}
// 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;
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);
private void init(HoodieRecord record) {
if (doInit) {
this.partitionPath = record.getPartitionPath();
// extract some information from the first record
Optional<FileSlice> fileSlice = fileSystemView.getLatestFileSlices(partitionPath)
.filter(fileSlice1 -> fileSlice1.getFileId().equals(fileId)).findFirst();
String baseInstantTime = commitTime;
if (fileSlice.isPresent()) {
baseInstantTime = fileSlice.get().getBaseCommitTime();
} else {
// This means there is no base data file, start appending to a new log file
fileSlice = Optional.of(new FileSlice(baseInstantTime, this.fileId));
logger.info("New InsertHandle for partition :" + partitionPath);
}
writeStatus.getStat().setPrevCommit(baseInstantTime);
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(partitionPath);
writeStatus.getStat().setFileId(fileId);
averageRecordSize = SizeEstimator.estimate(record);
try {
this.writer = createLogWriter(fileSlice, baseInstantTime);
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 HoodieAppendHandle for FileId: " + fileId + " on commit "
+ commitTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath()
+ partitionPath, e);
}
Path path = new Path(partitionPath, writer.getLogFile().getFileName());
writeStatus.getStat().setPath(path.toString());
doInit = false;
}
Path path = new Path(partitionPath,
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
writeStatus.getStat().setPath(path.toString());
}
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
@@ -160,38 +177,11 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
// TODO (NA) - Perform a schema check of current input record with the last schema on log file
// 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;
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
while (recordItr.hasNext()) {
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) {
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
logger.info("AvgRecordSize => " + averageRecordSize);
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2;
doAppend(header);
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
numberOfRecords = 0;
}
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
if (indexedRecord.isPresent()) {
recordList.add(indexedRecord.get());
} else {
keysToDelete.add(record.getRecordKey());
}
numberOfRecords++;
init(record);
flushToDiskIfRequired(record);
writeToBuffer(record);
}
doAppend(header);
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
@@ -199,6 +189,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
private void doAppend(Map<HoodieLogBlock.HeaderMetadataType, String> header) {
try {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
if (recordList.size() > 0) {
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, header));
recordList.clear();
@@ -214,11 +206,37 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
}
}
public void close() {
@Override
public boolean canWrite(HoodieRecord record) {
return config.getParquetMaxFileSize() >= estimatedNumberOfBytesWritten * config
.getLogFileToParquetCompressionRatio();
}
@Override
public void write(HoodieRecord record, Optional<IndexedRecord> insertValue) {
Optional recordMetadata = record.getData().getMetadata();
try {
init(record);
flushToDiskIfRequired(record);
writeToBuffer(record);
} catch (Throwable t) {
// Not throwing exception from here, since we don't want to fail the entire job
// for a single record
writeStatus.markFailure(record, t, recordMetadata);
logger.error("Error writing record " + record, t);
}
}
@Override
public WriteStatus close() {
try {
// flush any remaining records to disk
doAppend(header);
if (writer != null) {
writer.close();
}
writeStatus.getStat().setPrevCommit(commitTime);
writeStatus.getStat().setFileId(this.fileId);
writeStatus.getStat().setNumWrites(recordsWritten);
writeStatus.getStat().setNumDeletes(recordsDeleted);
writeStatus.getStat().setTotalWriteBytes(estimatedNumberOfBytesWritten);
@@ -226,13 +244,54 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
RuntimeStats runtimeStats = new RuntimeStats();
runtimeStats.setTotalUpsertTime(timer.endTimer());
writeStatus.getStat().setRuntimeStats(runtimeStats);
return writeStatus;
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
}
@Override
public WriteStatus getWriteStatus() {
return writeStatus;
}
private Writer createLogWriter(Optional<FileSlice> fileSlice, String baseCommitTime)
throws IOException, InterruptedException {
return HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(
fileSlice.get().getLogFiles().map(logFile -> logFile.getLogVersion())
.max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
.withSizeThreshold(config.getLogFileMaxSize()).withFs(fs)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
}
private void writeToBuffer(HoodieRecord<T> record) {
// update the new location of the record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
if (indexedRecord.isPresent()) {
recordList.add(indexedRecord.get());
} else {
keysToDelete.add(record.getRecordKey());
}
numberOfRecords++;
}
/**
* Checks if the number of records have reached the set threshold and then flushes the records to disk
*/
private void flushToDiskIfRequired(HoodieRecord record) {
// 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
logger.info("AvgRecordSize => " + averageRecordSize);
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2;
doAppend(header);
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
numberOfRecords = 0;
}
}
}

View File

@@ -38,7 +38,7 @@ import org.apache.log4j.Logger;
/**
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
* <p> 1) It provides sufficient time for existing queries running on older versions, to finish <p>
* <p> 1) It provides sufficient time for existing queries running on older versions, to close <p>
* 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done
* based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
*/

View File

@@ -52,6 +52,7 @@ 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.api.java.JavaSparkContext;
/**
* Archiver to bound the growth of <action>.commit files
@@ -99,9 +100,9 @@ public class HoodieCommitArchiveLog {
/**
* Check if commits need to be archived. If yes, archive commits.
*/
public boolean archiveIfRequired() {
public boolean archiveIfRequired(final JavaSparkContext jsc) {
try {
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
List<HoodieInstant> instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList());
boolean success = true;
if (instantsToArchive.iterator().hasNext()) {
this.writer = openWriter();
@@ -117,13 +118,13 @@ public class HoodieCommitArchiveLog {
}
}
private Stream<HoodieInstant> getInstantsToArchive() {
private Stream<HoodieInstant> getInstantsToArchive(JavaSparkContext jsc) {
// TODO : rename to max/minInstantsToKeep
int maxCommitsToKeep = config.getMaxCommitsToKeep();
int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
// GroupBy each action and limit each action timeline to maxCommitsToKeep
// TODO: Handle ROLLBACK_ACTION in future

View File

@@ -31,8 +31,8 @@ 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.Optional;
import java.util.UUID;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -49,12 +49,13 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
private Path tempPath = null;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private Iterator<HoodieRecord<T>> recordIterator;
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
String partitionPath) {
String partitionPath, String fileId) {
super(config, commitTime, hoodieTable);
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
status.setFileId(UUID.randomUUID().toString());
status.setFileId(fileId);
status.setPartitionPath(partitionPath);
final int sparkPartitionId = TaskContext.getPartitionId();
@@ -77,12 +78,13 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
logger.info("New InsertHandle for partition :" + partitionPath);
}
/**
* 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 HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordIterator) {
this(config, commitTime, hoodieTable, partitionPath, fileId);
this.recordIterator = recordIterator;
}
@Override
public boolean canWrite(HoodieRecord record) {
return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath());
}
@@ -114,9 +116,30 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
}
}
/**
* Writes all records passed
*/
public void write() {
try {
while (recordIterator.hasNext()) {
HoodieRecord<T> record = recordIterator.next();
write(record, record.getData().getInsertValue(schema));
}
} catch (IOException io) {
throw new HoodieInsertException(
"Failed to insert records for path " + getStorageWriterPath(), io);
}
}
@Override
public WriteStatus getWriteStatus() {
return status;
}
/**
* Performs actions to durably, persist the current changes and returns a WriteStatus object
*/
@Override
public WriteStatus close() {
logger.info("Closing the file " + status.getFileId() + " as we are done with all the records "
+ recordsWritten);

View File

@@ -16,6 +16,8 @@
package com.uber.hoodie.io;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
@@ -26,7 +28,9 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -103,4 +107,25 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
public Schema getSchema() {
return schema;
}
/**
* 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 false;
}
/**
* Perform the actual writing of the given record into the backing file.
*/
public void write(HoodieRecord record, Optional<IndexedRecord> insertValue) {
// NO_OP
}
public abstract WriteStatus close();
public abstract WriteStatus getWriteStatus();
}

View File

@@ -243,7 +243,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
}
}
public void close() {
@Override
public WriteStatus close() {
try {
// write out any pending records (this can happen when inserts are turned into updates)
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
@@ -269,6 +270,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
RuntimeStats runtimeStats = new RuntimeStats();
runtimeStats.setTotalUpsertTime(timer.endTimer());
writeStatus.getStat().setRuntimeStats(runtimeStats);
return writeStatus;
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
@@ -283,6 +285,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
return (this.tempPath == null) ? this.newFilePath : this.tempPath;
}
@Override
public WriteStatus getWriteStatus() {
return writeStatus;
}

View File

@@ -18,11 +18,13 @@ package com.uber.hoodie.io.compact;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
/**
@@ -33,10 +35,10 @@ import java.util.stream.Collectors;
*/
public class CompactionOperation implements Serializable {
private String dataFileCommitTime;
private long dataFileSize;
private Optional<String> dataFileCommitTime;
private Optional<Long> dataFileSize;
private List<String> deltaFilePaths;
private String dataFilePath;
private Optional<String> dataFilePath;
private String fileId;
private String partitionPath;
private Map<String, Object> metrics;
@@ -46,24 +48,32 @@ public class CompactionOperation implements Serializable {
public CompactionOperation() {
}
public CompactionOperation(HoodieDataFile dataFile, String partitionPath,
public CompactionOperation(Optional<HoodieDataFile> dataFile, String partitionPath,
List<HoodieLogFile> logFiles, HoodieWriteConfig writeConfig) {
this.dataFilePath = dataFile.getPath();
this.fileId = dataFile.getFileId();
if (dataFile.isPresent()) {
this.dataFilePath = Optional.of(dataFile.get().getPath());
this.fileId = dataFile.get().getFileId();
this.dataFileCommitTime = Optional.of(dataFile.get().getCommitTime());
this.dataFileSize = Optional.of(dataFile.get().getFileSize());
} else {
assert logFiles.size() > 0;
this.dataFilePath = Optional.empty();
this.fileId = FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath());
this.dataFileCommitTime = Optional.empty();
this.dataFileSize = Optional.empty();
}
this.partitionPath = partitionPath;
this.dataFileCommitTime = dataFile.getCommitTime();
this.dataFileSize = dataFile.getFileSize();
this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString())
.collect(Collectors.toList());
this.metrics = writeConfig.getCompactionStrategy()
.captureMetrics(dataFile, partitionPath, logFiles);
.captureMetrics(writeConfig, dataFile, partitionPath, logFiles);
}
public String getDataFileCommitTime() {
public Optional<String> getDataFileCommitTime() {
return dataFileCommitTime;
}
public long getDataFileSize() {
public Optional<Long> getDataFileSize() {
return dataFileSize;
}
@@ -71,7 +81,7 @@ public class CompactionOperation implements Serializable {
return deltaFilePaths;
}
public String getDataFilePath() {
public Optional<String> getDataFilePath() {
return dataFilePath;
}

View File

@@ -86,16 +86,19 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
private JavaRDD<WriteStatus> executeCompaction(JavaSparkContext jsc,
List<CompactionOperation> operations, HoodieTable hoodieTable, HoodieWriteConfig config,
String compactionCommitTime) throws IOException {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
// Compacting is very similar to applying updates to existing file
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
log.info("After filtering, Compacting " + operations + " files");
return jsc.parallelize(operations, operations.size())
.map(s -> compact(hoodieTable, config, s, compactionCommitTime))
.map(s -> compact(table, metaClient, config, s, compactionCommitTime))
.flatMap(writeStatusesItr -> writeStatusesItr.iterator());
}
private List<WriteStatus> compact(HoodieTable hoodieTable, HoodieWriteConfig config,
private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
HoodieWriteConfig config,
CompactionOperation operation, String commitTime) throws IOException {
FileSystem fs = hoodieTable.getMetaClient().getFs();
FileSystem fs = metaClient.getFs();
Schema readerSchema = HoodieAvroUtils
.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
@@ -107,7 +110,6 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// 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()
.getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
@@ -125,9 +127,16 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
}
// Compacting is very similar to applying updates to existing file
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient);
Iterator<List<WriteStatus>> result = table
.handleUpdate(commitTime, operation.getFileId(), scanner.getRecords());
Iterator<List<WriteStatus>> result;
// If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a
// new base parquet file.
if (operation.getDataFilePath().isPresent()) {
result = hoodieCopyOnWriteTable
.handleUpdate(commitTime, operation.getFileId(), scanner.getRecords());
} else {
result = hoodieCopyOnWriteTable
.handleInsert(commitTime, operation.getPartitionPath(), operation.getFileId(), scanner.iterator());
}
Iterable<List<WriteStatus>> resultIterable = () -> result;
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
.map(s -> {
@@ -176,7 +185,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
.getLogVersionComparator().reversed()).collect(Collectors.toList());
totalLogFiles.add((long) logFiles.size());
totalFileSlices.add(1L);
return new CompactionOperation(s.getDataFile().get(), partitionPath, logFiles, config);
return new CompactionOperation(s.getDataFile(), partitionPath, logFiles, config);
})
.filter(c -> !c.getDeltaFilePaths().isEmpty())
.collect(toList()).iterator()).collect();
@@ -195,4 +204,4 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
return operations;
}
}
}

View File

@@ -54,16 +54,19 @@ public abstract class CompactionStrategy implements Serializable {
* @param logFiles - List of log files to compact with the base file
* @return Map[String, Object] - metrics captured
*/
public Map<String, Object> captureMetrics(HoodieDataFile dataFile, String partitionPath,
List<HoodieLogFile> logFiles) {
public Map<String, Object> captureMetrics(HoodieWriteConfig writeConfig, Optional<HoodieDataFile> dataFile, String
partitionPath, List<HoodieLogFile> logFiles) {
Map<String, Object> metrics = Maps.newHashMap();
Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize();
// 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);
// Total read will be the base file + all the log files
Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize);
Long totalIORead = FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L)
+ totalLogFileSize);
// Total write will be similar to the size of the base file
Long totalIOWrite = FSUtils.getSizeInMB(dataFile.getFileSize());
Long totalIOWrite = FSUtils
.getSizeInMB(dataFile.isPresent() ? dataFile.get().getFileSize() : defaultMaxParquetFileSize);
// Total IO will the the IO for read + write
Long totalIO = totalIORead + totalIOWrite;
// Save these metrics and we will use during the filter

View File

@@ -39,10 +39,11 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat
private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE";
@Override
public Map<String, Object> captureMetrics(HoodieDataFile dataFile, String partitionPath,
public Map<String, Object> captureMetrics(HoodieWriteConfig config, Optional<HoodieDataFile> dataFile, String
partitionPath,
List<HoodieLogFile> logFiles) {
Map<String, Object> metrics = super.captureMetrics(dataFile, partitionPath, logFiles);
Map<String, Object> metrics = super.captureMetrics(config, 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)

View File

@@ -72,7 +72,7 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
this.schema = schema;
}
private static Configuration registerFileSystem(Path file, Configuration conf) {
public static Configuration registerFileSystem(Path file, Configuration conf) {
Configuration returnConf = new Configuration(conf);
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",

View File

@@ -30,18 +30,17 @@ 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,
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(
String commitTime, Path path, HoodieWriteConfig config, Schema schema,
HoodieTable hoodieTable) throws IOException {
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(

View File

@@ -40,10 +40,11 @@ import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.LazyInsertIterable;
import com.uber.hoodie.func.CopyOnWriteLazyInsertIterable;
import com.uber.hoodie.func.ParquetReaderIterator;
import com.uber.hoodie.func.SparkBoundedInMemoryExecutor;
import com.uber.hoodie.io.HoodieCleanHelper;
import com.uber.hoodie.io.HoodieCreateHandle;
import com.uber.hoodie.io.HoodieMergeHandle;
import java.io.IOException;
import java.io.Serializable;
@@ -64,6 +65,7 @@ import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroParquetReader;
@@ -90,8 +92,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class);
public HoodieCopyOnWriteTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
super(config, metaClient);
public HoodieCopyOnWriteTable(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
}
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String,
@@ -225,7 +227,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
public Iterator<List<WriteStatus>> handleInsert(String commitTime,
Iterator<HoodieRecord<T>> recordItr) throws Exception {
return new LazyInsertIterable<>(recordItr, config, commitTime, this);
return new CopyOnWriteLazyInsertIterable<>(recordItr, config, commitTime, this);
}
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String partitionPath, String fileId,
Iterator<HoodieRecord<T>> recordItr) {
HoodieCreateHandle createHandle = new HoodieCreateHandle(config, commitTime, this, partitionPath, fileId,
recordItr);
createHandle.write();
return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator();
}
@SuppressWarnings("unchecked")
@@ -289,17 +299,29 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
*/
protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits)
throws IOException {
Map<FileStatus, Boolean> results = Maps.newHashMap();
// PathFilter to get all parquet files and log files that need to be deleted
PathFilter filter = (path) -> {
if (path.toString().contains(".parquet")) {
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commits.contains(fileCommitTime);
}
return false;
};
deleteCleanedFiles(results, partitionPath, filter);
return results;
}
/**
* Common method used for cleaning out parquet files under a partition path during rollback of a
* set of commits
*/
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, String partitionPath,
PathFilter filter)
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();
FileStatus[] toBeDeleted = fs.listStatus(new Path(config.getBasePath(), partitionPath), filter);
for (FileStatus file : toBeDeleted) {
boolean success = fs.delete(file.getPath(), false);
results.put(file, success);
@@ -311,7 +333,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
@Override
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
throws IOException {
String actionType = this.getCommitActionType();
String actionType = metaClient.getCommitActionType();
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
List<String> inflights = this.getInflightCommitTimeline().getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());

View File

@@ -27,11 +27,12 @@ import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
@@ -39,6 +40,7 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.MergeOnReadLazyInsertIterable;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
@@ -55,6 +57,7 @@ import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
@@ -78,8 +81,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
// UpsertPartitioner for MergeOnRead table type
private MergeOnReadUpsertPartitioner mergeOnReadUpsertPartitioner;
public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
super(config, metaClient);
public HoodieMergeOnReadTable(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
}
@Override
@@ -110,6 +113,17 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
}
}
@Override
public Iterator<List<WriteStatus>> handleInsert(String commitTime,
Iterator<HoodieRecord<T>> recordItr) throws Exception {
// If canIndexLogFiles, write inserts to log files else write inserts to parquet files
if (index.canIndexLogFiles()) {
return new MergeOnReadLazyInsertIterable<>(recordItr, config, commitTime, this);
} else {
return super.handleInsert(commitTime, recordItr);
}
}
@Override
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionCommitTime) {
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
@@ -166,6 +180,20 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
return commits.stream().map(commit -> {
HoodieInstant instant = commitsAndCompactions.get(commit);
HoodieRollbackStat hoodieRollbackStats = null;
// Need to put the path filter here since Filter is not serializable
// PathFilter to get all parquet files and log files that need to be deleted
PathFilter filter = (path) -> {
if (path.toString().contains(".parquet")) {
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commits.contains(fileCommitTime);
} else if (path.toString().contains(".log")) {
// Since the baseCommitTime is the only commit for new log files, it's okay here
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
return commits.contains(fileCommitTime);
}
return false;
};
switch (instant.getAction()) {
case HoodieTimeline.COMMIT_ACTION:
try {
@@ -180,18 +208,18 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
case HoodieTimeline.DELTA_COMMIT_ACTION:
try {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
this.getCommitTimeline().getInstantDetails(
metaClient.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<>();
final 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)
filesToDeletedStatus = super
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
// In case all data was inserts and the commit failed, there is no partition stats
if (commitMetadata.getPartitionToWriteStats().size() == 0) {
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
}
// append rollback blocks for updates
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
@@ -201,15 +229,31 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseCommitTime)) : null;
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
.filter(wStat -> {
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT
&& wStat.getPrevCommit() != null;
}).forEach(wStat -> {
HoodieLogFormat.Writer writer = null;
if (wStat != null
&& wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT
&& wStat.getPrevCommit() != null) {
return true;
}
// we do not know fileIds for inserts (first inserts are either log files or parquet files),
// delete all files for the corresponding failed commit, if present (same as COW)
try {
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
} catch (IOException io) {
throw new UncheckedIOException(io);
}
return false;
})
.forEach(wStat -> {
Writer writer = null;
String baseCommitTime = wStat.getPrevCommit();
if (hoodieIndex.isGlobal()) {
baseCommitTime = fileIdToLatestCommitTimeMap.get(wStat.getFileId());
}
try {
// TODO : wStat.getPrevCommit() might not give the right commit time in the following
// scenario if a compaction was scheduled, the new commitTime will be used to write the
// new log file. In this case, the commit time for the log file is the
// getBaseCommitTime()
writer = HoodieLogFormat.newWriterBuilder().onParentPath(
new Path(this.getMetaClient().getBasePath(), partitionPath))
.withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime)
@@ -217,15 +261,15 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
Long numRollbackBlocks = 0L;
// generate metadata
Map<HoodieLogBlock.HeaderMetadataType, String> header =
Map<HeaderMetadataType, String> header =
Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME,
header.put(HeaderMetadataType.INSTANT_TIME,
metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME,
header.put(HeaderMetadataType.TARGET_INSTANT_TIME,
commit);
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String
.valueOf(
HoodieCommandBlock.HoodieCommandBlockTypeEnum
HoodieCommandBlockTypeEnum
.ROLLBACK_PREVIOUS_BLOCK
.ordinal()));
// if update belongs to an existing log file
@@ -263,7 +307,6 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
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));
@@ -298,6 +341,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
if (!commitTimeline.empty()) {
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
// find smallest file in partition and append to it
// TODO - check if index.isglobal then small files are log files too
Optional<FileSlice> smallFileSlice = getRTFileSystemView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter(
fileSlice -> fileSlice.getLogFiles().count() < 1
@@ -327,4 +372,5 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.collect(Collectors.toList());
}
}
}

View File

@@ -31,9 +31,9 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieSavepointException;
import com.uber.hoodie.index.HoodieIndex;
import java.io.IOException;
import java.io.Serializable;
import java.util.Iterator;
@@ -54,19 +54,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
protected final HoodieWriteConfig config;
protected final HoodieTableMetaClient metaClient;
protected final HoodieIndex<T> index;
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc) {
this.config = config;
this.metaClient = metaClient;
this.metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true);
this.index = HoodieIndex.createIndex(config, jsc);
}
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return new HoodieCopyOnWriteTable<>(config, metaClient);
return new HoodieCopyOnWriteTable<>(config, jsc);
case MERGE_ON_READ:
return new HoodieMergeOnReadTable<>(config, metaClient);
return new HoodieMergeOnReadTable<>(config, jsc);
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
@@ -124,21 +126,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
* Get the completed (commit + compaction) view of the file system for this table
*/
public TableFileSystemView getCompletedFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCommitsTimeline());
return new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline());
}
/**
* Get only the completed (no-inflights) commit timeline
*/
public HoodieTimeline getCompletedCommitTimeline() {
return getCommitsTimeline().filterCompletedInstants();
return metaClient.getCommitsTimeline().filterCompletedInstants();
}
/**
* Get only the inflights (no-completed) commit timeline
*/
public HoodieTimeline getInflightCommitTimeline() {
return getCommitsTimeline().filterInflights();
return metaClient.getCommitsTimeline().filterInflights();
}
/**
@@ -190,49 +192,10 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
}
/**
* Get the commit timeline visible for this table
* Return the index
*/
public HoodieTimeline getCommitsTimeline() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
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
return getActiveTimeline().getCommitsTimeline();
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
/**
* Get the compacted commit timeline visible for this table
*/
public HoodieTimeline getCommitTimeline() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits in tagging
return getActiveTimeline().getCommitTimeline();
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
/**
* Gets the commit action type
*/
public String getCommitActionType() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
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());
}
public HoodieIndex<T> getIndex() {
return index;
}
/**