1
0

Use hadoopConf in HoodieTableMetaClient and related tests

This commit is contained in:
Jian Xu
2018-03-08 17:21:11 -08:00
committed by vinoth chandar
parent 73534d467f
commit 7f079632a6
20 changed files with 57 additions and 50 deletions

View File

@@ -232,7 +232,7 @@ public class HDFSParquetImporter implements Serializable {
Properties properties = new Properties();
properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName);
properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType);
HoodieTableMetaClient.initializePathAsHoodieDataset(fs, cfg.targetPath, properties);
HoodieTableMetaClient.initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties);
HoodieWriteClient client = createHoodieClient(jsc, cfg.targetPath, schemaStr,
cfg.parallelism);

View File

@@ -202,8 +202,7 @@ public class HoodieDeltaStreamer implements Serializable {
Properties properties = new Properties();
properties.put(HoodieWriteConfig.TABLE_NAME, cfg.targetTableName);
HoodieTableMetaClient
.initializePathAsHoodieDataset(
FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()), cfg.targetBasePath,
.initializePathAsHoodieDataset(jssc.hadoopConfiguration(), cfg.targetBasePath,
properties);
}
log.info("Checkpoint to resume from : " + resumeCheckpointStr);

View File

@@ -25,6 +25,7 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.util.FSUtils;
import java.io.File;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf;
@@ -52,8 +53,9 @@ public class TestHoodieSnapshotCopier {
basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME;
outputPath = rootPath + "/output";
fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
HoodieTestUtils.init(fs, basePath);
final Configuration hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
fs = FSUtils.getFs(basePath, hadoopConf);
HoodieTestUtils.init(hadoopConf, basePath);
// Start a local Spark job
SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]");
jsc = new JavaSparkContext(conf);