1
0

Multi FS Support

- Reviving PR 191, to make FileSystem creation off actual path
 - Streamline all filesystem access to HoodieTableMetaClient
 - Hadoop Conf from Spark Context serialized & passed to executor code too
 - Pick up env vars prefixed with HOODIE_ENV_ into Configuration object
 - Cleanup usage of FSUtils.getFS, piggybacking off HoodieTableMetaClient.getFS
 - Adding s3a to supported schemes & support escaping "." in env vars
 - Tests use HoodieTestUtils.getDefaultHadoopConf
This commit is contained in:
Vinoth Chandar
2017-12-10 23:31:54 -08:00
committed by vinoth chandar
parent 44839b88c6
commit 0cd186c899
78 changed files with 851 additions and 535 deletions

View File

@@ -17,6 +17,7 @@
package com.uber.hoodie.cli; package com.uber.hoodie.cli;
import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.FSUtils;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@@ -36,7 +37,7 @@ public class HoodieCLI {
public static boolean initConf() { public static boolean initConf() {
if (HoodieCLI.conf == null) { if (HoodieCLI.conf == null) {
HoodieCLI.conf = new Configuration(); HoodieCLI.conf = FSUtils.prepareHadoopConf(new Configuration());
return true; return true;
} }
return false; return false;

View File

@@ -54,12 +54,14 @@ public class ArchivedCommitsCommand implements CommandMarker {
System.out System.out
.println("===============> Showing only " + limit + " archived commits <==============="); .println("===============> Showing only " + limit + " archived commits <===============");
FileStatus[] fsStatuses = FSUtils.getFs().globStatus( String basePath = HoodieCLI.tableMetadata.getBasePath();
new Path(HoodieCLI.tableMetadata.getBasePath() + "/.hoodie/.commits_.archive*")); FileStatus[] fsStatuses = FSUtils.getFs(basePath, HoodieCLI.conf)
.globStatus(new Path(basePath + "/.hoodie/.commits_.archive*"));
List<String[]> allCommits = new ArrayList<>(); List<String[]> allCommits = new ArrayList<>();
for (FileStatus fs : fsStatuses) { for (FileStatus fs : fsStatuses) {
//read the archived file //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); new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema(), false);
List<IndexedRecord> readRecords = new ArrayList<>(); List<IndexedRecord> readRecords = new ArrayList<>();

View File

@@ -77,7 +77,7 @@ public class CleansCommand implements CommandMarker {
@CliCommand(value = "cleans refresh", help = "Refresh the commits") @CliCommand(value = "cleans refresh", help = "Refresh the commits")
public String refreshCleans() throws IOException { public String refreshCleans() throws IOException {
HoodieTableMetaClient metadata = HoodieTableMetaClient metadata =
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath());
HoodieCLI.setTableMetadata(metadata); HoodieCLI.setTableMetadata(metadata);
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
} }

View File

@@ -96,7 +96,7 @@ public class CommitsCommand implements CommandMarker {
@CliCommand(value = "commits refresh", help = "Refresh the commits") @CliCommand(value = "commits refresh", help = "Refresh the commits")
public String refreshCommits() throws IOException { public String refreshCommits() throws IOException {
HoodieTableMetaClient metadata = HoodieTableMetaClient metadata =
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath());
HoodieCLI.setTableMetadata(metadata); HoodieCLI.setTableMetadata(metadata);
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
} }
@@ -224,14 +224,13 @@ public class CommitsCommand implements CommandMarker {
public String compareCommits( public String compareCommits(
@CliOption(key = {"path"}, help = "Path of the dataset to compare to") @CliOption(key = {"path"}, help = "Path of the dataset to compare to")
final String path) throws Exception { final String path) throws Exception {
HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.fs, path);
HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.conf, path);
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline() HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants(); .filterCompletedInstants();
;
HoodieTableMetaClient source = HoodieCLI.tableMetadata; HoodieTableMetaClient source = HoodieCLI.tableMetadata;
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline() HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants(); .filterCompletedInstants();
;
String targetLatestCommit = String targetLatestCommit =
targetTimeline.getInstants().iterator().hasNext() ? "0" targetTimeline.getInstants().iterator().hasNext() ? "0"
: targetTimeline.lastInstant().get().getTimestamp(); : targetTimeline.lastInstant().get().getTimestamp();
@@ -266,7 +265,7 @@ public class CommitsCommand implements CommandMarker {
public String syncCommits( public String syncCommits(
@CliOption(key = {"path"}, help = "Path of the dataset to compare to") @CliOption(key = {"path"}, help = "Path of the dataset to compare to")
final String path) throws Exception { final String path) throws Exception {
HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.fs, path); HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.conf, path);
HoodieCLI.state = HoodieCLI.CLIState.SYNC; HoodieCLI.state = HoodieCLI.CLIState.SYNC;
return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName() return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName()
+ " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName(); + " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName();

View File

@@ -33,7 +33,7 @@ public class DatasetsCommand implements CommandMarker {
final String path) throws IOException { final String path) throws IOException {
boolean initialized = HoodieCLI.initConf(); boolean initialized = HoodieCLI.initConf();
HoodieCLI.initFS(initialized); HoodieCLI.initFS(initialized);
HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.fs, path)); HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.conf, path));
HoodieCLI.state = HoodieCLI.CLIState.DATASET; HoodieCLI.state = HoodieCLI.CLIState.DATASET;
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName()
+ " loaded"; + " loaded";

View File

@@ -138,7 +138,7 @@ public class SavepointsCommand implements CommandMarker {
@CliCommand(value = "savepoints refresh", help = "Refresh the savepoints") @CliCommand(value = "savepoints refresh", help = "Refresh the savepoints")
public String refreshMetaClient() throws IOException { public String refreshMetaClient() throws IOException {
HoodieTableMetaClient metadata = HoodieTableMetaClient metadata =
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath()); new HoodieTableMetaClient(HoodieCLI.conf, HoodieCLI.tableMetadata.getBasePath());
HoodieCLI.setTableMetadata(metadata); HoodieCLI.setTableMetadata(metadata);
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed."; return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
} }

View File

@@ -97,8 +97,8 @@ public class SparkMain {
String repairedOutputPath, String repairedOutputPath,
String basePath) String basePath)
throws Exception { throws Exception {
DedupeSparkJob job = new DedupeSparkJob(basePath, DedupeSparkJob job = new DedupeSparkJob(basePath, duplicatedPartitionPath, repairedOutputPath,
duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc), FSUtils.getFs()); new SQLContext(jsc), FSUtils.getFs(basePath, jsc.hadoopConfiguration()));
job.fixDuplicates(true); job.fixDuplicates(true);
return 0; return 0;
} }

View File

@@ -18,6 +18,7 @@ package com.uber.hoodie.cli.utils;
import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.cli.commands.SparkMain; import com.uber.hoodie.cli.commands.SparkMain;
import com.uber.hoodie.common.util.FSUtils;
import java.io.File; import java.io.File;
import java.net.URISyntaxException; import java.net.URISyntaxException;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
@@ -66,6 +67,7 @@ public class SparkUtil {
sparkConf = HoodieWriteClient.registerClasses(sparkConf); sparkConf = HoodieWriteClient.registerClasses(sparkConf);
JavaSparkContext jsc = new JavaSparkContext(sparkConf); JavaSparkContext jsc = new JavaSparkContext(sparkConf);
jsc.hadoopConfiguration().setBoolean("parquet.enable.summary-metadata", false); jsc.hadoopConfiguration().setBoolean("parquet.enable.summary-metadata", false);
FSUtils.prepareHadoopConf(jsc.hadoopConfiguration());
return jsc; return jsc;
} }
} }

View File

@@ -75,7 +75,7 @@ class DedupeSparkJob(basePath: String,
val tmpTableName = s"htbl_${System.currentTimeMillis()}" val tmpTableName = s"htbl_${System.currentTimeMillis()}"
val dedupeTblName = s"${tmpTableName}_dupeKeys" 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 allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}"))
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
@@ -127,7 +127,7 @@ class DedupeSparkJob(basePath: String,
def fixDuplicates(dryRun: Boolean = true) = { 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 allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}"))
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles) val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)

View File

@@ -38,7 +38,7 @@ import scala.collection.mutable._
object SparkHelpers { object SparkHelpers {
@throws[Exception] @throws[Exception]
def skipKeysAndWriteNewFile(commitTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) { 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 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 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) val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter)
@@ -47,6 +47,7 @@ object SparkHelpers {
for (rec <- sourceRecords) { for (rec <- sourceRecords) {
val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString
if (!keysToSkip.contains(key)) { if (!keysToSkip.contains(key)) {
writer.writeAvro(key, rec) writer.writeAvro(key, rec)
} }
} }

View File

@@ -67,10 +67,10 @@ public class HoodieReadClient implements Serializable {
*/ */
public HoodieReadClient(JavaSparkContext jsc, String basePath) { public HoodieReadClient(JavaSparkContext jsc, String basePath) {
this.jsc = jsc; 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 // Create a Hoodie table which encapsulated the commits and files visible
this.hoodieTable = HoodieTable this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null); .getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), null);
this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants(); this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants();
this.index = this.index =
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc); new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
@@ -129,8 +129,8 @@ public class HoodieReadClient implements Serializable {
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD() JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD()
.mapToPair(row -> { .mapToPair(row -> {
HoodieKey key = new HoodieKey( HoodieKey key = new HoodieKey(
row.<String>getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD), row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
row.<String>getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD)); row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
return new Tuple2<>(key, row); return new Tuple2<>(key, row);
}); });

View File

@@ -53,18 +53,6 @@ import com.uber.hoodie.table.HoodieTable;
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner; import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
import com.uber.hoodie.table.WorkloadProfile; import com.uber.hoodie.table.WorkloadProfile;
import com.uber.hoodie.table.WorkloadStat; 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.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
@@ -76,6 +64,17 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors; 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 * Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient
@@ -112,7 +111,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/ */
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
boolean rollbackInFlight) { boolean rollbackInFlight) {
this.fs = FSUtils.getFs(); this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
this.jsc = jsc; this.jsc = jsc;
this.config = clientConfig; this.config = clientConfig;
this.index = HoodieIndex.createIndex(config, jsc); this.index = HoodieIndex.createIndex(config, jsc);
@@ -133,8 +132,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/ */
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) { public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, table); JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, table);
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown()); return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
@@ -146,8 +146,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) { public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
writeContext = metrics.getCommitCtx(); writeContext = metrics.getCommitCtx();
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
try { try {
// De-dupe/merge if needed // De-dupe/merge if needed
@@ -180,8 +181,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) { public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
writeContext = metrics.getCommitCtx(); writeContext = metrics.getCommitCtx();
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
try { try {
// De-dupe/merge if needed // De-dupe/merge if needed
JavaRDD<HoodieRecord<T>> dedupedRecords = JavaRDD<HoodieRecord<T>> dedupedRecords =
@@ -237,8 +239,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) { Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
writeContext = metrics.getCommitCtx(); writeContext = metrics.getCommitCtx();
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
try { try {
// De-dupe/merge if needed // De-dupe/merge if needed
@@ -306,8 +309,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* file instead of using HoodieCommitMetadata * file instead of using HoodieCommitMetadata
*/ */
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile,
HoodieTable<T> table, HoodieTable<T> table, String commitTime) throws HoodieCommitException {
String commitTime) throws HoodieCommitException {
try { try {
HoodieCommitMetadata metadata = new HoodieCommitMetadata(); HoodieCommitMetadata metadata = new HoodieCommitMetadata();
profile.getPartitionPaths().stream().forEach(path -> { profile.getPartitionPaths().stream().forEach(path -> {
@@ -409,8 +411,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
logger.info("Commiting " + commitTime); logger.info("Commiting " + commitTime);
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
@@ -485,8 +488,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @return true if the savepoint was created successfully * @return true if the savepoint was created successfully
*/ */
public boolean savepoint(String user, String comment) { public boolean savepoint(String user, String comment) {
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
if (table.getCompletedCommitTimeline().empty()) { if (table.getCompletedCommitTimeline().empty()) {
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
} }
@@ -512,8 +516,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @return true if the savepoint was created successfully * @return true if the savepoint was created successfully
*/ */
public boolean savepoint(String commitTime, String user, String comment) { public boolean savepoint(String commitTime, String user, String comment) {
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant(); Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
@@ -575,8 +580,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @return true if the savepoint was deleted successfully * @return true if the savepoint was deleted successfully
*/ */
public void deleteSavepoint(String savepointTime) { public void deleteSavepoint(String savepointTime) {
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieInstant savePoint = HoodieInstant savePoint =
@@ -602,8 +608,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @return true if the savepoint was rollecback to successfully * @return true if the savepoint was rollecback to successfully
*/ */
public boolean rollbackToSavepoint(String savepointTime) { public boolean rollbackToSavepoint(String savepointTime) {
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieTimeline commitTimeline = table.getCommitsTimeline(); HoodieTimeline commitTimeline = table.getCommitsTimeline();
@@ -653,8 +660,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
String startRollbackTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); String startRollbackTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieTimeline inflightTimeline = table.getInflightCommitTimeline(); HoodieTimeline inflightTimeline = table.getInflightCommitTimeline();
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline(); HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
@@ -761,8 +769,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
final Timer.Context context = metrics.getCleanCtx(); final Timer.Context context = metrics.getCleanCtx();
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(),
true), config);
List<HoodieCleanStat> cleanStats = table.clean(jsc); List<HoodieCleanStat> cleanStats = table.clean(jsc);
if (cleanStats.isEmpty()) { if (cleanStats.isEmpty()) {
@@ -810,8 +819,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public void startCommitWithTime(String commitTime) { public void startCommitWithTime(String commitTime) {
logger.info("Generate a new commit time " + commitTime); logger.info("Generate a new commit time " + commitTime);
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
String commitActionType = table.getCommitActionType(); String commitActionType = table.getCommitActionType();
activeTimeline.createInflight( activeTimeline.createInflight(
@@ -827,8 +837,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/ */
private void compact(String compactionCommitTime) throws IOException { private void compact(String compactionCommitTime) throws IOException {
// Create a Hoodie table which encapsulated the commits and files visible // Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
Optional<HoodieCommitMetadata> compactionMetadata = table.compact(jsc, compactionCommitTime); Optional<HoodieCommitMetadata> compactionMetadata = table.compact(jsc, compactionCommitTime);
if (compactionMetadata.isPresent()) { if (compactionMetadata.isPresent()) {
logger.info("Compacted successfully on commit " + compactionCommitTime); logger.info("Compacted successfully on commit " + compactionCommitTime);
@@ -876,8 +887,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* Cleanup all inflight commits * Cleanup all inflight commits
*/ */
private void rollbackInflightCommits() { private void rollbackInflightCommits() {
HoodieTable<T> table = HoodieTable HoodieTable<T> table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights(); HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights();
List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp) List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList()); .collect(Collectors.toList());

View File

@@ -72,7 +72,8 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig, HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig,
commitTime, commitTime,
record.getPartitionPath(), record.getPartitionPath(),
TaskContext.getPartitionId()); TaskContext.getPartitionId(),
hoodieTable);
partitionsCleaned.add(record.getPartitionPath()); partitionsCleaned.add(record.getPartitionPath());
} }

View File

@@ -155,7 +155,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it. // Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it.
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo,
partitionRecordKeyPairRDD); partitionRecordKeyPairRDD);
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo,
partitionRecordKeyPairRDD,
parallelism); parallelism);
} }
@@ -257,7 +258,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.mapToPair(ft -> { .mapToPair(ft -> {
try { try {
String[] minMaxKeys = ParquetUtils String[] minMaxKeys = ParquetUtils
.readMinMaxRecordKeys(ft._2().getFileStatus().getPath()); .readMinMaxRecordKeys(hoodieTable.getHadoopConf(),
ft._2().getFileStatus().getPath());
return new Tuple2<>(ft._1(), return new Tuple2<>(ft._1(),
new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1])); new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
} catch (MetadataNotFoundException me) { } catch (MetadataNotFoundException me) {
@@ -358,7 +360,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Make sure the parallelism is atleast the groupby parallelism for tagging location * Make sure the parallelism is atleast the groupby parallelism for tagging location
*/ */
@VisibleForTesting @VisibleForTesting
JavaPairRDD<String, String> findMatchingFilesForRecordKeys( JavaPairRDD<String, String> findMatchingFilesForRecordKeys(HoodieTable hoodieTable,
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo, final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD, JavaPairRDD<String, String> partitionRecordKeyPairRDD,
int totalSubpartitions) { int totalSubpartitions) {
@@ -372,7 +374,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.sortByKey(true, joinParallelism); .sortByKey(true, joinParallelism);
return fileSortedTripletRDD return fileSortedTripletRDD
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true) .mapPartitionsWithIndex(
new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true)
.flatMap(indexLookupResults -> indexLookupResults.iterator()) .flatMap(indexLookupResults -> indexLookupResults.iterator())
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0) .filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
.flatMapToPair(lookupResult -> { .flatMapToPair(lookupResult -> {

View File

@@ -24,10 +24,12 @@ import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIndexException; import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.func.LazyIterableIterator; import com.uber.hoodie.func.LazyIterableIterator;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
@@ -45,20 +47,24 @@ public class HoodieBloomIndexCheckFunction implements
private final String basePath; private final String basePath;
public HoodieBloomIndexCheckFunction(String basePath) { private final HoodieTable table;
public HoodieBloomIndexCheckFunction(HoodieTable table, String basePath) {
this.table = table;
this.basePath = basePath; this.basePath = basePath;
} }
/** /**
* Given a list of row keys and one file, return only row keys existing in that file. * Given a list of row keys and one file, return only row keys existing in that file.
*/ */
public static List<String> checkCandidatesAgainstFile(List<String> candidateRecordKeys, public static List<String> checkCandidatesAgainstFile(Configuration configuration,
List<String> candidateRecordKeys,
Path filePath) throws HoodieIndexException { Path filePath) throws HoodieIndexException {
List<String> foundRecordKeys = new ArrayList<>(); List<String> foundRecordKeys = new ArrayList<>();
try { try {
// Load all rowKeys from the file, to double-confirm // Load all rowKeys from the file, to double-confirm
if (!candidateRecordKeys.isEmpty()) { if (!candidateRecordKeys.isEmpty()) {
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath); Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(configuration, filePath);
logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath); logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath);
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("Keys from " + filePath + " => " + fileRowKeys); logger.debug("Keys from " + filePath + " => " + fileRowKeys);
@@ -107,7 +113,8 @@ public class HoodieBloomIndexCheckFunction implements
private void initState(String fileName, String partitionPath) throws HoodieIndexException { private void initState(String fileName, String partitionPath) throws HoodieIndexException {
try { try {
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName); Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(filePath); bloomFilter = ParquetUtils
.readBloomFilterFromParquetMetadata(table.getHadoopConf(), filePath);
candidateRecordKeys = new ArrayList<>(); candidateRecordKeys = new ArrayList<>();
currentFile = fileName; currentFile = fileName;
currentParitionPath = partitionPath; currentParitionPath = partitionPath;
@@ -154,7 +161,7 @@ public class HoodieBloomIndexCheckFunction implements
.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); .debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
} }
ret.add(new IndexLookupResult(currentFile, ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(candidateRecordKeys, filePath))); checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath)));
initState(fileName, partitionPath); initState(fileName, partitionPath);
if (bloomFilter.mightContain(recordKey)) { if (bloomFilter.mightContain(recordKey)) {
@@ -177,7 +184,7 @@ public class HoodieBloomIndexCheckFunction implements
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys); logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
} }
ret.add(new IndexLookupResult(currentFile, ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(candidateRecordKeys, filePath))); checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath)));
} }
} catch (Throwable e) { } catch (Throwable e) {

View File

@@ -33,7 +33,6 @@ import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
@@ -51,14 +50,12 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
private final HoodieTimeline commitTimeline; private final HoodieTimeline commitTimeline;
private HoodieTable<T> hoodieTable; private HoodieTable<T> hoodieTable;
private HoodieWriteConfig config; private HoodieWriteConfig config;
private FileSystem fs;
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) { public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
this.hoodieTable = hoodieTable; this.hoodieTable = hoodieTable;
this.fileSystemView = hoodieTable.getCompletedFileSystemView(); this.fileSystemView = hoodieTable.getCompletedFileSystemView();
this.commitTimeline = hoodieTable.getCompletedCommitTimeline(); this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
this.config = config; this.config = config;
this.fs = hoodieTable.getFs();
} }

View File

@@ -126,7 +126,8 @@ public class HoodieCommitArchiveLog {
int minCommitsToKeep = config.getMinCommitsToKeep(); int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTable table = HoodieTable HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config); .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true),
config);
// GroupBy each action and limit each action timeline to maxCommitsToKeep // GroupBy each action and limit each action timeline to maxCommitsToKeep
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline() HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
@@ -165,7 +166,7 @@ public class HoodieCommitArchiveLog {
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) { private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
log.info("Deleting instants " + archivedInstants); log.info("Deleting instants " + archivedInstants);
HoodieTableMetaClient metaClient = HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true); new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true);
boolean success = true; boolean success = true;
for (HoodieInstant archivedInstant : archivedInstants) { for (HoodieInstant archivedInstant : archivedInstants) {
@@ -188,7 +189,7 @@ public class HoodieCommitArchiveLog {
try { try {
HoodieTableMetaClient metaClient = HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true); new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true);
HoodieTimeline commitTimeline = HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants(); metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();

View File

@@ -47,7 +47,7 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
HoodieTable<T> hoodieTable) { HoodieTable<T> hoodieTable) {
this.commitTime = commitTime; this.commitTime = commitTime;
this.config = config; this.config = config;
this.fs = FSUtils.getFs(); this.fs = hoodieTable.getMetaClient().getFs();
this.hoodieTable = hoodieTable; this.hoodieTable = hoodieTable;
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline(); this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
this.fileSystemView = hoodieTable.getROFileSystemView(); this.fileSystemView = hoodieTable.getROFileSystemView();
@@ -73,8 +73,9 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config, public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
String commitTime, String commitTime,
String partitionPath, String partitionPath,
int taskPartitionId) { int taskPartitionId,
FileSystem fs = FSUtils.getFs(); HoodieTable hoodieTable) {
FileSystem fs = hoodieTable.getMetaClient().getFs();
try { try {
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
.format("%s/%s/%s", config.getBasePath(), partitionPath, .format("%s/%s/%s", config.getBasePath(), partitionPath,

View File

@@ -16,6 +16,8 @@
package com.uber.hoodie.io.compact; package com.uber.hoodie.io.compact;
import static java.util.stream.Collectors.toList;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
@@ -34,13 +36,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.table.HoodieCopyOnWriteTable; import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import com.uber.hoodie.table.HoodieTable; 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.io.IOException;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.Collection; import java.util.Collection;
@@ -49,8 +44,12 @@ import java.util.List;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.StreamSupport; import java.util.stream.StreamSupport;
import org.apache.avro.Schema;
import static java.util.stream.Collectors.toList; 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 * HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
@@ -73,7 +72,6 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
.getTableType().name()); .getTableType().name());
// TODO - rollback any compactions in flight // TODO - rollback any compactions in flight
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
List<String> partitionPaths = List<String> partitionPaths =
@@ -102,15 +100,9 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
log.info("After filtering, Compacting " + operations + " files"); log.info("After filtering, Compacting " + operations + " files");
List<HoodieWriteStat> updateStatusMap = List<HoodieWriteStat> updateStatusMap =
jsc.parallelize(operations, operations.size()) jsc.parallelize(operations, operations.size())
.map(s -> executeCompaction(metaClient, config, s, compactionCommitTime)) .map(s -> executeCompaction(hoodieTable, config, s, compactionCommitTime))
.flatMap(new FlatMapFunction<List<HoodieWriteStat>, HoodieWriteStat>() { .flatMap(writeStatList -> writeStatList.iterator())
@Override .collect();
public Iterator<HoodieWriteStat> call(
List<HoodieWriteStat> hoodieWriteStats)
throws Exception {
return hoodieWriteStats.iterator();
}
}).collect();
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
for (HoodieWriteStat stat : updateStatusMap) { for (HoodieWriteStat stat : updateStatusMap) {
@@ -134,10 +126,11 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
return true; return true;
} }
private List<HoodieWriteStat> executeCompaction(HoodieTableMetaClient metaClient,
private List<HoodieWriteStat> executeCompaction(HoodieTable hoodieTable,
HoodieWriteConfig config, CompactionOperation operation, String commitTime) HoodieWriteConfig config, CompactionOperation operation, String commitTime)
throws IOException { throws IOException {
FileSystem fs = FSUtils.getFs(); FileSystem fs = hoodieTable.getMetaClient().getFs();
Schema readerSchema = Schema readerSchema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
@@ -147,7 +140,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// Reads the entire avro file. Always only specific blocks should be read from the avro file (failure recover). // 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 // 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. // Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
String maxInstantTime = metaClient.getActiveTimeline() String maxInstantTime = metaClient.getActiveTimeline()
.getTimelineOfActions( .getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
@@ -162,8 +155,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
} }
// Compacting is very similar to applying updates to existing file // Compacting is very similar to applying updates to existing file
HoodieCopyOnWriteTable table = HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient);
new HoodieCopyOnWriteTable(config, metaClient);
Iterator<List<WriteStatus>> result = table Iterator<List<WriteStatus>> result = table
.handleUpdate(commitTime, operation.getFileId(), scanner.iterator()); .handleUpdate(commitTime, operation.getFileId(), scanner.iterator());
Iterable<List<WriteStatus>> resultIterable = () -> result; Iterable<List<WriteStatus>> resultIterable = () -> result;

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.io.storage;
import com.uber.hoodie.avro.HoodieAvroWriteSupport; import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.HoodieAvroUtils;
import java.io.IOException; import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
@@ -26,7 +27,6 @@ import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.parquet.hadoop.ParquetFileWriter; import org.apache.parquet.hadoop.ParquetFileWriter;
import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.ParquetWriter;
@@ -51,9 +51,9 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
private final Schema schema; private final Schema schema;
private static Configuration registerFileSystem(Configuration conf) { private static Configuration registerFileSystem(Path file, Configuration conf) {
Configuration returnConf = new Configuration(conf); Configuration returnConf = new Configuration(conf);
String scheme = FileSystem.getDefaultUri(conf).getScheme(); String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl", returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
HoodieWrapperFileSystem.class.getName()); HoodieWrapperFileSystem.class.getName());
return returnConf; return returnConf;
@@ -67,11 +67,11 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
parquetConfig.getPageSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION,
registerFileSystem(parquetConfig.getHadoopConf())); registerFileSystem(file, parquetConfig.getHadoopConf()));
this.file = this.file =
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()); HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
this.fs = (HoodieWrapperFileSystem) this.file this.fs = (HoodieWrapperFileSystem) this.file
.getFileSystem(registerFileSystem(parquetConfig.getHadoopConf())); .getFileSystem(registerFileSystem(file, parquetConfig.getHadoopConf()));
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10% // We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS // TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
this.maxFileSize = parquetConfig.getMaxFileSize() + Math this.maxFileSize = parquetConfig.getMaxFileSize() + Math

View File

@@ -19,7 +19,6 @@ package com.uber.hoodie.io.storage;
import com.uber.hoodie.avro.HoodieAvroWriteSupport; import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter; import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable; import com.uber.hoodie.table.HoodieTable;
import java.io.IOException; import java.io.IOException;
@@ -37,11 +36,12 @@ public class HoodieStorageWriterFactory {
throws IOException { throws IOException {
//TODO - based on the metadata choose the implementation of HoodieStorageWriter //TODO - based on the metadata choose the implementation of HoodieStorageWriter
// Currently only parquet is supported // Currently only parquet is supported
return newParquetStorageWriter(commitTime, path, config, schema); return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable);
} }
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter( private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
String commitTime, Path path, HoodieWriteConfig config, Schema schema) throws IOException { String commitTime, Path path, HoodieWriteConfig config, Schema schema,
HoodieTable hoodieTable) throws IOException {
BloomFilter filter = BloomFilter filter =
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP()); new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
HoodieAvroWriteSupport writeSupport = HoodieAvroWriteSupport writeSupport =
@@ -50,7 +50,7 @@ public class HoodieStorageWriterFactory {
HoodieParquetConfig parquetConfig = HoodieParquetConfig parquetConfig =
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetBlockSize(), config.getParquetPageSize(),
config.getParquetMaxFileSize(), FSUtils.getFs().getConf()); config.getParquetMaxFileSize(), hoodieTable.getHadoopConf());
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema); return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
} }

View File

@@ -16,7 +16,8 @@
package com.uber.hoodie.io.storage; 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.io.IOException;
import java.net.URI; import java.net.URI;
import java.net.URISyntaxException; import java.net.URISyntaxException;
@@ -33,7 +34,6 @@ import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@@ -41,17 +41,14 @@ import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.FsStatus; import org.apache.hadoop.fs.FsStatus;
import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.fs.XAttrSetFlag; import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
@@ -70,6 +67,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
SUPPORT_SCHEMES.add("file"); SUPPORT_SCHEMES.add("file");
SUPPORT_SCHEMES.add("hdfs"); SUPPORT_SCHEMES.add("hdfs");
SUPPORT_SCHEMES.add("s3"); SUPPORT_SCHEMES.add("s3");
SUPPORT_SCHEMES.add("s3a");
// Hoodie currently relies on underlying object store being fully // Hoodie currently relies on underlying object store being fully
// consistent so only regional buckets should be used. // consistent so only regional buckets should be used.
@@ -85,7 +84,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override @Override
public void initialize(URI uri, Configuration conf) throws IOException { public void initialize(URI uri, Configuration conf) throws IOException {
// Get the default filesystem to decorate // 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 // Do not need to explicitly initialize the default filesystem, its done already in the above FileSystem.get
// fileSystem.initialize(FileSystem.getDefaultUri(conf), conf); // fileSystem.initialize(FileSystem.getDefaultUri(conf), conf);
// fileSystem.setConf(conf); // fileSystem.setConf(conf);
@@ -219,7 +223,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
} }
@Override @Override
public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { public FileStatus[] listStatus(Path f) throws IOException {
return fileSystem.listStatus(convertToDefaultPath(f)); return fileSystem.listStatus(convertToDefaultPath(f));
} }
@@ -415,19 +419,19 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override @Override
public FileStatus[] listStatus(Path f, PathFilter filter) public FileStatus[] listStatus(Path f, PathFilter filter)
throws FileNotFoundException, IOException { throws IOException {
return fileSystem.listStatus(convertToDefaultPath(f), filter); return fileSystem.listStatus(convertToDefaultPath(f), filter);
} }
@Override @Override
public FileStatus[] listStatus(Path[] files) public FileStatus[] listStatus(Path[] files)
throws FileNotFoundException, IOException { throws IOException {
return fileSystem.listStatus(convertDefaults(files)); return fileSystem.listStatus(convertDefaults(files));
} }
@Override @Override
public FileStatus[] listStatus(Path[] files, PathFilter filter) public FileStatus[] listStatus(Path[] files, PathFilter filter)
throws FileNotFoundException, IOException { throws IOException {
return fileSystem.listStatus(convertDefaults(files), filter); return fileSystem.listStatus(convertDefaults(files), filter);
} }
@@ -444,13 +448,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override @Override
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f) public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
throws FileNotFoundException, IOException { throws IOException {
return fileSystem.listLocatedStatus(convertToDefaultPath(f)); return fileSystem.listLocatedStatus(convertToDefaultPath(f));
} }
@Override @Override
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive) public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive)
throws FileNotFoundException, IOException { throws IOException {
return fileSystem.listFiles(convertToDefaultPath(f), recursive); return fileSystem.listFiles(convertToDefaultPath(f), recursive);
} }
@@ -571,21 +575,21 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override @Override
public void access(Path path, FsAction mode) public void access(Path path, FsAction mode)
throws AccessControlException, FileNotFoundException, IOException { throws IOException {
fileSystem.access(convertToDefaultPath(path), mode); fileSystem.access(convertToDefaultPath(path), mode);
} }
@Override @Override
public void createSymlink(Path target, Path link, boolean createParent) public void createSymlink(Path target, Path link, boolean createParent)
throws AccessControlException, FileAlreadyExistsException, FileNotFoundException, throws
ParentNotDirectoryException, UnsupportedFileSystemException, IOException { IOException {
fileSystem fileSystem
.createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent); .createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent);
} }
@Override @Override
public FileStatus getFileLinkStatus(Path f) public FileStatus getFileLinkStatus(Path f)
throws AccessControlException, FileNotFoundException, UnsupportedFileSystemException, throws
IOException { IOException {
return fileSystem.getFileLinkStatus(convertToDefaultPath(f)); return fileSystem.getFileLinkStatus(convertToDefaultPath(f));
} }
@@ -759,8 +763,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
} }
public static Path convertToHoodiePath(Path file, Configuration conf) { public static Path convertToHoodiePath(Path file, Configuration conf) {
String scheme = FileSystem.getDefaultUri(conf).getScheme(); try {
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
return convertPathWithScheme(file, getHoodieScheme(scheme)); return convertPathWithScheme(file, getHoodieScheme(scheme));
} catch (HoodieIOException e) {
throw e;
}
} }
private Path convertToDefaultPath(Path oldPath) { private Path convertToDefaultPath(Path oldPath) {

View File

@@ -52,7 +52,6 @@ import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@@ -413,10 +412,10 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
throw new HoodieUpsertException("Error in finding the old file path at commit " + throw new HoodieUpsertException("Error in finding the old file path at commit " +
commitTime + " at fileLoc: " + fileLoc); commitTime + " at fileLoc: " + fileLoc);
} else { } else {
Configuration conf = FSUtils.getFs().getConf(); AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
AvroReadSupport.setAvroReadSchema(conf, upsertHandle.getSchema());
ParquetReader<IndexedRecord> reader = ParquetReader<IndexedRecord> reader =
AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(conf).build(); AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(getHadoopConf())
.build();
try { try {
IndexedRecord record; IndexedRecord record;
while ((record = reader.read()) != null) { while ((record = reader.read()) != null) {
@@ -500,8 +499,9 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
@Override @Override
public List<HoodieCleanStat> clean(JavaSparkContext jsc) { public List<HoodieCleanStat> clean(JavaSparkContext jsc) {
try { try {
FileSystem fs = getMetaClient().getFs();
List<String> partitionsToClean = List<String> partitionsToClean =
FSUtils.getAllPartitionPaths(getFs(), getMetaClient().getBasePath(), FSUtils.getAllPartitionPaths(fs, getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning()); config.shouldAssumeDatePartitioning());
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config
.getCleanerPolicy()); .getCleanerPolicy());
@@ -522,7 +522,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits) protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits)
throws IOException { throws IOException {
logger.info("Cleaning path " + partitionPath); logger.info("Cleaning path " + partitionPath);
FileSystem fs = FSUtils.getFs(); FileSystem fs = getMetaClient().getFs();
FileStatus[] toBeDeleted = FileStatus[] toBeDeleted =
fs.listStatus(new Path(config.getBasePath(), partitionPath), path -> { fs.listStatus(new Path(config.getBasePath(), partitionPath), path -> {
if (!path.toString().contains(".parquet")) { if (!path.toString().contains(".parquet")) {
@@ -558,7 +558,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
// delete all the data files for all these commits // delete all the data files for all these commits
logger.info("Clean out all parquet files generated for commits: " + commits); logger.info("Clean out all parquet files generated for commits: " + commits);
List<HoodieRollbackStat> stats = jsc.parallelize( List<HoodieRollbackStat> stats = jsc.parallelize(
FSUtils.getAllPartitionPaths(FSUtils.getFs(), this.getMetaClient().getBasePath(), FSUtils.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning())) config.shouldAssumeDatePartitioning()))
.map((Function<String, HoodieRollbackStat>) partitionPath -> { .map((Function<String, HoodieRollbackStat>) partitionPath -> {
// Scan all partitions files with this commit time // Scan all partitions files with this commit time
@@ -618,7 +618,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
.parallelize(partitionsToClean, cleanerParallelism) .parallelize(partitionsToClean, cleanerParallelism)
.flatMapToPair(getFilesToDeleteFunc(this, config)) .flatMapToPair(getFilesToDeleteFunc(this, config))
.repartition(cleanerParallelism) // repartition to remove skews .repartition(cleanerParallelism) // repartition to remove skews
.mapPartitionsToPair(deleteFilesFunc(this, config)) .mapPartitionsToPair(deleteFilesFunc(this))
.reduceByKey( .reduceByKey(
// merge partition level clean stats below // merge partition level clean stats below
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1 (Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1
@@ -646,17 +646,17 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}).collect(Collectors.toList()); }).collect(Collectors.toList());
} }
private PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat> deleteFilesFunc( private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat> deleteFilesFunc(
HoodieTable table, HoodieWriteConfig config) { HoodieTable table) {
return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>) iter -> { return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>) iter -> {
HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config);
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>(); Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
FileSystem fs = table.getMetaClient().getFs();
while (iter.hasNext()) { while (iter.hasNext()) {
Tuple2<String, String> partitionDelFileTuple = iter.next(); Tuple2<String, String> partitionDelFileTuple = iter.next();
String partitionPath = partitionDelFileTuple._1(); String partitionPath = partitionDelFileTuple._1();
String deletePathStr = partitionDelFileTuple._2(); String deletePathStr = partitionDelFileTuple._2();
Boolean deletedFileResult = deleteFileAndGetResult(deletePathStr); Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
if (!partitionCleanStatMap.containsKey(partitionPath)) { if (!partitionCleanStatMap.containsKey(partitionPath)) {
partitionCleanStatMap.put(partitionPath, partitionCleanStatMap.put(partitionPath,
new PartitionCleanStat(partitionPath)); new PartitionCleanStat(partitionPath));
@@ -682,10 +682,11 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}; };
} }
private Boolean deleteFileAndGetResult(String deletePathStr) throws IOException { private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr)
throws IOException {
Path deletePath = new Path(deletePathStr); Path deletePath = new Path(deletePathStr);
logger.debug("Working on delete path :" + deletePath); logger.debug("Working on delete path :" + deletePath);
boolean deleteResult = getFs().delete(deletePath, false); boolean deleteResult = fs.delete(deletePath, false);
if (deleteResult) { if (deleteResult) {
logger.debug("Cleaned file at path :" + deletePath); logger.debug("Cleaned file at path :" + deletePath);
} }

View File

@@ -32,19 +32,11 @@ import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock; import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant; 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.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.exception.HoodieRollbackException; import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.io.HoodieAppendHandle; import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; 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.IOException;
import java.io.UncheckedIOException; import java.io.UncheckedIOException;
import java.util.Arrays; import java.util.Arrays;
@@ -55,6 +47,12 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
/** /**
@@ -74,8 +72,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class); private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
public HoodieMergeOnReadTable(HoodieWriteConfig config, public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
HoodieTableMetaClient metaClient) {
super(config, metaClient); super(config, metaClient);
} }
@@ -195,7 +192,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.onParentPath( .onParentPath(
new Path(this.getMetaClient().getBasePath(), partitionPath)) new Path(this.getMetaClient().getBasePath(), partitionPath))
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit()) .withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
.withFs(FSUtils.getFs()) .withFs(getMetaClient().getFs())
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
Long numRollbackBlocks = 0L; Long numRollbackBlocks = 0L;
// generate metadata // generate metadata
@@ -215,7 +212,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
numRollbackBlocks++; numRollbackBlocks++;
} }
filesToNumBlocksRollback filesToNumBlocksRollback
.put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()), .put(getMetaClient().getFs()
.getFileStatus(writer.getLogFile().getPath()),
numRollbackBlocks); numRollbackBlocks);
} catch (IOException | InterruptedException io) { } catch (IOException | InterruptedException io) {
throw new HoodieRollbackException( throw new HoodieRollbackException(

View File

@@ -34,12 +34,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException; import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieSavepointException; 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.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.util.Iterator; import java.util.Iterator;
@@ -47,6 +41,11 @@ import java.util.List;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.Stream; 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 * Abstract implementation of a HoodieTable
@@ -55,6 +54,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
protected final HoodieWriteConfig config; protected final HoodieWriteConfig config;
protected final HoodieTableMetaClient metaClient; protected final HoodieTableMetaClient metaClient;
private static Logger logger = LogManager.getLogger(HoodieTable.class); private static Logger logger = LogManager.getLogger(HoodieTable.class);
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) { protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
@@ -87,8 +87,8 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
return metaClient; return metaClient;
} }
public FileSystem getFs() { public Configuration getHadoopConf() {
return metaClient.getFs(); return metaClient.getHadoopConf();
} }
/** /**

View File

@@ -81,7 +81,7 @@ public class HoodieClientExample {
// initialize the table, if not done already // initialize the table, if not done already
Path path = new Path(tablePath); Path path = new Path(tablePath);
FileSystem fs = FSUtils.getFs(); FileSystem fs = FSUtils.getFs(tablePath, jsc.hadoopConfiguration());
if (!fs.exists(path)) { if (!fs.exists(path)) {
HoodieTableMetaClient HoodieTableMetaClient
.initTableType(fs, tablePath, HoodieTableType.valueOf(tableType), tableName, .initTableType(fs, tablePath, HoodieTableType.valueOf(tableType), tableName,

View File

@@ -84,6 +84,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
private transient JavaSparkContext jsc = null; private transient JavaSparkContext jsc = null;
private transient SQLContext sqlContext; private transient SQLContext sqlContext;
private transient FileSystem fs;
private String basePath = null; private String basePath = null;
private transient HoodieTestDataGenerator dataGen = null; private transient HoodieTestDataGenerator dataGen = null;
private String[] partitionPaths = {"2016/01/01", "2016/02/02", "2016/06/02"}; private String[] partitionPaths = {"2016/01/01", "2016/02/02", "2016/06/02"};
@@ -100,6 +101,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
TemporaryFolder folder = new TemporaryFolder(); TemporaryFolder folder = new TemporaryFolder();
folder.create(); folder.create();
basePath = folder.getRoot().getAbsolutePath(); basePath = folder.getRoot().getAbsolutePath();
fs = FSUtils.getFs(basePath.toString(), jsc.hadoopConfiguration());
HoodieTestUtils.init(basePath); HoodieTestUtils.init(basePath);
dataGen = new HoodieTestDataGenerator(); dataGen = new HoodieTestDataGenerator();
} }
@@ -212,7 +214,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieWriteConfig cfg = getConfig(); HoodieWriteConfig cfg = getConfig();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
FileSystem fs = FSUtils.getFs();
/** /**
* Write 1 (only inserts) * Write 1 (only inserts)
@@ -230,8 +231,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs); assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs);
// verify that there is a commit // verify that there is a commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()) basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
.getCommitTimeline(); .getCommitTimeline();
assertEquals("Expecting a single commit.", 1, assertEquals("Expecting a single commit.", 1,
@@ -242,7 +244,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
records.size(), records.size(),
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
// Should have 100 records in table (check using Index), all in locations marked at commit // Should have 100 records in table (check using Index), all in locations marked at commit
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); HoodieTable table = HoodieTable
.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table) List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table)
.collect(); .collect();
@@ -268,13 +271,13 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
// verify there are now 2 commits // verify there are now 2 commits
timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline(); timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
assertEquals("Expecting two commits.", assertEquals("Expecting two commits.",
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2);
assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(),
newCommitTime); newCommitTime);
metaClient = new HoodieTableMetaClient(fs, basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig()); table = HoodieTable.getHoodieTable(metaClient, getConfig());
// Index should be able to locate all updates in correct locations. // Index should be able to locate all updates in correct locations.
@@ -305,7 +308,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieWriteConfig cfg = getConfig(); HoodieWriteConfig cfg = getConfig();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
FileSystem fs = FSUtils.getFs();
/** /**
* Write 1 (inserts and deletes) * Write 1 (inserts and deletes)
@@ -326,8 +328,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
// verify that there is a commit // verify that there is a commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()) basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
.getCommitTimeline(); .getCommitTimeline();
assertEquals("Expecting a single commit.", 1, assertEquals("Expecting a single commit.", 1,
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
@@ -336,7 +339,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertEquals("Must contain 200 records", fewRecordsForInsert.size(), assertEquals("Must contain 200 records", fewRecordsForInsert.size(),
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count()); HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
// Should have 100 records in table (check using Index), all in locations marked at commit // Should have 100 records in table (check using Index), all in locations marked at commit
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); HoodieTable table = HoodieTable
.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index List<HoodieRecord> taggedRecords = index
.tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect(); .tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect();
@@ -359,7 +363,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
// verify there are now 2 commits // verify there are now 2 commits
timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline(); timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
assertEquals("Expecting two commits.", assertEquals("Expecting two commits.",
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2); timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2);
assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(),
@@ -393,7 +397,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
.build()).build(); .build()).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
FileSystem fs = FSUtils.getFs();
HoodieTestDataGenerator HoodieTestDataGenerator
.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); .writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
@@ -451,8 +454,10 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
List<String> partitionPaths = FSUtils List<String> partitionPaths = FSUtils
.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); .getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); basePath);
HoodieTable table = HoodieTable
.getHoodieTable(metaClient, getConfig());
final TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView(); final TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView();
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> { List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002")); return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
@@ -472,7 +477,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Verify there are no errors // Verify there are no errors
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig()); table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> { dataFiles = partitionPaths.stream().flatMap(s -> {
@@ -490,7 +495,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
.build()).build(); .build()).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
FileSystem fs = FSUtils.getFs();
HoodieTestDataGenerator HoodieTestDataGenerator
.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath); .writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
@@ -531,7 +535,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
List<String> partitionPaths = FSUtils List<String> partitionPaths = FSUtils
.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); .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()); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView(); final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView();
@@ -551,7 +556,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Verify there are no errors // Verify there are no errors
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig()); table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView(); final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView();
@@ -573,7 +578,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
table.getCompletedSavepointTimeline().getInstants().findFirst().get(); table.getCompletedSavepointTimeline().getInstants().findFirst().get();
client.rollbackToSavepoint(savepoint.getTimestamp()); client.rollbackToSavepoint(savepoint.getTimestamp());
metaClient = new HoodieTableMetaClient(fs, basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig()); table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView(); final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> { dataFiles = partitionPaths.stream().flatMap(s -> {
@@ -602,7 +607,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
.retainFileVersions(maxVersions).build()).build(); .retainFileVersions(maxVersions).build()).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
FileSystem fs = FSUtils.getFs();
/** /**
* do a big insert * do a big insert
@@ -618,8 +622,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
// verify that there is a commit // verify that there is a commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()) basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
.getCommitTimeline(); .getCommitTimeline();
assertEquals("Expecting a single commit.", 1, assertEquals("Expecting a single commit.", 1,
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
@@ -647,7 +652,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Verify there are no errors // Verify there are no errors
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
table = HoodieTable.getHoodieTable(metadata, getConfig()); table = HoodieTable.getHoodieTable(metadata, getConfig());
timeline = table.getCommitsTimeline(); timeline = table.getCommitsTimeline();
@@ -702,7 +708,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
.retainCommits(maxCommits).build()).build(); .retainCommits(maxCommits).build()).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc); HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
FileSystem fs = FSUtils.getFs();
/** /**
* do a big insert * do a big insert
@@ -718,13 +723,15 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
// verify that there is a commit // verify that there is a commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()) basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
.getCommitTimeline(); .getCommitTimeline();
assertEquals("Expecting a single commit.", 1, assertEquals("Expecting a single commit.", 1,
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
// Should have 100 records in table (check using Index), all in locations marked at commit // 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()); assertFalse(table.getCompletedCommitTimeline().empty());
String commitTime = String commitTime =
@@ -747,7 +754,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Verify there are no errors // Verify there are no errors
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg); HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg);
HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline(); HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline();
Optional<HoodieInstant> Optional<HoodieInstant>
@@ -788,8 +796,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
String commitTime2 = "20160502020601"; String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611"; String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs(); new File(basePath + "/.hoodie").mkdirs();
HoodieTestDataGenerator.writePartitionMetadata(FSUtils.getFs(), HoodieTestDataGenerator
new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, .writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
basePath); basePath);
// Only first two have commit files // Only first two have commit files
@@ -878,8 +886,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
String commitTime2 = "20160502020601"; String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611"; String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs(); new File(basePath + "/.hoodie").mkdirs();
HoodieTestDataGenerator.writePartitionMetadata(FSUtils.getFs(), HoodieTestDataGenerator
new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, .writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
basePath); basePath);
// One good commit // One good commit
@@ -956,8 +964,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
@Test @Test
public void testSmallInsertHandlingForUpserts() throws Exception { public void testSmallInsertHandlingForUpserts() throws Exception {
FileSystem fs = FSUtils.getFs();
final String TEST_PARTITION_PATH = "2016/09/26"; final String TEST_PARTITION_PATH = "2016/09/26";
final int INSERT_SPLIT_LIMIT = 100; final int INSERT_SPLIT_LIMIT = 100;
// setup the small file handling params // setup the small file handling params
@@ -982,7 +988,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertEquals("Just 1 file needs to be added.", 1, statuses.size()); assertEquals("Just 1 file needs to be added.", 1, statuses.size());
String file1 = statuses.get(0).getFileId(); String file1 = statuses.get(0).getFileId();
assertEquals("file should contain 100 records", 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(), TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(),
100); 100);
@@ -1006,9 +1012,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
Path newFile = new Path(basePath, Path newFile = new Path(basePath,
TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1));
assertEquals("file should contain 140 records", assertEquals("file should contain 140 records",
ParquetUtils.readRowKeysFromParquet(newFile).size(), 140); ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
List<GenericRecord> records = ParquetUtils.readAvroRecords(newFile); List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile);
for (GenericRecord record : records) { for (GenericRecord record : records) {
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
assertEquals("only expect commit2", commitTime2, assertEquals("only expect commit2", commitTime2,
@@ -1030,7 +1036,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
assertEquals("2 files needs to be committed.", 2, statuses.size()); 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); HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView(); TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView();
List<HoodieDataFile> files = fileSystemView List<HoodieDataFile> files = fileSystemView
@@ -1040,7 +1046,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
for (HoodieDataFile file : files) { for (HoodieDataFile file : files) {
if (file.getFileName().contains(file1)) { if (file.getFileName().contains(file1)) {
assertEquals("Existing file should be expanded", commitTime3, file.getCommitTime()); 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) { for (GenericRecord record : records) {
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String recordCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); String recordCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
@@ -1057,7 +1063,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
keys2.size()); keys2.size());
} else { } else {
assertEquals("New file must be written for commit 3", commitTime3, file.getCommitTime()); 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) { for (GenericRecord record : records) {
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
assertEquals("only expect commit3", commitTime3, assertEquals("only expect commit3", commitTime3,
@@ -1091,12 +1097,12 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
List<WriteStatus> statuses = client.insert(insertRecordsRDD1, commitTime1).collect(); List<WriteStatus> statuses = client.insert(insertRecordsRDD1, commitTime1).collect();
assertNoWriteErrors(statuses); 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()); assertEquals("Just 1 file needs to be added.", 1, statuses.size());
String file1 = statuses.get(0).getFileId(); String file1 = statuses.get(0).getFileId();
assertEquals("file should contain 100 records", 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(), TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(),
100); 100);
@@ -1116,9 +1122,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
Path newFile = new Path(basePath, Path newFile = new Path(basePath,
TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1)); TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1));
assertEquals("file should contain 140 records", assertEquals("file should contain 140 records",
ParquetUtils.readRowKeysFromParquet(newFile).size(), 140); ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
List<GenericRecord> records = ParquetUtils.readAvroRecords(newFile); List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile);
for (GenericRecord record : records) { for (GenericRecord record : records) {
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String recCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); String recCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
@@ -1137,8 +1143,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
assertEquals("2 files needs to be committed.", 2, statuses.size()); assertEquals("2 files needs to be committed.", 2, statuses.size());
FileSystem fs = FSUtils.getFs(); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
List<HoodieDataFile> files = List<HoodieDataFile> files =
table.getROFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3) table.getROFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3)
@@ -1148,7 +1154,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
int totalInserts = 0; int totalInserts = 0;
for (HoodieDataFile file : files) { for (HoodieDataFile file : files) {
assertEquals("All files must be at commit 3", commitTime3, file.getCommitTime()); 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(); totalInserts += records.size();
} }
assertEquals("Total number of records must add up", totalInserts, assertEquals("Total number of records must add up", totalInserts,
@@ -1169,7 +1175,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000"); String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
HoodieTable table = HoodieTable HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), .getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc); List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
@@ -1183,7 +1190,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make next commit, with 1 insert & 1 update per partition // make next commit, with 1 insert & 1 update per partition
HoodieTestUtils.createCommitFiles(basePath, "001"); HoodieTestUtils.createCommitFiles(basePath, "001");
table = HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), .getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true),
config); config);
String file2P0C1 = HoodieTestUtils String file2P0C1 = HoodieTestUtils
@@ -1206,7 +1214,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make next commit, with 2 updates to existing files, and 1 insert // make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "002"); HoodieTestUtils.createCommitFiles(basePath, "002");
table = HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), .getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config);
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
@@ -1260,8 +1269,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make 1 compaction commit // make 1 compaction commit
HoodieTestUtils.createCompactionCommitFiles(basePath, "001"); HoodieTestUtils.createCompactionCommitFiles(basePath, "001");
HoodieTable table = HoodieTable HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
.getHoodieTable(metaClient, config);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc); List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
assertEquals("Must clean three files, one parquet and 2 log files", 3, assertEquals("Must clean three files, one parquet and 2 log files", 3,
getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size()); getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size());
@@ -1286,9 +1294,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000"); String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000"); String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
HoodieTable table = HoodieTable HoodieTable table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
config);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc); List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
assertEquals("Must not clean any files", 0, assertEquals("Must not clean any files", 0,
@@ -1300,9 +1307,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make next commit, with 1 insert & 1 update per partition // make next commit, with 1 insert & 1 update per partition
HoodieTestUtils.createCommitFiles(basePath, "001"); HoodieTestUtils.createCommitFiles(basePath, "001");
table = HoodieTable table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
config);
String file2P0C1 = HoodieTestUtils String file2P0C1 = HoodieTestUtils
.createNewDataFile(basePath, partitionPaths[0], "001"); // insert .createNewDataFile(basePath, partitionPaths[0], "001"); // insert
@@ -1324,7 +1330,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make next commit, with 2 updates to existing files, and 1 insert // make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "002"); HoodieTestUtils.createCommitFiles(basePath, "002");
table = HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), .getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config);
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
@@ -1341,7 +1348,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make next commit, with 2 updates to existing files, and 1 insert // make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "003"); HoodieTestUtils.createCommitFiles(basePath, "003");
table = HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), .getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config);
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update
@@ -1384,7 +1392,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieTestUtils.createCommitFiles(basePath, "000"); HoodieTestUtils.createCommitFiles(basePath, "000");
HoodieTable table = HoodieTable HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), .getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc); List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
@@ -1449,7 +1458,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
updateAllFilesInPartition(filesP2C0, partitionPaths[2], "003"); updateAllFilesInPartition(filesP2C0, partitionPaths[2], "003");
HoodieTable table = HoodieTable HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), .getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config); config);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc); List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
@@ -1476,8 +1486,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build(); HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg); HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
FileSystem fs = FSUtils.getFs(); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg);
String commitTime = "000"; String commitTime = "000";

View File

@@ -0,0 +1,159 @@
/*
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie;
import static org.junit.Assert.assertEquals;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.minicluster.HdfsTestService;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import java.io.Serializable;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
public class TestMultiFS implements Serializable {
private static String dfsBasePath;
private static HdfsTestService hdfsTestService;
private static MiniDFSCluster dfsCluster;
private static DistributedFileSystem dfs;
private static Logger logger = LogManager.getLogger(TestMultiFS.class);
private String tablePath = "file:///tmp/hoodie/sample-table";
private String tableName = "hoodie_rt";
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
private static JavaSparkContext jsc;
private static SQLContext sqlContext;
@Before
public void initClass() throws Exception {
hdfsTestService = new HdfsTestService();
dfsCluster = hdfsTestService.start(true);
// Create a temp folder as the base path
dfs = dfsCluster.getFileSystem();
dfsBasePath = dfs.getWorkingDirectory().toString();
dfs.mkdirs(new Path(dfsBasePath));
SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example");
sparkConf.setMaster("local[1]");
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
sparkConf.set("spark.kryoserializer.buffer.max", "512m");
jsc = new JavaSparkContext(sparkConf);
sqlContext = new SQLContext(jsc);
}
@After
public void cleanupClass() throws Exception {
if (hdfsTestService != null) {
hdfsTestService.stop();
}
if (jsc != null) {
jsc.stop();
}
FSUtils.setFs(null);
}
@Test
public void readLocalWriteHDFS() throws Exception {
// Generator of some records to be loaded in.
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
// Initialize table and filesystem
FileSystem hdfs = FSUtils.getFs(dfsBasePath, jsc.hadoopConfiguration());
HoodieTableMetaClient
.initTableType(hdfs, dfsBasePath, HoodieTableType.valueOf(tableType), tableName,
HoodieAvroPayload.class.getName());
//Create write client to write some records in
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(dfsBasePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable(tableName).withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.build();
HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc, cfg);
// Write generated data to hdfs (only inserts)
String readCommitTime = hdfsWriteClient.startCommit();
logger.info("Starting commit " + readCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(readCommitTime, 100);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
hdfsWriteClient.upsert(writeRecords, readCommitTime);
// Read from hdfs
FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
.getCommitTimeline();
Dataset<Row> readRecords = HoodieClientTestUtils
.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime);
assertEquals("Should contain 100 records", readRecords.count(), records.size());
// Write to local
FileSystem local = FSUtils.getFs(tablePath, jsc.hadoopConfiguration());
HoodieTableMetaClient
.initTableType(local, tablePath, HoodieTableType.valueOf(tableType), tableName,
HoodieAvroPayload.class.getName());
HoodieWriteConfig localConfig = HoodieWriteConfig.newBuilder().withPath(tablePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable(tableName).withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.build();
HoodieWriteClient localWriteClient = new HoodieWriteClient(jsc, localConfig);
String writeCommitTime = localWriteClient.startCommit();
logger.info("Starting write commit " + writeCommitTime);
List<HoodieRecord> localRecords = dataGen.generateInserts(writeCommitTime, 100);
JavaRDD<HoodieRecord> localWriteRecords = jsc.parallelize(localRecords, 1);
logger.info("Writing to path: " + tablePath);
localWriteClient.upsert(localWriteRecords, writeCommitTime);
logger.info("Reading from path: " + tablePath);
fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf());
metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath);
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
Dataset<Row> localReadRecords = HoodieClientTestUtils
.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size());
}
}

View File

@@ -174,7 +174,7 @@ public class HoodieClientTestUtils {
List<String> filteredPaths = new ArrayList<>(); List<String> filteredPaths = new ArrayList<>();
try { try {
HoodieTable hoodieTable = HoodieTable HoodieTable hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null); .getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null);
for (String path : paths) { for (String path : paths) {
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView( TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(
hoodieTable.getMetaClient(), hoodieTable.getMetaClient(),

View File

@@ -18,7 +18,7 @@ package com.uber.hoodie.common;
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; 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.common.util.HoodieAvroUtils;
import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat; import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
import java.io.IOException; import java.io.IOException;
@@ -42,12 +42,13 @@ import org.apache.hadoop.mapred.RecordReader;
*/ */
public class HoodieMergeOnReadTestUtils { public class HoodieMergeOnReadTestUtils {
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths) public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths,
String basePath)
throws IOException { throws IOException {
JobConf jobConf = new JobConf(); JobConf jobConf = new JobConf();
Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA)); Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA));
HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat(); HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
setPropsForInputFormat(inputFormat, jobConf, schema); setPropsForInputFormat(inputFormat, jobConf, schema, basePath);
return inputPaths.stream().map(path -> { return inputPaths.stream().map(path -> {
setInputPath(jobConf, path); setInputPath(jobConf, path);
List<GenericRecord> records = new ArrayList<>(); List<GenericRecord> records = new ArrayList<>();
@@ -76,12 +77,12 @@ public class HoodieMergeOnReadTestUtils {
} }
private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf,
Schema schema) { Schema schema, String basePath) {
List<Schema.Field> fields = schema.getFields(); List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())) String postions = fields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(",")); .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_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions); jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr"); jobConf.set("partition_columns", "datestr");

View File

@@ -20,6 +20,7 @@ import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.model.HoodieRecord; 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.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
@@ -193,7 +194,7 @@ public class HoodieTestDataGenerator {
Path commitFile = Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCommitFileName(commitTime)); .makeCommitFileName(commitTime));
FileSystem fs = FSUtils.getFs(); FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
FSDataOutputStream os = fs.create(commitFile, true); FSDataOutputStream os = fs.create(commitFile, true);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
try { try {
@@ -209,7 +210,7 @@ public class HoodieTestDataGenerator {
Path commitFile = Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeSavePointFileName(commitTime)); .makeSavePointFileName(commitTime));
FileSystem fs = FSUtils.getFs(); FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
FSDataOutputStream os = fs.create(commitFile, true); FSDataOutputStream os = fs.create(commitFile, true);
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
try { try {

View File

@@ -55,8 +55,9 @@ public class TestUpdateMapFunction {
public void testSchemaEvolutionOnUpdate() throws Exception { public void testSchemaEvolutionOnUpdate() throws Exception {
// Create a bunch of records with a old version of schema // Create a bunch of records with a old version of schema
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt"); HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieTestUtils.getDefaultHadoopConf(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient);
String recordStr1 = String recordStr1 =
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
@@ -80,16 +81,16 @@ public class TestUpdateMapFunction {
Iterator<List<WriteStatus>> insertResult = table.handleInsert("100", records.iterator()); Iterator<List<WriteStatus>> insertResult = table.handleInsert("100", records.iterator());
Path commitFile = Path commitFile =
new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100")); 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 // Now try an update with an evolved schema
// Evolved schema does not have guarantee on preserving the original field ordering // Evolved schema does not have guarantee on preserving the original field ordering
config = makeHoodieClientConfig("/exampleEvolvedSchema.txt"); config = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath);
String fileId = insertResult.next().get(0).getFileId(); String fileId = insertResult.next().get(0).getFileId();
System.out.println(fileId); System.out.println(fileId);
table = new HoodieCopyOnWriteTable(config, metadata); table = new HoodieCopyOnWriteTable(config, metaClient);
// New content with values for the newly added field // New content with values for the newly added field
recordStr1 = recordStr1 =
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}"; "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";

View File

@@ -54,7 +54,6 @@ import java.util.stream.Collectors;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.avro.AvroSchemaConverter;
@@ -73,12 +72,11 @@ public class TestHoodieBloomIndex {
private JavaSparkContext jsc = null; private JavaSparkContext jsc = null;
private String basePath = null; private String basePath = null;
private transient final FileSystem fs; private transient FileSystem fs;
private String schemaStr; private String schemaStr;
private Schema schema; private Schema schema;
public TestHoodieBloomIndex() throws Exception { public TestHoodieBloomIndex() throws Exception {
fs = FSUtils.getFs();
} }
@Before @Before
@@ -89,6 +87,7 @@ public class TestHoodieBloomIndex {
TemporaryFolder folder = new TemporaryFolder(); TemporaryFolder folder = new TemporaryFolder();
folder.create(); folder.create();
basePath = folder.getRoot().getAbsolutePath(); basePath = folder.getRoot().getAbsolutePath();
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
HoodieTestUtils.init(basePath); HoodieTestUtils.init(basePath);
// We have some records to be tagged (two different partitions) // We have some records to be tagged (two different partitions)
schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8"); schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
@@ -120,8 +119,6 @@ public class TestHoodieBloomIndex {
.parallelize(Arrays.asList(record1, record2, record3, record4)); .parallelize(Arrays.asList(record1, record2, record3, record4));
// Load to memory // Load to memory
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
Map<String, Iterable<String>> map = recordRDD Map<String, Iterable<String>> map = recordRDD
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey())) .mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()))
.groupByKey().collectAsMap(); .groupByKey().collectAsMap();
@@ -174,7 +171,7 @@ public class TestHoodieBloomIndex {
Arrays.asList(record2, record3, record4), schema, null, false); Arrays.asList(record2, record3, record4), schema, null, false);
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12"); List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, table); List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, table);
// Still 0, as no valid commit // Still 0, as no valid commit
@@ -291,7 +288,8 @@ public class TestHoodieBloomIndex {
List<String> uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), List<String> uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(),
record3.getRecordKey(), record4.getRecordKey()); record3.getRecordKey(), record4.getRecordKey());
List<String> results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(uuids, List<String> results = HoodieBloomIndexCheckFunction
.checkCandidatesAgainstFile(jsc.hadoopConfiguration(), uuids,
new Path(basePath + "/2016/01/31/" + filename)); new Path(basePath + "/2016/01/31/" + filename));
assertEquals(results.size(), 2); assertEquals(results.size(), 2);
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0") assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
@@ -308,7 +306,7 @@ public class TestHoodieBloomIndex {
// We have some records to be tagged (two different partitions) // We have some records to be tagged (two different partitions)
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD(); JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
// Also create the metadata and config // 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(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
@@ -348,7 +346,7 @@ public class TestHoodieBloomIndex {
.parallelize(Arrays.asList(record1, record2, record3, record4)); .parallelize(Arrays.asList(record1, record2, record3, record4));
// Also create the metadata and config // 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(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
@@ -367,7 +365,7 @@ public class TestHoodieBloomIndex {
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true); String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
// We do the tag again // We do the tag again
metadata = new HoodieTableMetaClient(fs, basePath); metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metadata, config); table = HoodieTable.getHoodieTable(metadata, config);
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table); taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
@@ -409,7 +407,7 @@ public class TestHoodieBloomIndex {
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4)); JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
// Also create the metadata and config // 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(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
@@ -429,7 +427,7 @@ public class TestHoodieBloomIndex {
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true); String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
// We do the tag again // We do the tag again
metadata = new HoodieTableMetaClient(fs, basePath); metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metadata, config); table = HoodieTable.getHoodieTable(metadata, config);
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table); taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
@@ -476,7 +474,7 @@ public class TestHoodieBloomIndex {
// We do the tag // We do the tag
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2)); JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config); HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
@@ -515,7 +513,7 @@ public class TestHoodieBloomIndex {
String commitTime = FSUtils.getCommitTime(filename); String commitTime = FSUtils.getCommitTime(filename);
HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
new Configuration()); HoodieTestUtils.getDefaultHadoopConf());
HoodieParquetWriter writer = new HoodieParquetWriter( HoodieParquetWriter writer = new HoodieParquetWriter(
commitTime, commitTime,
new Path(basePath + "/" + partitionPath + "/" + filename), new Path(basePath + "/" + partitionPath + "/" + filename),

View File

@@ -55,7 +55,7 @@ public class TestHoodieCommitArchiveLog {
folder.create(); folder.create();
basePath = folder.getRoot().getAbsolutePath(); basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(basePath); HoodieTestUtils.init(basePath);
fs = FSUtils.getFs(); fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
} }
@Test @Test
@@ -82,7 +82,7 @@ public class TestHoodieCommitArchiveLog {
HoodieTestDataGenerator.createCommitFile(basePath, "104"); HoodieTestDataGenerator.createCommitFile(basePath, "104");
HoodieTestDataGenerator.createCommitFile(basePath, "105"); HoodieTestDataGenerator.createCommitFile(basePath, "105");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieTimeline timeline = HoodieTimeline timeline =
metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
@@ -112,8 +112,8 @@ public class TestHoodieCommitArchiveLog {
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList())); originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
//read the file //read the file
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(), HoodieLogFormat.Reader reader = HoodieLogFormat
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), .newReader(fs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")),
HoodieArchivedMetaEntry.getClassSchema(), false); HoodieArchivedMetaEntry.getClassSchema(), false);
int archivedRecordsCount = 0; int archivedRecordsCount = 0;
@@ -147,7 +147,7 @@ public class TestHoodieCommitArchiveLog {
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig( .forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createCommitFile(basePath, "101");
@@ -173,7 +173,7 @@ public class TestHoodieCommitArchiveLog {
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig( .forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createCommitFile(basePath, "101");
@@ -206,7 +206,7 @@ public class TestHoodieCommitArchiveLog {
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig( .forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build(); HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs); HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
HoodieTestDataGenerator.createCommitFile(basePath, "100"); HoodieTestDataGenerator.createCommitFile(basePath, "100");
HoodieTestDataGenerator.createCommitFile(basePath, "101"); HoodieTestDataGenerator.createCommitFile(basePath, "101");

View File

@@ -16,6 +16,9 @@
package com.uber.hoodie.io; 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.HoodieWriteClient;
import com.uber.hoodie.WriteStatus; import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.HoodieClientTestUtils;
@@ -28,7 +31,6 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig; import com.uber.hoodie.config.HoodieStorageConfig;
@@ -38,7 +40,10 @@ import com.uber.hoodie.index.bloom.HoodieBloomIndex;
import com.uber.hoodie.io.compact.HoodieCompactor; import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import com.uber.hoodie.table.HoodieTable; import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileSystem; import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After; import org.junit.After;
@@ -46,14 +51,6 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.TemporaryFolder; 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 { public class TestHoodieCompactor {
private transient JavaSparkContext jsc = null; private transient JavaSparkContext jsc = null;
@@ -104,18 +101,19 @@ public class TestHoodieCompactor {
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
public void testCompactionOnCopyOnWriteFail() throws Exception { public void testCompactionOnCopyOnWriteFail() throws Exception {
HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE); HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath); basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig()); HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime());
} }
@Test @Test
public void testCompactionEmpty() throws Exception { public void testCompactionEmpty() throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieWriteConfig config = getConfig(); HoodieWriteConfig config = getConfig();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable
.getHoodieTable(metaClient, config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
String newCommitTime = writeClient.startCommit(); String newCommitTime = writeClient.startCommit();
@@ -132,7 +130,6 @@ public class TestHoodieCompactor {
@Test @Test
public void testLogFileCountsAfterCompaction() throws Exception { public void testLogFileCountsAfterCompaction() throws Exception {
FileSystem fs = FSUtils.getFs();
// insert 100 records // insert 100 records
HoodieWriteConfig config = getConfig(); HoodieWriteConfig config = getConfig();
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config); HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
@@ -144,7 +141,8 @@ public class TestHoodieCompactor {
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect(); List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
// Update all the 100 records // Update all the 100 records
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
newCommitTime = "101"; newCommitTime = "101";
@@ -161,7 +159,7 @@ public class TestHoodieCompactor {
updatedRecords); updatedRecords);
// Verify that all data file has one log file // 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); table = HoodieTable.getHoodieTable(metaClient, config);
for (String partitionPath : dataGen.getPartitionPaths()) { for (String partitionPath : dataGen.getPartitionPaths()) {
List<FileSlice> groupedLogFiles = List<FileSlice> groupedLogFiles =
@@ -174,14 +172,14 @@ public class TestHoodieCompactor {
} }
// Do a compaction // Do a compaction
metaClient = new HoodieTableMetaClient(fs, basePath); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config); table = HoodieTable.getHoodieTable(metaClient, config);
HoodieCommitMetadata result = HoodieCommitMetadata result =
compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime()); compactor.compact(jsc, getConfig(), table, HoodieActiveTimeline.createNewCommitTime());
// Verify that recently written compacted data file has no log file // 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); table = HoodieTable.getHoodieTable(metaClient, config);
HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); HoodieActiveTimeline timeline = metaClient.getActiveTimeline();

View File

@@ -89,7 +89,8 @@ public class TestCopyOnWriteTable {
String commitTime = HoodieTestUtils.makeNewCommitTime(); String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig(); HoodieWriteConfig config = makeHoodieClientConfig();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath); HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath);
@@ -115,7 +116,7 @@ public class TestCopyOnWriteTable {
// Prepare the AvroParquetIO // Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfig(); HoodieWriteConfig config = makeHoodieClientConfig();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime(); String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
String partitionPath = "/2016/01/31"; String partitionPath = "/2016/01/31";
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
@@ -154,7 +155,8 @@ public class TestCopyOnWriteTable {
// Read out the bloom filter and make sure filter can answer record exist or not // Read out the bloom filter and make sure filter can answer record exist or not
Path parquetFilePath = new Path(parquetFile.getAbsolutePath()); Path parquetFilePath = new Path(parquetFile.getAbsolutePath());
BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(parquetFilePath); BloomFilter filter = ParquetUtils
.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath);
for (HoodieRecord record : records) { for (HoodieRecord record : records) {
assertTrue(filter.mightContain(record.getRecordKey())); assertTrue(filter.mightContain(record.getRecordKey()));
} }
@@ -163,7 +165,8 @@ public class TestCopyOnWriteTable {
+ FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile(); + FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile();
// Read the parquet file, check the record content // Read the parquet file, check the record content
List<GenericRecord> fileRecords = ParquetUtils.readAvroRecords(parquetFilePath); List<GenericRecord> fileRecords = ParquetUtils
.readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath);
GenericRecord newRecord; GenericRecord newRecord;
int index = 0; int index = 0;
for (GenericRecord record : fileRecords) { for (GenericRecord record : fileRecords) {
@@ -188,7 +191,7 @@ public class TestCopyOnWriteTable {
Thread.sleep(1000); Thread.sleep(1000);
String newCommitTime = HoodieTestUtils.makeNewCommitTime(); String newCommitTime = HoodieTestUtils.makeNewCommitTime();
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = new HoodieCopyOnWriteTable(config, metadata); table = new HoodieCopyOnWriteTable(config, metadata);
Iterator<List<WriteStatus>> iter = table Iterator<List<WriteStatus>> iter = table
.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(), .handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(),
@@ -211,7 +214,7 @@ public class TestCopyOnWriteTable {
// Check whether the record has been updated // Check whether the record has been updated
Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath()); Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath());
BloomFilter updatedFilter = ParquetUtils BloomFilter updatedFilter = ParquetUtils
.readBloomFilterFromParquetMetadata(updatedParquetFilePath); .readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), updatedParquetFilePath);
for (HoodieRecord record : records) { for (HoodieRecord record : records) {
// No change to the _row_key // No change to the _row_key
assertTrue(updatedFilter.mightContain(record.getRecordKey())); assertTrue(updatedFilter.mightContain(record.getRecordKey()));
@@ -261,7 +264,7 @@ public class TestCopyOnWriteTable {
HoodieWriteConfig config = makeHoodieClientConfigBuilder() HoodieWriteConfig config = makeHoodieClientConfigBuilder()
.withWriteStatusClass(MetadataMergeWriteStatus.class).build(); .withWriteStatusClass(MetadataMergeWriteStatus.class).build();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime(); String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
@@ -298,8 +301,8 @@ public class TestCopyOnWriteTable {
public void testInsertWithPartialFailures() throws Exception { public void testInsertWithPartialFailures() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig(); HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime(); String commitTime = HoodieTestUtils.makeNewCommitTime();
FileSystem fs = FSUtils.getFs(); FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Write a few records, and get atleast one file // Write a few records, and get atleast one file
@@ -340,7 +343,7 @@ public class TestCopyOnWriteTable {
public void testInsertRecords() throws Exception { public void testInsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig(); HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime(); String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Case 1: // Case 1:
@@ -389,7 +392,7 @@ public class TestCopyOnWriteTable {
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024) HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024)
.parquetPageSize(64 * 1024).build()).build(); .parquetPageSize(64 * 1024).build()).build();
String commitTime = HoodieTestUtils.makeNewCommitTime(); String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
List<HoodieRecord> records = new ArrayList<>(); List<HoodieRecord> records = new ArrayList<>();
@@ -437,7 +440,7 @@ public class TestCopyOnWriteTable {
HoodieClientTestUtils.fakeCommitFile(basePath, "001"); HoodieClientTestUtils.fakeCommitFile(basePath, "001");
HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize); 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); HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator( HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(

View File

@@ -19,6 +19,11 @@
package com.uber.hoodie.table; 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.HoodieWriteClient;
import com.uber.hoodie.WriteStatus; import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.HoodieClientTestUtils;
@@ -45,6 +50,14 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex; import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.HoodieCompactor; import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor; 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.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@@ -61,25 +74,11 @@ import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.TemporaryFolder; 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 { public class TestMergeOnReadTable {
private transient JavaSparkContext jsc = null; private transient JavaSparkContext jsc = null;
private transient SQLContext sqlContext; private transient SQLContext sqlContext;
private String basePath = null; private static String basePath = null;
private HoodieCompactor compactor; private HoodieCompactor compactor;
private FileSystem fs; private FileSystem fs;
@@ -94,12 +93,11 @@ public class TestMergeOnReadTable {
if (hdfsTestService != null) { if (hdfsTestService != null) {
hdfsTestService.stop(); hdfsTestService.stop();
dfsCluster.shutdown(); dfsCluster.shutdown();
;
} }
FSUtils.setFs(null); FSUtils.setFs(null);
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM
FileSystem.closeAll(); FileSystem.closeAll();
HoodieTestUtils.resetFS(); HoodieTestUtils.resetFS(basePath);
} }
@BeforeClass @BeforeClass
@@ -113,30 +111,28 @@ public class TestMergeOnReadTable {
dfs = dfsCluster.getFileSystem(); dfs = dfsCluster.getFileSystem();
} }
FSUtils.setFs(dfs); FSUtils.setFs(dfs);
HoodieTestUtils.resetFS(); HoodieTestUtils.resetFS(basePath);
} }
@Before @Before
public void init() throws IOException { public void init() throws IOException {
this.fs = FSUtils.getFs();
// Initialize a local spark env // Initialize a local spark env
jsc = new JavaSparkContext( jsc = new JavaSparkContext(
HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeOnReadTable")); HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeOnReadTable"));
jsc.hadoopConfiguration().addResource(FSUtils.getFs().getConf());
// Create a temp folder as the base path // Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder(); TemporaryFolder folder = new TemporaryFolder();
folder.create(); folder.create();
basePath = folder.getRoot().getAbsolutePath(); basePath = folder.getRoot().getAbsolutePath();
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
jsc.hadoopConfiguration().addResource(fs.getConf());
dfs.mkdirs(new Path(basePath)); dfs.mkdirs(new Path(basePath));
FSUtils.setFs(dfs); FSUtils.setFs(dfs);
HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ); HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ);
sqlContext = new SQLContext(jsc); // SQLContext stuff
compactor = new HoodieRealtimeTableCompactor(); compactor = new HoodieRealtimeTableCompactor();
//SQLContext stuff
sqlContext = new SQLContext(jsc);
} }
@After @After
@@ -167,7 +163,8 @@ public class TestMergeOnReadTable {
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect(); List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
Optional<HoodieInstant> deltaCommit = Optional<HoodieInstant> deltaCommit =
@@ -209,7 +206,7 @@ public class TestMergeOnReadTable {
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors // Verify there are no errors
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
assertTrue(deltaCommit.isPresent()); assertTrue(deltaCommit.isPresent());
assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp()); assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp());
@@ -229,8 +226,9 @@ public class TestMergeOnReadTable {
assertTrue(dataFilesToRead.findAny().isPresent()); assertTrue(dataFilesToRead.findAny().isPresent());
// verify that there is a commit // verify that there is a commit
table = HoodieTable table = HoodieTable.getHoodieTable(
.getHoodieTable(new HoodieTableMetaClient(fs, cfg.getBasePath(), true), getConfig(false)); new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true),
getConfig(false));
HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants(); HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants();
assertEquals("Expecting a single commit.", 1, assertEquals("Expecting a single commit.", 1,
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants()); timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
@@ -284,7 +282,8 @@ public class TestMergeOnReadTable {
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect(); List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
Optional<HoodieInstant> deltaCommit = Optional<HoodieInstant> deltaCommit =
@@ -332,7 +331,7 @@ public class TestMergeOnReadTable {
// Verify there are no errors // Verify there are no errors
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
assertTrue(deltaCommit.isPresent()); assertTrue(deltaCommit.isPresent());
assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp()); assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp());
@@ -349,7 +348,7 @@ public class TestMergeOnReadTable {
List<String> dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) List<String> dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath())
.collect(Collectors.toList()); .collect(Collectors.toList());
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils
.getRecordsUsingInputFormat(dataFiles); .getRecordsUsingInputFormat(dataFiles, basePath);
//Wrote 40 records and deleted 20 records, so remaining 40-20 = 20 //Wrote 40 records and deleted 20 records, so remaining 40-20 = 20
assertEquals("Must contain 20 records", 20, recordsRead.size()); assertEquals("Must contain 20 records", 20, recordsRead.size());
} }
@@ -377,9 +376,8 @@ public class TestMergeOnReadTable {
//verify there are no errors //verify there are no errors
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); cfg.getBasePath());
Optional<HoodieInstant> commit = Optional<HoodieInstant> commit =
metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertTrue(commit.isPresent()); assertTrue(commit.isPresent());
@@ -403,8 +401,9 @@ public class TestMergeOnReadTable {
//rollback a COW commit when TableType is MOR //rollback a COW commit when TableType is MOR
client.rollback(newCommitTime); client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable hoodieTable = HoodieTable
.getHoodieTable(metaClient, cfg);
FileStatus[] allFiles = HoodieTestUtils FileStatus[] allFiles = HoodieTestUtils
.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); .listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient,
@@ -440,7 +439,8 @@ public class TestMergeOnReadTable {
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect(); List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
Optional<HoodieInstant> deltaCommit = Optional<HoodieInstant> deltaCommit =
@@ -476,7 +476,7 @@ public class TestMergeOnReadTable {
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors // Verify there are no errors
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
assertTrue(deltaCommit.isPresent()); assertTrue(deltaCommit.isPresent());
assertEquals("Latest Delta commit should be 002", "002", deltaCommit.get().getTimestamp()); assertEquals("Latest Delta commit should be 002", "002", deltaCommit.get().getTimestamp());
@@ -487,19 +487,19 @@ public class TestMergeOnReadTable {
List<String> dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()) List<String> dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath())
.collect(Collectors.toList()); .collect(Collectors.toList());
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils
.getRecordsUsingInputFormat(dataFiles); .getRecordsUsingInputFormat(dataFiles, basePath);
assertEquals(recordsRead.size(), 200); assertEquals(recordsRead.size(), 200);
// Test delta commit rollback // Test delta commit rollback
client.rollback(newCommitTime); client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(), roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(),
allFiles); allFiles);
dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList()); dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles); recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
assertEquals(recordsRead.size(), 200); assertEquals(recordsRead.size(), 200);
@@ -516,13 +516,14 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses); assertNoWriteErrors(statuses);
HoodieCompactor compactor = new HoodieRealtimeTableCompactor(); HoodieCompactor compactor = new HoodieRealtimeTableCompactor();
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(true)); HoodieTable table = HoodieTable
.getHoodieTable(metaClient, getConfig(true));
compactor.compact(jsc, getConfig(true), table, HoodieActiveTimeline.createNewCommitTime()); compactor.compact(jsc, getConfig(true), table, HoodieActiveTimeline.createNewCommitTime());
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); 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); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(),
allFiles); allFiles);
@@ -541,7 +542,7 @@ public class TestMergeOnReadTable {
client.rollback(compactedCommitTime); client.rollback(compactedCommitTime);
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); 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); hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(), roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(),
allFiles); allFiles);

View File

@@ -140,5 +140,11 @@
</exclusion> </exclusion>
</exclusions> </exclusions>
</dependency> </dependency>
<dependency>
<groupId>com.github.stefanbirkner</groupId>
<artifactId>system-rules</artifactId>
<version>1.16.0</version>
<scope>test</scope>
</dependency>
</dependencies> </dependencies>
</project> </project>

View File

@@ -77,7 +77,7 @@ public class MercifulJsonConverter {
switch (schema.getType()) { switch (schema.getType()) {
case BOOLEAN: case BOOLEAN:
if (value instanceof Boolean) { if (value instanceof Boolean) {
return (Boolean) value; return value;
} }
break; break;
case DOUBLE: case DOUBLE:

View File

@@ -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();
}
}

View File

@@ -17,5 +17,5 @@
package com.uber.hoodie.common.model; package com.uber.hoodie.common.model;
public enum ActionType { public enum ActionType {
commit, savepoint, compaction, clean, rollback; commit, savepoint, compaction, clean, rollback
} }

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie.common.table; package com.uber.hoodie.common.table;
import com.uber.hoodie.common.SerializableConfiguration;
import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline; import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
@@ -26,6 +27,7 @@ import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.util.Objects; import java.util.Objects;
import java.util.Properties; import java.util.Properties;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@@ -52,24 +54,28 @@ public class HoodieTableMetaClient implements Serializable {
private String basePath; private String basePath;
private transient FileSystem fs; private transient FileSystem fs;
private String metaPath; private String metaPath;
private SerializableConfiguration hadoopConf;
private HoodieTableType tableType; private HoodieTableType tableType;
private HoodieTableConfig tableConfig; private HoodieTableConfig tableConfig;
private HoodieActiveTimeline activeTimeline; private HoodieActiveTimeline activeTimeline;
private HoodieArchivedTimeline archivedTimeline; 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 // 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 { throws DatasetNotFoundException {
log.info("Loading HoodieTableMetaClient from " + basePath); log.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath; this.basePath = basePath;
this.fs = fs; this.hadoopConf = new SerializableConfiguration(conf);
Path basePathDir = new Path(this.basePath); Path basePathDir = new Path(this.basePath);
this.metaPath = basePath + File.separator + METAFOLDER_NAME; this.metaPath = basePath + File.separator + METAFOLDER_NAME;
Path metaPathDir = new Path(this.metaPath); Path metaPathDir = new Path(this.metaPath);
this.fs = getFs();
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir); DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
this.tableConfig = new HoodieTableConfig(fs, metaPath); this.tableConfig = new HoodieTableConfig(fs, metaPath);
this.tableType = tableConfig.getTableType(); this.tableType = tableConfig.getTableType();
@@ -96,7 +102,7 @@ public class HoodieTableMetaClient implements Serializable {
private void readObject(java.io.ObjectInputStream in) private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException { throws IOException, ClassNotFoundException {
in.defaultReadObject(); in.defaultReadObject();
this.fs = FSUtils.getFs(); fs = null; // will be lazily inited
} }
private void writeObject(java.io.ObjectOutputStream out) private void writeObject(java.io.ObjectOutputStream out)
@@ -136,9 +142,16 @@ public class HoodieTableMetaClient implements Serializable {
* Get the FS implementation for this table * Get the FS implementation for this table
*/ */
public FileSystem getFs() { public FileSystem getFs() {
if (fs == null) {
fs = FSUtils.getFs(metaPath, hadoopConf.get());
}
return fs; return fs;
} }
public Configuration getHadoopConf() {
return hadoopConf.get();
}
/** /**
* Get the active instants as a timeline * Get the active instants as a timeline
* *
@@ -146,7 +159,7 @@ public class HoodieTableMetaClient implements Serializable {
*/ */
public synchronized HoodieActiveTimeline getActiveTimeline() { public synchronized HoodieActiveTimeline getActiveTimeline() {
if (activeTimeline == null) { if (activeTimeline == null) {
activeTimeline = new HoodieActiveTimeline(fs, metaPath); activeTimeline = new HoodieActiveTimeline(this);
} }
return activeTimeline; return activeTimeline;
} }
@@ -159,7 +172,7 @@ public class HoodieTableMetaClient implements Serializable {
*/ */
public synchronized HoodieArchivedTimeline getArchivedTimeline() { public synchronized HoodieArchivedTimeline getArchivedTimeline() {
if (archivedTimeline == null) { if (archivedTimeline == null) {
archivedTimeline = new HoodieArchivedTimeline(fs, metaPath); archivedTimeline = new HoodieArchivedTimeline(this);
} }
return archivedTimeline; return archivedTimeline;
} }
@@ -196,7 +209,7 @@ public class HoodieTableMetaClient implements Serializable {
fs.mkdirs(metaPathDir); fs.mkdirs(metaPathDir);
} }
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); 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() log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType()
+ " from " + basePath); + " from " + basePath);
return metaClient; return metaClient;

View File

@@ -83,7 +83,7 @@ public class HoodieCompactedLogRecordScanner implements
Schema readerSchema, String latestInstantTime) { Schema readerSchema, String latestInstantTime) {
this.readerSchema = readerSchema; this.readerSchema = readerSchema;
this.latestInstantTime = latestInstantTime; this.latestInstantTime = latestInstantTime;
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs, basePath); this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
// load class from the payload fully qualified class name // load class from the payload fully qualified class name
this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass(); this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass();

View File

@@ -150,7 +150,7 @@ public interface HoodieLogFormat {
public Writer build() throws IOException, InterruptedException { public Writer build() throws IOException, InterruptedException {
log.info("Building HoodieLogFormat Writer"); log.info("Building HoodieLogFormat Writer");
if (fs == null) { if (fs == null) {
fs = FSUtils.getFs(); throw new IllegalArgumentException("fs is not specified");
} }
if (logFileId == null) { if (logFileId == null) {
throw new IllegalArgumentException("FileID is not specified"); throw new IllegalArgumentException("FileID is not specified");

View File

@@ -58,7 +58,7 @@ public abstract class HoodieLogBlock {
*/ */
public enum LogMetadataType { public enum LogMetadataType {
INSTANT_TIME, INSTANT_TIME,
TARGET_INSTANT_TIME; TARGET_INSTANT_TIME
} }
public HoodieLogBlock(Map<LogMetadataType, String> logMetadata) { public HoodieLogBlock(Map<LogMetadataType, String> logMetadata) {

View File

@@ -37,7 +37,6 @@ import java.util.stream.Stream;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
import org.apache.log4j.Logger; import org.apache.log4j.Logger;
@@ -54,11 +53,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss"); public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class); private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
private String metaPath; private HoodieTableMetaClient metaClient;
private transient FileSystem fs;
/** /**
* Returns next commit time in the {@link #COMMIT_FORMATTER} format. * Returns next commit time in the {@link #COMMIT_FORMATTER} format.
@@ -67,12 +63,14 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
return HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date()); 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 // Filter all the filter in the metapath and include only the extensions passed and
// convert them into HoodieInstant // convert them into HoodieInstant
try { try {
this.instants = 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 // Include only the meta files with extensions that needs to be included
String extension = FSUtils.getFileExtension(path.getName()); String extension = FSUtils.getFileExtension(path.getName());
return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension)); return Arrays.stream(includedExtensions).anyMatch(Predicate.isEqual(extension));
@@ -85,14 +83,13 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
} catch (IOException e) { } catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e); throw new HoodieIOException("Failed to scan metadata", e);
} }
this.fs = fs; this.metaClient = metaClient;
this.metaPath = metaPath;
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 // 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<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails; this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
} }
public HoodieActiveTimeline(FileSystem fs, String metaPath) { public HoodieActiveTimeline(HoodieTableMetaClient metaClient) {
this(fs, metaPath, this(metaClient,
new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION,
CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION}); CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION});
@@ -114,7 +111,6 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
private void readObject(java.io.ObjectInputStream in) private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException { throws IOException, ClassNotFoundException {
in.defaultReadObject(); in.defaultReadObject();
this.fs = FSUtils.getFs();
} }
/** /**
@@ -214,9 +210,9 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public void deleteInflight(HoodieInstant instant) { public void deleteInflight(HoodieInstant instant) {
log.info("Deleting in-flight " + instant); log.info("Deleting in-flight " + instant);
Path inFlightCommitFilePath = new Path(metaPath, instant.getFileName()); Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName());
try { try {
boolean result = fs.delete(inFlightCommitFilePath, false); boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false);
if (result) { if (result) {
log.info("Removed in-flight " + instant); log.info("Removed in-flight " + instant);
} else { } else {
@@ -230,18 +226,18 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
@Override @Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) { public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
Path detailPath = new Path(metaPath, instant.getFileName()); Path detailPath = new Path(metaClient.getMetaPath(), instant.getFileName());
return readDataFromPath(detailPath); return readDataFromPath(detailPath);
} }
protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed, protected void moveInflightToComplete(HoodieInstant inflight, HoodieInstant completed,
Optional<byte[]> data) { Optional<byte[]> data) {
Path commitFilePath = new Path(metaPath, completed.getFileName()); Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
try { try {
// open a new file and write the commit metadata in // 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); createFileInMetaPath(inflight.getFileName(), data);
boolean success = fs.rename(inflightCommitFile, commitFilePath); boolean success = metaClient.getFs().rename(inflightCommitFile, commitFilePath);
if (!success) { if (!success) {
throw new HoodieIOException( throw new HoodieIOException(
"Could not rename " + inflightCommitFile + " to " + commitFilePath); "Could not rename " + inflightCommitFile + " to " + commitFilePath);
@@ -252,11 +248,11 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
} }
protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { protected void moveCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
Path inFlightCommitFilePath = new Path(metaPath, inflight.getFileName()); Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName());
try { try {
if (!fs.exists(inFlightCommitFilePath)) { if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
Path commitFilePath = new Path(metaPath, completed.getFileName()); Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
boolean success = fs.rename(commitFilePath, inFlightCommitFilePath); boolean success = metaClient.getFs().rename(commitFilePath, inFlightCommitFilePath);
if (!success) { if (!success) {
throw new HoodieIOException( throw new HoodieIOException(
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath); "Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
@@ -272,15 +268,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
} }
protected void createFileInMetaPath(String filename, Optional<byte[]> content) { protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
Path fullPath = new Path(metaPath, filename); Path fullPath = new Path(metaClient.getMetaPath(), filename);
try { try {
if (!content.isPresent()) { if (!content.isPresent()) {
if (fs.createNewFile(fullPath)) { if (metaClient.getFs().createNewFile(fullPath)) {
log.info("Created a new file in meta path: " + fullPath); log.info("Created a new file in meta path: " + fullPath);
return; return;
} }
} else { } else {
FSDataOutputStream fsout = fs.create(fullPath, true); FSDataOutputStream fsout = metaClient.getFs().create(fullPath, true);
fsout.write(content.get()); fsout.write(content.get());
fsout.close(); fsout.close();
return; return;
@@ -292,7 +288,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
} }
protected Optional<byte[]> readDataFromPath(Path detailPath) { protected Optional<byte[]> readDataFromPath(Path detailPath) {
try (FSDataInputStream is = fs.open(detailPath)) { try (FSDataInputStream is = metaClient.getFs().open(detailPath)) {
return Optional.of(IOUtils.toByteArray(is)); return Optional.of(IOUtils.toByteArray(is));
} catch (IOException e) { } catch (IOException e) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e); throw new HoodieIOException("Could not read commit details from " + detailPath, e);
@@ -300,6 +296,6 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
} }
public HoodieActiveTimeline reload() { public HoodieActiveTimeline reload() {
return new HoodieActiveTimeline(fs, metaPath); return new HoodieActiveTimeline(metaClient);
} }
} }

View File

@@ -16,8 +16,8 @@
package com.uber.hoodie.common.table.timeline; 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.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
@@ -27,7 +27,6 @@ import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.function.Function; import java.util.function.Function;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
@@ -44,17 +43,17 @@ import org.apache.log4j.Logger;
public class HoodieArchivedTimeline extends HoodieDefaultTimeline { public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits"; private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits";
private transient FileSystem fs; private HoodieTableMetaClient metaClient;
private String metaPath;
private Map<String, byte[]> readCommits = new HashMap<>(); private Map<String, byte[]> readCommits = new HashMap<>();
private final transient static Logger log = LogManager.getLogger(HoodieArchivedTimeline.class); 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 // Read back the commits to make sure
Path archiveLogPath = getArchiveLogPath(metaPath); Path archiveLogPath = getArchiveLogPath(metaClient.getMetaPath());
try (SequenceFile.Reader reader = 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 key = new Text();
Text val = new Text(); Text val = new Text();
while (reader.next(key, val)) { while (reader.next(key, val)) {
@@ -71,8 +70,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
} }
// multiple casts will make this lambda serializable - http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 // 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<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails; this.details = (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
this.fs = fs; this.metaClient = metaClient;
this.metaPath = metaPath;
} }
/** /**
@@ -91,7 +89,6 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
private void readObject(java.io.ObjectInputStream in) private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException { throws IOException, ClassNotFoundException {
in.defaultReadObject(); in.defaultReadObject();
this.fs = FSUtils.getFs();
} }
@@ -105,7 +102,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
} }
public HoodieArchivedTimeline reload() { public HoodieArchivedTimeline reload() {
return new HoodieArchivedTimeline(fs, metaPath); return new HoodieArchivedTimeline(metaClient);
} }
} }

View File

@@ -40,7 +40,6 @@ import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
/** /**
@@ -57,7 +56,6 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
TableFileSystemView.RealtimeView, Serializable { TableFileSystemView.RealtimeView, Serializable {
protected HoodieTableMetaClient metaClient; protected HoodieTableMetaClient metaClient;
protected transient FileSystem fs;
// This is the commits that will be visible for all views extending this view // This is the commits that will be visible for all views extending this view
protected HoodieTimeline visibleActiveTimeline; protected HoodieTimeline visibleActiveTimeline;
@@ -72,7 +70,6 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline) { HoodieTimeline visibleActiveTimeline) {
this.metaClient = metaClient; this.metaClient = metaClient;
this.fs = metaClient.getFs();
this.visibleActiveTimeline = visibleActiveTimeline; this.visibleActiveTimeline = visibleActiveTimeline;
this.fileGroupMap = new HashMap<>(); this.fileGroupMap = new HashMap<>();
this.partitionToFileGroupsMap = new HashMap<>(); this.partitionToFileGroupsMap = new HashMap<>();
@@ -98,7 +95,6 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
private void readObject(java.io.ObjectInputStream in) private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException { throws IOException, ClassNotFoundException {
in.defaultReadObject(); in.defaultReadObject();
this.fs = FSUtils.getFs();
} }
private void writeObject(java.io.ObjectOutputStream out) private void writeObject(java.io.ObjectOutputStream out)
@@ -255,8 +251,8 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
try { try {
// Create the path if it does not exist already // Create the path if it does not exist already
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr); Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
FSUtils.createPathIfNotExists(fs, partitionPath); FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath);
FileStatus[] statuses = fs.listStatus(partitionPath); FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath);
List<HoodieFileGroup> fileGroups = addFilesToView(statuses); List<HoodieFileGroup> fileGroups = addFilesToView(statuses);
return fileGroups.stream(); return fileGroups.stream();
} catch (IOException e) { } catch (IOException e) {

View File

@@ -94,7 +94,7 @@ public class AvroUtils {
public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime, public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime,
Optional<Long> durationInMs, List<HoodieCleanStat> cleanStats) { Optional<Long> durationInMs, List<HoodieCleanStat> cleanStats) {
ImmutableMap.Builder<String, HoodieCleanPartitionMetadata> partitionMetadataBuilder = ImmutableMap.Builder<String, HoodieCleanPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.<String, HoodieCleanPartitionMetadata>builder(); ImmutableMap.builder();
int totalDeleted = 0; int totalDeleted = 0;
String earliestCommitToRetain = null; String earliestCommitToRetain = null;
for (HoodieCleanStat stat : cleanStats) { for (HoodieCleanStat stat : cleanStats) {
@@ -116,7 +116,7 @@ public class AvroUtils {
public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime, public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime,
Optional<Long> durationInMs, List<String> commits, List<HoodieRollbackStat> stats) { Optional<Long> durationInMs, List<String> commits, List<HoodieRollbackStat> stats) {
ImmutableMap.Builder<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder = ImmutableMap.Builder<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.<String, HoodieRollbackPartitionMetadata>builder(); ImmutableMap.builder();
int totalDeleted = 0; int totalDeleted = 0;
for (HoodieRollbackStat stat : stats) { for (HoodieRollbackStat stat : stats) {
HoodieRollbackPartitionMetadata metadata = HoodieRollbackPartitionMetadata metadata =
@@ -132,7 +132,7 @@ public class AvroUtils {
public static HoodieSavepointMetadata convertSavepointMetadata(String user, String comment, public static HoodieSavepointMetadata convertSavepointMetadata(String user, String comment,
Map<String, List<String>> latestFiles) { Map<String, List<String>> latestFiles) {
ImmutableMap.Builder<String, HoodieSavepointPartitionMetadata> partitionMetadataBuilder = ImmutableMap.Builder<String, HoodieSavepointPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.<String, HoodieSavepointPartitionMetadata>builder(); ImmutableMap.builder();
for (Map.Entry<String, List<String>> stat : latestFiles.entrySet()) { for (Map.Entry<String, List<String>> stat : latestFiles.entrySet()) {
HoodieSavepointPartitionMetadata metadata = HoodieSavepointPartitionMetadata metadata =
new HoodieSavepointPartitionMetadata(stat.getKey(), stat.getValue()); new HoodieSavepointPartitionMetadata(stat.getKey(), stat.getValue());

View File

@@ -29,6 +29,7 @@ import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Map.Entry;
import java.util.Optional; import java.util.Optional;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; import java.util.regex.Pattern;
@@ -55,6 +56,7 @@ public class FSUtils {
private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10; private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10;
private static final long MIN_CLEAN_TO_KEEP = 10; private static final long MIN_CLEAN_TO_KEEP = 10;
private static final long MIN_ROLLBACK_TO_KEEP = 10; private static final long MIN_ROLLBACK_TO_KEEP = 10;
private static final String HOODIE_ENV_PROPS_PREFIX = "HOODIE_ENV_";
private static FileSystem fs; private static FileSystem fs;
/** /**
@@ -65,17 +67,32 @@ public class FSUtils {
FSUtils.fs = 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());
public static FileSystem getFs() { // look for all properties, prefixed to be picked up
for (Entry<String, String> 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(String path, Configuration conf) {
if (fs != null) { if (fs != null) {
return fs; 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());
FileSystem fs; FileSystem fs;
conf = prepareHadoopConf(conf);
try { try {
fs = FileSystem.get(conf); fs = new Path(path).getFileSystem(conf);
} catch (IOException e) { } catch (IOException e) {
throw new HoodieIOException("Failed to get instance of " + FileSystem.class.getName(), throw new HoodieIOException("Failed to get instance of " + FileSystem.class.getName(),
e); e);
@@ -83,7 +100,6 @@ public class FSUtils {
LOG.info( LOG.info(
String.format("Hadoop Configuration: fs.defaultFS: [%s], Config:[%s], FileSystem: [%s]", String.format("Hadoop Configuration: fs.defaultFS: [%s], Config:[%s], FileSystem: [%s]",
conf.getRaw("fs.defaultFS"), conf.toString(), fs.toString())); conf.getRaw("fs.defaultFS"), conf.toString(), fs.toString()));
return fs; return fs;
} }

View File

@@ -51,10 +51,11 @@ public class ParquetUtils {
* Read the rowKey list from the given parquet file. * Read the rowKey list from the given parquet file.
* *
* @param filePath The parquet file path. * @param filePath The parquet file path.
* @param configuration configuration to build fs object
*/ */
public static Set<String> readRowKeysFromParquet(Path filePath) { public static Set<String> readRowKeysFromParquet(Configuration configuration, Path filePath) {
Configuration conf = new Configuration(); Configuration conf = new Configuration(configuration);
conf.addResource(getFs().getConf()); conf.addResource(getFs(filePath.toString(), conf).getConf());
Schema readSchema = HoodieAvroUtils.getRecordKeySchema(); Schema readSchema = HoodieAvroUtils.getRecordKeySchema();
AvroReadSupport.setAvroReadSchema(conf, readSchema); AvroReadSupport.setAvroReadSchema(conf, readSchema);
AvroReadSupport.setRequestedProjection(conf, readSchema); AvroReadSupport.setRequestedProjection(conf, readSchema);
@@ -84,19 +85,12 @@ public class ParquetUtils {
return rowKeys; 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) { public static ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
ParquetMetadata footer; ParquetMetadata footer;
try { try {
// TODO(vc): Should we use the parallel reading version here? // 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) { } catch (IOException e) {
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath, throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath,
e); e);
@@ -108,14 +102,15 @@ public class ParquetUtils {
/** /**
* Get the schema of the given parquet file. * Get the schema of the given parquet file.
*/ */
public static MessageType readSchema(Path parquetFilePath) { public static MessageType readSchema(Configuration configuration, Path parquetFilePath) {
return readMetadata(parquetFilePath).getFileMetaData().getSchema(); return readMetadata(configuration, parquetFilePath).getFileMetaData().getSchema();
} }
private static List<String> readParquetFooter(Path parquetFilePath, String... footerNames) { private static List<String> readParquetFooter(Configuration configuration, Path parquetFilePath,
String... footerNames) {
List<String> footerVals = new ArrayList<>(); List<String> footerVals = new ArrayList<>();
ParquetMetadata footer = readMetadata(parquetFilePath); ParquetMetadata footer = readMetadata(configuration, parquetFilePath);
Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData(); Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData();
for (String footerName : footerNames) { for (String footerName : footerNames) {
if (metadata.containsKey(footerName)) { if (metadata.containsKey(footerName)) {
@@ -128,21 +123,22 @@ public class ParquetUtils {
return footerVals; return footerVals;
} }
public static Schema readAvroSchema(Path parquetFilePath) { public static Schema readAvroSchema(Configuration configuration, Path parquetFilePath) {
return new AvroSchemaConverter().convert(readSchema(parquetFilePath)); return new AvroSchemaConverter().convert(readSchema(configuration, parquetFilePath));
} }
/** /**
* Read out the bloom filter from the parquet file meta data. * Read out the bloom filter from the parquet file meta data.
*/ */
public static BloomFilter readBloomFilterFromParquetMetadata(Path parquetFilePath) { public static BloomFilter readBloomFilterFromParquetMetadata(Configuration configuration,
String footerVal = readParquetFooter(parquetFilePath, Path parquetFilePath) {
String footerVal = readParquetFooter(configuration, parquetFilePath,
HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY).get(0); HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY).get(0);
return new BloomFilter(footerVal); return new BloomFilter(footerVal);
} }
public static String[] readMinMaxRecordKeys(Path parquetFilePath) { public static String[] readMinMaxRecordKeys(Configuration configuration, Path parquetFilePath) {
List<String> minMaxKeys = readParquetFooter(parquetFilePath, List<String> minMaxKeys = readParquetFooter(configuration, parquetFilePath,
HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER, HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER,
HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER); HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER);
if (minMaxKeys.size() != 2) { if (minMaxKeys.size() != 2) {
@@ -156,11 +152,11 @@ public class ParquetUtils {
/** /**
* NOTE: This literally reads the entire file contents, thus should be used with caution. * NOTE: This literally reads the entire file contents, thus should be used with caution.
*/ */
public static List<GenericRecord> readAvroRecords(Path filePath) { public static List<GenericRecord> readAvroRecords(Configuration configuration, Path filePath) {
ParquetReader reader = null; ParquetReader reader = null;
List<GenericRecord> records = new ArrayList<>(); List<GenericRecord> records = new ArrayList<>();
try { try {
reader = AvroParquetReader.builder(filePath).build(); reader = AvroParquetReader.builder(filePath).withConf(configuration).build();
Object obj = reader.read(); Object obj = reader.read();
while (obj != null) { while (obj != null) {
if (obj instanceof GenericRecord) { if (obj instanceof GenericRecord) {

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.common.minicluster;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.uber.hoodie.common.model.HoodieTestUtils;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
@@ -54,7 +55,7 @@ public class HdfsTestService {
private MiniDFSCluster miniDfsCluster; private MiniDFSCluster miniDfsCluster;
public HdfsTestService() { public HdfsTestService() {
hadoopConf = new Configuration(); hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
workDir = Files.createTempDir().getAbsolutePath(); workDir = Files.createTempDir().getAbsolutePath();
} }
@@ -67,7 +68,7 @@ public class HdfsTestService {
.checkState(workDir != null, "The work dir must be set before starting cluster."); .checkState(workDir != null, "The work dir must be set before starting cluster.");
if (hadoopConf == null) { if (hadoopConf == null) {
hadoopConf = new Configuration(); hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
} }
// If clean, then remove the work dir so we can start fresh. // If clean, then remove the work dir so we can start fresh.

View File

@@ -58,6 +58,7 @@ import java.util.stream.Stream;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@@ -69,18 +70,23 @@ import org.junit.rules.TemporaryFolder;
public class HoodieTestUtils { public class HoodieTestUtils {
public static FileSystem fs = FSUtils.getFs(); public static FileSystem fs;
public static final String TEST_EXTENSION = ".test"; public static final String TEST_EXTENSION = ".test";
public static final String RAW_TRIPS_TEST_NAME = "raw_trips"; public static final String RAW_TRIPS_TEST_NAME = "raw_trips";
public static final int DEFAULT_TASK_PARTITIONID = 1; public static final int DEFAULT_TASK_PARTITIONID = 1;
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"}; public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
private static Random rand = new Random(46474747); private static Random rand = new Random(46474747);
public static void resetFS() { public static void resetFS(String basePath) {
HoodieTestUtils.fs = FSUtils.getFs(); HoodieTestUtils.fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
}
public static Configuration getDefaultHadoopConf() {
return new Configuration();
} }
public static HoodieTableMetaClient init(String basePath) throws IOException { public static HoodieTableMetaClient init(String basePath) throws IOException {
fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
return initTableType(basePath, HoodieTableType.COPY_ON_WRITE); return initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
} }
@@ -211,7 +217,7 @@ public class HoodieTestUtils {
Path commitFile = Path commitFile =
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCleanerFileName(commitTime)); .makeCleanerFileName(commitTime));
FileSystem fs = FSUtils.getFs(); FileSystem fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
FSDataOutputStream os = fs.create(commitFile, true); FSDataOutputStream os = fs.create(commitFile, true);
try { try {
HoodieCleanStat cleanStats = new HoodieCleanStat( HoodieCleanStat cleanStats = new HoodieCleanStat(

View File

@@ -67,7 +67,7 @@ public class HoodieLogFormatTest {
private FileSystem fs; private FileSystem fs;
private Path partitionPath; private Path partitionPath;
private String basePath; private static String basePath;
@BeforeClass @BeforeClass
public static void setUpClass() throws IOException, InterruptedException { public static void setUpClass() throws IOException, InterruptedException {
@@ -78,7 +78,7 @@ public class HoodieLogFormatTest {
@AfterClass @AfterClass
public static void tearDownClass() { public static void tearDownClass() {
MiniClusterUtil.shutdown(); MiniClusterUtil.shutdown();
HoodieTestUtils.resetFS(); HoodieTestUtils.resetFS(basePath);
} }
@Before @Before
@@ -343,7 +343,7 @@ public class HoodieLogFormatTest {
writer.close(); writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit) // 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()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with // create a block with
outputStream.write(HoodieLogFormat.MAGIC); outputStream.write(HoodieLogFormat.MAGIC);
@@ -533,7 +533,7 @@ public class HoodieLogFormatTest {
writer.close(); writer.close();
// Append some arbit byte[] to thee end of the log (mimics a partially written commit) // 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()); FSDataOutputStream outputStream = fs.append(writer.getLogFile().getPath());
// create a block with // create a block with
outputStream.write(HoodieLogFormat.MAGIC); outputStream.write(HoodieLogFormat.MAGIC);

View File

@@ -74,7 +74,7 @@ public class HoodieActiveTimelineTest {
HoodieInstant instant5 = HoodieInstant instant5 =
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9"); 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(instant1, Optional.empty());
timeline.saveAsComplete(instant2, Optional.empty()); timeline.saveAsComplete(instant2, Optional.empty());
timeline.saveAsComplete(instant3, Optional.empty()); timeline.saveAsComplete(instant3, Optional.empty());
@@ -98,7 +98,7 @@ public class HoodieActiveTimelineTest {
@Test @Test
public void testTimelineOperationsBasic() throws Exception { public void testTimelineOperationsBasic() throws Exception {
timeline = new HoodieActiveTimeline(HoodieTestUtils.fs, metaClient.getMetaPath()); timeline = new HoodieActiveTimeline(metaClient);
assertTrue(timeline.empty()); assertTrue(timeline.empty());
assertEquals("", 0, timeline.countInstants()); assertEquals("", 0, timeline.countInstants());
assertEquals("", Optional.empty(), timeline.firstInstant()); assertEquals("", Optional.empty(), timeline.firstInstant());

View File

@@ -69,7 +69,7 @@ public class HoodieTableFileSystemViewTest {
} }
private void refreshFsView(FileStatus[] statuses) { private void refreshFsView(FileStatus[] statuses) {
metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs, basePath, true); metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs.getConf(), basePath, true);
if (statuses != null) { if (statuses != null) {
fsView = new HoodieTableFileSystemView(metaClient, fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(), metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(),

View File

@@ -16,15 +16,24 @@
package com.uber.hoodie.common.util; package com.uber.hoodie.common.util;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.model.HoodieTestUtils;
import java.text.SimpleDateFormat; import java.text.SimpleDateFormat;
import java.util.Date; import java.util.Date;
import java.util.UUID; import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.contrib.java.lang.system.EnvironmentVariables;
public class TestFSUtils { public class TestFSUtils {
@Rule
public final EnvironmentVariables environmentVariables
= new EnvironmentVariables();
@Test @Test
public void testMakeDataFileName() { public void testMakeDataFileName() {
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date()); String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
@@ -59,4 +68,15 @@ public class TestFSUtils {
String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName); String fullFileName = FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName);
assertTrue(FSUtils.getFileId(fullFileName).equals(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"));
}
} }

View File

@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
import com.uber.hoodie.avro.HoodieAvroWriteSupport; import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter; import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTestUtils;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@@ -79,12 +80,15 @@ public class TestParquetUtils {
// Read and verify // Read and verify
List<String> rowKeysInFile = new ArrayList<>( List<String> rowKeysInFile = new ArrayList<>(
ParquetUtils.readRowKeysFromParquet(new Path(filePath))); ParquetUtils
.readRowKeysFromParquet(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath)));
Collections.sort(rowKeysInFile); Collections.sort(rowKeysInFile);
Collections.sort(rowKeys); Collections.sort(rowKeys);
assertEquals("Did not read back the expected list of keys", rowKeys, rowKeysInFile); 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) { for (String rowKey : rowKeys) {
assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey)); assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey));
} }

View File

@@ -295,6 +295,6 @@ public class HoodieInputFormat extends MapredParquetInputFormat
} }
Path baseDir = HoodieHiveUtil.getNthParent(dataPath, levels); Path baseDir = HoodieHiveUtil.getNthParent(dataPath, levels);
LOG.info("Reading hoodie metadata from path " + baseDir.toString()); LOG.info("Reading hoodie metadata from path " + baseDir.toString());
return new HoodieTableMetaClient(fs, baseDir.toString()); return new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
} }
} }

View File

@@ -19,6 +19,7 @@ import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; 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.DatasetNotFoundException;
import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieException;
import java.io.Serializable; import java.io.Serializable;
@@ -86,7 +87,7 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
} }
Path folder = null; Path folder = null;
try { try {
FileSystem fs = path.getFileSystem(new Configuration()); FileSystem fs = path.getFileSystem(FSUtils.prepareHadoopConf(new Configuration()));
if (fs.isDirectory(path)) { if (fs.isDirectory(path)) {
return true; return true;
} }
@@ -123,7 +124,7 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
if (baseDir != null) { if (baseDir != null) {
try { try {
HoodieTableMetaClient metaClient = HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, baseDir.toString()); new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline() metaClient.getActiveTimeline().getCommitTimeline()
.filterCompletedInstants(), .filterCompletedInstants(),

View File

@@ -84,7 +84,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)); LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR));
try { try {
baseFileSchema = readSchema(jobConf, split.getPath()); baseFileSchema = readSchema(jobConf, split.getPath());
readAndCompactLog(); readAndCompactLog(jobConf);
} catch (IOException e) { } catch (IOException e) {
throw new HoodieIOException( throw new HoodieIOException(
"Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e); "Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e);
@@ -110,7 +110,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
* Goes through the log files and populates a map with latest version of each key logged, since * Goes through the log files and populates a map with latest version of each key logged, since
* the base split was written. * the base split was written.
*/ */
private void readAndCompactLog() throws IOException { private void readAndCompactLog(JobConf jobConf) throws IOException {
Schema writerSchema = new AvroSchemaConverter().convert(baseFileSchema); Schema writerSchema = new AvroSchemaConverter().convert(baseFileSchema);
List<String> projectionFields = orderFields( List<String> projectionFields = orderFields(
jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR), jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
@@ -123,7 +123,8 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
String.format("About to read compacted logs %s for base split %s, projecting cols %s", String.format("About to read compacted logs %s for base split %s, projecting cols %s",
split.getDeltaFilePaths(), split.getPath(), projectionFields)); split.getDeltaFilePaths(), split.getPath(), projectionFields));
HoodieCompactedLogRecordScanner compactedLogRecordScanner = HoodieCompactedLogRecordScanner compactedLogRecordScanner =
new HoodieCompactedLogRecordScanner(FSUtils.getFs(), split.getBasePath(), new HoodieCompactedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf),
split.getBasePath(),
split.getDeltaFilePaths(), split.getDeltaFilePaths(),
readerSchema, split.getMaxCommitTime()); readerSchema, split.getMaxCommitTime());
// NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit

View File

@@ -41,6 +41,7 @@ import java.util.Map;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord; import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
@@ -64,10 +65,13 @@ import org.junit.rules.TemporaryFolder;
public class HoodieRealtimeRecordReaderTest { public class HoodieRealtimeRecordReaderTest {
private JobConf jobConf; private JobConf jobConf;
private FileSystem fs;
@Before @Before
public void setUp() { public void setUp() {
jobConf = new JobConf(); jobConf = new JobConf();
fs = FSUtils
.getFs(basePath.getRoot().getAbsolutePath(), HoodieTestUtils.getDefaultHadoopConf());
} }
@Rule @Rule
@@ -79,7 +83,9 @@ public class HoodieRealtimeRecordReaderTest {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionDir.getPath())) .onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
.overBaseCommit(baseCommit).withFs(FSUtils.getFs()).build(); .overBaseCommit(baseCommit)
.withFs(fs)
.build();
List<IndexedRecord> records = new ArrayList<>(); List<IndexedRecord> records = new ArrayList<>();
for (int i = 0; i < numberOfRecords; i++) { for (int i = 0; i < numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0")); records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
@@ -123,8 +129,8 @@ public class HoodieRealtimeRecordReaderTest {
//create a RecordReader to be used by HoodieRealtimeRecordReader //create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<Void, ArrayWritable> reader = RecordReader<Void, ArrayWritable> reader =
new MapredParquetInputFormat(). new MapredParquetInputFormat().
getRecordReader(new FileSplit(split.getPath(), 0, getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()),
FSUtils.getFs().getLength(split.getPath()), (String[]) null), jobConf, null); (String[]) null), jobConf, null);
JobConf jobConf = new JobConf(); JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields(); List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(",")); String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
@@ -183,8 +189,8 @@ public class HoodieRealtimeRecordReaderTest {
//create a RecordReader to be used by HoodieRealtimeRecordReader //create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<Void, ArrayWritable> reader = RecordReader<Void, ArrayWritable> reader =
new MapredParquetInputFormat(). new MapredParquetInputFormat().
getRecordReader(new FileSplit(split.getPath(), 0, getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()),
FSUtils.getFs().getLength(split.getPath()), (String[]) null), jobConf, null); (String[]) null), jobConf, null);
JobConf jobConf = new JobConf(); JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields(); List<Schema.Field> fields = schema.getFields();

View File

@@ -30,6 +30,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Partition;
@@ -183,7 +184,7 @@ public class HiveSyncTool {
cmd.usage(); cmd.usage();
System.exit(1); System.exit(1);
} }
FileSystem fs = FSUtils.getFs(); FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
HiveConf hiveConf = new HiveConf(); HiveConf hiveConf = new HiveConf();
hiveConf.addResource(fs.getConf()); hiveConf.addResource(fs.getConf());
new HiveSyncTool(cfg, hiveConf, fs).syncHoodieTable(); new HiveSyncTool(cfg, hiveConf, fs).syncHoodieTable();

View File

@@ -91,7 +91,7 @@ public class HoodieHiveClient {
HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
this.syncConfig = cfg; this.syncConfig = cfg;
this.fs = fs; this.fs = fs;
this.metaClient = new HoodieTableMetaClient(fs, cfg.basePath, true); this.metaClient = new HoodieTableMetaClient(fs.getConf(), cfg.basePath, true);
this.tableType = metaClient.getTableType(); this.tableType = metaClient.getTableType();
LOG.info("Creating hive connection " + cfg.jdbcUrl); LOG.info("Creating hive connection " + cfg.jdbcUrl);

View File

@@ -20,6 +20,7 @@ package com.uber.hoodie.hive.util;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.uber.hoodie.common.model.HoodieTestUtils;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
@@ -87,7 +88,7 @@ public class HiveTestService {
.checkState(workDir != null, "The work dir must be set before starting cluster."); .checkState(workDir != null, "The work dir must be set before starting cluster.");
if (hadoopConf == null) { if (hadoopConf == null) {
hadoopConf = new Configuration(); hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
} }
String localHiveLocation = getHiveLocation(workDir); String localHiveLocation = getHiveLocation(workDir);

View File

@@ -67,7 +67,7 @@ public class HoodieDataSourceHelpers {
*/ */
public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) { public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) {
HoodieTable table = HoodieTable 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)) { if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) {
return table.getActiveTimeline().getTimelineOfActions( return table.getActiveTimeline().getTimelineOfActions(
Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,

View File

@@ -47,7 +47,7 @@ class IncrementalRelation(val sqlContext: SQLContext,
private val log = LogManager.getLogger(classOf[IncrementalRelation]) private val log = LogManager.getLogger(classOf[IncrementalRelation])
val fs = new Path(basePath).getFileSystem(sqlContext.sparkContext.hadoopConfiguration) 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 // MOR datasets not supported yet
if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) { if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) {
throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets") 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 val latestMeta = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(commitsToReturn.last).get) .fromBytes(commitTimeline.getInstantDetails(commitsToReturn.last).get)
val metaFilePath = latestMeta.getFileIdAndFullPaths(basePath).values().iterator().next() 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 override def schema: StructType = latestSchema

View File

@@ -57,7 +57,7 @@ class DataSourceTest extends AssertionsForJUnit {
val folder = new TemporaryFolder val folder = new TemporaryFolder
folder.create folder.create
basePath = folder.getRoot.getAbsolutePath basePath = folder.getRoot.getAbsolutePath
fs = FSUtils.getFs fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
} }
@Test def testCopyOnWriteStorage() { @Test def testCopyOnWriteStorage() {

View File

@@ -56,21 +56,18 @@ import org.apache.spark.Accumulator;
import org.apache.spark.SparkConf; import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext; 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; import scala.Tuple2;
public class HDFSParquetImporter implements Serializable { public class HDFSParquetImporter implements Serializable {
private static volatile Logger logger = LogManager.getLogger(HDFSParquetImporter.class); private static volatile Logger logger = LogManager.getLogger(HDFSParquetImporter.class);
private final Config cfg; 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 static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd");
public HDFSParquetImporter( public HDFSParquetImporter(
Config cfg) throws IOException { Config cfg) throws IOException {
this.cfg = cfg; this.cfg = cfg;
fs = FSUtils.getFs();
} }
public static class FormatValidator implements IValueValidator<String> { public static class FormatValidator implements IValueValidator<String> {
@@ -203,6 +200,7 @@ public class HDFSParquetImporter implements Serializable {
} }
public int dataImport(JavaSparkContext jsc, int retry) throws Exception { public int dataImport(JavaSparkContext jsc, int retry) throws Exception {
this.fs = FSUtils.getFs(cfg.targetPath, jsc.hadoopConfiguration());
int ret = -1; int ret = -1;
try { try {
// Verify that targetPath is not present. // Verify that targetPath is not present.
@@ -251,11 +249,8 @@ public class HDFSParquetImporter implements Serializable {
GenericRecord.class, job.getConfiguration()) GenericRecord.class, job.getConfiguration())
// To reduce large number of tasks. // To reduce large number of tasks.
.coalesce(16 * cfg.parallelism) .coalesce(16 * cfg.parallelism)
.map(new Function<Tuple2<Void, GenericRecord>, HoodieRecord<HoodieJsonPayload>>() { .map(entry -> {
@Override GenericRecord genericRecord = ((Tuple2<Void, GenericRecord>) entry)._2();
public HoodieRecord<HoodieJsonPayload> call(Tuple2<Void, GenericRecord> entry)
throws Exception {
GenericRecord genericRecord = entry._2();
Object partitionField = genericRecord.get(cfg.partitionKey); Object partitionField = genericRecord.get(cfg.partitionKey);
if (partitionField == null) { if (partitionField == null) {
throw new HoodieIOException( throw new HoodieIOException(
@@ -268,26 +263,22 @@ public class HDFSParquetImporter implements Serializable {
} }
long ts = (long) ((Double) partitionField * 1000l); long ts = (long) ((Double) partitionField * 1000l);
String partitionPath = PARTITION_FORMATTER.format(new Date(ts)); String partitionPath = PARTITION_FORMATTER.format(new Date(ts));
return new HoodieRecord<HoodieJsonPayload>( return new HoodieRecord<>(
new HoodieKey((String) rowField, partitionPath), new HoodieKey((String) rowField, partitionPath),
new HoodieJsonPayload(genericRecord.toString())); new HoodieJsonPayload(genericRecord.toString()));
} }
}
); );
// Get commit time. // Get commit time.
String commitTime = client.startCommit(); String commitTime = client.startCommit();
JavaRDD<WriteStatus> writeResponse = client.bulkInsert(hoodieRecords, commitTime); JavaRDD<WriteStatus> writeResponse = client.bulkInsert(hoodieRecords, commitTime);
Accumulator<Integer> errors = jsc.accumulator(0); Accumulator<Integer> errors = jsc.accumulator(0);
writeResponse.foreach(new VoidFunction<WriteStatus>() { writeResponse.foreach(writeStatus -> {
@Override
public void call(WriteStatus writeStatus) throws Exception {
if (writeStatus.hasErrors()) { if (writeStatus.hasErrors()) {
errors.add(1); errors.add(1);
logger.error(String.format("Error processing records :writeStatus:%s", logger.error(String.format("Error processing records :writeStatus:%s",
writeStatus.getStat().toString())); writeStatus.getStat().toString()));
} }
}
}); });
if (errors.value() == 0) { if (errors.value() == 0) {
logger.info(String logger.info(String

View File

@@ -291,7 +291,7 @@ public class HiveIncrementalPuller {
if (!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) { if (!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) {
return "0"; return "0";
} }
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath); HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), targetDataPath);
Optional<HoodieInstant> Optional<HoodieInstant>
lastCommit = metadata.getActiveTimeline().getCommitsTimeline() lastCommit = metadata.getActiveTimeline().getCommitsTimeline()
@@ -331,7 +331,7 @@ public class HiveIncrementalPuller {
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation)
throws IOException { throws IOException {
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation); HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), sourceTableLocation);
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsTimeline() List<String> commitsToSync = metadata.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants() .filterCompletedInstants()
.findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants() .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstants()

View File

@@ -20,6 +20,7 @@ package com.uber.hoodie.utilities;
import com.beust.jcommander.JCommander; import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameter;
import com.uber.hoodie.common.SerializableConfiguration;
import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodiePartitionMetadata; import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.table.HoodieTableConfig; import com.uber.hoodie.common.table.HoodieTableConfig;
@@ -70,8 +71,10 @@ public class HoodieSnapshotCopier implements Serializable {
public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir, public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir,
final boolean shouldAssumeDatePartitioning) throws IOException { final boolean shouldAssumeDatePartitioning) throws IOException {
FileSystem fs = FSUtils.getFs(); FileSystem fs = FSUtils.getFs(baseDir, jsc.hadoopConfiguration());
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir); final SerializableConfiguration serConf = new SerializableConfiguration(
jsc.hadoopConfiguration());
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir);
final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView( final TableFileSystemView.ReadOptimizedView fsView = new HoodieTableFileSystemView(
tableMetadata, tableMetadata,
tableMetadata.getActiveTimeline().getCommitsTimeline() tableMetadata.getActiveTimeline().getCommitsTimeline()
@@ -104,7 +107,7 @@ public class HoodieSnapshotCopier implements Serializable {
jsc.parallelize(partitions, partitions.size()) jsc.parallelize(partitions, partitions.size())
.flatMap(partition -> { .flatMap(partition -> {
// Only take latest version files <= latestCommit. // Only take latest version files <= latestCommit.
FileSystem fs1 = FSUtils.getFs(); FileSystem fs1 = FSUtils.getFs(baseDir, serConf.get());
List<Tuple2<String, String>> filePaths = new ArrayList<>(); List<Tuple2<String, String>> filePaths = new ArrayList<>();
Stream<HoodieDataFile> dataFiles = fsView Stream<HoodieDataFile> dataFiles = fsView
.getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp); .getLatestDataFilesBeforeOrOn(partition, latestCommitTimestamp);
@@ -123,13 +126,13 @@ public class HoodieSnapshotCopier implements Serializable {
String partition = tuple._1(); String partition = tuple._1();
Path sourceFilePath = new Path(tuple._2()); Path sourceFilePath = new Path(tuple._2());
Path toPartitionPath = new Path(outputDir, partition); Path toPartitionPath = new Path(outputDir, partition);
FileSystem fs1 = FSUtils.getFs(); FileSystem ifs = FSUtils.getFs(baseDir, serConf.get());
if (!fs1.exists(toPartitionPath)) { if (!ifs.exists(toPartitionPath)) {
fs1.mkdirs(toPartitionPath); ifs.mkdirs(toPartitionPath);
} }
FileUtil.copy(fs1, sourceFilePath, fs1, FileUtil.copy(ifs, sourceFilePath, ifs,
new Path(toPartitionPath, sourceFilePath.getName()), false, fs1.getConf()); new Path(toPartitionPath, sourceFilePath.getName()), false, ifs.getConf());
}); });
// Also copy the .commit files // Also copy the .commit files

View File

@@ -115,10 +115,11 @@ public class HoodieDeltaStreamer implements Serializable {
public HoodieDeltaStreamer(Config cfg) throws IOException { public HoodieDeltaStreamer(Config cfg) throws IOException {
this.cfg = cfg; 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))) { 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 this.commitTimelineOpt = Optional
.of(meta.getActiveTimeline().getCommitsTimeline() .of(meta.getActiveTimeline().getCommitsTimeline()
.filterCompletedInstants()); .filterCompletedInstants());
@@ -129,8 +130,6 @@ public class HoodieDeltaStreamer implements Serializable {
//TODO(vc) Should these be passed from outside? //TODO(vc) Should these be passed from outside?
initSchemaProvider(); initSchemaProvider();
initKeyGenerator(); initKeyGenerator();
this.jssc = getSparkContext();
initSource(); initSource();
} }
@@ -203,7 +202,9 @@ public class HoodieDeltaStreamer implements Serializable {
Properties properties = new Properties(); Properties properties = new Properties();
properties.put(HoodieWriteConfig.TABLE_NAME, cfg.targetTableName); properties.put(HoodieWriteConfig.TABLE_NAME, cfg.targetTableName);
HoodieTableMetaClient HoodieTableMetaClient
.initializePathAsHoodieDataset(FSUtils.getFs(), cfg.targetBasePath, properties); .initializePathAsHoodieDataset(
FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()), cfg.targetBasePath,
properties);
} }
log.info("Checkpoint to resume from : " + resumeCheckpointStr); log.info("Checkpoint to resume from : " + resumeCheckpointStr);

View File

@@ -25,6 +25,7 @@ import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.commons.configuration.PropertiesConfiguration; import org.apache.commons.configuration.PropertiesConfiguration;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@@ -50,7 +51,7 @@ public class FilebasedSchemaProvider extends SchemaProvider {
public FilebasedSchemaProvider(PropertiesConfiguration config) { public FilebasedSchemaProvider(PropertiesConfiguration config) {
super(config); super(config);
this.fs = FSUtils.getFs(); this.fs = FSUtils.getFs(config.getBasePath(), new Configuration());
DataSourceUtils.checkRequiredProperties(config, DataSourceUtils.checkRequiredProperties(config,
Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP)); Arrays.asList(Config.SOURCE_SCHEMA_FILE_PROP, Config.TARGET_SCHEMA_FILE_PROP));

View File

@@ -65,7 +65,7 @@ public class DFSSource extends Source {
public DFSSource(PropertiesConfiguration config, JavaSparkContext sparkContext, public DFSSource(PropertiesConfiguration config, JavaSparkContext sparkContext,
SourceDataFormat dataFormat, SchemaProvider schemaProvider) { SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
super(config, sparkContext, dataFormat, 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)); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
} }

View File

@@ -72,7 +72,7 @@ public class HiveIncrPullSource extends Source {
public HiveIncrPullSource(PropertiesConfiguration config, JavaSparkContext sparkContext, public HiveIncrPullSource(PropertiesConfiguration config, JavaSparkContext sparkContext,
SourceDataFormat dataFormat, SchemaProvider schemaProvider) { SourceDataFormat dataFormat, SchemaProvider schemaProvider) {
super(config, sparkContext, dataFormat, 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)); DataSourceUtils.checkRequiredProperties(config, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
this.incrPullRootPath = config.getString(Config.ROOT_INPUT_PATH_PROP); this.incrPullRootPath = config.getString(Config.ROOT_INPUT_PATH_PROP);
} }

View File

@@ -48,7 +48,6 @@ import org.apache.spark.streaming.kafka.KafkaCluster;
import org.apache.spark.streaming.kafka.KafkaUtils; import org.apache.spark.streaming.kafka.KafkaUtils;
import org.apache.spark.streaming.kafka.OffsetRange; import org.apache.spark.streaming.kafka.OffsetRange;
import scala.Predef; import scala.Predef;
import scala.Tuple2;
import scala.collection.JavaConverters; import scala.collection.JavaConverters;
import scala.collection.immutable.Map; import scala.collection.immutable.Map;
import scala.collection.immutable.Set; import scala.collection.immutable.Set;
@@ -134,16 +133,16 @@ public class KafkaSource extends Source {
public static <K, V> Map<K, V> toScalaMap(HashMap<K, V> m) { public static <K, V> Map<K, V> toScalaMap(HashMap<K, V> m) {
return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap( return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap(
Predef.<Tuple2<K, V>>conforms() Predef.conforms()
); );
} }
public static Set<String> toScalaSet(HashSet<String> s) { public static Set<String> toScalaSet(HashSet<String> s) {
return JavaConverters.asScalaSetConverter(s).asScala().<String>toSet(); return JavaConverters.asScalaSetConverter(s).asScala().toSet();
} }
public static <K, V> java.util.Map<K, V> toJavaMap(Map<K, V> m) { public static <K, V> java.util.Map<K, V> toJavaMap(Map<K, V> m) {
return JavaConverters.<K, V>mapAsJavaMapConverter(m).asJava(); return JavaConverters.mapAsJavaMapConverter(m).asJava();
} }
} }

View File

@@ -24,6 +24,7 @@ import com.uber.hoodie.HoodieReadClient;
import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.minicluster.HdfsTestService; 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.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.FSUtils;
@@ -38,7 +39,6 @@ import java.util.Map.Entry;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@@ -174,7 +174,7 @@ public class TestHDFSParquetImporter implements Serializable {
ParquetWriter<GenericRecord> writer = AvroParquetWriter ParquetWriter<GenericRecord> writer = AvroParquetWriter
.<GenericRecord>builder(srcFile) .<GenericRecord>builder(srcFile)
.withSchema(HoodieTestDataGenerator.avroSchema) .withSchema(HoodieTestDataGenerator.avroSchema)
.withConf(new Configuration()) .withConf(HoodieTestUtils.getDefaultHadoopConf())
.build(); .build();
for (GenericRecord record : records) { for (GenericRecord record : records) {
writer.write(record); writer.write(record);

View File

@@ -44,17 +44,22 @@ public class TestHoodieSnapshotCopier {
@Before @Before
public void init() throws IOException { public void init() throws IOException {
try {
// Prepare directories // Prepare directories
TemporaryFolder folder = new TemporaryFolder(); TemporaryFolder folder = new TemporaryFolder();
folder.create(); folder.create();
rootPath = folder.getRoot().getAbsolutePath(); rootPath = "file://" + folder.getRoot().getAbsolutePath();
basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME; basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME;
HoodieTestUtils.init(basePath);
outputPath = rootPath + "/output"; outputPath = rootPath + "/output";
fs = FSUtils.getFs();
fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
HoodieTestUtils.init(basePath);
// Start a local Spark job // Start a local Spark job
SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]"); SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]");
jsc = new JavaSparkContext(conf); jsc = new JavaSparkContext(conf);
} catch (Exception e) {
e.printStackTrace();
}
} }
@Test @Test