Multi FS Support
- Reviving PR 191, to make FileSystem creation off actual path - Streamline all filesystem access to HoodieTableMetaClient - Hadoop Conf from Spark Context serialized & passed to executor code too - Pick up env vars prefixed with HOODIE_ENV_ into Configuration object - Cleanup usage of FSUtils.getFS, piggybacking off HoodieTableMetaClient.getFS - Adding s3a to supported schemes & support escaping "." in env vars - Tests use HoodieTestUtils.getDefaultHadoopConf
This commit is contained in:
committed by
vinoth chandar
parent
44839b88c6
commit
0cd186c899
@@ -67,10 +67,10 @@ public class HoodieReadClient implements Serializable {
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
|
||||
this.jsc = jsc;
|
||||
this.fs = FSUtils.getFs();
|
||||
this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
this.hoodieTable = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
|
||||
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), null);
|
||||
this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants();
|
||||
this.index =
|
||||
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
|
||||
@@ -129,8 +129,8 @@ public class HoodieReadClient implements Serializable {
|
||||
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD()
|
||||
.mapToPair(row -> {
|
||||
HoodieKey key = new HoodieKey(
|
||||
row.<String>getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
|
||||
row.<String>getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
|
||||
row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
|
||||
row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
|
||||
return new Tuple2<>(key, row);
|
||||
});
|
||||
|
||||
|
||||
@@ -53,18 +53,6 @@ import com.uber.hoodie.table.HoodieTable;
|
||||
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
|
||||
import com.uber.hoodie.table.WorkloadProfile;
|
||||
import com.uber.hoodie.table.WorkloadStat;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
@@ -76,6 +64,17 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient
|
||||
@@ -112,7 +111,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
|
||||
boolean rollbackInFlight) {
|
||||
this.fs = FSUtils.getFs();
|
||||
this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
|
||||
this.jsc = jsc;
|
||||
this.config = clientConfig;
|
||||
this.index = HoodieIndex.createIndex(config, jsc);
|
||||
@@ -133,8 +132,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(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, table);
|
||||
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
||||
@@ -146,8 +146,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
@@ -180,8 +181,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
@@ -237,8 +239,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
@@ -306,8 +309,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* file instead of using HoodieCommitMetadata
|
||||
*/
|
||||
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile,
|
||||
HoodieTable<T> table,
|
||||
String commitTime) throws HoodieCommitException {
|
||||
HoodieTable<T> table, String commitTime) throws HoodieCommitException {
|
||||
try {
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
profile.getPartitionPaths().stream().forEach(path -> {
|
||||
@@ -409,8 +411,9 @@ 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(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
|
||||
@@ -485,8 +488,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @return true if the savepoint was created successfully
|
||||
*/
|
||||
public boolean savepoint(String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
if (table.getCompletedCommitTimeline().empty()) {
|
||||
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
|
||||
}
|
||||
@@ -512,8 +516,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @return true if the savepoint was created successfully
|
||||
*/
|
||||
public boolean savepoint(String commitTime, String user, String comment) {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
|
||||
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
|
||||
@@ -575,8 +580,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @return true if the savepoint was deleted successfully
|
||||
*/
|
||||
public void deleteSavepoint(String savepointTime) {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
|
||||
HoodieInstant savePoint =
|
||||
@@ -602,8 +608,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @return true if the savepoint was rollecback to successfully
|
||||
*/
|
||||
public boolean rollbackToSavepoint(String savepointTime) {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieTimeline commitTimeline = table.getCommitsTimeline();
|
||||
|
||||
@@ -653,8 +660,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
String startRollbackTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieTimeline inflightTimeline = table.getInflightCommitTimeline();
|
||||
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
|
||||
@@ -761,8 +769,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
final Timer.Context context = metrics.getCleanCtx();
|
||||
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(),
|
||||
true), config);
|
||||
|
||||
List<HoodieCleanStat> cleanStats = table.clean(jsc);
|
||||
if (cleanStats.isEmpty()) {
|
||||
@@ -810,8 +819,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(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
String commitActionType = table.getCommitActionType();
|
||||
activeTimeline.createInflight(
|
||||
@@ -827,8 +837,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
private void compact(String compactionCommitTime) throws IOException {
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
Optional<HoodieCommitMetadata> compactionMetadata = table.compact(jsc, compactionCommitTime);
|
||||
if (compactionMetadata.isPresent()) {
|
||||
logger.info("Compacted successfully on commit " + compactionCommitTime);
|
||||
@@ -876,8 +887,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Cleanup all inflight commits
|
||||
*/
|
||||
private void rollbackInflightCommits() {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights();
|
||||
List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
@@ -72,7 +72,8 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
|
||||
HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig,
|
||||
commitTime,
|
||||
record.getPartitionPath(),
|
||||
TaskContext.getPartitionId());
|
||||
TaskContext.getPartitionId(),
|
||||
hoodieTable);
|
||||
partitionsCleaned.add(record.getPartitionPath());
|
||||
}
|
||||
|
||||
|
||||
@@ -155,7 +155,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it.
|
||||
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo,
|
||||
partitionRecordKeyPairRDD);
|
||||
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD,
|
||||
return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo,
|
||||
partitionRecordKeyPairRDD,
|
||||
parallelism);
|
||||
}
|
||||
|
||||
@@ -257,7 +258,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
.mapToPair(ft -> {
|
||||
try {
|
||||
String[] minMaxKeys = ParquetUtils
|
||||
.readMinMaxRecordKeys(ft._2().getFileStatus().getPath());
|
||||
.readMinMaxRecordKeys(hoodieTable.getHadoopConf(),
|
||||
ft._2().getFileStatus().getPath());
|
||||
return new Tuple2<>(ft._1(),
|
||||
new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
|
||||
} catch (MetadataNotFoundException me) {
|
||||
@@ -358,7 +360,7 @@ 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(
|
||||
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(HoodieTable hoodieTable,
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD,
|
||||
int totalSubpartitions) {
|
||||
@@ -372,7 +374,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
.sortByKey(true, joinParallelism);
|
||||
|
||||
return fileSortedTripletRDD
|
||||
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true)
|
||||
.mapPartitionsWithIndex(
|
||||
new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true)
|
||||
.flatMap(indexLookupResults -> indexLookupResults.iterator())
|
||||
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
|
||||
.flatMapToPair(lookupResult -> {
|
||||
|
||||
@@ -24,10 +24,12 @@ 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.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -45,20 +47,24 @@ public class HoodieBloomIndexCheckFunction implements
|
||||
|
||||
private final String basePath;
|
||||
|
||||
public HoodieBloomIndexCheckFunction(String basePath) {
|
||||
private final HoodieTable table;
|
||||
|
||||
public HoodieBloomIndexCheckFunction(HoodieTable table, String basePath) {
|
||||
this.table = table;
|
||||
this.basePath = basePath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a list of row keys and one file, return only row keys existing in that file.
|
||||
*/
|
||||
public static List<String> checkCandidatesAgainstFile(List<String> candidateRecordKeys,
|
||||
public static List<String> checkCandidatesAgainstFile(Configuration configuration,
|
||||
List<String> candidateRecordKeys,
|
||||
Path filePath) throws HoodieIndexException {
|
||||
List<String> foundRecordKeys = new ArrayList<>();
|
||||
try {
|
||||
// Load all rowKeys from the file, to double-confirm
|
||||
if (!candidateRecordKeys.isEmpty()) {
|
||||
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath);
|
||||
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(configuration, filePath);
|
||||
logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Keys from " + filePath + " => " + fileRowKeys);
|
||||
@@ -107,7 +113,8 @@ public class HoodieBloomIndexCheckFunction implements
|
||||
private void initState(String fileName, String partitionPath) throws HoodieIndexException {
|
||||
try {
|
||||
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
|
||||
bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(filePath);
|
||||
bloomFilter = ParquetUtils
|
||||
.readBloomFilterFromParquetMetadata(table.getHadoopConf(), filePath);
|
||||
candidateRecordKeys = new ArrayList<>();
|
||||
currentFile = fileName;
|
||||
currentParitionPath = partitionPath;
|
||||
@@ -154,7 +161,7 @@ public class HoodieBloomIndexCheckFunction implements
|
||||
.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
|
||||
}
|
||||
ret.add(new IndexLookupResult(currentFile,
|
||||
checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
|
||||
checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath)));
|
||||
|
||||
initState(fileName, partitionPath);
|
||||
if (bloomFilter.mightContain(recordKey)) {
|
||||
@@ -177,7 +184,7 @@ public class HoodieBloomIndexCheckFunction implements
|
||||
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
|
||||
}
|
||||
ret.add(new IndexLookupResult(currentFile,
|
||||
checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
|
||||
checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath)));
|
||||
}
|
||||
|
||||
} catch (Throwable e) {
|
||||
|
||||
@@ -33,7 +33,6 @@ import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -51,14 +50,12 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTable<T> hoodieTable;
|
||||
private HoodieWriteConfig config;
|
||||
private FileSystem fs;
|
||||
|
||||
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileSystemView = hoodieTable.getCompletedFileSystemView();
|
||||
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.config = config;
|
||||
this.fs = hoodieTable.getFs();
|
||||
}
|
||||
|
||||
|
||||
|
||||
@@ -126,7 +126,8 @@ public class HoodieCommitArchiveLog {
|
||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
// GroupBy each action and limit each action timeline to maxCommitsToKeep
|
||||
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
|
||||
@@ -165,7 +166,7 @@ public class HoodieCommitArchiveLog {
|
||||
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
|
||||
log.info("Deleting instants " + archivedInstants);
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true);
|
||||
|
||||
boolean success = true;
|
||||
for (HoodieInstant archivedInstant : archivedInstants) {
|
||||
@@ -188,7 +189,7 @@ public class HoodieCommitArchiveLog {
|
||||
|
||||
try {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline =
|
||||
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
|
||||
@@ -47,7 +47,7 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
HoodieTable<T> hoodieTable) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.fs = FSUtils.getFs();
|
||||
this.fs = hoodieTable.getMetaClient().getFs();
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.fileSystemView = hoodieTable.getROFileSystemView();
|
||||
@@ -73,8 +73,9 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
String partitionPath,
|
||||
int taskPartitionId) {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
int taskPartitionId,
|
||||
HoodieTable hoodieTable) {
|
||||
FileSystem fs = hoodieTable.getMetaClient().getFs();
|
||||
try {
|
||||
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
|
||||
@@ -16,6 +16,8 @@
|
||||
|
||||
package com.uber.hoodie.io.compact;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
@@ -34,13 +36,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCompactionException;
|
||||
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Collection;
|
||||
@@ -49,8 +44,12 @@ import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
|
||||
/**
|
||||
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
|
||||
@@ -73,7 +72,6 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
.getTableType().name());
|
||||
|
||||
// TODO - rollback any compactions in flight
|
||||
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
|
||||
List<String> partitionPaths =
|
||||
@@ -102,15 +100,9 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
log.info("After filtering, Compacting " + operations + " files");
|
||||
List<HoodieWriteStat> updateStatusMap =
|
||||
jsc.parallelize(operations, operations.size())
|
||||
.map(s -> executeCompaction(metaClient, config, s, compactionCommitTime))
|
||||
.flatMap(new FlatMapFunction<List<HoodieWriteStat>, HoodieWriteStat>() {
|
||||
@Override
|
||||
public Iterator<HoodieWriteStat> call(
|
||||
List<HoodieWriteStat> hoodieWriteStats)
|
||||
throws Exception {
|
||||
return hoodieWriteStats.iterator();
|
||||
}
|
||||
}).collect();
|
||||
.map(s -> executeCompaction(hoodieTable, config, s, compactionCommitTime))
|
||||
.flatMap(writeStatList -> writeStatList.iterator())
|
||||
.collect();
|
||||
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
|
||||
for (HoodieWriteStat stat : updateStatusMap) {
|
||||
@@ -134,10 +126,11 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
return true;
|
||||
}
|
||||
|
||||
private List<HoodieWriteStat> executeCompaction(HoodieTableMetaClient metaClient,
|
||||
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
|
||||
|
||||
private List<HoodieWriteStat> executeCompaction(HoodieTable hoodieTable,
|
||||
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
|
||||
throws IOException {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
FileSystem fs = hoodieTable.getMetaClient().getFs();
|
||||
Schema readerSchema =
|
||||
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
|
||||
@@ -147,7 +140,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file (failure recover).
|
||||
// Load all the delta commits since the last compaction commit and get all the blocks to be loaded and load it using CompositeAvroLogReader
|
||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
String maxInstantTime = metaClient.getActiveTimeline()
|
||||
.getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
|
||||
@@ -162,8 +155,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
}
|
||||
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
HoodieCopyOnWriteTable table =
|
||||
new HoodieCopyOnWriteTable(config, metaClient);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient);
|
||||
Iterator<List<WriteStatus>> result = table
|
||||
.handleUpdate(commitTime, operation.getFileId(), scanner.iterator());
|
||||
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
||||
|
||||
@@ -19,6 +19,7 @@ package com.uber.hoodie.io.storage;
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
@@ -26,7 +27,6 @@ import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.hadoop.ParquetFileWriter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
@@ -51,9 +51,9 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
private final Schema schema;
|
||||
|
||||
|
||||
private static Configuration registerFileSystem(Configuration conf) {
|
||||
private static Configuration registerFileSystem(Path file, Configuration conf) {
|
||||
Configuration returnConf = new Configuration(conf);
|
||||
String scheme = FileSystem.getDefaultUri(conf).getScheme();
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
|
||||
HoodieWrapperFileSystem.class.getName());
|
||||
return returnConf;
|
||||
@@ -67,11 +67,11 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
|
||||
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION,
|
||||
registerFileSystem(parquetConfig.getHadoopConf()));
|
||||
registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
this.file =
|
||||
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file
|
||||
.getFileSystem(registerFileSystem(parquetConfig.getHadoopConf()));
|
||||
.getFileSystem(registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
|
||||
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
|
||||
|
||||
@@ -19,7 +19,6 @@ package com.uber.hoodie.io.storage;
|
||||
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.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
@@ -37,11 +36,12 @@ public class HoodieStorageWriterFactory {
|
||||
throws IOException {
|
||||
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
|
||||
// Currently only parquet is supported
|
||||
return newParquetStorageWriter(commitTime, path, config, schema);
|
||||
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) throws IOException {
|
||||
String commitTime, Path path, HoodieWriteConfig config, Schema schema,
|
||||
HoodieTable hoodieTable) throws IOException {
|
||||
BloomFilter filter =
|
||||
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
|
||||
HoodieAvroWriteSupport writeSupport =
|
||||
@@ -50,7 +50,7 @@ public class HoodieStorageWriterFactory {
|
||||
HoodieParquetConfig parquetConfig =
|
||||
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(),
|
||||
config.getParquetMaxFileSize(), FSUtils.getFs().getConf());
|
||||
config.getParquetMaxFileSize(), hoodieTable.getHadoopConf());
|
||||
|
||||
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
|
||||
}
|
||||
|
||||
@@ -16,7 +16,8 @@
|
||||
|
||||
package com.uber.hoodie.io.storage;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
@@ -33,7 +34,6 @@ import org.apache.hadoop.fs.ContentSummary;
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
||||
import org.apache.hadoop.fs.FileChecksum;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -41,17 +41,14 @@ import org.apache.hadoop.fs.FsServerDefaults;
|
||||
import org.apache.hadoop.fs.FsStatus;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Options;
|
||||
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
||||
import org.apache.hadoop.fs.XAttrSetFlag;
|
||||
import org.apache.hadoop.fs.permission.AclEntry;
|
||||
import org.apache.hadoop.fs.permission.AclStatus;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
@@ -70,6 +67,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
SUPPORT_SCHEMES.add("file");
|
||||
SUPPORT_SCHEMES.add("hdfs");
|
||||
SUPPORT_SCHEMES.add("s3");
|
||||
SUPPORT_SCHEMES.add("s3a");
|
||||
|
||||
|
||||
// Hoodie currently relies on underlying object store being fully
|
||||
// consistent so only regional buckets should be used.
|
||||
@@ -85,7 +84,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
@Override
|
||||
public void initialize(URI uri, Configuration conf) throws IOException {
|
||||
// Get the default filesystem to decorate
|
||||
fileSystem = FileSystem.get(conf);
|
||||
Path path = new Path(uri);
|
||||
// Remove 'hoodie-' prefix from path
|
||||
if (path.toString().startsWith(HOODIE_SCHEME_PREFIX)) {
|
||||
path = new Path(path.toString().replace(HOODIE_SCHEME_PREFIX, ""));
|
||||
}
|
||||
this.fileSystem = FSUtils.getFs(path.toString(), conf);
|
||||
// Do not need to explicitly initialize the default filesystem, its done already in the above FileSystem.get
|
||||
// fileSystem.initialize(FileSystem.getDefaultUri(conf), conf);
|
||||
// fileSystem.setConf(conf);
|
||||
@@ -219,7 +223,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException {
|
||||
public FileStatus[] listStatus(Path f) throws IOException {
|
||||
return fileSystem.listStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@@ -415,19 +419,19 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path f, PathFilter filter)
|
||||
throws FileNotFoundException, IOException {
|
||||
throws IOException {
|
||||
return fileSystem.listStatus(convertToDefaultPath(f), filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path[] files)
|
||||
throws FileNotFoundException, IOException {
|
||||
throws IOException {
|
||||
return fileSystem.listStatus(convertDefaults(files));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path[] files, PathFilter filter)
|
||||
throws FileNotFoundException, IOException {
|
||||
throws IOException {
|
||||
return fileSystem.listStatus(convertDefaults(files), filter);
|
||||
}
|
||||
|
||||
@@ -444,13 +448,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
|
||||
throws FileNotFoundException, IOException {
|
||||
throws IOException {
|
||||
return fileSystem.listLocatedStatus(convertToDefaultPath(f));
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive)
|
||||
throws FileNotFoundException, IOException {
|
||||
throws IOException {
|
||||
return fileSystem.listFiles(convertToDefaultPath(f), recursive);
|
||||
}
|
||||
|
||||
@@ -571,21 +575,21 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public void access(Path path, FsAction mode)
|
||||
throws AccessControlException, FileNotFoundException, IOException {
|
||||
throws IOException {
|
||||
fileSystem.access(convertToDefaultPath(path), mode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createSymlink(Path target, Path link, boolean createParent)
|
||||
throws AccessControlException, FileAlreadyExistsException, FileNotFoundException,
|
||||
ParentNotDirectoryException, UnsupportedFileSystemException, IOException {
|
||||
throws
|
||||
IOException {
|
||||
fileSystem
|
||||
.createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus getFileLinkStatus(Path f)
|
||||
throws AccessControlException, FileNotFoundException, UnsupportedFileSystemException,
|
||||
throws
|
||||
IOException {
|
||||
return fileSystem.getFileLinkStatus(convertToDefaultPath(f));
|
||||
}
|
||||
@@ -759,8 +763,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
}
|
||||
|
||||
public static Path convertToHoodiePath(Path file, Configuration conf) {
|
||||
String scheme = FileSystem.getDefaultUri(conf).getScheme();
|
||||
return convertPathWithScheme(file, getHoodieScheme(scheme));
|
||||
try {
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
return convertPathWithScheme(file, getHoodieScheme(scheme));
|
||||
} catch (HoodieIOException e) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private Path convertToDefaultPath(Path oldPath) {
|
||||
|
||||
@@ -52,7 +52,6 @@ import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -413,10 +412,10 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
throw new HoodieUpsertException("Error in finding the old file path at commit " +
|
||||
commitTime + " at fileLoc: " + fileLoc);
|
||||
} else {
|
||||
Configuration conf = FSUtils.getFs().getConf();
|
||||
AvroReadSupport.setAvroReadSchema(conf, upsertHandle.getSchema());
|
||||
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
|
||||
ParquetReader<IndexedRecord> reader =
|
||||
AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(conf).build();
|
||||
AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(getHadoopConf())
|
||||
.build();
|
||||
try {
|
||||
IndexedRecord record;
|
||||
while ((record = reader.read()) != null) {
|
||||
@@ -500,8 +499,9 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
@Override
|
||||
public List<HoodieCleanStat> clean(JavaSparkContext jsc) {
|
||||
try {
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
List<String> partitionsToClean =
|
||||
FSUtils.getAllPartitionPaths(getFs(), getMetaClient().getBasePath(),
|
||||
FSUtils.getAllPartitionPaths(fs, getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config
|
||||
.getCleanerPolicy());
|
||||
@@ -522,7 +522,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits)
|
||||
throws IOException {
|
||||
logger.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
FileStatus[] toBeDeleted =
|
||||
fs.listStatus(new Path(config.getBasePath(), partitionPath), path -> {
|
||||
if (!path.toString().contains(".parquet")) {
|
||||
@@ -558,7 +558,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
// delete all the data files for all these commits
|
||||
logger.info("Clean out all parquet files generated for commits: " + commits);
|
||||
List<HoodieRollbackStat> stats = jsc.parallelize(
|
||||
FSUtils.getAllPartitionPaths(FSUtils.getFs(), this.getMetaClient().getBasePath(),
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
@@ -618,7 +618,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
.parallelize(partitionsToClean, cleanerParallelism)
|
||||
.flatMapToPair(getFilesToDeleteFunc(this, config))
|
||||
.repartition(cleanerParallelism) // repartition to remove skews
|
||||
.mapPartitionsToPair(deleteFilesFunc(this, config))
|
||||
.mapPartitionsToPair(deleteFilesFunc(this))
|
||||
.reduceByKey(
|
||||
// merge partition level clean stats below
|
||||
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1
|
||||
@@ -646,17 +646,17 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat> deleteFilesFunc(
|
||||
HoodieTable table, HoodieWriteConfig config) {
|
||||
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat> deleteFilesFunc(
|
||||
HoodieTable table) {
|
||||
return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>) iter -> {
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config);
|
||||
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
|
||||
|
||||
FileSystem fs = table.getMetaClient().getFs();
|
||||
while (iter.hasNext()) {
|
||||
Tuple2<String, String> partitionDelFileTuple = iter.next();
|
||||
String partitionPath = partitionDelFileTuple._1();
|
||||
String deletePathStr = partitionDelFileTuple._2();
|
||||
Boolean deletedFileResult = deleteFileAndGetResult(deletePathStr);
|
||||
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
|
||||
if (!partitionCleanStatMap.containsKey(partitionPath)) {
|
||||
partitionCleanStatMap.put(partitionPath,
|
||||
new PartitionCleanStat(partitionPath));
|
||||
@@ -682,10 +682,11 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
};
|
||||
}
|
||||
|
||||
private Boolean deleteFileAndGetResult(String deletePathStr) throws IOException {
|
||||
private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr)
|
||||
throws IOException {
|
||||
Path deletePath = new Path(deletePathStr);
|
||||
logger.debug("Working on delete path :" + deletePath);
|
||||
boolean deleteResult = getFs().delete(deletePath, false);
|
||||
boolean deleteResult = fs.delete(deletePath, false);
|
||||
if (deleteResult) {
|
||||
logger.debug("Cleaned file at path :" + deletePath);
|
||||
}
|
||||
|
||||
@@ -32,19 +32,11 @@ import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCompactionException;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.io.HoodieAppendHandle;
|
||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.Arrays;
|
||||
@@ -55,6 +47,12 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
|
||||
|
||||
/**
|
||||
@@ -74,8 +72,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
||||
|
||||
public HoodieMergeOnReadTable(HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
super(config, metaClient);
|
||||
}
|
||||
|
||||
@@ -195,7 +192,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
.onParentPath(
|
||||
new Path(this.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
|
||||
.withFs(FSUtils.getFs())
|
||||
.withFs(getMetaClient().getFs())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
Long numRollbackBlocks = 0L;
|
||||
// generate metadata
|
||||
@@ -215,7 +212,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
numRollbackBlocks++;
|
||||
}
|
||||
filesToNumBlocksRollback
|
||||
.put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()),
|
||||
.put(getMetaClient().getFs()
|
||||
.getFileStatus(writer.getLogFile().getPath()),
|
||||
numRollbackBlocks);
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException(
|
||||
|
||||
@@ -34,12 +34,6 @@ 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 org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Iterator;
|
||||
@@ -47,6 +41,11 @@ import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Abstract implementation of a HoodieTable
|
||||
@@ -55,6 +54,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieTable.class);
|
||||
|
||||
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
@@ -87,8 +87,8 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
return metaClient;
|
||||
}
|
||||
|
||||
public FileSystem getFs() {
|
||||
return metaClient.getFs();
|
||||
public Configuration getHadoopConf() {
|
||||
return metaClient.getHadoopConf();
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -81,7 +81,7 @@ public class HoodieClientExample {
|
||||
|
||||
// initialize the table, if not done already
|
||||
Path path = new Path(tablePath);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
FileSystem fs = FSUtils.getFs(tablePath, jsc.hadoopConfiguration());
|
||||
if (!fs.exists(path)) {
|
||||
HoodieTableMetaClient
|
||||
.initTableType(fs, tablePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
|
||||
@@ -84,6 +84,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
|
||||
private transient JavaSparkContext jsc = null;
|
||||
private transient SQLContext sqlContext;
|
||||
private transient FileSystem fs;
|
||||
private String basePath = null;
|
||||
private transient HoodieTestDataGenerator dataGen = null;
|
||||
private String[] partitionPaths = {"2016/01/01", "2016/02/02", "2016/06/02"};
|
||||
@@ -100,6 +101,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
fs = FSUtils.getFs(basePath.toString(), jsc.hadoopConfiguration());
|
||||
HoodieTestUtils.init(basePath);
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
@@ -212,7 +214,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
HoodieWriteConfig cfg = getConfig();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
@@ -230,8 +231,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath())
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
|
||||
.getCommitTimeline();
|
||||
|
||||
assertEquals("Expecting a single commit.", 1,
|
||||
@@ -242,7 +244,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
records.size(),
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(metaClient, getConfig());
|
||||
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table)
|
||||
.collect();
|
||||
@@ -268,13 +271,13 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify there are now 2 commits
|
||||
timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline();
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting two commits.",
|
||||
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2);
|
||||
assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(),
|
||||
newCommitTime);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
|
||||
// Index should be able to locate all updates in correct locations.
|
||||
@@ -305,7 +308,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
HoodieWriteConfig cfg = getConfig();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
/**
|
||||
* Write 1 (inserts and deletes)
|
||||
@@ -326,8 +328,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath())
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
|
||||
.getCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1,
|
||||
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
@@ -336,7 +339,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertEquals("Must contain 200 records", fewRecordsForInsert.size(),
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(metaClient, getConfig());
|
||||
|
||||
List<HoodieRecord> taggedRecords = index
|
||||
.tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect();
|
||||
@@ -359,7 +363,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify there are now 2 commits
|
||||
timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline();
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals("Expecting two commits.",
|
||||
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2);
|
||||
assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(),
|
||||
@@ -393,7 +397,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
|
||||
.build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTestDataGenerator
|
||||
.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
|
||||
|
||||
@@ -451,8 +454,10 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
|
||||
List<String> partitionPaths = FSUtils
|
||||
.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(metaClient, getConfig());
|
||||
final TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView();
|
||||
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
|
||||
@@ -472,7 +477,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView();
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
@@ -490,7 +495,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
|
||||
.build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTestDataGenerator
|
||||
.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
|
||||
|
||||
@@ -531,7 +535,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
List<String> partitionPaths = FSUtils
|
||||
.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView();
|
||||
|
||||
@@ -551,7 +556,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView();
|
||||
|
||||
@@ -573,7 +578,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
table.getCompletedSavepointTimeline().getInstants().findFirst().get();
|
||||
client.rollbackToSavepoint(savepoint.getTimestamp());
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView();
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
@@ -602,7 +607,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
.retainFileVersions(maxVersions).build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
/**
|
||||
* do a big insert
|
||||
@@ -618,8 +622,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath())
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
|
||||
.getCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1,
|
||||
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
@@ -647,7 +652,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
table = HoodieTable.getHoodieTable(metadata, getConfig());
|
||||
timeline = table.getCommitsTimeline();
|
||||
|
||||
@@ -702,7 +708,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
.retainCommits(maxCommits).build()).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
/**
|
||||
* do a big insert
|
||||
@@ -718,13 +723,15 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath())
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
|
||||
.getCommitTimeline();
|
||||
assertEquals("Expecting a single commit.", 1,
|
||||
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(metaClient, getConfig());
|
||||
|
||||
assertFalse(table.getCompletedCommitTimeline().empty());
|
||||
String commitTime =
|
||||
@@ -747,7 +754,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg);
|
||||
HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline();
|
||||
Optional<HoodieInstant>
|
||||
@@ -788,8 +796,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
String commitTime2 = "20160502020601";
|
||||
String commitTime3 = "20160506030611";
|
||||
new File(basePath + "/.hoodie").mkdirs();
|
||||
HoodieTestDataGenerator.writePartitionMetadata(FSUtils.getFs(),
|
||||
new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
|
||||
HoodieTestDataGenerator
|
||||
.writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
|
||||
basePath);
|
||||
|
||||
// Only first two have commit files
|
||||
@@ -878,9 +886,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
String commitTime2 = "20160502020601";
|
||||
String commitTime3 = "20160506030611";
|
||||
new File(basePath + "/.hoodie").mkdirs();
|
||||
HoodieTestDataGenerator.writePartitionMetadata(FSUtils.getFs(),
|
||||
new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
|
||||
basePath);
|
||||
HoodieTestDataGenerator
|
||||
.writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
|
||||
basePath);
|
||||
|
||||
// One good commit
|
||||
HoodieTestUtils.createCommitFiles(basePath, commitTime1);
|
||||
@@ -956,8 +964,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
|
||||
@Test
|
||||
public void testSmallInsertHandlingForUpserts() throws Exception {
|
||||
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
final String TEST_PARTITION_PATH = "2016/09/26";
|
||||
final int INSERT_SPLIT_LIMIT = 100;
|
||||
// setup the small file handling params
|
||||
@@ -982,7 +988,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
|
||||
String file1 = statuses.get(0).getFileId();
|
||||
assertEquals("file should contain 100 records",
|
||||
ParquetUtils.readRowKeysFromParquet(new Path(basePath,
|
||||
ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), new Path(basePath,
|
||||
TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(),
|
||||
100);
|
||||
|
||||
@@ -1006,9 +1012,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
Path newFile = new Path(basePath,
|
||||
TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1));
|
||||
assertEquals("file should contain 140 records",
|
||||
ParquetUtils.readRowKeysFromParquet(newFile).size(), 140);
|
||||
ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
|
||||
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(newFile);
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile);
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertEquals("only expect commit2", commitTime2,
|
||||
@@ -1030,7 +1036,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("2 files needs to be committed.", 2, statuses.size());
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||
TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView();
|
||||
List<HoodieDataFile> files = fileSystemView
|
||||
@@ -1040,7 +1046,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
for (HoodieDataFile file : files) {
|
||||
if (file.getFileName().contains(file1)) {
|
||||
assertEquals("Existing file should be expanded", commitTime3, file.getCommitTime());
|
||||
records = ParquetUtils.readAvroRecords(new Path(file.getPath()));
|
||||
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String recordCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
@@ -1057,7 +1063,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
keys2.size());
|
||||
} else {
|
||||
assertEquals("New file must be written for commit 3", commitTime3, file.getCommitTime());
|
||||
records = ParquetUtils.readAvroRecords(new Path(file.getPath()));
|
||||
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertEquals("only expect commit3", commitTime3,
|
||||
@@ -1091,12 +1097,12 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
List<WriteStatus> statuses = client.insert(insertRecordsRDD1, commitTime1).collect();
|
||||
|
||||
assertNoWriteErrors(statuses);
|
||||
assertPartitionMetadata(new String[]{TEST_PARTITION_PATH}, FSUtils.getFs());
|
||||
assertPartitionMetadata(new String[]{TEST_PARTITION_PATH}, fs);
|
||||
|
||||
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
|
||||
String file1 = statuses.get(0).getFileId();
|
||||
assertEquals("file should contain 100 records",
|
||||
ParquetUtils.readRowKeysFromParquet(new Path(basePath,
|
||||
ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), new Path(basePath,
|
||||
TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(),
|
||||
100);
|
||||
|
||||
@@ -1116,9 +1122,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
Path newFile = new Path(basePath,
|
||||
TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1));
|
||||
assertEquals("file should contain 140 records",
|
||||
ParquetUtils.readRowKeysFromParquet(newFile).size(), 140);
|
||||
ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
|
||||
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(newFile);
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile);
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String recCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
@@ -1137,8 +1143,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
assertEquals("2 files needs to be committed.", 2, statuses.size());
|
||||
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
List<HoodieDataFile> files =
|
||||
table.getROFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3)
|
||||
@@ -1148,7 +1154,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
int totalInserts = 0;
|
||||
for (HoodieDataFile file : files) {
|
||||
assertEquals("All files must be at commit 3", commitTime3, file.getCommitTime());
|
||||
records = ParquetUtils.readAvroRecords(new Path(file.getPath()));
|
||||
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
|
||||
totalInserts += records.size();
|
||||
}
|
||||
assertEquals("Total number of records must add up", totalInserts,
|
||||
@@ -1169,7 +1175,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
|
||||
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
|
||||
.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
|
||||
@@ -1183,7 +1190,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
// make next commit, with 1 insert & 1 update per partition
|
||||
HoodieTestUtils.createCommitFiles(basePath, "001");
|
||||
table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
|
||||
.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true),
|
||||
config);
|
||||
|
||||
String file2P0C1 = HoodieTestUtils
|
||||
@@ -1206,7 +1214,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
// make next commit, with 2 updates to existing files, and 1 insert
|
||||
HoodieTestUtils.createCommitFiles(basePath, "002");
|
||||
table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
|
||||
.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
|
||||
@@ -1260,8 +1269,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
// make 1 compaction commit
|
||||
HoodieTestUtils.createCompactionCommitFiles(basePath, "001");
|
||||
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(metaClient, config);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
|
||||
assertEquals("Must clean three files, one parquet and 2 log files", 3,
|
||||
getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size());
|
||||
@@ -1286,9 +1294,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
|
||||
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
|
||||
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
|
||||
config);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
|
||||
assertEquals("Must not clean any files", 0,
|
||||
@@ -1300,9 +1307,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
|
||||
// make next commit, with 1 insert & 1 update per partition
|
||||
HoodieTestUtils.createCommitFiles(basePath, "001");
|
||||
table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
|
||||
config);
|
||||
table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
|
||||
|
||||
String file2P0C1 = HoodieTestUtils
|
||||
.createNewDataFile(basePath, partitionPaths[0], "001"); // insert
|
||||
@@ -1324,7 +1330,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
// make next commit, with 2 updates to existing files, and 1 insert
|
||||
HoodieTestUtils.createCommitFiles(basePath, "002");
|
||||
table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
|
||||
.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
|
||||
@@ -1341,7 +1348,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
// make next commit, with 2 updates to existing files, and 1 insert
|
||||
HoodieTestUtils.createCommitFiles(basePath, "003");
|
||||
table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
|
||||
.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update
|
||||
@@ -1384,7 +1392,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
HoodieTestUtils.createCommitFiles(basePath, "000");
|
||||
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
|
||||
.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
|
||||
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
|
||||
@@ -1449,7 +1458,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
updateAllFilesInPartition(filesP2C0, partitionPaths[2], "003");
|
||||
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
|
||||
.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
|
||||
config);
|
||||
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
|
||||
|
||||
@@ -1476,8 +1486,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
|
||||
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
|
||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
|
||||
String commitTime = "000";
|
||||
|
||||
159
hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java
Normal file
159
hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java
Normal file
@@ -0,0 +1,159 @@
|
||||
/*
|
||||
* Copyright (c) 2017 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;
|
||||
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.minicluster.HdfsTestService;
|
||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
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.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestMultiFS implements Serializable {
|
||||
|
||||
private static String dfsBasePath;
|
||||
private static HdfsTestService hdfsTestService;
|
||||
private static MiniDFSCluster dfsCluster;
|
||||
private static DistributedFileSystem dfs;
|
||||
private static Logger logger = LogManager.getLogger(TestMultiFS.class);
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
private String tableName = "hoodie_rt";
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
private static JavaSparkContext jsc;
|
||||
private static SQLContext sqlContext;
|
||||
|
||||
@Before
|
||||
public void initClass() throws Exception {
|
||||
hdfsTestService = new HdfsTestService();
|
||||
dfsCluster = hdfsTestService.start(true);
|
||||
|
||||
// Create a temp folder as the base path
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
dfsBasePath = dfs.getWorkingDirectory().toString();
|
||||
dfs.mkdirs(new Path(dfsBasePath));
|
||||
|
||||
SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example");
|
||||
sparkConf.setMaster("local[1]");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.kryoserializer.buffer.max", "512m");
|
||||
jsc = new JavaSparkContext(sparkConf);
|
||||
sqlContext = new SQLContext(jsc);
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanupClass() throws Exception {
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
}
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
FSUtils.setFs(null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void readLocalWriteHDFS() throws Exception {
|
||||
|
||||
// Generator of some records to be loaded in.
|
||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||
|
||||
// Initialize table and filesystem
|
||||
FileSystem hdfs = FSUtils.getFs(dfsBasePath, jsc.hadoopConfiguration());
|
||||
HoodieTableMetaClient
|
||||
.initTableType(hdfs, dfsBasePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
HoodieAvroPayload.class.getName());
|
||||
|
||||
//Create write client to write some records in
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(dfsBasePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable(tableName).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.build();
|
||||
HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc, cfg);
|
||||
|
||||
// Write generated data to hdfs (only inserts)
|
||||
String readCommitTime = hdfsWriteClient.startCommit();
|
||||
logger.info("Starting commit " + readCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(readCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
hdfsWriteClient.upsert(writeRecords, readCommitTime);
|
||||
|
||||
// Read from hdfs
|
||||
FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
|
||||
.getCommitTimeline();
|
||||
Dataset<Row> readRecords = HoodieClientTestUtils
|
||||
.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime);
|
||||
assertEquals("Should contain 100 records", readRecords.count(), records.size());
|
||||
|
||||
// Write to local
|
||||
FileSystem local = FSUtils.getFs(tablePath, jsc.hadoopConfiguration());
|
||||
HoodieTableMetaClient
|
||||
.initTableType(local, tablePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
HoodieAvroPayload.class.getName());
|
||||
HoodieWriteConfig localConfig = HoodieWriteConfig.newBuilder().withPath(tablePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable(tableName).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.build();
|
||||
HoodieWriteClient localWriteClient = new HoodieWriteClient(jsc, localConfig);
|
||||
|
||||
String writeCommitTime = localWriteClient.startCommit();
|
||||
logger.info("Starting write commit " + writeCommitTime);
|
||||
List<HoodieRecord> localRecords = dataGen.generateInserts(writeCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> localWriteRecords = jsc.parallelize(localRecords, 1);
|
||||
logger.info("Writing to path: " + tablePath);
|
||||
localWriteClient.upsert(localWriteRecords, writeCommitTime);
|
||||
|
||||
logger.info("Reading from path: " + tablePath);
|
||||
fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> localReadRecords = HoodieClientTestUtils
|
||||
.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
|
||||
assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size());
|
||||
}
|
||||
}
|
||||
@@ -174,7 +174,7 @@ public class HoodieClientTestUtils {
|
||||
List<String> filteredPaths = new ArrayList<>();
|
||||
try {
|
||||
HoodieTable hoodieTable = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null);
|
||||
for (String path : paths) {
|
||||
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(
|
||||
hoodieTable.getMetaClient(),
|
||||
|
||||
@@ -18,7 +18,7 @@ package com.uber.hoodie.common;
|
||||
|
||||
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
|
||||
import java.io.IOException;
|
||||
@@ -42,12 +42,13 @@ import org.apache.hadoop.mapred.RecordReader;
|
||||
*/
|
||||
public class HoodieMergeOnReadTestUtils {
|
||||
|
||||
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths)
|
||||
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths,
|
||||
String basePath)
|
||||
throws IOException {
|
||||
JobConf jobConf = new JobConf();
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA));
|
||||
HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
|
||||
setPropsForInputFormat(inputFormat, jobConf, schema);
|
||||
setPropsForInputFormat(inputFormat, jobConf, schema, basePath);
|
||||
return inputPaths.stream().map(path -> {
|
||||
setInputPath(jobConf, path);
|
||||
List<GenericRecord> records = new ArrayList<>();
|
||||
@@ -76,12 +77,12 @@ public class HoodieMergeOnReadTestUtils {
|
||||
}
|
||||
|
||||
private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf,
|
||||
Schema schema) {
|
||||
Schema schema, String basePath) {
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
|
||||
String postions = fields.stream().map(f -> String.valueOf(f.pos()))
|
||||
.collect(Collectors.joining(","));
|
||||
Configuration conf = FSUtils.getFs().getConf();
|
||||
Configuration conf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
|
||||
@@ -20,6 +20,7 @@ import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
@@ -193,7 +194,7 @@ public class HoodieTestDataGenerator {
|
||||
Path commitFile =
|
||||
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
|
||||
.makeCommitFileName(commitTime));
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
try {
|
||||
@@ -209,7 +210,7 @@ public class HoodieTestDataGenerator {
|
||||
Path commitFile =
|
||||
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
|
||||
.makeSavePointFileName(commitTime));
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
try {
|
||||
|
||||
@@ -55,8 +55,9 @@ public class TestUpdateMapFunction {
|
||||
public void testSchemaEvolutionOnUpdate() throws Exception {
|
||||
// Create a bunch of records with a old version of schema
|
||||
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(
|
||||
HoodieTestUtils.getDefaultHadoopConf(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient);
|
||||
|
||||
String recordStr1 =
|
||||
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
@@ -80,16 +81,16 @@ public class TestUpdateMapFunction {
|
||||
Iterator<List<WriteStatus>> insertResult = table.handleInsert("100", records.iterator());
|
||||
Path commitFile =
|
||||
new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100"));
|
||||
FSUtils.getFs().create(commitFile);
|
||||
FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()).create(commitFile);
|
||||
|
||||
// Now try an update with an evolved schema
|
||||
// Evolved schema does not have guarantee on preserving the original field ordering
|
||||
config = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
|
||||
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath);
|
||||
String fileId = insertResult.next().get(0).getFileId();
|
||||
System.out.println(fileId);
|
||||
|
||||
table = new HoodieCopyOnWriteTable(config, metadata);
|
||||
table = new HoodieCopyOnWriteTable(config, metaClient);
|
||||
// New content with values for the newly added field
|
||||
recordStr1 =
|
||||
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
|
||||
|
||||
@@ -54,7 +54,6 @@ import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
@@ -73,12 +72,11 @@ public class TestHoodieBloomIndex {
|
||||
|
||||
private JavaSparkContext jsc = null;
|
||||
private String basePath = null;
|
||||
private transient final FileSystem fs;
|
||||
private transient FileSystem fs;
|
||||
private String schemaStr;
|
||||
private Schema schema;
|
||||
|
||||
public TestHoodieBloomIndex() throws Exception {
|
||||
fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
@Before
|
||||
@@ -89,6 +87,7 @@ public class TestHoodieBloomIndex {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
HoodieTestUtils.init(basePath);
|
||||
// We have some records to be tagged (two different partitions)
|
||||
schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||
@@ -120,8 +119,6 @@ public class TestHoodieBloomIndex {
|
||||
.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
|
||||
Map<String, Iterable<String>> map = recordRDD
|
||||
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()))
|
||||
.groupByKey().collectAsMap();
|
||||
@@ -174,7 +171,7 @@ public class TestHoodieBloomIndex {
|
||||
Arrays.asList(record2, record3, record4), schema, null, false);
|
||||
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, table);
|
||||
// Still 0, as no valid commit
|
||||
@@ -291,7 +288,8 @@ public class TestHoodieBloomIndex {
|
||||
List<String> uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(),
|
||||
record3.getRecordKey(), record4.getRecordKey());
|
||||
|
||||
List<String> results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(uuids,
|
||||
List<String> results = HoodieBloomIndexCheckFunction
|
||||
.checkCandidatesAgainstFile(jsc.hadoopConfiguration(), uuids,
|
||||
new Path(basePath + "/2016/01/31/" + filename));
|
||||
assertEquals(results.size(), 2);
|
||||
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|
||||
@@ -308,7 +306,7 @@ public class TestHoodieBloomIndex {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
|
||||
// Also create the metadata and config
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||
|
||||
@@ -348,7 +346,7 @@ public class TestHoodieBloomIndex {
|
||||
.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
|
||||
// Also create the metadata and config
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||
|
||||
@@ -367,7 +365,7 @@ public class TestHoodieBloomIndex {
|
||||
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
|
||||
|
||||
// We do the tag again
|
||||
metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metadata, config);
|
||||
|
||||
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
|
||||
@@ -409,7 +407,7 @@ public class TestHoodieBloomIndex {
|
||||
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
|
||||
|
||||
// Also create the metadata and config
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||
|
||||
@@ -429,7 +427,7 @@ public class TestHoodieBloomIndex {
|
||||
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
|
||||
|
||||
// We do the tag again
|
||||
metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metadata, config);
|
||||
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
|
||||
|
||||
@@ -476,7 +474,7 @@ public class TestHoodieBloomIndex {
|
||||
|
||||
// We do the tag
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||
|
||||
@@ -515,7 +513,7 @@ public class TestHoodieBloomIndex {
|
||||
String commitTime = FSUtils.getCommitTime(filename);
|
||||
HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
|
||||
new Configuration());
|
||||
HoodieTestUtils.getDefaultHadoopConf());
|
||||
HoodieParquetWriter writer = new HoodieParquetWriter(
|
||||
commitTime,
|
||||
new Path(basePath + "/" + partitionPath + "/" + filename),
|
||||
|
||||
@@ -55,7 +55,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.init(basePath);
|
||||
fs = FSUtils.getFs();
|
||||
fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -82,7 +82,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
|
||||
HoodieTimeline timeline =
|
||||
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
@@ -112,8 +112,8 @@ public class TestHoodieCommitArchiveLog {
|
||||
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
|
||||
|
||||
//read the file
|
||||
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(),
|
||||
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")),
|
||||
HoodieLogFormat.Reader reader = HoodieLogFormat
|
||||
.newReader(fs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")),
|
||||
HoodieArchivedMetaEntry.getClassSchema(), false);
|
||||
|
||||
int archivedRecordsCount = 0;
|
||||
@@ -147,7 +147,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable("test-trip-table").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||
@@ -173,7 +173,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable("test-trip-table").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||
@@ -206,7 +206,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable("test-trip-table").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||
|
||||
@@ -16,6 +16,9 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
@@ -28,7 +31,6 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
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.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieStorageConfig;
|
||||
@@ -38,7 +40,10 @@ import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
||||
import com.uber.hoodie.io.compact.HoodieCompactor;
|
||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.After;
|
||||
@@ -46,14 +51,6 @@ import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestHoodieCompactor {
|
||||
|
||||
private transient JavaSparkContext jsc = null;
|
||||
@@ -104,18 +101,19 @@ public class TestHoodieCompactor {
|
||||
@Test(expected = IllegalArgumentException.class)
|
||||
public void testCompactionOnCopyOnWriteFail() throws Exception {
|
||||
HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||
|
||||
compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompactionEmpty() throws Exception {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(metaClient, config);
|
||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
||||
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
@@ -132,7 +130,6 @@ public class TestHoodieCompactor {
|
||||
|
||||
@Test
|
||||
public void testLogFileCountsAfterCompaction() throws Exception {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
// insert 100 records
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
||||
@@ -144,7 +141,8 @@ public class TestHoodieCompactor {
|
||||
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
// Update all the 100 records
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
|
||||
newCommitTime = "101";
|
||||
@@ -161,7 +159,7 @@ public class TestHoodieCompactor {
|
||||
updatedRecords);
|
||||
|
||||
// Verify that all data file has one log file
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles =
|
||||
@@ -174,14 +172,14 @@ public class TestHoodieCompactor {
|
||||
}
|
||||
|
||||
// Do a compaction
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
|
||||
HoodieCommitMetadata result =
|
||||
compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime());
|
||||
|
||||
// Verify that recently written compacted data file has no log file
|
||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
|
||||
|
||||
@@ -89,7 +89,8 @@ public class TestCopyOnWriteTable {
|
||||
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
basePath);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
|
||||
|
||||
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath);
|
||||
@@ -115,7 +116,7 @@ public class TestCopyOnWriteTable {
|
||||
// Prepare the AvroParquetIO
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
|
||||
String partitionPath = "/2016/01/31";
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
|
||||
@@ -154,7 +155,8 @@ public class TestCopyOnWriteTable {
|
||||
|
||||
// Read out the bloom filter and make sure filter can answer record exist or not
|
||||
Path parquetFilePath = new Path(parquetFile.getAbsolutePath());
|
||||
BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(parquetFilePath);
|
||||
BloomFilter filter = ParquetUtils
|
||||
.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath);
|
||||
for (HoodieRecord record : records) {
|
||||
assertTrue(filter.mightContain(record.getRecordKey()));
|
||||
}
|
||||
@@ -163,7 +165,8 @@ public class TestCopyOnWriteTable {
|
||||
+ FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile();
|
||||
|
||||
// Read the parquet file, check the record content
|
||||
List<GenericRecord> fileRecords = ParquetUtils.readAvroRecords(parquetFilePath);
|
||||
List<GenericRecord> fileRecords = ParquetUtils
|
||||
.readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath);
|
||||
GenericRecord newRecord;
|
||||
int index = 0;
|
||||
for (GenericRecord record : fileRecords) {
|
||||
@@ -188,7 +191,7 @@ public class TestCopyOnWriteTable {
|
||||
|
||||
Thread.sleep(1000);
|
||||
String newCommitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
table = new HoodieCopyOnWriteTable(config, metadata);
|
||||
Iterator<List<WriteStatus>> iter = table
|
||||
.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(),
|
||||
@@ -211,7 +214,7 @@ public class TestCopyOnWriteTable {
|
||||
// Check whether the record has been updated
|
||||
Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath());
|
||||
BloomFilter updatedFilter = ParquetUtils
|
||||
.readBloomFilterFromParquetMetadata(updatedParquetFilePath);
|
||||
.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), updatedParquetFilePath);
|
||||
for (HoodieRecord record : records) {
|
||||
// No change to the _row_key
|
||||
assertTrue(updatedFilter.mightContain(record.getRecordKey()));
|
||||
@@ -261,7 +264,7 @@ public class TestCopyOnWriteTable {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
|
||||
.withWriteStatusClass(MetadataMergeWriteStatus.class).build();
|
||||
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
|
||||
|
||||
@@ -298,8 +301,8 @@ public class TestCopyOnWriteTable {
|
||||
public void testInsertWithPartialFailures() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
|
||||
|
||||
// Write a few records, and get atleast one file
|
||||
@@ -340,7 +343,7 @@ public class TestCopyOnWriteTable {
|
||||
public void testInsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
|
||||
|
||||
// Case 1:
|
||||
@@ -389,7 +392,7 @@ public class TestCopyOnWriteTable {
|
||||
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024)
|
||||
.parquetPageSize(64 * 1024).build()).build();
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
@@ -437,7 +440,7 @@ public class TestCopyOnWriteTable {
|
||||
HoodieClientTestUtils.fakeCommitFile(basePath, "001");
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize);
|
||||
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
|
||||
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(
|
||||
|
||||
@@ -19,6 +19,11 @@
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
|
||||
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
@@ -45,6 +50,14 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.io.compact.HoodieCompactor;
|
||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -61,25 +74,11 @@ import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestMergeOnReadTable {
|
||||
|
||||
private transient JavaSparkContext jsc = null;
|
||||
private transient SQLContext sqlContext;
|
||||
private String basePath = null;
|
||||
private static String basePath = null;
|
||||
private HoodieCompactor compactor;
|
||||
private FileSystem fs;
|
||||
|
||||
@@ -94,12 +93,11 @@ public class TestMergeOnReadTable {
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
dfsCluster.shutdown();
|
||||
;
|
||||
}
|
||||
FSUtils.setFs(null);
|
||||
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM
|
||||
FileSystem.closeAll();
|
||||
HoodieTestUtils.resetFS();
|
||||
HoodieTestUtils.resetFS(basePath);
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
@@ -113,30 +111,28 @@ public class TestMergeOnReadTable {
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
}
|
||||
FSUtils.setFs(dfs);
|
||||
HoodieTestUtils.resetFS();
|
||||
HoodieTestUtils.resetFS(basePath);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
this.fs = FSUtils.getFs();
|
||||
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(
|
||||
HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeOnReadTable"));
|
||||
jsc.hadoopConfiguration().addResource(FSUtils.getFs().getConf());
|
||||
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
|
||||
jsc.hadoopConfiguration().addResource(fs.getConf());
|
||||
|
||||
dfs.mkdirs(new Path(basePath));
|
||||
FSUtils.setFs(dfs);
|
||||
HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ);
|
||||
|
||||
sqlContext = new SQLContext(jsc); // SQLContext stuff
|
||||
compactor = new HoodieRealtimeTableCompactor();
|
||||
|
||||
//SQLContext stuff
|
||||
sqlContext = new SQLContext(jsc);
|
||||
}
|
||||
|
||||
@After
|
||||
@@ -167,7 +163,8 @@ public class TestMergeOnReadTable {
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
|
||||
Optional<HoodieInstant> deltaCommit =
|
||||
@@ -209,7 +206,7 @@ public class TestMergeOnReadTable {
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp());
|
||||
@@ -229,8 +226,9 @@ public class TestMergeOnReadTable {
|
||||
assertTrue(dataFilesToRead.findAny().isPresent());
|
||||
|
||||
// verify that there is a commit
|
||||
table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, cfg.getBasePath(), true), getConfig(false));
|
||||
table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true),
|
||||
getConfig(false));
|
||||
HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants();
|
||||
assertEquals("Expecting a single commit.", 1,
|
||||
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
|
||||
@@ -284,7 +282,8 @@ public class TestMergeOnReadTable {
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
|
||||
Optional<HoodieInstant> deltaCommit =
|
||||
@@ -332,7 +331,7 @@ public class TestMergeOnReadTable {
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp());
|
||||
@@ -349,7 +348,7 @@ public class TestMergeOnReadTable {
|
||||
List<String> dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath())
|
||||
.collect(Collectors.toList());
|
||||
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils
|
||||
.getRecordsUsingInputFormat(dataFiles);
|
||||
.getRecordsUsingInputFormat(dataFiles, basePath);
|
||||
//Wrote 40 records and deleted 20 records, so remaining 40-20 = 20
|
||||
assertEquals("Must contain 20 records", 20, recordsRead.size());
|
||||
}
|
||||
@@ -377,9 +376,8 @@ public class TestMergeOnReadTable {
|
||||
//verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
cfg.getBasePath());
|
||||
Optional<HoodieInstant> commit =
|
||||
metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
||||
assertTrue(commit.isPresent());
|
||||
@@ -403,8 +401,9 @@ public class TestMergeOnReadTable {
|
||||
//rollback a COW commit when TableType is MOR
|
||||
client.rollback(newCommitTime);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable
|
||||
.getHoodieTable(metaClient, cfg);
|
||||
FileStatus[] allFiles = HoodieTestUtils
|
||||
.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient,
|
||||
@@ -440,7 +439,8 @@ public class TestMergeOnReadTable {
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
|
||||
Optional<HoodieInstant> deltaCommit =
|
||||
@@ -476,7 +476,7 @@ public class TestMergeOnReadTable {
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals("Latest Delta commit should be 002", "002", deltaCommit.get().getTimestamp());
|
||||
@@ -487,19 +487,19 @@ public class TestMergeOnReadTable {
|
||||
List<String> dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath())
|
||||
.collect(Collectors.toList());
|
||||
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils
|
||||
.getRecordsUsingInputFormat(dataFiles);
|
||||
.getRecordsUsingInputFormat(dataFiles, basePath);
|
||||
|
||||
assertEquals(recordsRead.size(), 200);
|
||||
|
||||
// Test delta commit rollback
|
||||
client.rollback(newCommitTime);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(),
|
||||
allFiles);
|
||||
dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
|
||||
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles);
|
||||
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
|
||||
|
||||
assertEquals(recordsRead.size(), 200);
|
||||
|
||||
@@ -516,13 +516,14 @@ public class TestMergeOnReadTable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieCompactor compactor = new HoodieRealtimeTableCompactor();
|
||||
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(true));
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(metaClient, getConfig(true));
|
||||
|
||||
compactor.compact(jsc, getConfig(true), table, HoodieActiveTimeline.createNewCommitTime());
|
||||
|
||||
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(),
|
||||
allFiles);
|
||||
@@ -541,7 +542,7 @@ public class TestMergeOnReadTable {
|
||||
client.rollback(compactedCommitTime);
|
||||
|
||||
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
|
||||
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(),
|
||||
allFiles);
|
||||
|
||||
Reference in New Issue
Block a user