New Features in DeltaStreamer :
(1) Apply transformation when using delta-streamer to ingest data. (2) Add Hudi Incremental Source for Delta Streamer (3) Allow delta-streamer config-property to be passed as command-line (4) Add Hive Integration to Delta-Streamer and address Review comments (5) Ensure MultiPartKeysValueExtractor handle hive style partition description (6) Reuse same spark session on both source and transformer (7) Support extracting partition fields from _hoodie_partition_path for HoodieIncrSource (8) Reuse Binary Avro coders (9) Add push down filter for Incremental source (10) Add Hoodie DeltaStreamer metrics to track total time taken
This commit is contained in:
committed by
vinoth chandar
parent
c70dbc13e9
commit
3a0044216c
@@ -18,10 +18,16 @@
|
||||
|
||||
package com.uber.hoodie.utilities;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.uber.hoodie.common.TestRawTripPayload;
|
||||
import com.uber.hoodie.common.minicluster.HdfsTestService;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.util.TypedProperties;
|
||||
import com.uber.hoodie.hive.HiveSyncConfig;
|
||||
import com.uber.hoodie.hive.HoodieHiveClient;
|
||||
import com.uber.hoodie.hive.util.HiveTestService;
|
||||
import com.uber.hoodie.utilities.sources.TestDataSource;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
@@ -32,8 +38,11 @@ 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.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hive.service.server.HiveServer2;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
@@ -51,15 +60,26 @@ public class UtilitiesTestBase {
|
||||
protected static MiniDFSCluster dfsCluster;
|
||||
protected static DistributedFileSystem dfs;
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
protected transient SparkSession sparkSession = null;
|
||||
protected transient SQLContext sqlContext;
|
||||
protected static HiveServer2 hiveServer;
|
||||
|
||||
@BeforeClass
|
||||
public static void initClass() throws Exception {
|
||||
initClass(false);
|
||||
}
|
||||
|
||||
static void initClass(boolean startHiveService) throws Exception {
|
||||
hdfsTestService = new HdfsTestService();
|
||||
dfsCluster = hdfsTestService.start(true);
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
dfsBasePath = dfs.getWorkingDirectory().toString();
|
||||
dfs.mkdirs(new Path(dfsBasePath));
|
||||
if (startHiveService) {
|
||||
HiveTestService hiveService = new HiveTestService(hdfsTestService.getHadoopConf());
|
||||
hiveServer = hiveService.start();
|
||||
clearHiveDb();
|
||||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
@@ -67,6 +87,9 @@ public class UtilitiesTestBase {
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
}
|
||||
if (hiveServer != null) {
|
||||
hiveServer.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
@@ -74,6 +97,7 @@ public class UtilitiesTestBase {
|
||||
TestDataSource.initDataGen();
|
||||
jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[2]");
|
||||
sqlContext = new SQLContext(jsc);
|
||||
sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
|
||||
}
|
||||
|
||||
@After
|
||||
@@ -84,6 +108,42 @@ public class UtilitiesTestBase {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to get hive sync config
|
||||
* @param basePath
|
||||
* @param tableName
|
||||
* @return
|
||||
*/
|
||||
protected static HiveSyncConfig getHiveSyncConfig(String basePath, String tableName) {
|
||||
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
|
||||
hiveSyncConfig.jdbcUrl = "jdbc:hive2://127.0.0.1:9999/";
|
||||
hiveSyncConfig.hiveUser = "";
|
||||
hiveSyncConfig.hivePass = "";
|
||||
hiveSyncConfig.databaseName = "testdb1";
|
||||
hiveSyncConfig.tableName = tableName;
|
||||
hiveSyncConfig.basePath = basePath;
|
||||
hiveSyncConfig.assumeDatePartitioning = false;
|
||||
hiveSyncConfig.partitionFields = new ImmutableList.Builder<String>().add("datestr").build();
|
||||
return hiveSyncConfig;
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize Hive DB
|
||||
* @throws IOException
|
||||
*/
|
||||
private static void clearHiveDb() throws IOException {
|
||||
HiveConf hiveConf = new HiveConf();
|
||||
// Create Dummy hive sync config
|
||||
HiveSyncConfig hiveSyncConfig = getHiveSyncConfig("/dummy", "dummy");
|
||||
hiveConf.addResource(hiveServer.getHiveConf());
|
||||
HoodieTableMetaClient.initTableType(dfs.getConf(), hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE,
|
||||
hiveSyncConfig.tableName, null);
|
||||
HoodieHiveClient client = new HoodieHiveClient(hiveSyncConfig, hiveConf, dfs);
|
||||
client.updateHiveSQL("drop database if exists " + hiveSyncConfig.databaseName);
|
||||
client.updateHiveSQL("create database " + hiveSyncConfig.databaseName);
|
||||
client.close();
|
||||
}
|
||||
|
||||
public static class Helpers {
|
||||
|
||||
// to get hold of resources bundled with jar
|
||||
|
||||
Reference in New Issue
Block a user