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,102 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* A simplified versions of Apache commons - PropertiesConfiguration, that supports limited field types and hierarchical
|
||||
* configurations within the same folder as the root file.
|
||||
*
|
||||
* Includes denoted by the same include=filename.properties syntax, with relative path from root file's folder. Lines
|
||||
* beginning with '#' are ignored as comments. Final values for properties are resolved by the order in which they are
|
||||
* specified in the files, with included files treated as if they are inline.
|
||||
*
|
||||
* Note: Not reusing commons-configuration since it has too many conflicting runtime deps.
|
||||
*/
|
||||
public class DFSPropertiesConfiguration {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(DFSPropertiesConfiguration.class);
|
||||
|
||||
private final FileSystem fs;
|
||||
|
||||
private final Path rootFile;
|
||||
|
||||
private final TypedProperties props;
|
||||
|
||||
// Keep track of files visited, to detect loops
|
||||
private final Set<String> visitedFiles;
|
||||
|
||||
public DFSPropertiesConfiguration(FileSystem fs, Path rootFile, TypedProperties defaults) {
|
||||
this.fs = fs;
|
||||
this.rootFile = rootFile;
|
||||
this.props = defaults;
|
||||
this.visitedFiles = new HashSet<>();
|
||||
visitFile(rootFile);
|
||||
}
|
||||
|
||||
private String[] splitProperty(String line) {
|
||||
int ind = line.indexOf('=');
|
||||
String k = line.substring(0, ind).trim();
|
||||
String v = line.substring(ind + 1).trim();
|
||||
return new String[]{k, v};
|
||||
}
|
||||
|
||||
private void visitFile(Path file) {
|
||||
try {
|
||||
if (visitedFiles.contains(file.getName())) {
|
||||
throw new IllegalStateException("Loop detected; file " + file + " already referenced");
|
||||
}
|
||||
visitedFiles.add(file.getName());
|
||||
BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(file)));
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
if (line.startsWith("#") || line.equals("") || !line.contains("=")) {
|
||||
continue;
|
||||
}
|
||||
String[] split = splitProperty(line);
|
||||
if (line.startsWith("include=") || line.startsWith("include =")) {
|
||||
visitFile(new Path(rootFile.getParent(), split[1]));
|
||||
} else {
|
||||
props.setProperty(split[0], split[1]);
|
||||
}
|
||||
}
|
||||
reader.close();
|
||||
} catch (IOException ioe) {
|
||||
log.error("Error reading in properies from dfs", ioe);
|
||||
throw new IllegalArgumentException("Cannot read properties from dfs", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
public DFSPropertiesConfiguration(FileSystem fs, Path rootFile) {
|
||||
this(fs, rootFile, new TypedProperties());
|
||||
}
|
||||
|
||||
public TypedProperties getConfig() {
|
||||
return props;
|
||||
}
|
||||
}
|
||||
@@ -19,6 +19,7 @@ package com.uber.hoodie.common.util;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
@@ -26,18 +27,22 @@ public class ReflectionUtils {
|
||||
|
||||
private static Map<String, Class<?>> clazzCache = new HashMap<>();
|
||||
|
||||
private static Class<?> getClass(String clazzName) {
|
||||
if (!clazzCache.containsKey(clazzName)) {
|
||||
try {
|
||||
Class<?> clazz = Class.forName(clazzName);
|
||||
clazzCache.put(clazzName, clazz);
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new HoodieException("Unable to load class", e);
|
||||
}
|
||||
}
|
||||
return clazzCache.get(clazzName);
|
||||
}
|
||||
|
||||
public static <T> T loadClass(String fqcn) {
|
||||
try {
|
||||
if (clazzCache.get(fqcn) == null) {
|
||||
Class<?> clazz = Class.<HoodieRecordPayload>forName(fqcn);
|
||||
clazzCache.put(fqcn, clazz);
|
||||
}
|
||||
return (T) clazzCache.get(fqcn).newInstance();
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new HoodieException("Could not load class " + fqcn, e);
|
||||
} catch (InstantiationException e) {
|
||||
throw new HoodieException("Could not load class " + fqcn, e);
|
||||
} catch (IllegalAccessException e) {
|
||||
return (T) getClass(fqcn).newInstance();
|
||||
} catch (InstantiationException | IllegalAccessException e) {
|
||||
throw new HoodieException("Could not load class " + fqcn, e);
|
||||
}
|
||||
}
|
||||
@@ -49,18 +54,32 @@ public class ReflectionUtils {
|
||||
Object[] payloadArgs,
|
||||
Class<?>... constructorArgTypes) {
|
||||
try {
|
||||
if (clazzCache.get(recordPayloadClass) == null) {
|
||||
Class<?> clazz = Class.<HoodieRecordPayload>forName(recordPayloadClass);
|
||||
clazzCache.put(recordPayloadClass, clazz);
|
||||
}
|
||||
return (T) clazzCache.get(recordPayloadClass).getConstructor(constructorArgTypes)
|
||||
return (T) getClass(recordPayloadClass).getConstructor(constructorArgTypes)
|
||||
.newInstance(payloadArgs);
|
||||
} catch (InstantiationException | IllegalAccessException
|
||||
| InvocationTargetException | NoSuchMethodException e) {
|
||||
throw new HoodieException("Unable to instantiate payload class ", e);
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new HoodieException("Unable to instantiate payload class ", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an instnace of the given class. Use this version when dealing with interface types as constructor args.
|
||||
*/
|
||||
public static Object loadClass(String clazz, Class<?>[] constructorArgTypes, Object... constructorArgs) {
|
||||
try {
|
||||
return getClass(clazz).getConstructor(constructorArgTypes).newInstance(constructorArgs);
|
||||
} catch (InstantiationException | IllegalAccessException
|
||||
| InvocationTargetException | NoSuchMethodException e) {
|
||||
throw new HoodieException("Unable to instantiate class ", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an instance of the given class. Constructor arg types are inferred.
|
||||
*/
|
||||
public static Object loadClass(String clazz, Object... constructorArgs) {
|
||||
Class<?>[] constructorArgTypes = Arrays.stream(constructorArgs)
|
||||
.map(arg -> arg.getClass()).toArray(Class<?>[]::new);
|
||||
return loadClass(clazz, constructorArgTypes, constructorArgs);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,87 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Type-aware extension of {@link java.util.Properties}
|
||||
*/
|
||||
public class TypedProperties extends Properties implements Serializable {
|
||||
|
||||
public TypedProperties() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
public TypedProperties(Properties defaults) {
|
||||
super(defaults);
|
||||
}
|
||||
|
||||
private void checkKey(String property) {
|
||||
if (!containsKey(property)) {
|
||||
throw new IllegalArgumentException("Property " + property + " not found");
|
||||
}
|
||||
}
|
||||
|
||||
public String getString(String property) {
|
||||
checkKey(property);
|
||||
return getProperty(property);
|
||||
}
|
||||
|
||||
public String getString(String property, String defaultValue) {
|
||||
return containsKey(property) ? getProperty(property) : defaultValue;
|
||||
}
|
||||
|
||||
public int getInteger(String property) {
|
||||
checkKey(property);
|
||||
return Integer.valueOf(getProperty(property));
|
||||
}
|
||||
|
||||
public int getInteger(String property, int defaultValue) {
|
||||
return containsKey(property) ? Integer.valueOf(getProperty(property)) : defaultValue;
|
||||
}
|
||||
|
||||
public long getLong(String property) {
|
||||
checkKey(property);
|
||||
return Long.valueOf(getProperty(property));
|
||||
}
|
||||
|
||||
public long getLong(String property, long defaultValue) {
|
||||
return containsKey(property) ? Long.valueOf(getProperty(property)) : defaultValue;
|
||||
}
|
||||
|
||||
public boolean getBoolean(String property) {
|
||||
checkKey(property);
|
||||
return Boolean.valueOf(getProperty(property));
|
||||
}
|
||||
|
||||
public boolean getBoolean(String property, boolean defaultValue) {
|
||||
return containsKey(property) ? Boolean.valueOf(getProperty(property)) : defaultValue;
|
||||
}
|
||||
|
||||
public double getDouble(String property) {
|
||||
checkKey(property);
|
||||
return Double.valueOf(getProperty(property));
|
||||
}
|
||||
|
||||
public double getDouble(String property, double defaultValue) {
|
||||
return containsKey(property) ? Double.valueOf(getProperty(property)) : defaultValue;
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user