Implement Merge on Read Storage (#76)
1. Create HoodieTable abstraction for commits and fileSystemView 2. HoodieMergeOnReadTable created 3. View is now always obtained from the table and the correct view based on the table type is returned
This commit is contained in:
committed by
Prasanna Rajaperumal
parent
11d2fd3428
commit
eb46e7c72b
@@ -26,12 +26,12 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.index.HoodieBloomIndex;
|
||||
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -79,7 +79,7 @@ public class HoodieReadClient implements Serializable {
|
||||
*/
|
||||
private transient final HoodieBloomIndex index;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private HoodieTable hoodieTable;
|
||||
private transient Optional<SQLContext> sqlContextOpt;
|
||||
|
||||
|
||||
@@ -89,9 +89,11 @@ public class HoodieReadClient implements Serializable {
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
|
||||
this.jsc = jsc;
|
||||
this.fs = FSUtils.getFs();
|
||||
this.metaClient = new HoodieTableMetaClient(fs, basePath, true);
|
||||
this.commitTimeline =
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
this.hoodieTable = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
|
||||
this.commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline();
|
||||
|
||||
this.index =
|
||||
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
|
||||
this.sqlContextOpt = Optional.absent();
|
||||
@@ -134,7 +136,7 @@ public class HoodieReadClient implements Serializable {
|
||||
|
||||
assertSqlContext();
|
||||
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
|
||||
index.fetchRecordLocation(hoodieKeys, metaClient);
|
||||
index.fetchRecordLocation(hoodieKeys, hoodieTable);
|
||||
List<String> paths = keyToFileRDD
|
||||
.filter(new Function<Tuple2<HoodieKey, Optional<String>>, Boolean>() {
|
||||
@Override
|
||||
@@ -184,14 +186,14 @@ public class HoodieReadClient implements Serializable {
|
||||
public Dataset<Row> read(String... paths) {
|
||||
assertSqlContext();
|
||||
List<String> filteredPaths = new ArrayList<>();
|
||||
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metaClient);
|
||||
TableFileSystemView fileSystemView = hoodieTable.getFileSystemView();
|
||||
|
||||
try {
|
||||
for (String path : paths) {
|
||||
if (!path.contains(metaClient.getBasePath())) {
|
||||
if (!path.contains(hoodieTable.getMetaClient().getBasePath())) {
|
||||
throw new HoodieException("Path " + path
|
||||
+ " does not seem to be a part of a Hoodie dataset at base path "
|
||||
+ metaClient.getBasePath());
|
||||
+ hoodieTable.getMetaClient().getBasePath());
|
||||
}
|
||||
|
||||
List<HoodieDataFile> latestFiles = fileSystemView.getLatestVersions(fs.globStatus(new Path(path))).collect(
|
||||
@@ -243,8 +245,9 @@ public class HoodieReadClient implements Serializable {
|
||||
*/
|
||||
public Dataset<Row> readCommit(String commitTime) {
|
||||
assertSqlContext();
|
||||
String actionType = hoodieTable.getCompactedCommitActionType();
|
||||
HoodieInstant commitInstant =
|
||||
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
new HoodieInstant(false, actionType, commitTime);
|
||||
if (!commitTimeline.containsInstant(commitInstant)) {
|
||||
new HoodieException("No commit exists at " + commitTime);
|
||||
}
|
||||
@@ -261,6 +264,7 @@ public class HoodieReadClient implements Serializable {
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Checks if the given [Keys] exists in the hoodie table and returns [Key,
|
||||
* Optional[FullFilePath]] If the optional FullFilePath value is not present, then the key is
|
||||
@@ -269,7 +273,7 @@ public class HoodieReadClient implements Serializable {
|
||||
*/
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(
|
||||
JavaRDD<HoodieKey> hoodieKeys) {
|
||||
return index.fetchRecordLocation(hoodieKeys, metaClient);
|
||||
return index.fetchRecordLocation(hoodieKeys, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -280,7 +284,7 @@ public class HoodieReadClient implements Serializable {
|
||||
* @return A subset of hoodieRecords RDD, with existing records filtered out.
|
||||
*/
|
||||
public JavaRDD<HoodieRecord> filterExists(JavaRDD<HoodieRecord> hoodieRecords) {
|
||||
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, metaClient);
|
||||
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, hoodieTable);
|
||||
return recordsWithLocation.filter(new Function<HoodieRecord, Boolean>() {
|
||||
@Override
|
||||
public Boolean call(HoodieRecord v1) throws Exception {
|
||||
|
||||
@@ -128,9 +128,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @return A subset of hoodieRecords RDD, with existing records filtered out.
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, metaClient);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, table);
|
||||
return recordsWithLocation.filter(new Function<HoodieRecord<T>, Boolean>() {
|
||||
@Override
|
||||
public Boolean call(HoodieRecord<T> v1) throws Exception {
|
||||
@@ -144,11 +146,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
|
||||
final HoodieTable table =
|
||||
HoodieTable.getHoodieTable(metaClient, commitTime, config);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
@@ -157,15 +157,14 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
config.getUpsertShuffleParallelism());
|
||||
|
||||
// perform index loop up to get existing location of records
|
||||
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, metaClient);
|
||||
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, table);
|
||||
|
||||
// Cache the tagged records, so we don't end up computing both
|
||||
taggedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||
|
||||
|
||||
WorkloadProfile profile = null;
|
||||
if (table.isWorkloadProfileNeeded()) {
|
||||
profile = new WorkloadProfile(taggedRecords);
|
||||
profile = new WorkloadProfile<>(taggedRecords);
|
||||
logger.info("Workload profile :" + profile);
|
||||
}
|
||||
|
||||
@@ -196,7 +195,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> call(Integer partition,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
return table.handleUpsertPartition(partition, recordItr, upsertPartitioner);
|
||||
return table.handleUpsertPartition(commitTime, partition, recordItr,
|
||||
upsertPartitioner);
|
||||
}
|
||||
}, true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
|
||||
@Override
|
||||
@@ -207,7 +207,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
});
|
||||
|
||||
// Update the index back.
|
||||
JavaRDD<WriteStatus> resultRDD = index.updateLocation(upsertStatusRDD, metaClient);
|
||||
JavaRDD<WriteStatus> resultRDD = index.updateLocation(upsertStatusRDD, table);
|
||||
resultRDD = resultRDD.persist(config.getWriteStatusStorageLevel());
|
||||
commitOnAutoCommit(commitTime, resultRDD);
|
||||
return resultRDD;
|
||||
@@ -250,8 +250,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
@@ -272,7 +273,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
}, true, config.getInsertShuffleParallelism());
|
||||
JavaRDD<WriteStatus> writeStatusRDD = sortedRecords
|
||||
.mapPartitionsWithIndex(new InsertMapFunction<T>(commitTime, config, metaClient),
|
||||
.mapPartitionsWithIndex(new InsertMapFunction<T>(commitTime, config, table),
|
||||
true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
|
||||
@Override
|
||||
public Iterable<WriteStatus> call(List<WriteStatus> writeStatuses)
|
||||
@@ -281,7 +282,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
});
|
||||
// Update the index back
|
||||
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, metaClient);
|
||||
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, table);
|
||||
// Trigger the insert and collect statuses
|
||||
statuses = statuses.persist(config.getWriteStatusStorageLevel());
|
||||
commitOnAutoCommit(commitTime, statuses);
|
||||
@@ -299,9 +300,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses) {
|
||||
logger.info("Comitting " + commitTime);
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
|
||||
List<Tuple2<String, HoodieWriteStat>> stats =
|
||||
writeStatuses.mapToPair(new PairFunction<WriteStatus, String, HoodieWriteStat>() {
|
||||
@@ -318,8 +321,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
try {
|
||||
String actionType = table.getCommitActionType();
|
||||
activeTimeline.saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime),
|
||||
new HoodieInstant(true, actionType, commitTime),
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Save was a success
|
||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||
@@ -333,7 +337,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
metadata);
|
||||
writeContext = null;
|
||||
}
|
||||
logger.info("Status of the commit " + commitTime);
|
||||
logger.info("Committed " + commitTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException(
|
||||
"Failed to commit " + config.getBasePath() + " at time " + commitTime, e);
|
||||
@@ -355,9 +359,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public boolean rollback(final String commitTime) throws HoodieRollbackException {
|
||||
final Timer.Context context = metrics.getRollbackCtx();
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieTimeline inflightTimeline = activeTimeline.getCommitTimeline().filterInflights();
|
||||
HoodieTimeline commitTimeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||
|
||||
@@ -394,7 +399,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
// 3. Delete the new generated parquet files
|
||||
logger.info("Clean out all parquet files generated at time: " + commitTime);
|
||||
final Accumulator<Integer> numFilesDeletedAccu = jsc.accumulator(0);
|
||||
jsc.parallelize(FSUtils.getAllPartitionPaths(fs, metaClient.getBasePath()))
|
||||
jsc.parallelize(
|
||||
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath()))
|
||||
.foreach(new VoidFunction<String>() {
|
||||
@Override
|
||||
public void call(String partitionPath) throws Exception {
|
||||
@@ -450,10 +456,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
try {
|
||||
logger.info("Cleaner started");
|
||||
final Timer.Context context = metrics.getCleanCtx();
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
|
||||
List<String> partitionsToClean = FSUtils.getAllPartitionPaths(fs, metaClient.getBasePath());
|
||||
List<String> partitionsToClean = FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath());
|
||||
// shuffle to distribute cleaning work across partitions evenly
|
||||
Collections.shuffle(partitionsToClean);
|
||||
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config.getCleanerPolicy());
|
||||
@@ -468,7 +475,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
@Override
|
||||
public Integer call(String partitionPathToClean) throws Exception {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieCleaner cleaner = new HoodieCleaner(metaClient, config, fs);
|
||||
HoodieCleaner cleaner = new HoodieCleaner(table, config);
|
||||
return cleaner.clean(partitionPathToClean);
|
||||
}
|
||||
}).reduce(new Function2<Integer, Integer, Integer>() {
|
||||
|
||||
@@ -36,7 +36,7 @@ import java.util.Properties;
|
||||
public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
private static final String BASE_PATH_PROP = "hoodie.base.path";
|
||||
private static final String AVRO_SCHEMA = "hoodie.avro.schema";
|
||||
private static final String TABLE_NAME = "hoodie.table.name";
|
||||
public static final String TABLE_NAME = "hoodie.table.name";
|
||||
private static final String DEFAULT_PARALLELISM = "200";
|
||||
private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism";
|
||||
private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism";
|
||||
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta commit
|
||||
* </p>
|
||||
*/
|
||||
public class HoodieAppendException extends HoodieException {
|
||||
public HoodieAppendException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieAppendException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -21,6 +21,7 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
import java.util.Iterator;
|
||||
@@ -35,18 +36,18 @@ public class InsertMapFunction<T extends HoodieRecordPayload>
|
||||
|
||||
private String commitTime;
|
||||
private HoodieWriteConfig config;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private HoodieTable<T> hoodieTable;
|
||||
|
||||
public InsertMapFunction(String commitTime, HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
HoodieTable<T> hoodieTable) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.metaClient = metaClient;
|
||||
this.hoodieTable = hoodieTable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> call(Integer partition, Iterator<HoodieRecord<T>> sortedRecordItr)
|
||||
throws Exception {
|
||||
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, metaClient);
|
||||
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -24,6 +24,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
|
||||
import com.uber.hoodie.io.HoodieIOHandle;
|
||||
import com.uber.hoodie.io.HoodieInsertHandle;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.util.ArrayList;
|
||||
@@ -40,17 +41,17 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyItera
|
||||
|
||||
private final HoodieWriteConfig hoodieConfig;
|
||||
private final String commitTime;
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieTable<T> hoodieTable;
|
||||
private Set<String> partitionsCleaned;
|
||||
private HoodieInsertHandle handle;
|
||||
|
||||
public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
||||
String commitTime, HoodieTableMetaClient metaClient) {
|
||||
String commitTime, HoodieTable<T> hoodieTable) {
|
||||
super(sortedRecordItr);
|
||||
this.partitionsCleaned = new HashSet<>();
|
||||
this.hoodieConfig = config;
|
||||
this.commitTime = commitTime;
|
||||
this.metaClient = metaClient;
|
||||
this.hoodieTable = hoodieTable;
|
||||
}
|
||||
|
||||
@Override protected void start() {
|
||||
@@ -78,7 +79,7 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyItera
|
||||
// lazily initialize the handle, for the first time
|
||||
if (handle == null) {
|
||||
handle =
|
||||
new HoodieInsertHandle(hoodieConfig, commitTime, metaClient,
|
||||
new HoodieInsertHandle(hoodieConfig, commitTime, hoodieTable,
|
||||
record.getPartitionPath());
|
||||
}
|
||||
|
||||
@@ -90,7 +91,7 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyItera
|
||||
statuses.add(handle.close());
|
||||
// Need to handle the rejected record & open new handle
|
||||
handle =
|
||||
new HoodieInsertHandle(hoodieConfig, commitTime, metaClient,
|
||||
new HoodieInsertHandle(hoodieConfig, commitTime, hoodieTable,
|
||||
record.getPartitionPath());
|
||||
handle.write(record); // we should be able to write 1 record.
|
||||
break;
|
||||
|
||||
@@ -30,6 +30,7 @@ import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException;
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
@@ -67,7 +68,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, HoodieTableMetaClient metaClient) {
|
||||
JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> hoodieTable) {
|
||||
throw new UnsupportedOperationException("HBase index does not implement check exist yet");
|
||||
}
|
||||
|
||||
@@ -93,10 +94,10 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
class LocationTagFunction
|
||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieTable<T> hoodieTable;
|
||||
|
||||
LocationTagFunction(HoodieTableMetaClient metaClient) {
|
||||
this.metaClient = metaClient;
|
||||
LocationTagFunction(HoodieTable<T> hoodieTable) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -129,9 +130,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
String fileId =
|
||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
||||
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants();
|
||||
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
// if the last commit ts for this row is less than the system commit ts
|
||||
if (!commitTimeline.empty() && commitTimeline.containsInstant(
|
||||
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))) {
|
||||
@@ -160,9 +159,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(metaClient), true);
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieTable<T> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(hoodieTable), true);
|
||||
}
|
||||
|
||||
class UpdateLocationTask implements Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> {
|
||||
@@ -223,7 +221,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true);
|
||||
}
|
||||
|
||||
|
||||
@@ -18,21 +18,16 @@ package com.uber.hoodie.index;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Optional;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
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.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -43,9 +38,12 @@ import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
import org.apache.spark.api.java.function.PairFlatMapFunction;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
|
||||
import scala.Tuple2;
|
||||
import java.util.*;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
@@ -67,7 +65,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTableMetaClient metaClient) {
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTable<T> hoodieTable) {
|
||||
|
||||
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
|
||||
@@ -80,7 +78,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
||||
lookupIndex(partitionRecordKeyPairRDD, metaClient);
|
||||
lookupIndex(partitionRecordKeyPairRDD, hoodieTable);
|
||||
|
||||
// Cache the result, for subsequent stages.
|
||||
rowKeyFilenamePairRDD.cache();
|
||||
@@ -94,7 +92,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient) {
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> hoodieTable) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
||||
hoodieKeys.mapToPair(new PairFunction<HoodieKey, String, String>() {
|
||||
@Override
|
||||
@@ -105,7 +103,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
||||
lookupIndex(partitionRecordKeyPairRDD, metaClient);
|
||||
lookupIndex(partitionRecordKeyPairRDD, hoodieTable);
|
||||
|
||||
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
|
||||
hoodieKeys.mapToPair(new PairFunction<HoodieKey, String, HoodieKey>() {
|
||||
@@ -125,9 +123,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
if (keyPathTuple._2._2.isPresent()) {
|
||||
String fileName = keyPathTuple._2._2.get();
|
||||
String partitionPath = keyPathTuple._2._1.getPartitionPath();
|
||||
recordLocationPath = Optional
|
||||
.of(new Path(new Path(metaClient.getBasePath(), partitionPath), fileName)
|
||||
.toUri().getPath());
|
||||
recordLocationPath = Optional.of(new Path(
|
||||
new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath),
|
||||
fileName).toUri().getPath());
|
||||
} else {
|
||||
recordLocationPath = Optional.absent();
|
||||
}
|
||||
@@ -141,18 +139,18 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* record keys already present and drop the record keys if not present
|
||||
*
|
||||
* @param partitionRecordKeyPairRDD
|
||||
* @param metaClient
|
||||
* @param hoodieTable
|
||||
* @return
|
||||
*/
|
||||
private JavaPairRDD<String, String> lookupIndex(
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTableMetaClient metaClient) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTable<T> hoodieTable) {
|
||||
// Obtain records per partition, in the incoming records
|
||||
Map<String, Object> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||
|
||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||
JavaPairRDD<String, String> partitionFilePairRDD =
|
||||
loadInvolvedFiles(affectedPartitionPathList, metaClient);
|
||||
loadInvolvedFiles(affectedPartitionPathList, hoodieTable);
|
||||
Map<String, Object> filesPerPartition = partitionFilePairRDD.countByKey();
|
||||
|
||||
// Compute total subpartitions, to split partitions into.
|
||||
@@ -212,19 +210,17 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
*/
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<String, String> loadInvolvedFiles(List<String> partitions,
|
||||
final HoodieTableMetaClient metaClient) {
|
||||
final HoodieTable<T> hoodieTable) {
|
||||
return jsc.parallelize(partitions, Math.max(partitions.size(), 1))
|
||||
.flatMapToPair(new PairFlatMapFunction<String, String, String>() {
|
||||
@Override
|
||||
public Iterable<Tuple2<String, String>> call(String partitionPath) {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
TableFileSystemView view = new ReadOptimizedTableView(fs, metaClient);
|
||||
java.util.Optional<HoodieInstant> latestCommitTime =
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
|
||||
hoodieTable.getCommitTimeline().filterCompletedInstants().lastInstant();
|
||||
List<Tuple2<String, String>> list = new ArrayList<>();
|
||||
if (latestCommitTime.isPresent()) {
|
||||
List<HoodieDataFile> filteredFiles =
|
||||
view.getLatestVersionInPartition(partitionPath,
|
||||
hoodieTable.getFileSystemView().getLatestVersionInPartition(partitionPath,
|
||||
latestCommitTime.get().getTimestamp()).collect(Collectors.toList());
|
||||
for (HoodieDataFile file : filteredFiles) {
|
||||
list.add(new Tuple2<>(partitionPath, file.getFileName()));
|
||||
@@ -235,6 +231,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// Nope, don't need to do anything.
|
||||
@@ -424,7 +421,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTableMetaClient metaClient) {
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -26,6 +26,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -63,15 +64,14 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
* @return
|
||||
*/
|
||||
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient);
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> metaClient);
|
||||
|
||||
/**
|
||||
* 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,
|
||||
HoodieTableMetaClient metaClient) throws
|
||||
HoodieIndexException;
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Extracts the location of written records, and updates the index.
|
||||
@@ -79,7 +79,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTableMetaClient metaClient) throws HoodieIndexException;
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Rollback the efffects of the commit made at commitTime.
|
||||
|
||||
@@ -25,6 +25,7 @@ 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.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -55,7 +56,7 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient) {
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> hoodieTable) {
|
||||
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
|
||||
}
|
||||
|
||||
@@ -81,13 +82,13 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
|
||||
@Override
|
||||
public WriteStatus call(WriteStatus writeStatus) {
|
||||
|
||||
@@ -0,0 +1,163 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import com.clearspring.analytics.util.Lists;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieDeltaWriteStat;
|
||||
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.log.HoodieLogAppendConfig;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.log.avro.RollingAvroLogAppender;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieAppendException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieUpdateHandle.class);
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
private final WriteStatus writeStatus;
|
||||
private final String fileId;
|
||||
private String partitionPath;
|
||||
private RollingAvroLogAppender logAppender;
|
||||
private List<HoodieRecord<T>> records;
|
||||
private long recordsWritten = 0;
|
||||
private HoodieLogFile currentLogFile;
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTable<T> hoodieTable, String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
WriteStatus writeStatus = new WriteStatus();
|
||||
writeStatus.setStat(new HoodieDeltaWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
this.fileId = fileId;
|
||||
init(recordItr);
|
||||
}
|
||||
|
||||
private void init(Iterator<HoodieRecord<T>> recordItr) {
|
||||
List<HoodieRecord<T>> records = Lists.newArrayList();
|
||||
recordItr.forEachRemaining(record -> {
|
||||
records.add(record);
|
||||
// extract some information from the first record
|
||||
if (partitionPath == null) {
|
||||
partitionPath = record.getPartitionPath();
|
||||
String latestValidFilePath =
|
||||
fileSystemView.getLatestDataFilesForFileId(record.getPartitionPath(), fileId)
|
||||
.findFirst().get().getFileName();
|
||||
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
|
||||
writeStatus.getStat().setPrevCommit(baseCommitTime);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(record.getPartitionPath());
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
|
||||
try {
|
||||
HoodieLogAppendConfig logConfig = HoodieLogAppendConfig.newBuilder()
|
||||
.onPartitionPath(
|
||||
new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).withBaseCommitTime(baseCommitTime).withSchema(schema)
|
||||
.withFs(fs).withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
this.logAppender = new RollingAvroLogAppender(logConfig);
|
||||
this.currentLogFile = logAppender.getConfig().getLogFile();
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||
.setLogVersion(currentLogFile.getLogVersion());
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||
.setLogOffset(logAppender.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());
|
||||
}
|
||||
writeStatus.getStat().setFullPath(currentLogFile.getPath().toString());
|
||||
}
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
});
|
||||
this.records = records;
|
||||
}
|
||||
|
||||
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
try {
|
||||
IndexedRecord avroRecord = hoodieRecord.getData().getInsertValue(schema);
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils
|
||||
.addHoodieKeyToRecord((GenericRecord) avroRecord, hoodieRecord.getRecordKey(),
|
||||
hoodieRecord.getPartitionPath(), fileId);
|
||||
HoodieAvroUtils
|
||||
.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||
hoodieRecord.deflate();
|
||||
writeStatus.markSuccess(hoodieRecord);
|
||||
recordsWritten++;
|
||||
return Optional.of(avroRecord);
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record " + hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e);
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
public void doAppend() {
|
||||
Iterator<IndexedRecord> recordItr =
|
||||
records.stream().map(this::getIndexedRecord).filter(Optional::isPresent)
|
||||
.map(Optional::get).iterator();
|
||||
try {
|
||||
logAppender.append(recordItr);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieAppendException(
|
||||
"Failed while appeding records to " + currentLogFile.getPath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
try {
|
||||
if (logAppender != null) {
|
||||
logAppender.close();
|
||||
}
|
||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
}
|
||||
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -17,14 +17,12 @@
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -58,18 +56,16 @@ public class HoodieCleaner {
|
||||
|
||||
private final TableFileSystemView fileSystemView;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private HoodieTable hoodieTable;
|
||||
private HoodieWriteConfig config;
|
||||
private FileSystem fs;
|
||||
|
||||
public HoodieCleaner(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
|
||||
FileSystem fs) {
|
||||
this.metaClient = metaClient;
|
||||
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
|
||||
this.commitTimeline =
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
public HoodieCleaner(HoodieTable hoodieTable, HoodieWriteConfig config) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileSystemView = hoodieTable.getCompactedFileSystemView();
|
||||
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.config = config;
|
||||
this.fs = fs;
|
||||
this.fs = hoodieTable.getFs();
|
||||
}
|
||||
|
||||
|
||||
|
||||
@@ -16,15 +16,14 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -39,20 +38,19 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
protected final String commitTime;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final HoodieTimeline hoodieTimeline;
|
||||
protected final TableFileSystemView fileSystemView;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
protected HoodieTimeline hoodieTimeline;
|
||||
protected TableFileSystemView fileSystemView;
|
||||
protected final Schema schema;
|
||||
|
||||
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
HoodieTable<T> hoodieTable) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.fs = FSUtils.getFs();
|
||||
this.metaClient = metaClient;
|
||||
this.hoodieTimeline =
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.fileSystemView = hoodieTable.getFileSystemView();
|
||||
this.schema =
|
||||
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
}
|
||||
|
||||
@@ -16,17 +16,17 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
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.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieInsertException;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -45,8 +45,8 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private int recordsWritten = 0;
|
||||
|
||||
public HoodieInsertHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTableMetaClient metadata, String partitionPath) {
|
||||
super(config, commitTime, metadata);
|
||||
HoodieTable<T> hoodieTable, String partitionPath) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
this.status = new WriteStatus();
|
||||
status.setFileId(UUID.randomUUID().toString());
|
||||
status.setPartitionPath(partitionPath);
|
||||
@@ -54,7 +54,7 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId());
|
||||
try {
|
||||
this.storageWriter =
|
||||
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, metadata, config, schema);
|
||||
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, hoodieTable, config, schema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||
|
||||
@@ -16,7 +16,6 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
@@ -27,6 +26,7 @@ import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -38,11 +38,12 @@ import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
|
||||
@SuppressWarnings("Duplicates") public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
@SuppressWarnings("Duplicates")
|
||||
public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieUpdateHandle.class);
|
||||
|
||||
private final WriteStatus writeStatus;
|
||||
private final HashMap<String, HoodieRecord<T>> keyToNewRecords;
|
||||
private WriteStatus writeStatus;
|
||||
private HashMap<String, HoodieRecord<T>> keyToNewRecords;
|
||||
private HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private Path newFilePath;
|
||||
private Path oldFilePath;
|
||||
@@ -52,22 +53,23 @@ import java.util.Iterator;
|
||||
|
||||
public HoodieUpdateHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTableMetaClient metaClient,
|
||||
HoodieTable<T> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr,
|
||||
String fileId) {
|
||||
super(config, commitTime, metaClient);
|
||||
WriteStatus writeStatus = new WriteStatus();
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
this.fileId = fileId;
|
||||
this.keyToNewRecords = new HashMap<>();
|
||||
init(recordItr);
|
||||
super(config, commitTime, hoodieTable);
|
||||
init(fileId, recordItr);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load the new incoming records in a map, and extract the old file path.
|
||||
*/
|
||||
private void init(Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
WriteStatus writeStatus = new WriteStatus();
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
this.fileId = fileId;
|
||||
this.keyToNewRecords = new HashMap<>();
|
||||
|
||||
try {
|
||||
// Load the new records in a map
|
||||
while (newRecordsItr.hasNext()) {
|
||||
@@ -104,14 +106,14 @@ import java.util.Iterator;
|
||||
}
|
||||
// Create the writer for writing the new version file
|
||||
storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, newFilePath, metaClient, config, schema);
|
||||
.getStorageWriter(commitTime, newFilePath, hoodieTable, config, schema);
|
||||
|
||||
} 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 " + metaClient.getBasePath());
|
||||
+ commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -16,18 +16,15 @@
|
||||
|
||||
package com.uber.hoodie.io.compact;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.RealtimeTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Date;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* A HoodieCompactor runs compaction on a hoodie table
|
||||
@@ -38,14 +35,13 @@ public interface HoodieCompactor extends Serializable {
|
||||
* @throws Exception
|
||||
*/
|
||||
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient, RealtimeTableView fsView,
|
||||
CompactionFilter compactionFilter) throws Exception;
|
||||
HoodieTable hoodieTable, CompactionFilter compactionFilter) throws Exception;
|
||||
|
||||
|
||||
// Helper methods
|
||||
default String startCompactionCommit(HoodieTableMetaClient metaClient) {
|
||||
default String startCompactionCommit(HoodieTable hoodieTable) {
|
||||
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
|
||||
HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline();
|
||||
activeTimeline
|
||||
.createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime));
|
||||
return commitTime;
|
||||
|
||||
@@ -24,13 +24,14 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.RealtimeTableView;
|
||||
import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
||||
import com.uber.hoodie.table.HoodieMergeOnReadTable;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.commons.collections.IteratorUtils;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -61,11 +62,12 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
|
||||
@Override
|
||||
public HoodieCompactionMetadata compact(JavaSparkContext jsc, HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient, RealtimeTableView fsView,
|
||||
HoodieTable hoodieTable,
|
||||
CompactionFilter compactionFilter) throws Exception {
|
||||
// TODO - rollback any compactions in flight
|
||||
|
||||
String compactionCommit = startCompactionCommit(metaClient);
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
String compactionCommit = startCompactionCommit(hoodieTable);
|
||||
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommit);
|
||||
List<String> partitionPaths =
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath());
|
||||
@@ -74,9 +76,9 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
List<CompactionOperation> operations =
|
||||
jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> {
|
||||
FileSystem fileSystem = FSUtils.getFs();
|
||||
return fsView.groupLatestDataFileWithLogFiles(fileSystem, partitionPath)
|
||||
.entrySet().stream()
|
||||
return hoodieTable.getFileSystemView()
|
||||
.groupLatestDataFileWithLogFiles(partitionPath).entrySet()
|
||||
.stream()
|
||||
.map(s -> new CompactionOperation(s.getKey(), partitionPath, s.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
}).collect();
|
||||
@@ -145,9 +147,15 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
List<HoodieRecord<HoodieAvroPayload>> readDeltaFilesInMemory =
|
||||
AvroUtils.loadFromFiles(fs, operation.getDeltaFilePaths(), schema);
|
||||
|
||||
if(readDeltaFilesInMemory.isEmpty()) {
|
||||
return IteratorUtils.emptyIterator();
|
||||
}
|
||||
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
HoodieCopyOnWriteTable<HoodieAvroPayload> table =
|
||||
new HoodieCopyOnWriteTable<>(commitTime, config, metaClient);
|
||||
return table.handleUpdate(operation.getFileId(), readDeltaFilesInMemory.iterator());
|
||||
new HoodieCopyOnWriteTable<>(config, metaClient);
|
||||
return table
|
||||
.handleUpdate(commitTime, operation.getFileId(), readDeltaFilesInMemory.iterator());
|
||||
}
|
||||
|
||||
public boolean commitCompaction(String commitTime, HoodieTableMetaClient metaClient,
|
||||
|
||||
@@ -22,6 +22,7 @@ import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -32,7 +33,7 @@ import java.io.IOException;
|
||||
|
||||
public class HoodieStorageWriterFactory {
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
||||
String commitTime, Path path, HoodieTableMetaClient metaClient, HoodieWriteConfig config, Schema schema)
|
||||
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
|
||||
|
||||
@@ -19,9 +19,7 @@ package com.uber.hoodie.table;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
@@ -70,6 +68,11 @@ import scala.Tuple2;
|
||||
*
|
||||
*/
|
||||
public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieTable {
|
||||
public HoodieCopyOnWriteTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
super(config, metaClient);
|
||||
}
|
||||
|
||||
|
||||
|
||||
// seed for random number generator. No particular significance, just makes testing deterministic
|
||||
private static final long RANDOM_NUMBER_SEED = 356374L;
|
||||
@@ -137,10 +140,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
|
||||
public HoodieCopyOnWriteTable(String commitTime, HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
super(commitTime, config, metaClient);
|
||||
}
|
||||
|
||||
/**
|
||||
* Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition)
|
||||
*/
|
||||
@@ -291,13 +290,11 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metaClient);
|
||||
HoodieTimeline commitTimeline = getCompletedCommitTimeline();
|
||||
|
||||
if (!commitTimeline.empty()) { // if we have some commits
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
List<HoodieDataFile> allFiles = fileSystemView
|
||||
List<HoodieDataFile> allFiles = getFileSystemView()
|
||||
.getLatestVersionInPartition(partitionPath, latestCommitTime.getTimestamp())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
@@ -399,11 +396,10 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
|
||||
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String fileLoc, Iterator<HoodieRecord<T>> recordItr)
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc, Iterator<HoodieRecord<T>> recordItr)
|
||||
throws IOException {
|
||||
// these are updates
|
||||
HoodieUpdateHandle upsertHandle =
|
||||
new HoodieUpdateHandle<>(config, commitTime, metaClient, recordItr, fileLoc);
|
||||
HoodieUpdateHandle upsertHandle = getUpdateHandle(commitTime, fileLoc, recordItr);
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException("Error in finding the old file path at commit " +
|
||||
commitTime +" at fileLoc: " + fileLoc);
|
||||
@@ -437,23 +433,27 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
return new LazyInsertIterable<>(recordItr, config, commitTime, metaClient);
|
||||
protected HoodieUpdateHandle getUpdateHandle(String commitTime, String fileLoc, Iterator<HoodieRecord<T>> recordItr) {
|
||||
return new HoodieUpdateHandle<>(config, commitTime, this, recordItr, fileLoc);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime, Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
return new LazyInsertIterable<>(recordItr, config, commitTime, this);
|
||||
}
|
||||
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpsertPartition(Integer partition,
|
||||
Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
public Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime, Integer partition,
|
||||
Iterator recordItr, Partitioner partitioner) {
|
||||
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
|
||||
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
|
||||
BucketType btype = binfo.bucketType;
|
||||
try {
|
||||
if (btype.equals(BucketType.INSERT)) {
|
||||
return handleInsert(recordItr);
|
||||
return handleInsert(commitTime, recordItr);
|
||||
} else if (btype.equals(BucketType.UPDATE)) {
|
||||
return handleUpdate(binfo.fileLoc, recordItr);
|
||||
return handleUpdate(commitTime, binfo.fileLoc, recordItr);
|
||||
} else {
|
||||
throw new HoodieUpsertException("Unknown bucketType " + btype + " for partition :" + partition);
|
||||
}
|
||||
|
||||
@@ -0,0 +1,62 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
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.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.io.HoodieAppendHandle;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Implementation of a more real-time read-optimized Hoodie Table where
|
||||
*
|
||||
* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or)
|
||||
* Merge with the smallest existing file, to expand it
|
||||
*
|
||||
* UPDATES - Appends the changes to a rolling log file maintained per file Id.
|
||||
* Compaction merges the log file into the base file.
|
||||
*
|
||||
*/
|
||||
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieCopyOnWriteTable<T> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
||||
|
||||
public HoodieMergeOnReadTable(HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
super(config, metaClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
|
||||
HoodieAppendHandle<T> appendHandle =
|
||||
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
|
||||
appendHandle.doAppend();
|
||||
appendHandle.close();
|
||||
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
|
||||
.iterator();
|
||||
}
|
||||
}
|
||||
@@ -16,14 +16,20 @@
|
||||
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.spark.Partitioner;
|
||||
|
||||
import java.io.Serializable;
|
||||
@@ -34,16 +40,10 @@ import java.util.List;
|
||||
* Abstract implementation of a HoodieTable
|
||||
*/
|
||||
public abstract class HoodieTable<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
protected final String commitTime;
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
|
||||
protected HoodieTable(String commitTime, HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
this.commitTime = commitTime;
|
||||
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
this.config = config;
|
||||
this.metaClient = metaClient;
|
||||
}
|
||||
@@ -73,6 +73,124 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
*/
|
||||
public abstract boolean isWorkloadProfileNeeded();
|
||||
|
||||
public HoodieWriteConfig getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient getMetaClient() {
|
||||
return metaClient;
|
||||
}
|
||||
|
||||
public FileSystem getFs() {
|
||||
return metaClient.getFs();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the view of the file system for this table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public TableFileSystemView getFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the view of the file system for this table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public TableFileSystemView getCompactedFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCompactionCommitTimeline());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) commit timeline
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCompletedCommitTimeline() {
|
||||
return getCommitTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
public HoodieActiveTimeline getActiveTimeline() {
|
||||
return metaClient.getActiveTimeline();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the commit timeline visible for this table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCommitTimeline() {
|
||||
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
|
||||
return getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION,
|
||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION));
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) compaction commit timeline
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCompletedCompactionCommitTimeline() {
|
||||
return getCompactionCommitTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the compacted commit timeline visible for this table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCompactionCommitTimeline() {
|
||||
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 in tagging
|
||||
return getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION));
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the commit action type
|
||||
* @return
|
||||
*/
|
||||
public String getCommitActionType() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return HoodieActiveTimeline.COMMIT_ACTION;
|
||||
case MERGE_ON_READ:
|
||||
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
|
||||
}
|
||||
throw new HoodieCommitException(
|
||||
"Could not commit on unknown storage type " + metaClient.getTableType());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the action type for a compaction commit
|
||||
* @return
|
||||
*/
|
||||
public String getCompactedCommitActionType() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return HoodieTimeline.COMMIT_ACTION;
|
||||
case MERGE_ON_READ:
|
||||
return HoodieTimeline.COMPACTION_ACTION;
|
||||
}
|
||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Perform the ultimate IO for a given upserted (RDD) partition
|
||||
@@ -81,28 +199,19 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
* @param recordIterator
|
||||
* @param partitioner
|
||||
*/
|
||||
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(Integer partition,
|
||||
Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
/**
|
||||
* Perform the ultimate IO for a given inserted (RDD) partition
|
||||
*
|
||||
* @param partition
|
||||
* @param recordIterator
|
||||
* @param partitioner
|
||||
*/
|
||||
public abstract Iterator<List<WriteStatus>> handleInsertPartition(Integer partition,
|
||||
Iterator<HoodieRecord<T>> recordIterator,
|
||||
Partitioner partitioner);
|
||||
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime,
|
||||
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
|
||||
public static HoodieTable getHoodieTable(HoodieTableMetaClient metaClient,
|
||||
String commitTime,
|
||||
HoodieWriteConfig config) {
|
||||
if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) {
|
||||
return new HoodieCopyOnWriteTable(commitTime, config, metaClient);
|
||||
} else {
|
||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
||||
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return new HoodieCopyOnWriteTable<>(config, metaClient);
|
||||
case MERGE_ON_READ:
|
||||
return new HoodieMergeOnReadTable<>(config, metaClient);
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user