diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java index 27f0ab516f10b..15cce03e23e9b 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodieCLI.java @@ -17,6 +17,7 @@ package com.uber.hoodie.cli; import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.util.FSUtils; import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -36,7 +37,7 @@ public enum CLIState { public static boolean initConf() { if (HoodieCLI.conf == null) { - HoodieCLI.conf = new Configuration(); + HoodieCLI.conf = FSUtils.prepareHadoopConf(new Configuration()); return true; } return false; diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java index e1b03b271406b..d4945c0120aec 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/ArchivedCommitsCommand.java @@ -54,12 +54,14 @@ public String showCommits( System.out .println("===============> Showing only " + limit + " archived commits <==============="); - FileStatus[] fsStatuses = FSUtils.getFs().globStatus( - new Path(HoodieCLI.tableMetadata.getBasePath() + "/.hoodie/.commits_.archive*")); + String basePath = HoodieCLI.tableMetadata.getBasePath(); + FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf) + .globStatus(new Path(basePath + "/.hoodie/.commits_.archive*")); List allCommits = new ArrayList<>(); for (FileStatus fs : fsStatuses) { //read the archived file - HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(), + HoodieLogFormat.Reader reader = HoodieLogFormat + .newReader(FSUtils.getFs(basePath, HoodieCLI.conf), new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema(), false); List readRecords = new ArrayList<>(); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java index 1b5a9602df995..38638866b647d 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CleansCommand.java @@ -77,7 +77,7 @@ public String showCleans() throws IOException { @CliCommand(value = "cleans refresh", help = "Refresh the commits") public String refreshCleans() throws IOException { HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); + new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java index d6446a2c4dd55..d7b6594d138c4 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java @@ -96,7 +96,7 @@ public String showCommits( @CliCommand(value = "commits refresh", help = "Refresh the commits") public String refreshCommits() throws IOException { HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); + new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } @@ -224,14 +224,13 @@ public boolean isCompareCommitsAvailable() { public String compareCommits( @CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) throws Exception { - HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.fs, path); + + HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.conf, path); HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants(); - ; HoodieTableMetaClient source = HoodieCLI.tableMetadata; HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants(); - ; String targetLatestCommit = targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp(); @@ -266,7 +265,7 @@ public boolean isSyncCommitsAvailable() { public String syncCommits( @CliOption(key = {"path"}, help = "Path of the dataset to compare to") final String path) throws Exception { - HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.fs, path); + HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.conf, path); HoodieCLI.state = HoodieCLI.CLIState.SYNC; return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName(); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java index fc1f22a3a28fc..de59d06694d86 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/DatasetsCommand.java @@ -33,7 +33,7 @@ public String connect( final String path) throws IOException { boolean initialized = HoodieCLI.initConf(); HoodieCLI.initFS(initialized); - HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.fs, path)); + HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.conf, path)); HoodieCLI.state = HoodieCLI.CLIState.DATASET; return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded"; diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java index 4f5b2c9a3bfef..db267d6a656e5 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SavepointsCommand.java @@ -138,7 +138,7 @@ public String rollbackToSavepoint( @CliCommand(value = "savepoints refresh", help = "Refresh the savepoints") public String refreshMetaClient() throws IOException { HoodieTableMetaClient metadata = - new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); + new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath()); HoodieCLI.setTableMetadata(metadata); return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java index aba2d9da8d452..4f8f2f856b306 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java @@ -97,8 +97,8 @@ private static int deduplicatePartitionPath(JavaSparkContext jsc, String repairedOutputPath, String basePath) throws Exception { - DedupeSparkJob job = new DedupeSparkJob(basePath, - duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc), FSUtils.getFs()); + DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath, + new SQLContext(jsc), FSUtils.getFs(basePath, jsc.hadoopConfiguration())); job.fixDuplicates(true); return 0; } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java index 4b4ab2a2d6353..d85626d358bf6 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/utils/SparkUtil.java @@ -18,6 +18,7 @@ import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.cli.commands.SparkMain; +import com.uber.hoodie.common.util.FSUtils; import java.io.File; import java.net.URISyntaxException; import org.apache.log4j.Logger; @@ -66,6 +67,7 @@ public static JavaSparkContext initJavaSparkConf(String name) { sparkConf = HoodieWriteClient.registerClasses(sparkConf); JavaSparkContext jsc = new JavaSparkContext(sparkConf); jsc.hadoopConfiguration().setBoolean("parquet.enable.summary-metadata", false); + FSUtils.prepareHadoopConf(jsc.hadoopConfiguration()); return jsc; } } diff --git a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala index 82c97e0a4b7d7..0058043ed6388 100644 --- a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala +++ b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/DedupeSparkJob.scala @@ -75,7 +75,7 @@ class DedupeSparkJob(basePath: String, val tmpTableName = s"htbl_${System.currentTimeMillis()}" val dedupeTblName = s"${tmpTableName}_dupeKeys" - val metadata = new HoodieTableMetaClient(fs, basePath) + val metadata = new HoodieTableMetaClient(fs.getConf, basePath) val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}")) val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) @@ -127,7 +127,7 @@ class DedupeSparkJob(basePath: String, def fixDuplicates(dryRun: Boolean = true) = { - val metadata = new HoodieTableMetaClient(fs, basePath) + val metadata = new HoodieTableMetaClient(fs.getConf, basePath) val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}")) val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) diff --git a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala index 3fc18c8e4d5f3..8e1eccfeedb46 100644 --- a/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala +++ b/hoodie-cli/src/main/scala/com/uber/hoodie/cli/SparkHelpers.scala @@ -38,7 +38,7 @@ import scala.collection.mutable._ object SparkHelpers { @throws[Exception] def skipKeysAndWriteNewFile(commitTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) { - val sourceRecords = ParquetUtils.readAvroRecords(sourceFile) + val sourceRecords = ParquetUtils.readAvroRecords(fs.getConf, sourceFile) val schema: Schema = sourceRecords.get(0).getSchema val filter: BloomFilter = new BloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble) val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter) @@ -47,6 +47,7 @@ object SparkHelpers { for (rec <- sourceRecords) { val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString if (!keysToSkip.contains(key)) { + writer.writeAvro(key, rec) } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java index 6c329c330bef9..eb9ad62e470ec 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -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 Dataset read(JavaRDD hoodieKeys, int parallelism) JavaPairRDD keyRowRDD = originalDF.javaRDD() .mapToPair(row -> { HoodieKey key = new HoodieKey( - row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD), - row.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); }); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 0de261c8f706a..6b0adf67fbb74 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -53,18 +53,6 @@ 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.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 @@ -92,7 +91,6 @@ public class HoodieWriteClient implements Seriali private final HoodieWriteConfig config; private transient final HoodieMetrics metrics; private transient final HoodieIndex index; - private transient final HoodieCommitArchiveLog archiveLog; private transient Timer.Context writeContext = null; /** @@ -112,12 +110,11 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) */ 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); this.metrics = new HoodieMetrics(config, config.getTableName()); - this.archiveLog = new HoodieCommitArchiveLog(clientConfig, fs); if (rollbackInFlight) { rollbackInflightCommits(); @@ -133,8 +130,9 @@ public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, */ public JavaRDD> filterExists(JavaRDD> hoodieRecords) { // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); JavaRDD> recordsWithLocation = index.tagLocation(hoodieRecords, table); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); @@ -146,8 +144,9 @@ public JavaRDD> filterExists(JavaRDD> hoodieReco public JavaRDD upsert(JavaRDD> records, final String commitTime) { writeContext = metrics.getCommitCtx(); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); try { // De-dupe/merge if needed @@ -180,8 +179,9 @@ public JavaRDD upsert(JavaRDD> records, final Strin public JavaRDD insert(JavaRDD> records, final String commitTime) { writeContext = metrics.getCommitCtx(); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); try { // De-dupe/merge if needed JavaRDD> dedupedRecords = @@ -237,8 +237,9 @@ public JavaRDD bulkInsert(JavaRDD> records, Option bulkInsertPartitioner) { writeContext = metrics.getCommitCtx(); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); try { // De-dupe/merge if needed @@ -306,8 +307,7 @@ private JavaRDD> combineOnCondition(boolean condition, * file instead of using HoodieCommitMetadata */ private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, - HoodieTable table, - String commitTime) throws HoodieCommitException { + HoodieTable table, String commitTime) throws HoodieCommitException { try { HoodieCommitMetadata metadata = new HoodieCommitMetadata(); profile.getPartitionPaths().stream().forEach(path -> { @@ -409,8 +409,9 @@ public boolean commit(String commitTime, logger.info("Commiting " + commitTime); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); @@ -443,6 +444,8 @@ public boolean commit(String commitTime, } // We cannot have unbounded commit files. Archive commits if we have to archive + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(config, + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true)); archiveLog.archiveIfRequired(); if (config.isAutoClean()) { // Call clean to cleanup if there is anything to cleanup after the commit, @@ -485,8 +488,9 @@ public boolean commit(String commitTime, * @return true if the savepoint was created successfully */ public boolean savepoint(String user, String comment) { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable 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 boolean savepoint(String user, String comment) { * @return true if the savepoint was created successfully */ public boolean savepoint(String commitTime, String user, String comment) { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); Optional cleanInstant = table.getCompletedCleanTimeline().lastInstant(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, @@ -575,8 +580,9 @@ public boolean savepoint(String commitTime, String user, String comment) { * @return true if the savepoint was deleted successfully */ public void deleteSavepoint(String savepointTime) { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieInstant savePoint = @@ -602,8 +608,9 @@ public void deleteSavepoint(String savepointTime) { * @return true if the savepoint was rollecback to successfully */ public boolean rollbackToSavepoint(String savepointTime) { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieTimeline commitTimeline = table.getCommitsTimeline(); @@ -653,8 +660,9 @@ private void rollback(List commits) { String startRollbackTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable 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 @@ private void clean(String startCleanTime) throws HoodieIOException { final Timer.Context context = metrics.getCleanCtx(); // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), + true), config); List cleanStats = table.clean(jsc); if (cleanStats.isEmpty()) { @@ -810,8 +819,9 @@ public String startCommit() { public void startCommitWithTime(String commitTime) { logger.info("Generate a new commit time " + commitTime); - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable 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 void startCommitWithTime(String commitTime) { */ private void compact(String compactionCommitTime) throws IOException { // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); Optional compactionMetadata = table.compact(jsc, compactionCommitTime); if (compactionMetadata.isPresent()) { logger.info("Compacted successfully on commit " + compactionCommitTime); @@ -876,8 +887,9 @@ private JavaRDD> deduplicateRecords(JavaRDD> rec * Cleanup all inflight commits */ private void rollbackInflightCommits() { - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable( + new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), + config); HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); List commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java index cc038f21ac981..8b49897e7e783 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/LazyInsertIterable.java @@ -72,7 +72,8 @@ protected List computeNext() { HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, commitTime, record.getPartitionPath(), - TaskContext.getPartitionId()); + TaskContext.getPartitionId(), + hoodieTable); partitionsCleaned.add(record.getPartitionPath()); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index 5a0d69002c552..bf452bc8f30f0 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -155,7 +155,8 @@ private JavaPairRDD lookupIndex( // 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 @@ List> loadInvolvedFiles(List partitio .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 @@ JavaPairRDD> explodeRecordRDDWithFileCompariso * Make sure the parallelism is atleast the groupby parallelism for tagging location */ @VisibleForTesting - JavaPairRDD findMatchingFilesForRecordKeys( + JavaPairRDD findMatchingFilesForRecordKeys(HoodieTable hoodieTable, final Map> partitionToFileIndexInfo, JavaPairRDD partitionRecordKeyPairRDD, int totalSubpartitions) { @@ -372,7 +374,8 @@ JavaPairRDD findMatchingFilesForRecordKeys( .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 -> { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java index 0d562ae8680de..cf5a12007de4e 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndexCheckFunction.java @@ -24,10 +24,12 @@ 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 checkCandidatesAgainstFile(List candidateRecordKeys, + public static List checkCandidatesAgainstFile(Configuration configuration, + List candidateRecordKeys, Path filePath) throws HoodieIndexException { List foundRecordKeys = new ArrayList<>(); try { // Load all rowKeys from the file, to double-confirm if (!candidateRecordKeys.isEmpty()) { - Set fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath); + Set 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 @@ protected void start() { 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 @@ protected List computeNext() { .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 @@ protected List computeNext() { 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) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java index 086b87f89a7d6..85028ba292fcb 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java @@ -33,7 +33,6 @@ 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> { private final HoodieTimeline commitTimeline; private HoodieTable hoodieTable; private HoodieWriteConfig config; - private FileSystem fs; public HoodieCleanHelper(HoodieTable hoodieTable, HoodieWriteConfig config) { this.hoodieTable = hoodieTable; this.fileSystemView = hoodieTable.getCompletedFileSystemView(); this.commitTimeline = hoodieTable.getCompletedCommitTimeline(); this.config = config; - this.fs = hoodieTable.getFs(); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index 3cb697c7a1078..bb295b64d2b51 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -40,7 +40,6 @@ import com.uber.hoodie.table.HoodieTable; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -60,15 +59,14 @@ public class HoodieCommitArchiveLog { private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class); private final Path archiveFilePath; - private final FileSystem fs; + private final HoodieTableMetaClient metaClient; private final HoodieWriteConfig config; private HoodieLogFormat.Writer writer; - public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) { - this.fs = fs; + public HoodieCommitArchiveLog(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { this.config = config; - this.archiveFilePath = HoodieArchivedTimeline - .getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME); + this.metaClient = metaClient; + this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); } private HoodieLogFormat.Writer openWriter() { @@ -78,7 +76,7 @@ private HoodieLogFormat.Writer openWriter() { .onParentPath(archiveFilePath.getParent()) .withFileId(archiveFilePath.getName()) .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION) - .withFs(fs) + .withFs(metaClient.getFs()) .overBaseCommit("").build(); } else { return this.writer; @@ -125,8 +123,7 @@ private Stream getInstantsToArchive() { int maxCommitsToKeep = config.getMaxCommitsToKeep(); int minCommitsToKeep = config.getMinCommitsToKeep(); - HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); // GroupBy each action and limit each action timeline to maxCommitsToKeep HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline() @@ -164,16 +161,13 @@ private Stream getInstantsToArchive() { private boolean deleteArchivedInstants(List archivedInstants) { log.info("Deleting instants " + archivedInstants); - HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(fs, config.getBasePath(), true); - boolean success = true; for (HoodieInstant archivedInstant : archivedInstants) { Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName()); try { - if (fs.exists(commitFile)) { - success &= fs.delete(commitFile, false); + if (metaClient.getFs().exists(commitFile)) { + success &= metaClient.getFs().delete(commitFile, false); log.info("Archived and deleted instant file " + commitFile); } } catch (IOException e) { @@ -185,13 +179,9 @@ private boolean deleteArchivedInstants(List archivedInstants) { } public void archive(List instants) throws HoodieCommitException { - try { - HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(fs, config.getBasePath(), true); HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants(); - Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); log.info("Wrapper schema " + wrapperSchema.toString()); List records = new ArrayList<>(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java index 53ec545fc059b..332c9a2d3ba1f 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java @@ -47,7 +47,7 @@ public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable 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 Path makeNewPath(String partitionPath, int taskPartitionId, String fileNa 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, diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 92107c0ccfc9a..fda6b5a268f1d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -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.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.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 HoodieCommitMetadata compact(JavaSparkContext jsc, HoodieWriteConfig conf .getTableType().name()); // TODO - rollback any compactions in flight - HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); List partitionPaths = @@ -102,15 +100,9 @@ public HoodieCommitMetadata compact(JavaSparkContext jsc, HoodieWriteConfig conf log.info("After filtering, Compacting " + operations + " files"); List updateStatusMap = jsc.parallelize(operations, operations.size()) - .map(s -> executeCompaction(metaClient, config, s, compactionCommitTime)) - .flatMap(new FlatMapFunction, HoodieWriteStat>() { - @Override - public Iterator call( - List 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 @@ private boolean isCompactionSucceeded(HoodieCommitMetadata result) { return true; } - private List executeCompaction(HoodieTableMetaClient metaClient, - HoodieWriteConfig config, CompactionOperation operation, String commitTime) + + private List 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 @@ private List executeCompaction(HoodieTableMetaClient metaClient // 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 @@ private List executeCompaction(HoodieTableMetaClient metaClient } // Compacting is very similar to applying updates to existing file - HoodieCopyOnWriteTable table = - new HoodieCopyOnWriteTable(config, metaClient); + HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient); Iterator> result = table .handleUpdate(commitTime, operation.getFileId(), scanner.iterator()); Iterable> resultIterable = () -> result; diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java index 2e207fd7f12fb..48bdfda59e6c7 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieParquetWriter.java @@ -19,6 +19,7 @@ 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.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 HoodieSto 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 HoodieStorageWriter 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 @@ private static HoodieSt 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); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java index 87a628447a434..a9d2a2298e910 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/storage/HoodieWrapperFileSystem.java @@ -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.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.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 boolean delete(Path f, boolean recursive) throws IOException { } @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 RemoteIterator listCorruptFileBlocks(Path path) throws IOException @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 FileStatus[] globStatus(Path pathPattern, PathFilter filter) @Override public RemoteIterator listLocatedStatus(Path f) - throws FileNotFoundException, IOException { + throws IOException { return fileSystem.listLocatedStatus(convertToDefaultPath(f)); } @Override public RemoteIterator listFiles(Path f, boolean recursive) - throws FileNotFoundException, IOException { + throws IOException { return fileSystem.listFiles(convertToDefaultPath(f), recursive); } @@ -571,21 +575,21 @@ public short getDefaultReplication(Path path) { @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 Path convertToHoodiePath(Path oldPath) { } 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) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index a509411d771cb..3e4b1dc86fcc5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -52,7 +52,6 @@ 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 Iterator> handleUpdate(String commitTime, String fileLo 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 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 Optional compact(JavaSparkContext jsc, String commi @Override public List clean(JavaSparkContext jsc) { try { + FileSystem fs = getMetaClient().getFs(); List 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 List clean(JavaSparkContext jsc) { protected Map deleteCleanedFiles(String partitionPath, List 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 List rollback(JavaSparkContext jsc, List comm // delete all the data files for all these commits logger.info("Clean out all parquet files generated for commits: " + commits); List stats = jsc.parallelize( - FSUtils.getAllPartitionPaths(FSUtils.getFs(), this.getMetaClient().getBasePath(), + FSUtils.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning())) .map((Function) partitionPath -> { // Scan all partitions files with this commit time @@ -618,7 +618,7 @@ private List cleanPartitionPaths(List partitionsToClean .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) (e1, e2) -> e1 @@ -646,17 +646,17 @@ private List cleanPartitionPaths(List partitionsToClean }).collect(Collectors.toList()); } - private PairFlatMapFunction>, String, PartitionCleanStat> deleteFilesFunc( - HoodieTable table, HoodieWriteConfig config) { + private static PairFlatMapFunction>, String, PartitionCleanStat> deleteFilesFunc( + HoodieTable table) { return (PairFlatMapFunction>, String, PartitionCleanStat>) iter -> { - HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config); Map partitionCleanStatMap = new HashMap<>(); + FileSystem fs = table.getMetaClient().getFs(); while (iter.hasNext()) { Tuple2 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 @@ private static PairFlatMapFunction getFilesToDeleteFunc( }; } - 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); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 19461b025368d..839aa1840ebeb 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -32,19 +32,11 @@ 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.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 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 List rollback(JavaSparkContext jsc, List comm .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 List rollback(JavaSparkContext jsc, List comm numRollbackBlocks++; } filesToNumBlocksRollback - .put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()), + .put(getMetaClient().getFs() + .getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks); } catch (IOException | InterruptedException io) { throw new HoodieRollbackException( diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java index 76ab92d99ca21..3140e34eec286 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java @@ -34,12 +34,6 @@ 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.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 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 HoodieTableMetaClient getMetaClient() { return metaClient; } - public FileSystem getFs() { - return metaClient.getFs(); + public Configuration getHadoopConf() { + return metaClient.getHadoopConf(); } /** diff --git a/hoodie-client/src/test/java/HoodieClientExample.java b/hoodie-client/src/test/java/HoodieClientExample.java index ef31fea1cf084..14c1e6eac239f 100644 --- a/hoodie-client/src/test/java/HoodieClientExample.java +++ b/hoodie-client/src/test/java/HoodieClientExample.java @@ -24,9 +24,10 @@ import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; -import com.uber.hoodie.index.HoodieIndex; +import com.uber.hoodie.index.HoodieIndex.IndexType; import java.util.List; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -81,7 +82,7 @@ public void run() throws Exception { // 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, @@ -92,7 +93,8 @@ public void run() throws Exception { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .forTable(tableName).withIndexConfig( - HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).build()) .build(); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index f6995762d3e05..184ea82e086e3 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -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,7 +101,8 @@ public void init() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.init(basePath); + fs = FSUtils.getFs(basePath.toString(), jsc.hadoopConfiguration()); + HoodieTestUtils.init(fs, basePath); dataGen = new HoodieTestDataGenerator(); } @@ -212,7 +214,6 @@ public void testUpserts() throws Exception { 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 void testUpserts() throws Exception { 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 void testUpserts() throws Exception { 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 taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table) .collect(); @@ -268,13 +271,13 @@ public void testUpserts() throws Exception { 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 void testDeletes() throws Exception { 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 void testDeletes() throws Exception { 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 void testDeletes() throws Exception { 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 taggedRecords = index .tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect(); @@ -359,7 +363,7 @@ public void testDeletes() throws Exception { 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 void testCreateSavepoint() throws Exception { .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 void testCreateSavepoint() throws Exception { List 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 dataFiles = partitionPaths.stream().flatMap(s -> { return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); @@ -472,7 +477,7 @@ public void testCreateSavepoint() throws Exception { // 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 void testRollbackToSavepoint() throws Exception { .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 void testRollbackToSavepoint() throws Exception { assertNoWriteErrors(statuses); List 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 void testRollbackToSavepoint() throws Exception { // 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 void testRollbackToSavepoint() throws Exception { 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 void testInsertAndCleanByVersions() throws Exception { .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 void testInsertAndCleanByVersions() throws Exception { 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 void testInsertAndCleanByVersions() throws Exception { // 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 void testInsertAndCleanByCommits() throws Exception { .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 void testInsertAndCleanByCommits() throws Exception { 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 void testInsertAndCleanByCommits() throws Exception { // 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 @@ -788,8 +796,8 @@ public void testRollbackCommit() throws Exception { 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 void testAutoRollbackCommit() throws Exception { 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 @@ private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) { @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 void testSmallInsertHandlingForUpserts() throws Exception { 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 void testSmallInsertHandlingForUpserts() throws Exception { 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 records = ParquetUtils.readAvroRecords(newFile); + List 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 void testSmallInsertHandlingForUpserts() throws Exception { 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 files = fileSystemView @@ -1040,7 +1046,7 @@ public void testSmallInsertHandlingForUpserts() throws Exception { 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 void testSmallInsertHandlingForUpserts() throws Exception { 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 void testSmallInsertHandlingForInserts() throws Exception { List 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 void testSmallInsertHandlingForInserts() throws Exception { 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 records = ParquetUtils.readAvroRecords(newFile); + List 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 void testSmallInsertHandlingForInserts() throws Exception { 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 files = table.getROFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3) @@ -1148,7 +1154,7 @@ public void testSmallInsertHandlingForInserts() throws Exception { 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 void testKeepLatestFileVersions() throws IOException { 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 hoodieCleanStatsOne = table.clean(jsc); @@ -1183,7 +1190,8 @@ public void testKeepLatestFileVersions() throws IOException { // 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 void testKeepLatestFileVersions() throws IOException { // 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 @@ -1238,30 +1247,29 @@ public void testKeepLatestFileVersionsMOR() throws IOException { .retainFileVersions(1).build()).build(); HoodieTableMetaClient metaClient = HoodieTestUtils - .initTableType(basePath, HoodieTableType.MERGE_ON_READ); + .initTableType(fs, basePath, HoodieTableType.MERGE_ON_READ); // Make 3 files, one base file and 2 log files associated with base file String file1P0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); String file2P0L0 = HoodieTestUtils - .createNewLogFile(basePath, partitionPaths[0], "000", file1P0, Optional.empty()); + .createNewLogFile(fs, basePath, partitionPaths[0], "000", file1P0, Optional.empty()); String file2P0L1 = HoodieTestUtils - .createNewLogFile(basePath, partitionPaths[0], "000", file1P0, Optional.of(2)); + .createNewLogFile(fs, basePath, partitionPaths[0], "000", file1P0, Optional.of(2)); // make 1 compaction commit - HoodieTestUtils.createCompactionCommitFiles(basePath, "000"); + HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "000"); // Make 4 files, one base file and 3 log files associated with base file HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0); file2P0L0 = HoodieTestUtils - .createNewLogFile(basePath, partitionPaths[0], "001", file1P0, Optional.empty()); + .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.empty()); file2P0L0 = HoodieTestUtils - .createNewLogFile(basePath, partitionPaths[0], "001", file1P0, Optional.of(2)); + .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(2)); file2P0L0 = HoodieTestUtils - .createNewLogFile(basePath, partitionPaths[0], "001", file1P0, Optional.of(3)); + .createNewLogFile(fs, basePath, partitionPaths[0], "001", file1P0, Optional.of(3)); // make 1 compaction commit - HoodieTestUtils.createCompactionCommitFiles(basePath, "001"); + HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); - HoodieTable table = HoodieTable - .getHoodieTable(metaClient, config); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); List 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 void testKeepLatestCommits() throws IOException { 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 hoodieCleanStatsOne = table.clean(jsc); assertEquals("Must not clean any files", 0, @@ -1300,9 +1307,8 @@ public void testKeepLatestCommits() throws IOException { // 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 void testKeepLatestCommits() throws IOException { // 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 void testKeepLatestCommits() throws IOException { // 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 void testCleaningWithZeroPartitonPaths() throws IOException { 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 hoodieCleanStatsOne = table.clean(jsc); @@ -1449,7 +1458,8 @@ public void onTaskEnd(SparkListenerTaskEnd taskEnd) { 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 hoodieCleanStats = table.clean(jsc); @@ -1476,8 +1486,8 @@ public void testCommitWritesRelativePaths() throws Exception { 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"; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java new file mode 100644 index 0000000000000..676b970d9bf9c --- /dev/null +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java @@ -0,0 +1,162 @@ +/* + * 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.AfterClass; +import org.junit.BeforeClass; +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; + + @BeforeClass + public static 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); + } + + @AfterClass + public static void cleanupClass() throws Exception { + if (jsc != null) { + jsc.stop(); + } + + if (hdfsTestService != null) { + hdfsTestService.stop(); + dfsCluster.shutdown(); + } + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM + FileSystem.closeAll(); + } + + @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 records = dataGen.generateInserts(readCommitTime, 100); + JavaRDD 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 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 localRecords = dataGen.generateInserts(writeCommitTime, 100); + JavaRDD 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 localReadRecords = HoodieClientTestUtils + .readCommit(tablePath, sqlContext, timeline, writeCommitTime); + assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size()); + } +} diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java index c2db12d5e86e1..295d415259ed4 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieClientTestUtils.java @@ -174,7 +174,7 @@ public static Dataset read(String basePath, List 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(), diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java index ec3d5728cf5be..c7e3e915618aa 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieMergeOnReadTestUtils.java @@ -18,7 +18,7 @@ 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 @@ */ public class HoodieMergeOnReadTestUtils { - public static List getRecordsUsingInputFormat(List inputPaths) + public static List getRecordsUsingInputFormat(List 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 records = new ArrayList<>(); @@ -76,12 +77,12 @@ public static List getRecordsUsingInputFormat(List inputP } private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, - Schema schema) { + Schema schema, String basePath) { List 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"); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index c197e6b5167b6..beb76df651ab7 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -20,6 +20,7 @@ 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 static void createCommitFile(String basePath, String commitTime) throws I 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 static void createSavepointFile(String basePath, String commitTime) throw 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 { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java index 8433c03661da9..352a0036b85dd 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestUpdateMapFunction.java @@ -48,15 +48,16 @@ public void init() throws Exception { TemporaryFolder folder = new TemporaryFolder(); folder.create(); this.basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.init(basePath); + HoodieTestUtils.init(FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()), basePath); } @Test 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 void testSchemaEvolutionOnUpdate() throws Exception { Iterator> 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}"; diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java index 8d4cc2558380b..c3ace401eff67 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java @@ -54,7 +54,6 @@ 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,7 +87,8 @@ public void init() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.init(basePath); + fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration()); + HoodieTestUtils.init(fs, basePath); // We have some records to be tagged (two different partitions) schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8"); schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr)); @@ -120,8 +119,6 @@ public void testLoadUUIDsInMemory() throws IOException { .parallelize(Arrays.asList(record1, record2, record3, record4)); // Load to memory - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); - Map> map = recordRDD .mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())) .groupByKey().collectAsMap(); @@ -174,7 +171,7 @@ public void testLoadInvolvedFiles() throws IOException { Arrays.asList(record2, record3, record4), schema, null, false); List 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> filesList = index.loadInvolvedFiles(partitions, table); // Still 0, as no valid commit @@ -291,7 +288,8 @@ public void testCheckUUIDsAgainstOneFile() List uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), record3.getRecordKey(), record4.getRecordKey()); - List results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(uuids, + List 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 void testTagLocationWithEmptyRDD() throws Exception { // We have some records to be tagged (two different partitions) JavaRDD 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 void testTagLocation() throws Exception { .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 void testTagLocation() throws Exception { 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 void testCheckExists() throws Exception { JavaRDD 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 void testCheckExists() throws Exception { 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 void testBloomFilterFalseError() throws IOException, InterruptedException // We do the tag JavaRDD 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 @@ private String writeParquetFile(String partitionPath, String filename, List originalCommits = timeline.getInstants().collect(Collectors.toList()); assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants()); - HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, + new HoodieTableMetaClient(fs.getConf(), basePath, true)); assertTrue(archiveLog.archiveIfRequired()); //reload the timeline and remove the remaining commits - timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline() + timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline() .filterCompletedInstants(); 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,22 +149,20 @@ public void testArchiveDatasetWithNoArchival() throws IOException { .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); - HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createCommitFile(basePath, "102"); HoodieTestDataGenerator.createCommitFile(basePath, "103"); - HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants(); assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = - metadata.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() + .filterCompletedInstants(); assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants()); } @@ -173,8 +173,8 @@ public void testArchiveCommitSafety() throws IOException { .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); - HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createCommitFile(basePath, "102"); @@ -182,14 +182,13 @@ public void testArchiveCommitSafety() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "104"); HoodieTestDataGenerator.createCommitFile(basePath, "105"); - HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = - metadata.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() + .filterCompletedInstants(); assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100")); assertTrue("Archived commits should always be safe", @@ -206,8 +205,8 @@ public void testArchiveCommitSavepointNoHole() throws IOException { .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); - HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); + HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient); HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createSavepointFile(basePath, "101"); @@ -216,14 +215,13 @@ public void testArchiveCommitSavepointNoHole() throws IOException { HoodieTestDataGenerator.createCommitFile(basePath, "104"); HoodieTestDataGenerator.createCommitFile(basePath, "105"); - HoodieTimeline timeline = - metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants(); assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants()); boolean result = archiveLog.archiveIfRequired(); assertTrue(result); - timeline = - metadata.getActiveTimeline().reload().getCommitsTimeline() - .filterCompletedInstants(); + timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline() + .filterCompletedInstants(); assertEquals( "Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)", 5, timeline.countInstants()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java index c842bf5922cd9..e05e73364d859 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/TestHoodieCompactor.java @@ -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; @@ -38,6 +41,10 @@ import com.uber.hoodie.io.compact.HoodieCompactor; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; import com.uber.hoodie.table.HoodieTable; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; import org.apache.hadoop.fs.FileSystem; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -46,20 +53,13 @@ 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; private String basePath = null; private HoodieCompactor compactor; private transient HoodieTestDataGenerator dataGen = null; + private transient FileSystem fs; @Before public void init() throws IOException { @@ -70,7 +70,8 @@ public void init() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ); + fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + HoodieTestUtils.initTableType(fs, basePath, HoodieTableType.MERGE_ON_READ); dataGen = new HoodieTestDataGenerator(); compactor = new HoodieRealtimeTableCompactor(); @@ -103,19 +104,20 @@ private HoodieWriteConfig.Builder getConfigBuilder() { @Test(expected = IllegalArgumentException.class) public void testCompactionOnCopyOnWriteFail() throws Exception { - HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE); - - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath); + HoodieTestUtils.initTableType(fs, basePath, HoodieTableType.COPY_ON_WRITE); + 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 +134,6 @@ public void testCompactionEmpty() throws Exception { @Test public void testLogFileCountsAfterCompaction() throws Exception { - FileSystem fs = FSUtils.getFs(); // insert 100 records HoodieWriteConfig config = getConfig(); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); @@ -144,7 +145,8 @@ public void testLogFileCountsAfterCompaction() throws Exception { List 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"; @@ -157,11 +159,11 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Write them to corresponding avro logfiles HoodieTestUtils - .writeRecordsToLogFiles(metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, + .writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema, 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 groupedLogFiles = @@ -174,14 +176,14 @@ public void testLogFileCountsAfterCompaction() throws Exception { } // 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(); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java index a98b76838316b..9432be72d01c2 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java @@ -76,7 +76,7 @@ public void init() throws Exception { TemporaryFolder folder = new TemporaryFolder(); folder.create(); this.basePath = folder.getRoot().getAbsolutePath(); - HoodieTestUtils.init(basePath); + HoodieTestUtils.init(FSUtils.getFs(basePath, jsc.hadoopConfiguration()), basePath); } @Test @@ -89,7 +89,8 @@ public void testMakeNewPath() throws Exception { 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 void testUpdateRecords() throws Exception { // 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 void testUpdateRecords() throws Exception { // 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 void testUpdateRecords() throws Exception { + FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile(); // Read the parquet file, check the record content - List fileRecords = ParquetUtils.readAvroRecords(parquetFilePath); + List fileRecords = ParquetUtils + .readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath); GenericRecord newRecord; int index = 0; for (GenericRecord record : fileRecords) { @@ -188,7 +191,7 @@ public void testUpdateRecords() throws Exception { 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> iter = table .handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(), @@ -211,7 +214,7 @@ public void testUpdateRecords() throws Exception { // 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 void testMetadataAggregateFromWriteStatus() throws Exception { 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 void testMetadataAggregateFromWriteStatus() throws Exception { 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 void testInsertWithPartialFailures() throws Exception { 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 void testFileSizeUpsertRecords() throws Exception { 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 records = new ArrayList<>(); @@ -437,7 +440,7 @@ private List testUpsertPartitioner(int smal 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( diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 42649159c86f3..a9ed76379f0fe 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -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; @@ -37,7 +42,6 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; -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; @@ -45,6 +49,14 @@ 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,27 +73,12 @@ 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; //NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class) //The implementation and gurantees of many API's differ, for example check rename(src,dst) @@ -94,12 +91,9 @@ public static void cleanUp() throws Exception { 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(); } @BeforeClass @@ -112,31 +106,25 @@ public static void setUpDFS() throws IOException { // Create a temp folder as the base path dfs = dfsCluster.getFileSystem(); } - FSUtils.setFs(dfs); - HoodieTestUtils.resetFS(); } @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(); + jsc.hadoopConfiguration().addResource(dfs.getConf()); + dfs.mkdirs(new Path(basePath)); - FSUtils.setFs(dfs); - HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(dfs, basePath, HoodieTableType.MERGE_ON_READ); + sqlContext = new SQLContext(jsc); // SQLContext stuff compactor = new HoodieRealtimeTableCompactor(); - - //SQLContext stuff - sqlContext = new SQLContext(jsc); } @After @@ -167,7 +155,8 @@ public void testSimpleInsertAndUpdate() throws Exception { List 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 deltaCommit = @@ -209,7 +198,7 @@ public void testSimpleInsertAndUpdate() throws Exception { 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()); @@ -222,15 +211,16 @@ public void testSimpleInsertAndUpdate() throws Exception { compactor.compact(jsc, getConfig(true), table, HoodieActiveTimeline.createNewCommitTime()); - allFiles = HoodieTestUtils.listAllDataFilesInPath(fs, cfg.getBasePath()); + allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); 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 +274,8 @@ public void testSimpleInsertAndDelete() throws Exception { List 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 deltaCommit = @@ -332,7 +323,7 @@ public void testSimpleInsertAndDelete() throws Exception { // 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()); @@ -340,7 +331,7 @@ public void testSimpleInsertAndDelete() throws Exception { commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - allFiles = HoodieTestUtils.listAllDataFilesInPath(fs, cfg.getBasePath()); + allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), allFiles); dataFilesToRead = roView.getLatestDataFiles(); @@ -349,7 +340,7 @@ public void testSimpleInsertAndDelete() throws Exception { List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) .collect(Collectors.toList()); List 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()); } @@ -358,7 +349,7 @@ public void testSimpleInsertAndDelete() throws Exception { public void testCOWToMORConvertedDatasetRollback() throws Exception { //Set TableType to COW - HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE); + HoodieTestUtils.initTableType(dfs, basePath, HoodieTableType.COPY_ON_WRITE); HoodieWriteConfig cfg = getConfig(true); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); @@ -377,9 +368,8 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { //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 commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertTrue(commit.isPresent()); @@ -398,13 +388,14 @@ public void testCOWToMORConvertedDatasetRollback() throws Exception { assertNoWriteErrors(statuses); //Set TableType to MOR - HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(dfs, basePath, HoodieTableType.MERGE_ON_READ); //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 +431,8 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { List 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 deltaCommit = @@ -476,7 +468,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { 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 +479,19 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { List dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) .collect(Collectors.toList()); List 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 +508,14 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { 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 +534,7 @@ public void testRollbackWithDeltaAndCompactionCommit() throws Exception { 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); diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml index db973dfa7f37f..181fed895818d 100644 --- a/hoodie-common/pom.xml +++ b/hoodie-common/pom.xml @@ -140,5 +140,11 @@ + + com.github.stefanbirkner + system-rules + 1.16.0 + test + diff --git a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java index b775d906876e2..b39d641746f8b 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/avro/MercifulJsonConverter.java @@ -77,7 +77,7 @@ private Object typeConvert(Object value, String name, Schema schema) throws IOEx switch (schema.getType()) { case BOOLEAN: if (value instanceof Boolean) { - return (Boolean) value; + return value; } break; case DOUBLE: diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/SerializableConfiguration.java b/hoodie-common/src/main/java/com/uber/hoodie/common/SerializableConfiguration.java new file mode 100644 index 0000000000000..01a1a6c09a862 --- /dev/null +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/SerializableConfiguration.java @@ -0,0 +1,56 @@ +/* + * 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.common; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import org.apache.hadoop.conf.Configuration; + +public class SerializableConfiguration implements Serializable { + + private transient Configuration configuration; + + public SerializableConfiguration(Configuration configuration) { + this.configuration = configuration; + } + + public Configuration get() { + return configuration; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); + configuration.write(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + configuration = new Configuration(false); + configuration.readFields(in); + } + + @Override + public String toString() { + StringBuilder str = new StringBuilder(); + configuration.iterator().forEachRemaining(e -> + str.append(String.format("%s => %s \n", e.getKey(), e.getValue()))); + return configuration.toString(); + } +} diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/ActionType.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/ActionType.java index 4a44276966412..96e5b97d0b063 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/ActionType.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/ActionType.java @@ -17,5 +17,5 @@ package com.uber.hoodie.common.model; public enum ActionType { - commit, savepoint, compaction, clean, rollback; + commit, savepoint, compaction, clean, rollback } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java index 8cc6c18c6dd5a..577d7cf4e346c 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java @@ -52,11 +52,13 @@ public class HoodieTableConfig implements Serializable { public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME = "hoodie.table.rt.file.format"; public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class"; + public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = "hoodie.archivelog.folder"; public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE; public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET; public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG; public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName(); + public static final String DEFAULT_ARCHIVELOG_FOLDER = ""; private Properties props; public HoodieTableConfig(FileSystem fs, String metaPath) { @@ -105,6 +107,9 @@ public static void createHoodieProperties(FileSystem fs, Path metadataFolder, && !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) { properties.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS); } + if (!properties.containsKey(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) { + properties.setProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER); + } properties .store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); } finally { @@ -161,4 +166,10 @@ public HoodieFileFormat getRTFileFormat() { return DEFAULT_RT_FILE_FORMAT; } + /** + * Get the relative path of archive log folder under metafolder, for this dataset + */ + public String getArchivelogFolder() { + return props.getProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER); + } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java index d012d9799f690..841cf47ab5e2f 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java @@ -16,6 +16,7 @@ package com.uber.hoodie.common.table; +import com.uber.hoodie.common.SerializableConfiguration; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline; @@ -26,6 +27,7 @@ import java.io.Serializable; import java.util.Objects; import java.util.Properties; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -52,24 +54,28 @@ public class HoodieTableMetaClient implements Serializable { private String basePath; private transient FileSystem fs; private String metaPath; + private SerializableConfiguration hadoopConf; private HoodieTableType tableType; private HoodieTableConfig tableConfig; private HoodieActiveTimeline activeTimeline; private HoodieArchivedTimeline archivedTimeline; - public HoodieTableMetaClient(FileSystem fs, String basePath) throws DatasetNotFoundException { + public HoodieTableMetaClient(Configuration conf, String basePath) + throws DatasetNotFoundException { // Do not load any timeline by default - this(fs, basePath, false); + this(conf, basePath, false); } - public HoodieTableMetaClient(FileSystem fs, String basePath, boolean loadActiveTimelineOnLoad) + public HoodieTableMetaClient(Configuration conf, String basePath, + boolean loadActiveTimelineOnLoad) throws DatasetNotFoundException { log.info("Loading HoodieTableMetaClient from " + basePath); this.basePath = basePath; - this.fs = fs; + this.hadoopConf = new SerializableConfiguration(conf); Path basePathDir = new Path(this.basePath); this.metaPath = basePath + File.separator + METAFOLDER_NAME; Path metaPathDir = new Path(this.metaPath); + this.fs = getFs(); DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir); this.tableConfig = new HoodieTableConfig(fs, metaPath); this.tableType = tableConfig.getTableType(); @@ -96,7 +102,7 @@ public HoodieTableMetaClient() { private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.fs = FSUtils.getFs(); + fs = null; // will be lazily inited } private void writeObject(java.io.ObjectOutputStream out) @@ -125,6 +131,18 @@ public String getMetaPath() { return metaPath; } + /** + * @return path where archived timeline is stored + */ + public String getArchivePath() { + String archiveFolder = tableConfig.getArchivelogFolder(); + if (archiveFolder.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) { + return getMetaPath(); + } else { + return getMetaPath() + "/" + archiveFolder; + } + } + /** * @return Table Config */ @@ -136,9 +154,16 @@ public HoodieTableConfig getTableConfig() { * Get the FS implementation for this table */ public FileSystem getFs() { + if (fs == null) { + fs = FSUtils.getFs(metaPath, hadoopConf.get()); + } return fs; } + public Configuration getHadoopConf() { + return hadoopConf.get(); + } + /** * Get the active instants as a timeline * @@ -146,7 +171,7 @@ public FileSystem getFs() { */ public synchronized HoodieActiveTimeline getActiveTimeline() { if (activeTimeline == null) { - activeTimeline = new HoodieActiveTimeline(fs, metaPath); + activeTimeline = new HoodieActiveTimeline(this); } return activeTimeline; } @@ -159,7 +184,7 @@ public synchronized HoodieActiveTimeline getActiveTimeline() { */ public synchronized HoodieArchivedTimeline getArchivedTimeline() { if (archivedTimeline == null) { - archivedTimeline = new HoodieArchivedTimeline(fs, metaPath); + archivedTimeline = new HoodieArchivedTimeline(this); } return archivedTimeline; } @@ -195,8 +220,20 @@ public static HoodieTableMetaClient initializePathAsHoodieDataset(FileSystem fs, if (!fs.exists(metaPathDir)) { fs.mkdirs(metaPathDir); } + + // if anything other than default archive log folder is specified, create that too + String archiveLogPropVal = props + .getProperty(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, + HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER); + if (!archiveLogPropVal.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) { + Path archiveLogDir = new Path(metaPathDir, archiveLogPropVal); + if (!fs.exists(archiveLogDir)) { + fs.mkdirs(archiveLogDir); + } + } + HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath); log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType() + " from " + basePath); return metaClient; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java index 6671992337d8b..140d5d4fa9098 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieCompactedLogRecordScanner.java @@ -83,7 +83,7 @@ public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List logMetadata) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index ae004991c1c7d..4c9f5c5232fc9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -37,7 +37,6 @@ import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -54,11 +53,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss"); - private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class); - private String metaPath; - private transient FileSystem fs; - + private HoodieTableMetaClient metaClient; /** * Returns next commit time in the {@link #COMMIT_FORMATTER} format. @@ -67,12 +63,14 @@ public static String createNewCommitTime() { return HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); } - protected HoodieActiveTimeline(FileSystem fs, String metaPath, String[] includedExtensions) { + protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, String[] includedExtensions) { // Filter all the filter in the metapath and include only the extensions passed and // convert them into HoodieInstant try { this.instants = - Arrays.stream(HoodieTableMetaClient.scanFiles(fs, new Path(metaPath), path -> { + Arrays.stream( + HoodieTableMetaClient + .scanFiles(metaClient.getFs(), new Path(metaClient.getMetaPath()), path -> { // Include only the meta files with extensions that needs to be included String extension = FSUtils.getFileExtension(path.getName()); return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension)); @@ -85,14 +83,13 @@ protected HoodieActiveTimeline(FileSystem fs, String metaPath, String[] included } catch (IOException e) { throw new HoodieIOException("Failed to scan metadata", e); } - this.fs = fs; - this.metaPath = metaPath; + this.metaClient = metaClient; // multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 this.details = (Function> & Serializable) this::getInstantDetails; } - public HoodieActiveTimeline(FileSystem fs, String metaPath) { - this(fs, metaPath, + public HoodieActiveTimeline(HoodieTableMetaClient metaClient) { + this(metaClient, new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION}); @@ -114,7 +111,6 @@ public HoodieActiveTimeline() { private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.fs = FSUtils.getFs(); } /** @@ -214,9 +210,9 @@ public void revertToInflight(HoodieInstant instant) { public void deleteInflight(HoodieInstant instant) { log.info("Deleting in-flight " + instant); - Path inFlightCommitFilePath = new Path(metaPath, instant.getFileName()); + Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName()); try { - boolean result = fs.delete(inFlightCommitFilePath, false); + boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false); if (result) { log.info("Removed in-flight " + instant); } else { @@ -230,18 +226,18 @@ public void deleteInflight(HoodieInstant instant) { @Override public Optional getInstantDetails(HoodieInstant instant) { - Path detailPath = new Path(metaPath, instant.getFileName()); + Path detailPath = new Path(metaClient.getMetaPath(), instant.getFileName()); return readDataFromPath(detailPath); } protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed, Optional data) { - Path commitFilePath = new Path(metaPath, completed.getFileName()); + Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName()); try { // open a new file and write the commit metadata in - Path inflightCommitFile = new Path(metaPath, inflight.getFileName()); + Path inflightCommitFile = new Path(metaClient.getMetaPath(), inflight.getFileName()); createFileInMetaPath(inflight.getFileName(), data); - boolean success = fs.rename(inflightCommitFile, commitFilePath); + boolean success = metaClient.getFs().rename(inflightCommitFile, commitFilePath); if (!success) { throw new HoodieIOException( "Could not rename " + inflightCommitFile + " to " + commitFilePath); @@ -252,11 +248,11 @@ protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant comp } protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { - Path inFlightCommitFilePath = new Path(metaPath, inflight.getFileName()); + Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName()); try { - if (!fs.exists(inFlightCommitFilePath)) { - Path commitFilePath = new Path(metaPath, completed.getFileName()); - boolean success = fs.rename(commitFilePath, inFlightCommitFilePath); + if (!metaClient.getFs().exists(inFlightCommitFilePath)) { + Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName()); + boolean success = metaClient.getFs().rename(commitFilePath, inFlightCommitFilePath); if (!success) { throw new HoodieIOException( "Could not rename " + commitFilePath + " to " + inFlightCommitFilePath); @@ -272,15 +268,15 @@ public void saveToInflight(HoodieInstant instant, Optional content) { } protected void createFileInMetaPath(String filename, Optional content) { - Path fullPath = new Path(metaPath, filename); + Path fullPath = new Path(metaClient.getMetaPath(), filename); try { if (!content.isPresent()) { - if (fs.createNewFile(fullPath)) { + if (metaClient.getFs().createNewFile(fullPath)) { log.info("Created a new file in meta path: " + fullPath); return; } } else { - FSDataOutputStream fsout = fs.create(fullPath, true); + FSDataOutputStream fsout = metaClient.getFs().create(fullPath, true); fsout.write(content.get()); fsout.close(); return; @@ -292,7 +288,7 @@ protected void createFileInMetaPath(String filename, Optional content) { } protected Optional readDataFromPath(Path detailPath) { - try (FSDataInputStream is = fs.open(detailPath)) { + try (FSDataInputStream is = metaClient.getFs().open(detailPath)) { return Optional.of(IOUtils.toByteArray(is)); } catch (IOException e) { throw new HoodieIOException("Could not read commit details from " + detailPath, e); @@ -300,6 +296,6 @@ protected Optional readDataFromPath(Path detailPath) { } public HoodieActiveTimeline reload() { - return new HoodieActiveTimeline(fs, metaPath); + return new HoodieActiveTimeline(metaClient); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java index 37e5e94145341..793d9d9961778 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java @@ -16,8 +16,8 @@ package com.uber.hoodie.common.table.timeline; +import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; -import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.io.Serializable; @@ -27,7 +27,6 @@ import java.util.Optional; import java.util.function.Function; import java.util.stream.Collectors; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; @@ -44,17 +43,17 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline { private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits"; - private transient FileSystem fs; - private String metaPath; + private HoodieTableMetaClient metaClient; private Map readCommits = new HashMap<>(); private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class); - public HoodieArchivedTimeline(FileSystem fs, String metaPath) { + public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) { // Read back the commits to make sure - Path archiveLogPath = getArchiveLogPath(metaPath); + Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); try (SequenceFile.Reader reader = - new SequenceFile.Reader(fs.getConf(), SequenceFile.Reader.file(archiveLogPath))) { + new SequenceFile.Reader(metaClient.getHadoopConf(), + SequenceFile.Reader.file(archiveLogPath))) { Text key = new Text(); Text val = new Text(); while (reader.next(key, val)) { @@ -71,8 +70,7 @@ public HoodieArchivedTimeline(FileSystem fs, String metaPath) { } // multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 this.details = (Function> & Serializable) this::getInstantDetails; - this.fs = fs; - this.metaPath = metaPath; + this.metaClient = metaClient; } /** @@ -91,12 +89,11 @@ public HoodieArchivedTimeline() { private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.fs = FSUtils.getFs(); } - public static Path getArchiveLogPath(String metaPath) { - return new Path(metaPath, HOODIE_COMMIT_ARCHIVE_LOG_FILE); + public static Path getArchiveLogPath(String archiveFolder) { + return new Path(archiveFolder, HOODIE_COMMIT_ARCHIVE_LOG_FILE); } @Override @@ -105,7 +102,7 @@ public Optional getInstantDetails(HoodieInstant instant) { } public HoodieArchivedTimeline reload() { - return new HoodieArchivedTimeline(fs, metaPath); + return new HoodieArchivedTimeline(metaClient); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java index afd2c89dc36fb..60df7f4715b38 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java @@ -40,7 +40,6 @@ import java.util.stream.Stream; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; /** @@ -57,7 +56,6 @@ public class HoodieTableFileSystemView implements TableFileSystemView, TableFileSystemView.RealtimeView, Serializable { protected HoodieTableMetaClient metaClient; - protected transient FileSystem fs; // This is the commits that will be visible for all views extending this view protected HoodieTimeline visibleActiveTimeline; @@ -72,7 +70,6 @@ public class HoodieTableFileSystemView implements TableFileSystemView, public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) { this.metaClient = metaClient; - this.fs = metaClient.getFs(); this.visibleActiveTimeline = visibleActiveTimeline; this.fileGroupMap = new HashMap<>(); this.partitionToFileGroupsMap = new HashMap<>(); @@ -98,7 +95,6 @@ public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.fs = FSUtils.getFs(); } private void writeObject(java.io.ObjectOutputStream out) @@ -255,8 +251,8 @@ public Stream getAllFileGroups(String partitionPathStr) { try { // Create the path if it does not exist already Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr); - FSUtils.createPathIfNotExists(fs, partitionPath); - FileStatus[] statuses = fs.listStatus(partitionPath); + FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath); + FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath); List fileGroups = addFilesToView(statuses); return fileGroups.stream(); } catch (IOException e) { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java index ae0dbd3f004a3..633960d0a3a2a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java @@ -94,7 +94,7 @@ public static List> loadFromFile(FileSystem fs, public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime, Optional durationInMs, List cleanStats) { ImmutableMap.Builder partitionMetadataBuilder = - ImmutableMap.builder(); + ImmutableMap.builder(); int totalDeleted = 0; String earliestCommitToRetain = null; for (HoodieCleanStat stat : cleanStats) { @@ -116,7 +116,7 @@ public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime, public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime, Optional durationInMs, List commits, List stats) { ImmutableMap.Builder partitionMetadataBuilder = - ImmutableMap.builder(); + ImmutableMap.builder(); int totalDeleted = 0; for (HoodieRollbackStat stat : stats) { HoodieRollbackPartitionMetadata metadata = @@ -132,7 +132,7 @@ public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbac public static HoodieSavepointMetadata convertSavepointMetadata(String user, String comment, Map> latestFiles) { ImmutableMap.Builder partitionMetadataBuilder = - ImmutableMap.builder(); + ImmutableMap.builder(); for (Map.Entry> stat : latestFiles.entrySet()) { HoodieSavepointPartitionMetadata metadata = new HoodieSavepointPartitionMetadata(stat.getKey(), stat.getValue()); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index daecf6237df63..5417f01c9b328 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -16,7 +16,6 @@ package com.uber.hoodie.common.util; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodiePartitionMetadata; @@ -29,6 +28,7 @@ import java.util.Arrays; import java.util.Comparator; import java.util.List; +import java.util.Map.Entry; import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -55,27 +55,31 @@ public class FSUtils { private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10; private static final long MIN_CLEAN_TO_KEEP = 10; private static final long MIN_ROLLBACK_TO_KEEP = 10; - private static FileSystem fs; + private static final String HOODIE_ENV_PROPS_PREFIX = "HOODIE_ENV_"; - /** - * Only to be used for testing. - */ - @VisibleForTesting - public static void setFs(FileSystem fs) { - FSUtils.fs = fs; + public static Configuration prepareHadoopConf(Configuration conf) { + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + + // look for all properties, prefixed to be picked up + for (Entry prop : System.getenv().entrySet()) { + if (prop.getKey().startsWith(HOODIE_ENV_PROPS_PREFIX)) { + LOG.info("Picking up value for hoodie env var :" + prop.getKey()); + conf.set(prop.getKey() + .replace(HOODIE_ENV_PROPS_PREFIX, "") + .replaceAll("_DOT_", "."), + prop.getValue()); + } + } + return conf; } - public static FileSystem getFs() { - if (fs != null) { - return fs; - } - Configuration conf = new Configuration(); - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + public static FileSystem getFs(String path, Configuration conf) { FileSystem fs; + conf = prepareHadoopConf(conf); try { - fs = FileSystem.get(conf); + fs = new Path(path).getFileSystem(conf); } catch (IOException e) { throw new HoodieIOException("Failed to get instance of " + FileSystem.class.getName(), e); @@ -83,7 +87,6 @@ public static FileSystem getFs() { LOG.info( String.format("Hadoop Configuration: fs.defaultFS: [%s], Config:[%s], FileSystem: [%s]", conf.getRaw("fs.defaultFS"), conf.toString(), fs.toString())); - return fs; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java index a4a683350e7a6..ecbaad1c96980 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java @@ -51,10 +51,11 @@ public class ParquetUtils { * Read the rowKey list from the given parquet file. * * @param filePath The parquet file path. + * @param configuration configuration to build fs object */ - public static Set readRowKeysFromParquet(Path filePath) { - Configuration conf = new Configuration(); - conf.addResource(getFs().getConf()); + public static Set readRowKeysFromParquet(Configuration configuration, Path filePath) { + Configuration conf = new Configuration(configuration); + conf.addResource(getFs(filePath.toString(), conf).getConf()); Schema readSchema = HoodieAvroUtils.getRecordKeySchema(); AvroReadSupport.setAvroReadSchema(conf, readSchema); AvroReadSupport.setRequestedProjection(conf, readSchema); @@ -84,19 +85,12 @@ public static Set readRowKeysFromParquet(Path filePath) { return rowKeys; } - - /** - * Read the metadata from a parquet file - */ - public static ParquetMetadata readMetadata(Path parquetFilePath) { - return readMetadata(new Configuration(), parquetFilePath); - } - public static ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) { ParquetMetadata footer; try { // TODO(vc): Should we use the parallel reading version here? - footer = ParquetFileReader.readFooter(getFs().getConf(), parquetFilePath); + footer = ParquetFileReader + .readFooter(getFs(parquetFilePath.toString(), conf).getConf(), parquetFilePath); } catch (IOException e) { throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, e); @@ -108,14 +102,15 @@ public static ParquetMetadata readMetadata(Configuration conf, Path parquetFileP /** * Get the schema of the given parquet file. */ - public static MessageType readSchema(Path parquetFilePath) { - return readMetadata(parquetFilePath).getFileMetaData().getSchema(); + public static MessageType readSchema(Configuration configuration, Path parquetFilePath) { + return readMetadata(configuration, parquetFilePath).getFileMetaData().getSchema(); } - private static List readParquetFooter(Path parquetFilePath, String... footerNames) { + private static List readParquetFooter(Configuration configuration, Path parquetFilePath, + String... footerNames) { List footerVals = new ArrayList<>(); - ParquetMetadata footer = readMetadata(parquetFilePath); + ParquetMetadata footer = readMetadata(configuration, parquetFilePath); Map metadata = footer.getFileMetaData().getKeyValueMetaData(); for (String footerName : footerNames) { if (metadata.containsKey(footerName)) { @@ -128,21 +123,22 @@ private static List readParquetFooter(Path parquetFilePath, String... fo return footerVals; } - public static Schema readAvroSchema(Path parquetFilePath) { - return new AvroSchemaConverter().convert(readSchema(parquetFilePath)); + public static Schema readAvroSchema(Configuration configuration, Path parquetFilePath) { + return new AvroSchemaConverter().convert(readSchema(configuration, parquetFilePath)); } /** * Read out the bloom filter from the parquet file meta data. */ - public static BloomFilter readBloomFilterFromParquetMetadata(Path parquetFilePath) { - String footerVal = readParquetFooter(parquetFilePath, + public static BloomFilter readBloomFilterFromParquetMetadata(Configuration configuration, + Path parquetFilePath) { + String footerVal = readParquetFooter(configuration, parquetFilePath, HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY).get(0); return new BloomFilter(footerVal); } - public static String[] readMinMaxRecordKeys(Path parquetFilePath) { - List minMaxKeys = readParquetFooter(parquetFilePath, + public static String[] readMinMaxRecordKeys(Configuration configuration, Path parquetFilePath) { + List minMaxKeys = readParquetFooter(configuration, parquetFilePath, HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER, HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER); if (minMaxKeys.size() != 2) { @@ -156,11 +152,11 @@ public static String[] readMinMaxRecordKeys(Path parquetFilePath) { /** * NOTE: This literally reads the entire file contents, thus should be used with caution. */ - public static List readAvroRecords(Path filePath) { + public static List readAvroRecords(Configuration configuration, Path filePath) { ParquetReader reader = null; List records = new ArrayList<>(); try { - reader = AvroParquetReader.builder(filePath).build(); + reader = AvroParquetReader.builder(filePath).withConf(configuration).build(); Object obj = reader.read(); while (obj != null) { if (obj instanceof GenericRecord) { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java index 0b3a6bdebfee0..a90739f75c4b4 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/minicluster/HdfsTestService.java @@ -19,6 +19,7 @@ import com.google.common.base.Preconditions; import com.google.common.io.Files; +import com.uber.hoodie.common.model.HoodieTestUtils; import java.io.File; import java.io.IOException; import org.apache.commons.io.FileUtils; @@ -54,7 +55,6 @@ public class HdfsTestService { private MiniDFSCluster miniDfsCluster; public HdfsTestService() { - hadoopConf = new Configuration(); workDir = Files.createTempDir().getAbsolutePath(); } @@ -65,10 +65,7 @@ public Configuration getHadoopConf() { public MiniDFSCluster start(boolean format) throws IOException { Preconditions .checkState(workDir != null, "The work dir must be set before starting cluster."); - - if (hadoopConf == null) { - hadoopConf = new Configuration(); - } + hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); // If clean, then remove the work dir so we can start fresh. String localDFSLocation = getDFSLocation(workDir); @@ -90,8 +87,8 @@ public MiniDFSCluster start(boolean format) throws IOException { } public void stop() throws IOException { + logger.info("HDFS Minicluster service being shut down."); miniDfsCluster.shutdown(); - logger.info("HDFS Minicluster service shut down."); miniDfsCluster = null; hadoopConf = null; } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index 9489e57ab3664..74103f8fa6ac2 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -58,6 +58,7 @@ 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.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -69,22 +70,22 @@ public class HoodieTestUtils { - public static FileSystem fs = FSUtils.getFs(); public static final String TEST_EXTENSION = ".test"; public static final String RAW_TRIPS_TEST_NAME = "raw_trips"; public static final int DEFAULT_TASK_PARTITIONID = 1; public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"}; private static Random rand = new Random(46474747); - public static void resetFS() { - HoodieTestUtils.fs = FSUtils.getFs(); + public static Configuration getDefaultHadoopConf() { + return new Configuration(); } - public static HoodieTableMetaClient init(String basePath) throws IOException { - return initTableType(basePath, HoodieTableType.COPY_ON_WRITE); + public static HoodieTableMetaClient init(FileSystem fs, String basePath) throws IOException { + return initTableType(fs, basePath, HoodieTableType.COPY_ON_WRITE); } - public static HoodieTableMetaClient initTableType(String basePath, HoodieTableType tableType) + public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath, + HoodieTableType tableType) throws IOException { Properties properties = new Properties(); properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME); @@ -99,7 +100,8 @@ public static HoodieTableMetaClient initOnTemp() throws IOException { TemporaryFolder folder = new TemporaryFolder(); folder.create(); String basePath = folder.getRoot().getAbsolutePath(); - return HoodieTestUtils.init(basePath); + return HoodieTestUtils + .init(FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()), basePath); } public static String makeNewCommitTime() { @@ -137,7 +139,7 @@ public static final String createDataFile(String basePath, String partitionPath, return fileID; } - public static final String createNewLogFile(String basePath, String partitionPath, + public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime, String fileID, Optional version) throws IOException { String folderPath = basePath + "/" + partitionPath + "/"; boolean makeDir = fs.mkdirs(new Path(folderPath)); @@ -153,7 +155,8 @@ public static final String createNewLogFile(String basePath, String partitionPat return fileID; } - public static final void createCompactionCommitFiles(String basePath, String... commitTimes) + public static final void createCompactionCommitFiles(FileSystem fs, String basePath, + String... commitTimes) throws IOException { for (String commitTime : commitTimes) { boolean createFile = fs.createNewFile(new Path( @@ -211,7 +214,7 @@ public static void createCleanFiles(String basePath, String commitTime) throws I Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline .makeCleanerFileName(commitTime)); - FileSystem fs = FSUtils.getFs(); + FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); FSDataOutputStream os = fs.create(commitFile, true); try { HoodieCleanStat cleanStats = new HoodieCleanStat( @@ -262,7 +265,7 @@ public static T serializeDeserialize(T object, Class return deseralizedObject; } - public static void writeRecordsToLogFiles(String basePath, Schema schema, + public static void writeRecordsToLogFiles(FileSystem fs, String basePath, Schema schema, List updatedRecords) { Map> groupedUpdated = updatedRecords.stream() .collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation)); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java index d771b8236e9d9..bd710d1a99e52 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java @@ -36,7 +36,6 @@ import org.apache.hadoop.io.Text; import org.junit.Before; import org.junit.Test; -import org.junit.rules.TemporaryFolder; public class HoodieTableMetaClientTest { @@ -45,10 +44,8 @@ public class HoodieTableMetaClientTest { @Before public void init() throws IOException { - TemporaryFolder folder = new TemporaryFolder(); - folder.create(); - this.basePath = folder.getRoot().getAbsolutePath(); - metaClient = HoodieTestUtils.init(basePath); + metaClient = HoodieTestUtils.initOnTemp(); + basePath = metaClient.getBasePath(); } @Test @@ -107,9 +104,9 @@ public void checkCommitTimeline() throws IOException { @Test public void checkArchiveCommitTimeline() throws IOException { - Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getMetaPath()); + Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); SequenceFile.Writer writer = SequenceFile - .createWriter(HoodieTestUtils.fs.getConf(), SequenceFile.Writer.file(archiveLogPath), + .createWriter(metaClient.getHadoopConf(), SequenceFile.Writer.file(archiveLogPath), SequenceFile.Writer.keyClass(Text.class), SequenceFile.Writer.valueClass(Text.class)); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java index 8fc7fb46f8a64..e9f3a3a877487 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/log/HoodieLogFormatTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.Maps; import com.uber.hoodie.common.minicluster.MiniClusterUtil; +import com.uber.hoodie.common.model.HoodieArchivedLogFile; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieTableType; @@ -53,6 +54,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.junit.After; @@ -67,7 +69,7 @@ public class HoodieLogFormatTest { private FileSystem fs; private Path partitionPath; - private String basePath; + private static String basePath; @BeforeClass public static void setUpClass() throws IOException, InterruptedException { @@ -78,7 +80,6 @@ public static void setUpClass() throws IOException, InterruptedException { @AfterClass public static void tearDownClass() { MiniClusterUtil.shutdown(); - HoodieTestUtils.resetFS(); } @Before @@ -89,8 +90,7 @@ public void setUp() throws IOException, InterruptedException { assertTrue(fs.mkdirs(new Path(folder.getRoot().getPath()))); this.partitionPath = new Path(folder.getRoot().getPath()); this.basePath = folder.getRoot().getParent(); - HoodieTestUtils.fs = fs; - HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestUtils.initTableType(fs, basePath, HoodieTableType.MERGE_ON_READ); } @After @@ -241,6 +241,34 @@ public void testLeaseRecovery() throws IOException, URISyntaxException, Interrup writer.close(); } + @Test + public void testAppendNotSupported() + throws IOException, URISyntaxException, InterruptedException { + // Use some fs like LocalFileSystem, that does not support appends + Path localPartitionPath = new Path("file://" + partitionPath); + FileSystem localFs = FSUtils + .getFs(localPartitionPath.toString(), HoodieTestUtils.getDefaultHadoopConf()); + Path testPath = new Path(localPartitionPath, "append_test"); + localFs.mkdirs(testPath); + + // Some data & append two times. + List records = SchemaTestUtil.generateTestRecords(0, 100); + Map metadata = Maps.newHashMap(); + metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100"); + HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, + getSimpleSchema(), metadata); + + for (int i = 0; i < 2; i++) { + HoodieLogFormat.newWriterBuilder().onParentPath(testPath) + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive") + .overBaseCommit("").withFs(localFs).build().appendBlock(dataBlock).close(); + } + + // ensure there are two log file versions, with same data. + FileStatus[] statuses = localFs.listStatus(testPath); + assertEquals(2, statuses.length); + } + @SuppressWarnings("unchecked") @Test public void testBasicWriteAndScan() @@ -343,7 +371,7 @@ public void testAppendAndReadOnCorruptedLog() writer.close(); // Append some arbit byte[] to thee end of the log (mimics a partially written commit) - fs = FileSystem.get(fs.getConf()); + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); // create a block with outputStream.write(HoodieLogFormat.MAGIC); @@ -533,7 +561,7 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() writer.close(); // Append some arbit byte[] to thee end of the log (mimics a partially written commit) - fs = FileSystem.get(fs.getConf()); + fs = FSUtils.getFs(fs.getUri().toString(), fs.getConf()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath()); // create a block with outputStream.write(HoodieLogFormat.MAGIC); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java index 22285a6c51b14..096c75d779320 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java @@ -49,7 +49,7 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { - HoodieTestUtils.fs.delete(new Path(this.metaClient.getBasePath()), true); + metaClient.getFs().delete(new Path(this.metaClient.getBasePath()), true); } @Test @@ -74,7 +74,7 @@ public void testLoadingInstantsFromFiles() throws IOException { HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9"); - timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath()); + timeline = new HoodieActiveTimeline(metaClient); timeline.saveAsComplete(instant1, Optional.empty()); timeline.saveAsComplete(instant2, Optional.empty()); timeline.saveAsComplete(instant3, Optional.empty()); @@ -98,7 +98,7 @@ public void testLoadingInstantsFromFiles() throws IOException { @Test public void testTimelineOperationsBasic() throws Exception { - timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath()); + timeline = new HoodieActiveTimeline(metaClient); assertTrue(timeline.empty()); assertEquals("", 0, timeline.countInstants()); assertEquals("", Optional.empty(), timeline.firstInstant()); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index 992b864162b6d..fab8e371459f4 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -45,7 +45,6 @@ import org.apache.hadoop.fs.Path; import org.junit.Before; import org.junit.Test; -import org.junit.rules.TemporaryFolder; @SuppressWarnings("ResultOfMethodCallIgnored") public class HoodieTableFileSystemViewTest { @@ -58,10 +57,8 @@ public class HoodieTableFileSystemViewTest { @Before public void init() throws IOException { - TemporaryFolder folder = new TemporaryFolder(); - folder.create(); - this.basePath = folder.getRoot().getAbsolutePath(); - metaClient = HoodieTestUtils.init(basePath); + metaClient = HoodieTestUtils.initOnTemp(); + basePath = metaClient.getBasePath(); fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants()); roView = (TableFileSystemView.ReadOptimizedView) fsView; @@ -69,7 +66,7 @@ public void init() throws IOException { } private void refreshFsView(FileStatus[] statuses) { - metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs, basePath, true); + metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); if (statuses != null) { fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), @@ -184,7 +181,7 @@ public void testStreamLatestVersionInPartition() throws IOException { new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); // Now we list the entire partition - FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath)); + FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath)); assertEquals(11, statuses.length); refreshFsView(null); @@ -285,7 +282,7 @@ public void testStreamEveryVersionInPartition() throws IOException { new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); // Now we list the entire partition - FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath)); + FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath)); assertEquals(7, statuses.length); refreshFsView(null); @@ -359,7 +356,7 @@ public void streamLatestVersionInRange() throws IOException { new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); // Now we list the entire partition - FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath)); + FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath)); assertEquals(9, statuses.length); refreshFsView(statuses); @@ -430,7 +427,7 @@ public void streamLatestVersionsBefore() throws IOException { new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); // Now we list the entire partition - FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath)); + FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath)); assertEquals(7, statuses.length); refreshFsView(null); @@ -492,7 +489,7 @@ public void streamLatestVersions() throws IOException { new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile(); // Now we list the entire partition - FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath)); + FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath)); assertEquals(10, statuses.length); refreshFsView(statuses); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java index edcc1509b25c3..43b0ebdf32ea2 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestFSUtils.java @@ -16,15 +16,24 @@ package com.uber.hoodie.common.util; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.uber.hoodie.common.model.HoodieTestUtils; import java.text.SimpleDateFormat; import java.util.Date; import java.util.UUID; +import org.apache.hadoop.conf.Configuration; +import org.junit.Rule; import org.junit.Test; +import org.junit.contrib.java.lang.system.EnvironmentVariables; public class TestFSUtils { + @Rule + public final EnvironmentVariables environmentVariables + = new EnvironmentVariables(); + @Test public void testMakeDataFileName() { String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); @@ -59,4 +68,15 @@ public void testGetFileNameWithoutMeta() { String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName); assertTrue(FSUtils.getFileId(fullFileName).equals(fileName)); } + + @Test + public void testEnvVarVariablesPickedup() { + environmentVariables.set("HOODIE_ENV_fs_DOT_key1", "value1"); + Configuration conf = FSUtils.prepareHadoopConf(HoodieTestUtils.getDefaultHadoopConf()); + assertEquals("value1", conf.get("fs.key1")); + conf.set("fs.key1", "value11"); + conf.set("fs.key2", "value2"); + assertEquals("value11", conf.get("fs.key1")); + assertEquals("value2", conf.get("fs.key2")); + } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java index 266cb1158cf3d..fd0a4475a13b1 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestParquetUtils.java @@ -22,6 +22,7 @@ import com.uber.hoodie.avro.HoodieAvroWriteSupport; import com.uber.hoodie.common.BloomFilter; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.model.HoodieTestUtils; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -79,12 +80,15 @@ public void testHoodieWriteSupport() throws Exception { // Read and verify List rowKeysInFile = new ArrayList<>( - ParquetUtils.readRowKeysFromParquet(new Path(filePath))); + ParquetUtils + .readRowKeysFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath))); Collections.sort(rowKeysInFile); Collections.sort(rowKeys); assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile); - BloomFilter filterInFile = ParquetUtils.readBloomFilterFromParquetMetadata(new Path(filePath)); + BloomFilter filterInFile = ParquetUtils + .readBloomFilterFromParquetMetadata(HoodieTestUtils.getDefaultHadoopConf(), + new Path(filePath)); for (String rowKey : rowKeys) { assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey)); } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java index ee4f5a9544286..302373e1beb92 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieInputFormat.java @@ -295,6 +295,6 @@ protected static HoodieTableMetaClient getTableMetaClient(FileSystem fs, Path da } Path baseDir = HoodieHiveUtil.getNthParent(dataPath, levels); LOG.info("Reading hoodie metadata from path " + baseDir.toString()); - return new HoodieTableMetaClient(fs, baseDir.toString()); + return new HoodieTableMetaClient(fs.getConf(), baseDir.toString()); } } diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java index c8ffbcc9a6b2a..2c01ed666a220 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/HoodieROTablePathFilter.java @@ -19,6 +19,7 @@ import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; +import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.exception.DatasetNotFoundException; import com.uber.hoodie.exception.HoodieException; import java.io.Serializable; @@ -86,7 +87,7 @@ public boolean accept(Path path) { } Path folder = null; try { - FileSystem fs = path.getFileSystem(new Configuration()); + FileSystem fs = path.getFileSystem(FSUtils.prepareHadoopConf(new Configuration())); if (fs.isDirectory(path)) { return true; } @@ -123,7 +124,7 @@ public boolean accept(Path path) { if (baseDir != null) { try { HoodieTableMetaClient metaClient = - new HoodieTableMetaClient(fs, baseDir.toString()); + new HoodieTableMetaClient(fs.getConf(), baseDir.toString()); HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitTimeline() .filterCompletedInstants(), diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java index 00ef57e42d287..35ef39eda8ca4 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReader.java @@ -84,7 +84,7 @@ public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); try { baseFileSchema = readSchema(jobConf, split.getPath()); - readAndCompactLog(); + readAndCompactLog(jobConf); } catch (IOException e) { throw new HoodieIOException( "Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e); @@ -110,7 +110,7 @@ private static MessageType readSchema(Configuration conf, Path parquetFilePath) * Goes through the log files and populates a map with latest version of each key logged, since * the base split was written. */ - private void readAndCompactLog() throws IOException { + private void readAndCompactLog(JobConf jobConf) throws IOException { Schema writerSchema = new AvroSchemaConverter().convert(baseFileSchema); List projectionFields = orderFields( jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), @@ -123,7 +123,8 @@ private void readAndCompactLog() throws IOException { String.format("About to read compacted logs %s for base split %s, projecting cols %s", split.getDeltaFilePaths(), split.getPath(), projectionFields)); HoodieCompactedLogRecordScanner compactedLogRecordScanner = - new HoodieCompactedLogRecordScanner(FSUtils.getFs(), split.getBasePath(), + new HoodieCompactedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf), + split.getBasePath(), split.getDeltaFilePaths(), readerSchema, split.getMaxCommitTime()); // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java index ac14e64842f9a..4b060a132bd26 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/InputFormatTestUtil.java @@ -39,7 +39,9 @@ public class InputFormatTestUtil { public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles, String commitNumber) throws IOException { basePath.create(); - HoodieTestUtils.init(basePath.getRoot().toString()); + HoodieTestUtils + .init(FSUtils.getFs(basePath.getRoot().toString(), HoodieTestUtils.getDefaultHadoopConf()), + basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); for (int i = 0; i < numberOfFiles; i++) { File dataFile = @@ -99,7 +101,9 @@ public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException { basePath.create(); - HoodieTestUtils.init(basePath.getRoot().toString()); + HoodieTestUtils + .init(FSUtils.getFs(basePath.getRoot().toString(), HoodieTestUtils.getDefaultHadoopConf()), + basePath.getRoot().toString()); File partitionPath = basePath.newFolder("2016", "05", "01"); AvroParquetWriter parquetWriter; for (int i = 0; i < numberOfFiles; i++) { diff --git a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java index 84f02b8688fd0..2bef7780a1ed9 100644 --- a/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java +++ b/hoodie-hadoop-mr/src/test/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeRecordReaderTest.java @@ -41,6 +41,7 @@ import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; @@ -64,10 +65,13 @@ public class HoodieRealtimeRecordReaderTest { private JobConf jobConf; + private FileSystem fs; @Before public void setUp() { jobConf = new JobConf(); + fs = FSUtils + .getFs(basePath.getRoot().getAbsolutePath(), HoodieTestUtils.getDefaultHadoopConf()); } @Rule @@ -79,7 +83,9 @@ private HoodieLogFormat.Writer writeLogFile(File partitionDir, Schema schema, St HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() .onParentPath(new Path(partitionDir.getPath())) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId) - .overBaseCommit(baseCommit).withFs(FSUtils.getFs()).build(); + .overBaseCommit(baseCommit) + .withFs(fs) + .build(); List records = new ArrayList<>(); for (int i = 0; i < numberOfRecords; i++) { records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0")); @@ -98,7 +104,7 @@ public void testReader() throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema()); HoodieTestUtils - .initTableType(basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); + .initTableType(fs, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); String commitTime = "100"; File partitionDir = InputFormatTestUtil .prepareParquetDataset(basePath, schema, 1, 100, commitTime); @@ -123,8 +129,8 @@ public void testReader() throws Exception { //create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat(). - getRecordReader(new FileSplit(split.getPath(), 0, - FSUtils.getFs().getLength(split.getPath()), (String[]) null), jobConf, null); + getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), + (String[]) null), jobConf, null); JobConf jobConf = new JobConf(); List fields = schema.getFields(); String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); @@ -156,7 +162,7 @@ public void testReaderWithNestedAndComplexSchema() throws Exception { // initial commit Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema()); HoodieTestUtils - .initTableType(basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); + .initTableType(fs, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ); String commitTime = "100"; int numberOfRecords = 100; int numberOfLogRecords = numberOfRecords / 2; @@ -183,8 +189,8 @@ public void testReaderWithNestedAndComplexSchema() throws Exception { //create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat(). - getRecordReader(new FileSplit(split.getPath(), 0, - FSUtils.getFs().getLength(split.getPath()), (String[]) null), jobConf, null); + getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), + (String[]) null), jobConf, null); JobConf jobConf = new JobConf(); List fields = schema.getFields(); diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java index 088e24a9c962b..472001ede9e69 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java @@ -30,6 +30,7 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.Partition; @@ -183,7 +184,7 @@ public static void main(String[] args) throws Exception { cmd.usage(); System.exit(1); } - FileSystem fs = FSUtils.getFs(); + FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration()); HiveConf hiveConf = new HiveConf(); hiveConf.addResource(fs.getConf()); new HiveSyncTool(cfg, hiveConf, fs).syncHoodieTable(); diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index dede5e5f5d2d8..95d1d58212a0c 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -91,7 +91,7 @@ public class HoodieHiveClient { HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { this.syncConfig = cfg; this.fs = fs; - this.metaClient = new HoodieTableMetaClient(fs, cfg.basePath, true); + this.metaClient = new HoodieTableMetaClient(fs.getConf(), cfg.basePath, true); this.tableType = metaClient.getTableType(); LOG.info("Creating hive connection " + cfg.jdbcUrl); diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java index 26ed1b0f95e50..7b9172ead70bd 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/util/HiveTestService.java @@ -20,6 +20,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.google.common.io.Files; +import com.uber.hoodie.common.model.HoodieTestUtils; import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; @@ -87,7 +88,7 @@ public HiveServer2 start() throws IOException { .checkState(workDir != null, "The work dir must be set before starting cluster."); if (hadoopConf == null) { - hadoopConf = new Configuration(); + hadoopConf = HoodieTestUtils.getDefaultHadoopConf(); } String localHiveLocation = getHiveLocation(workDir); diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java index b74d917071ed2..996786a7d4bed 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/HoodieDataSourceHelpers.java @@ -67,7 +67,7 @@ public static String latestCommit(FileSystem fs, String basePath) { */ public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) { HoodieTable table = HoodieTable - .getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null); + .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null); if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) { return table.getActiveTimeline().getTimelineOfActions( Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala index 26579630d7815..285ba87b1b021 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/DefaultSource.scala @@ -189,6 +189,7 @@ class DefaultSource extends RelationProvider val properties = new Properties(); properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tblName.get); properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, storageType); + properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, "archived"); HoodieTableMetaClient.initializePathAsHoodieDataset(fs, path.get, properties); } diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala index 4aca81f1725b7..cf0a2c1aa8082 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/IncrementalRelation.scala @@ -47,7 +47,7 @@ class IncrementalRelation(val sqlContext: SQLContext, private val log = LogManager.getLogger(classOf[IncrementalRelation]) val fs = new Path(basePath).getFileSystem(sqlContext.sparkContext.hadoopConfiguration) - val metaClient = new HoodieTableMetaClient(fs, basePath, true) + val metaClient = new HoodieTableMetaClient(sqlContext.sparkContext.hadoopConfiguration, basePath, true) // MOR datasets not supported yet if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) { throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets") @@ -72,7 +72,8 @@ class IncrementalRelation(val sqlContext: SQLContext, val latestMeta = HoodieCommitMetadata .fromBytes(commitTimeline.getInstantDetails(commitsToReturn.last).get) val metaFilePath = latestMeta.getFileIdAndFullPaths(basePath).values().iterator().next() - AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema(new Path(metaFilePath))) + AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema( + sqlContext.sparkContext.hadoopConfiguration, new Path(metaFilePath))) } override def schema: StructType = latestSchema diff --git a/hoodie-spark/src/test/scala/DataSourceTest.scala b/hoodie-spark/src/test/scala/DataSourceTest.scala index b9fed1cfe7db4..1bdc92777c1a1 100644 --- a/hoodie-spark/src/test/scala/DataSourceTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceTest.scala @@ -57,7 +57,7 @@ class DataSourceTest extends AssertionsForJUnit { val folder = new TemporaryFolder folder.create basePath = folder.getRoot.getAbsolutePath - fs = FSUtils.getFs + fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration) } @Test def testCopyOnWriteStorage() { diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java index 27f2649740b06..1d3a2b1c10eae 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java @@ -56,21 +56,18 @@ 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.Function; -import org.apache.spark.api.java.function.VoidFunction; import scala.Tuple2; public class HDFSParquetImporter implements Serializable { private static volatile Logger logger = LogManager.getLogger(HDFSParquetImporter.class); private final Config cfg; - private final transient FileSystem fs; + private transient FileSystem fs; public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd"); public HDFSParquetImporter( Config cfg) throws IOException { this.cfg = cfg; - fs = FSUtils.getFs(); } public static class FormatValidator implements IValueValidator { @@ -203,6 +200,7 @@ private String getSchema() throws Exception { } public int dataImport(JavaSparkContext jsc, int retry) throws Exception { + this.fs = FSUtils.getFs(cfg.targetPath, jsc.hadoopConfiguration()); int ret = -1; try { // Verify that targetPath is not present. @@ -251,43 +249,36 @@ protected int dataImport(JavaSparkContext jsc) throws IOException { GenericRecord.class, job.getConfiguration()) // To reduce large number of tasks. .coalesce(16 * cfg.parallelism) - .map(new Function, HoodieRecord>() { - @Override - public HoodieRecord call(Tuple2 entry) - throws Exception { - GenericRecord genericRecord = entry._2(); - Object partitionField = genericRecord.get(cfg.partitionKey); - if (partitionField == null) { - throw new HoodieIOException( - "partition key is missing. :" + cfg.partitionKey); - } - Object rowField = genericRecord.get(cfg.rowKey); - if (rowField == null) { - throw new HoodieIOException( - "row field is missing. :" + cfg.rowKey); - } - long ts = (long) ((Double) partitionField * 1000l); - String partitionPath = PARTITION_FORMATTER.format(new Date(ts)); - return new HoodieRecord( - new HoodieKey((String) rowField, partitionPath), - new HoodieJsonPayload(genericRecord.toString())); - } - } + .map(entry -> { + GenericRecord genericRecord = ((Tuple2) entry)._2(); + Object partitionField = genericRecord.get(cfg.partitionKey); + if (partitionField == null) { + throw new HoodieIOException( + "partition key is missing. :" + cfg.partitionKey); + } + Object rowField = genericRecord.get(cfg.rowKey); + if (rowField == null) { + throw new HoodieIOException( + "row field is missing. :" + cfg.rowKey); + } + long ts = (long) ((Double) partitionField * 1000l); + String partitionPath = PARTITION_FORMATTER.format(new Date(ts)); + return new HoodieRecord<>( + new HoodieKey((String) rowField, partitionPath), + new HoodieJsonPayload(genericRecord.toString())); + } ); // Get commit time. String commitTime = client.startCommit(); JavaRDD writeResponse = client.bulkInsert(hoodieRecords, commitTime); Accumulator errors = jsc.accumulator(0); - writeResponse.foreach(new VoidFunction() { - @Override - public void call(WriteStatus writeStatus) throws Exception { + writeResponse.foreach(writeStatus -> { if (writeStatus.hasErrors()) { errors.add(1); logger.error(String.format("Error processing records :writeStatus:%s", writeStatus.getStat().toString())); } - } }); if (errors.value() == 0) { logger.info(String diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java index 5bcba8cb8361b..a6fbc3f363cfb 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java @@ -291,7 +291,7 @@ private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IO if (!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) { return "0"; } - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), targetDataPath); Optional lastCommit = metadata.getActiveTimeline().getCommitsTimeline() @@ -331,7 +331,7 @@ private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException { - HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation); + HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), sourceTableLocation); List commitsToSync = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants() .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants() diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java index 3d199e0f52593..41ef772769288 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java @@ -20,6 +20,7 @@ import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; +import com.uber.hoodie.common.SerializableConfiguration; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.table.HoodieTableConfig; @@ -70,8 +71,10 @@ static class Config implements Serializable { public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir, final boolean shouldAssumeDatePartitioning) throws IOException { - FileSystem fs = FSUtils.getFs(); - final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir); + FileSystem fs = FSUtils.getFs(baseDir, jsc.hadoopConfiguration()); + final SerializableConfiguration serConf = new SerializableConfiguration( + jsc.hadoopConfiguration()); + final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir); final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView( tableMetadata, tableMetadata.getActiveTimeline().getCommitsTimeline() @@ -104,7 +107,7 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi jsc.parallelize(partitions, partitions.size()) .flatMap(partition -> { // Only take latest version files <= latestCommit. - FileSystem fs1 = FSUtils.getFs(); + FileSystem fs1 = FSUtils.getFs(baseDir, serConf.get()); List> filePaths = new ArrayList<>(); Stream dataFiles = fsView .getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp); @@ -123,13 +126,13 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi String partition = tuple._1(); Path sourceFilePath = new Path(tuple._2()); Path toPartitionPath = new Path(outputDir, partition); - FileSystem fs1 = FSUtils.getFs(); + FileSystem ifs = FSUtils.getFs(baseDir, serConf.get()); - if (!fs1.exists(toPartitionPath)) { - fs1.mkdirs(toPartitionPath); + if (!ifs.exists(toPartitionPath)) { + ifs.mkdirs(toPartitionPath); } - FileUtil.copy(fs1, sourceFilePath, fs1, - new Path(toPartitionPath, sourceFilePath.getName()), false, fs1.getConf()); + FileUtil.copy(ifs, sourceFilePath, ifs, + new Path(toPartitionPath, sourceFilePath.getName()), false, ifs.getConf()); }); // Also copy the .commit files diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index 2baafd036382f..6a63c33fc84ed 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -115,10 +115,11 @@ public class HoodieDeltaStreamer implements Serializable { public HoodieDeltaStreamer(Config cfg) throws IOException { this.cfg = cfg; - this.fs = FSUtils.getFs(); + this.jssc = getSparkContext(); + this.fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()); if (fs.exists(new Path(cfg.targetBasePath))) { - HoodieTableMetaClient meta = new HoodieTableMetaClient(fs, cfg.targetBasePath); + HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), cfg.targetBasePath); this.commitTimelineOpt = Optional .of(meta.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants()); @@ -129,8 +130,6 @@ public HoodieDeltaStreamer(Config cfg) throws IOException { //TODO(vc) Should these be passed from outside? initSchemaProvider(); initKeyGenerator(); - this.jssc = getSparkContext(); - initSource(); } @@ -203,7 +202,9 @@ private void sync() throws Exception { Properties properties = new Properties(); properties.put(HoodieWriteConfig.TABLE_NAME, cfg.targetTableName); HoodieTableMetaClient - .initializePathAsHoodieDataset(FSUtils.getFs(), cfg.targetBasePath, properties); + .initializePathAsHoodieDataset( + FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()), cfg.targetBasePath, + properties); } log.info("Checkpoint to resume from : " + resumeCheckpointStr); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java index f6ea67f0155d6..44745b093fd2e 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/schema/FilebasedSchemaProvider.java @@ -25,6 +25,7 @@ import java.util.Arrays; import org.apache.avro.Schema; import org.apache.commons.configuration.PropertiesConfiguration; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -50,7 +51,7 @@ static class Config { public FilebasedSchemaProvider(PropertiesConfiguration config) { super(config); - this.fs = FSUtils.getFs(); + this.fs = FSUtils.getFs(config.getBasePath(), new Configuration()); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP)); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java index 128a449a43e98..f9c9a9a1dbd63 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/DFSSource.java @@ -65,7 +65,7 @@ static class Config { public DFSSource(PropertiesConfiguration config, JavaSparkContext sparkContext, SourceDataFormat dataFormat, SchemaProvider schemaProvider) { super(config, sparkContext, dataFormat, schemaProvider); - this.fs = FSUtils.getFs(); + this.fs = FSUtils.getFs(config.getBasePath(), sparkContext.hadoopConfiguration()); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP)); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java index 08c9193663576..aeecb9db014ca 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java @@ -72,7 +72,7 @@ static class Config { public HiveIncrPullSource(PropertiesConfiguration config, JavaSparkContext sparkContext, SourceDataFormat dataFormat, SchemaProvider schemaProvider) { super(config, sparkContext, dataFormat, schemaProvider); - this.fs = FSUtils.getFs(); + this.fs = FSUtils.getFs(config.getBasePath(), sparkContext.hadoopConfiguration()); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP)); this.incrPullRootPath = config.getString(Config.ROOT_INPUT_PATH_PROP); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java index 2f2941e5d675b..36c43c5968988 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/KafkaSource.java @@ -48,7 +48,6 @@ import org.apache.spark.streaming.kafka.KafkaUtils; import org.apache.spark.streaming.kafka.OffsetRange; import scala.Predef; -import scala.Tuple2; import scala.collection.JavaConverters; import scala.collection.immutable.Map; import scala.collection.immutable.Set; @@ -134,16 +133,16 @@ static class ScalaHelpers { public static Map toScalaMap(HashMap m) { return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap( - Predef.>conforms() + Predef.conforms() ); } public static Set toScalaSet(HashSet s) { - return JavaConverters.asScalaSetConverter(s).asScala().toSet(); + return JavaConverters.asScalaSetConverter(s).asScala().toSet(); } public static java.util.Map toJavaMap(Map m) { - return JavaConverters.mapAsJavaMapConverter(m).asJava(); + return JavaConverters.mapAsJavaMapConverter(m).asJava(); } } diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java index af0a5233078a3..c1e58153fb4c5 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHDFSParquetImporter.java @@ -24,9 +24,9 @@ import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.minicluster.HdfsTestService; +import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; -import com.uber.hoodie.common.util.FSUtils; import java.io.IOException; import java.io.Serializable; import java.text.ParseException; @@ -38,7 +38,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -71,7 +70,6 @@ public static void initClass() throws Exception { dfs = dfsCluster.getFileSystem(); dfsBasePath = dfs.getWorkingDirectory().toString(); dfs.mkdirs(new Path(dfsBasePath)); - FSUtils.setFs(dfs); } @AfterClass @@ -79,7 +77,6 @@ public static void cleanupClass() throws Exception { if (hdfsTestService != null) { hdfsTestService.stop(); } - FSUtils.setFs(null); } /** @@ -174,7 +171,7 @@ private void createRecords(Path srcFolder) throws ParseException, IOException { ParquetWriter writer = AvroParquetWriter .builder(srcFile) .withSchema(HoodieTestDataGenerator.avroSchema) - .withConf(new Configuration()) + .withConf(HoodieTestUtils.getDefaultHadoopConf()) .build(); for (GenericRecord record : records) { writer.write(record); diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java index 6f9acc4892fc6..326894dfeab0f 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieSnapshotCopier.java @@ -44,17 +44,22 @@ public class TestHoodieSnapshotCopier { @Before public void init() throws IOException { - // Prepare directories - TemporaryFolder folder = new TemporaryFolder(); - folder.create(); - rootPath = folder.getRoot().getAbsolutePath(); - basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME; - HoodieTestUtils.init(basePath); - outputPath = rootPath + "/output"; - fs = FSUtils.getFs(); - // Start a local Spark job - SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]"); - jsc = new JavaSparkContext(conf); + try { + // Prepare directories + TemporaryFolder folder = new TemporaryFolder(); + folder.create(); + rootPath = "file://" + folder.getRoot().getAbsolutePath(); + basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME; + outputPath = rootPath + "/output"; + + fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()); + HoodieTestUtils.init(fs, basePath); + // Start a local Spark job + SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]"); + jsc = new JavaSparkContext(conf); + } catch (Exception e) { + e.printStackTrace(); + } } @Test