Reworking the deltastreamer tool
- Standardize version of jackson - DFSPropertiesConfiguration replaces usage of commons PropertiesConfiguration - Remove dependency on ConstructorUtils - Throw error if ordering value is not present, during key generation - Switch to shade plugin for hoodie-utilities - Added support for consumption for Confluent avro kafka serdes - Support for Confluent schema registry - KafkaSource now deals with skews nicely, by doing round robin allocation of source limit across partitions - Added support for BULK_INSERT operations as well - Pass in the payload class config properly into HoodieWriteClient - Fix documentation based on new usage - Adding tests on deltastreamer, sources and all new util classes.
This commit is contained in:
committed by
vinoth chandar
parent
fb95dbdedb
commit
d58ddbd999
@@ -0,0 +1,137 @@
|
||||
/*
|
||||
* 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.utilities;
|
||||
|
||||
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.util.TypedProperties;
|
||||
import com.uber.hoodie.utilities.sources.TestDataSource;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.PrintStream;
|
||||
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.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
/**
|
||||
* Abstract test that provides a dfs & spark contexts.
|
||||
*
|
||||
* TODO(vc): this needs to be done across the board.
|
||||
*/
|
||||
public class UtilitiesTestBase {
|
||||
|
||||
protected static String dfsBasePath;
|
||||
protected static HdfsTestService hdfsTestService;
|
||||
protected static MiniDFSCluster dfsCluster;
|
||||
protected static DistributedFileSystem dfs;
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
protected transient SQLContext sqlContext;
|
||||
|
||||
@BeforeClass
|
||||
public static void initClass() throws Exception {
|
||||
hdfsTestService = new HdfsTestService();
|
||||
dfsCluster = hdfsTestService.start(true);
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
dfsBasePath = dfs.getWorkingDirectory().toString();
|
||||
dfs.mkdirs(new Path(dfsBasePath));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void cleanupClass() throws Exception {
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
TestDataSource.initDataGen();
|
||||
jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[2]");
|
||||
sqlContext = new SQLContext(jsc);
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() throws Exception {
|
||||
TestDataSource.resetDataGen();
|
||||
if (jsc != null) {
|
||||
jsc.stop();
|
||||
}
|
||||
}
|
||||
|
||||
public static class Helpers {
|
||||
|
||||
// to get hold of resources bundled with jar
|
||||
private static ClassLoader classLoader = Helpers.class.getClassLoader();
|
||||
|
||||
public static void copyToDFS(String testResourcePath, FileSystem fs, String targetPath) throws IOException {
|
||||
BufferedReader reader = new BufferedReader(
|
||||
new InputStreamReader(classLoader.getResourceAsStream(testResourcePath)));
|
||||
PrintStream os = new PrintStream(fs.create(new Path(targetPath), true));
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
os.println(line);
|
||||
}
|
||||
os.flush();
|
||||
os.close();
|
||||
}
|
||||
|
||||
public static void savePropsToDFS(TypedProperties props, FileSystem fs, String targetPath) throws IOException {
|
||||
String[] lines = props.keySet().stream().map(k -> String.format("%s=%s", k, props.get(k))).toArray(String[]::new);
|
||||
saveStringsToDFS(lines, fs, targetPath);
|
||||
}
|
||||
|
||||
public static void saveStringsToDFS(String[] lines, FileSystem fs, String targetPath) throws IOException {
|
||||
PrintStream os = new PrintStream(fs.create(new Path(targetPath), true));
|
||||
for (String l : lines) {
|
||||
os.println(l);
|
||||
}
|
||||
os.flush();
|
||||
os.close();
|
||||
}
|
||||
|
||||
public static TypedProperties setupSchemaOnDFS() throws IOException {
|
||||
UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc");
|
||||
TypedProperties props = new TypedProperties();
|
||||
props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
|
||||
return props;
|
||||
}
|
||||
|
||||
public static String toJsonString(HoodieRecord hr) {
|
||||
try {
|
||||
return ((TestRawTripPayload) hr.getData()).getJsonData();
|
||||
} catch (IOException ioe) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
public static String[] jsonifyRecords(List<HoodieRecord> records) throws IOException {
|
||||
return records.stream().map(Helpers::toJsonString).toArray(String[]::new);
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user