diff --git a/README.md b/README.md index 60f988e..7de3150 100644 --- a/README.md +++ b/README.md @@ -17,17 +17,53 @@ LinkedIn Data Integration Library (DIL) is a collection of generic data integrat - Ingest data from one Rest API and egress to another (Rest API) on cloud # Requirements -* Java >= 1.8 +* JDK 1.8 If building the distribution with tests turned on: * Maven version 3.5.3 # Instructions to build the distribution 1. Extract the archive file to your local directory. -2. Set JAVA_HOME +2. Set JAVA_HOME to use JDK 1.8 (JDK 11+ not supported) 3. Build -* Skip tests and build the distribution -> `./gradlew build -x findbugsMain -x test -x rat -x checkstyleMain` - -* Tests and build the distribution (requires Maven): > `./gradlew build` + +# Instructions to contribute +To contribute, please use submit Pull Request (PR) for committers to merge. +1. Create your own fork on GitHub off the main repository +2. Clone your fork to your local computer + - `git clone https://github.com/<>/data-integration-library.git` +3. Add upstream and verify + - `git remote add upstream https://github.com/linkedin/data-integration-library.git` + - `git remote -v` +4. Change, test, commit, and push to your fork + - `git status` + - `git add .` + - `git commit -m "comments"` + - `git push origin master` +5. Create Pull Request on GitHub with the following details + - Title + - Detailed description + - Document the tests done + - Links to the updated documents +6. Publish to local Maven repository + - `./gradlew publishToMavenLocal` +7. Refresh your fork + - if upstream has no conflict with your fork, you can go to your forked + repository, and use "Fetch upstream" function to sync up your fork. + - if upstream has conflicts with your fork, GitHub will ask you to create + a pull request to merge. + - if the conflicts are too significant, it is better to just copy + everything from upstream (the main repository) to your fork; that can + be done with the following procedure: + - Follow step 2 and step 3 above + - `git fetch upstream` + - `git reset --hard upstream/master` + - `git push origin +master` + - check your fork should be in sync with the main repository + +# Detailed Documents + +- [Job Properties](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/summary.md) +- [Job Properties by Category](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/categories.md) +- [Deprecated Job Properties](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/deprecated.md) \ No newline at end of file diff --git a/build.gradle b/build.gradle index 218aa80..fc8436f 100644 --- a/build.gradle +++ b/build.gradle @@ -44,11 +44,6 @@ allprojects { subprojects { dependencies { - // Gradle 5 compatibility - compileOnly externalDependency.lombok - testCompileOnly externalDependency.lombok - annotationProcessor externalDependency.lombok - testAnnotationProcessor externalDependency.lombok } project.buildDir = new File(project.rootProject.buildDir, project.name) } diff --git a/buildSrc/src/main/groovy/org/apache/gobblin/gradle/BuildProperties.groovy b/buildSrc/src/main/groovy/org/apache/gobblin/gradle/BuildProperties.groovy deleted file mode 100644 index e051673..0000000 --- a/buildSrc/src/main/groovy/org/apache/gobblin/gradle/BuildProperties.groovy +++ /dev/null @@ -1,50 +0,0 @@ -package org.apache.gobblin.gradle; - -import java.util.TreeMap -import org.gradle.api.Project - -/** - * The manages the collection of all known build properties for the project. It is - * essentially a map from the property name to the BuildProperty object. - */ -public class BuildProperties extends TreeMap { - final Project project; - - public BuildProperties(Project project) { - super(); - this.project = project - } - - public BuildProperties register(BuildProperty prop) { - put(prop.name, prop); - return this; - } - - public void ensureDefined(String propName) { - if (! containsKey(propName)) { - throw new RuntimeException ("Property not defined: " + propName) - } - def defaultValue = get(propName).defaultValue - - // Special treatment for Boolean flags -- just specifying the property - // is treated as setting to true. - if (null != defaultValue && defaultValue instanceof Boolean && - !((Boolean)defaultValue).booleanValue()) { - this.project.ext.set(propName, this.project.hasProperty(propName)) - } - else if (! this.project.hasProperty(propName)) { - this.project.ext.set(propName, defaultValue) - } - - println String.format("Build property: %s=%s", propName, this.project.ext.get(propName)) - } - - public void printHelp() { - println "\n\n" - println "BUILD PROPERTIES" - println "" - this.each { propName, propHelp -> - println propHelp.getHelp() - } - } -} \ No newline at end of file diff --git a/buildSrc/src/main/groovy/org/apache/gobblin/gradle/BuildProperty.groovy b/buildSrc/src/main/groovy/org/apache/gobblin/gradle/BuildProperty.groovy deleted file mode 100644 index 24a8349..0000000 --- a/buildSrc/src/main/groovy/org/apache/gobblin/gradle/BuildProperty.groovy +++ /dev/null @@ -1,23 +0,0 @@ -package org.apache.gobblin.gradle; - -/** - * Encapsulates various aspects of a project property that can be used to customize the build through - * the gradle -P switch. - */ -public class BuildProperty { - private final String HELP_FORMAT = "\t%-20s - %s. Default: %s"; - - public final String name; - public final Object defaultValue; - public final String description; - - public BuildProperty(String name, Object defaultValue, String description) { - this.name = name; - this.defaultValue = defaultValue; - this.description = description; - } - - public String getHelp() { - return String.format(HELP_FORMAT, this.name, this.description, this.defaultValue) - } -} \ No newline at end of file diff --git a/cdi-core/build.gradle b/cdi-core/build.gradle index f5643a6..45d5bca 100644 --- a/cdi-core/build.gradle +++ b/cdi-core/build.gradle @@ -31,10 +31,8 @@ dependencies { compile externalDependency.'awsUtils' compile externalDependency.'commonsValidator' compile externalDependency.'guava' - compile externalDependency.'lombok' compile externalDependency.'commonsLang3' compile externalDependency.'testng' - compile externalDependency.'okhttp' compile externalDependency.'jhyde' runtime externalDependency.'gobblin-azkaban' diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/BooleanProperties.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/BooleanProperties.java new file mode 100644 index 0000000..d7d82f5 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/BooleanProperties.java @@ -0,0 +1,91 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.State; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * A Boolean type of property has no default defaultValue, and each property + * has to supply a default value, true or false + */ +public class BooleanProperties extends MultistageProperties { + private static final Logger LOG = LoggerFactory.getLogger(BooleanProperties.class); + + /** + * Constructor with explicit default value + * @param config property name + * @param defaultValue default value + */ + BooleanProperties(String config, Boolean defaultValue) { + super(config, Boolean.class, defaultValue); + } + + /** + * Validates the value when it is blank + * - No configuration is considered blank + * - A blank string is considered blank + * + * @param state state + * @return true if blank + */ + @Override + public boolean isBlank(State state) { + return !state.contains(getConfig()) + || StringUtils.isBlank(state.getProp(getConfig())); + } + + /** + * Validates the value when it is non-blank and accepts blank value + * - A blank configuration is considered valid + * - Any properly formed Boolean is considered valid + * @param state state + * @return true if blank or non-blank and valid + */ + @Override + public boolean isValid(State state) { + if (!isBlank(state)) try { + String value = state.getProp(getConfig()); + if (!value.toLowerCase().matches("true|false")) { + LOG.error(errorMessage(state)); + return false; + } + // Properly formed Boolean string is valid + Boolean.parseBoolean(state.getProp(getConfig())); + } catch (Exception e) { + LOG.error(errorMessage(state), e.getMessage()); + return false; + } + return true; + } + + /** + * Validates the value when it is non-blank and rejects blank value + * - only properly formed Boolean string is considered valid + * + * @param state source state + * @return true when the configuration is non-blank and valid + */ + public boolean isValidNonblank(State state) { + return !isBlank(state) && isValid(state); + } + + /** + * Retrieves property value from state object if valid and not blank + * otherwise, return default value + * + * @param state state + * @return property value if non-blank and valid, otherwise the default value + */ + protected Boolean getValidNonblankWithDefault(State state) { + if (isValidNonblank(state)) { + return Boolean.parseBoolean(state.getProp(getConfig())); + } + return getDefaultValue(); + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/CsvProperties.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/CsvProperties.java new file mode 100644 index 0000000..6ff075e --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/CsvProperties.java @@ -0,0 +1,186 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import com.google.common.collect.Lists; +import com.google.gson.JsonObject; +import com.linkedin.cdi.util.CsvUtils; +import java.util.List; +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.State; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; +import static com.linkedin.cdi.configuration.StaticConstants.*; + + +/** + * CSV Parameters + */ +public class CsvProperties extends JsonObjectProperties{ + final private static String DEFAULT_FIELD_TYPE = "defaultFieldType"; + final private static String ESCAPE_CHARACTER = "escapeCharacter"; + final private static String ESCAPE_CHARACTER_DEFAULT = "u005C"; + final private static String QUOTE_CHARACTER = "quoteCharacter"; + final private static String QUOTE_CHARACTER_DEFAULT = "\""; + final private static String FIELD_SEPARATOR = "fieldSeparator"; + final private static String FIELD_SEPARATOR_DEFAULT = KEY_WORD_COMMA; + final private static String RECORD_SEPARATOR = "recordSeparator"; + final private static String RECORD_SEPARATOR_DEFAULT = System.lineSeparator(); + final private static String LINES_TO_SKIP = "linesToSkip"; + final private static String COLUMN_HEADER_INDEX = "columnHeaderIndex"; + final private static String COLUMN_PROJECTION = "columnProjection"; + final private static String MAX_FAILURES = "maxFailures"; + final private static String KEEP_NULL_STRING = "keepNullString"; + + final private static IntegerProperties linesToSkip = new IntegerProperties(LINES_TO_SKIP); + final private static IntegerProperties columnHeaderIndex = new IntegerProperties(COLUMN_HEADER_INDEX, -1, Integer.MAX_VALUE, -1); + final private static LongProperties maxFailures = new LongProperties(MAX_FAILURES); + final private static BooleanProperties keepNullString = new BooleanProperties(KEEP_NULL_STRING, Boolean.FALSE); + + final private static List csvAttributes = Lists.newArrayList( + DEFAULT_FIELD_TYPE, + ESCAPE_CHARACTER, QUOTE_CHARACTER, FIELD_SEPARATOR, RECORD_SEPARATOR, + LINES_TO_SKIP, COLUMN_HEADER_INDEX, COLUMN_PROJECTION, + MAX_FAILURES, KEEP_NULL_STRING + ); + + @Override + public boolean isValid(State state) { + if (super.isValid(state) && !super.isBlank(state)) { + JsonObject value = GSON.fromJson(state.getProp(getConfig()), JsonObject.class); + if (!value.entrySet().stream().allMatch(p -> csvAttributes.contains(p.getKey()))) { + return false; + } + + if (value.has(COLUMN_PROJECTION)) { + String columnProjections = value.get(COLUMN_PROJECTION).getAsString(); + if (columnProjections.trim().isEmpty()) { + return false; + } + } + + State tmpState = new State(); + if (value.has(COLUMN_HEADER_INDEX)) { + tmpState.setProp(COLUMN_HEADER_INDEX, value.get(COLUMN_HEADER_INDEX).getAsString()); + if (!columnHeaderIndex.isValid(tmpState)) { + return false; + } + } + if (value.has(LINES_TO_SKIP)) { + tmpState.setProp(LINES_TO_SKIP, value.get(LINES_TO_SKIP).getAsString()); + if (!linesToSkip.isValid(tmpState)) { + return false; + } + + if (linesToSkip.get(tmpState) < columnHeaderIndex.get(tmpState) + 1) { + return false; + } + } + if (value.has(MAX_FAILURES)) { + tmpState.setProp(MAX_FAILURES, value.get(MAX_FAILURES).getAsString()); + if (!maxFailures.isValid(tmpState)) { + return false; + } + } + + if (value.has(KEEP_NULL_STRING)) { + tmpState.setProp(KEEP_NULL_STRING, value.get(KEEP_NULL_STRING).getAsString()); + if (!keepNullString.isValid(tmpState)) { + return false; + } + } + } + return super.isValid(state); + } + + /** + * Constructor with implicit default value + * @param config property name + */ + CsvProperties(String config) { + super(config); + } + + public String getDefaultFieldType(State state) { + JsonObject value = get(state); + if (value.has(DEFAULT_FIELD_TYPE)) { + return value.get(DEFAULT_FIELD_TYPE).getAsString(); + } + return StringUtils.EMPTY; + } + + public String getEscapeCharacter(State state) { + JsonObject value = get(state); + if (value.has(ESCAPE_CHARACTER)) { + return CsvUtils.unescape(value.get(ESCAPE_CHARACTER).getAsString().trim()); + } + return CsvUtils.unescape(ESCAPE_CHARACTER_DEFAULT); + } + + public String getQuoteCharacter(State state) { + JsonObject value = get(state); + if (value.has(QUOTE_CHARACTER)) { + return CsvUtils.unescape(value.get(QUOTE_CHARACTER).getAsString().trim()); + } + return QUOTE_CHARACTER_DEFAULT; + } + + public String getFieldSeparator(State state) { + JsonObject value = get(state); + if (value.has(FIELD_SEPARATOR)) { + return CsvUtils.unescape(value.get(FIELD_SEPARATOR).getAsString().trim()); + } + return FIELD_SEPARATOR_DEFAULT; + } + + public String getRecordSeparator(State state) { + JsonObject value = get(state); + if (value.has(RECORD_SEPARATOR)) { + return CsvUtils.unescape(value.get(RECORD_SEPARATOR).getAsString().trim()); + } + return RECORD_SEPARATOR_DEFAULT; + } + + public Integer getLinesToSkip(State state) { + JsonObject value = get(state); + int skip = 0; + if (value.has(LINES_TO_SKIP) && StringUtils.isNotBlank(value.get(LINES_TO_SKIP).getAsString())) { + skip = value.get(LINES_TO_SKIP).getAsInt(); + } + return Math.max(skip, getColumnHeaderIndex(state) + 1); + } + + public Integer getColumnHeaderIndex(State state) { + JsonObject value = get(state); + if (value.has(COLUMN_HEADER_INDEX) && StringUtils.isNotBlank(value.get(COLUMN_HEADER_INDEX).getAsString())) { + return value.get(COLUMN_HEADER_INDEX).getAsInt(); + } + return -1; + } + + public String getColumnProjection(State state) { + JsonObject value = get(state); + if (value.has(COLUMN_PROJECTION)) { + return value.get(COLUMN_PROJECTION).getAsString(); + } + return StringUtils.EMPTY; + } + + public Long getMaxFailures(State state) { + JsonObject value = get(state); + if (value.has(MAX_FAILURES) && StringUtils.isNotBlank(value.get(MAX_FAILURES).getAsString())) { + return value.get(MAX_FAILURES).getAsLong(); + } + return 0L; + } + + public Boolean getKeepNullString(State state) { + JsonObject value = get(state); + if (value.has(KEEP_NULL_STRING) && StringUtils.isNotBlank(value.get(KEEP_NULL_STRING).getAsString())) { + return value.get(KEEP_NULL_STRING).getAsBoolean(); + } + return false; + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/IntegerProperties.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/IntegerProperties.java new file mode 100644 index 0000000..8e2965d --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/IntegerProperties.java @@ -0,0 +1,117 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.State; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * A Integer type of property has default defaultValue of 0 + */ +public class IntegerProperties extends MultistageProperties { + private static final Logger LOG = LoggerFactory.getLogger(IntegerProperties.class); + + + /** + * Constructor with implicit default, max, and min + * @param config the property name + */ + IntegerProperties(String config) { + super(config, Integer.class, 0, Integer.MAX_VALUE, 0); + } + + /** + * Constructor with explicit default, and implicit max and min + * @param config the property name + * @param defaultValue default value + */ + IntegerProperties(String config, Integer defaultValue) { + super(config, Integer.class, defaultValue, Integer.MAX_VALUE, 0); + } + + /** + * Constructor with explicit default and max, and implicit min + * @param config the property name + * @param defaultValue default value + * @param maxValue max value + */ + IntegerProperties(String config, Integer defaultValue, Integer maxValue) { + super(config, Integer.class, defaultValue, maxValue, 0); + } + + /** + * Constructor with explicit default, max and min + * @param config the property name + * @param defaultValue default value + * @param maxValue max value + * @param minValue min value + */ + IntegerProperties(String config, Integer defaultValue, Integer maxValue, Integer minValue) { + super(config, Integer.class, defaultValue, maxValue, minValue); + } + + /** + * Validates the value when it is blank + * - No configuration is considered blank + * - A blank string is considered blank + * + * @param state state + * @return true if blank + */ + @Override + public boolean isBlank(State state) { + return !state.contains(getConfig()) + || StringUtils.isBlank(state.getProp(getConfig())); + } + + /** + * Validates the value when it is non-blank and accepts blank value + * - A blank configuration is considered valid + * - Any properly formed Integer is considered valid + * @param state state + * @return true if blank or non-blank and valid + */ + @Override + public boolean isValid(State state) { + if (!isBlank(state)) try { + // Properly formed Integer string is valid + int value = Integer.parseInt(state.getProp(getConfig())); + return value >= getMinValue() && value <= getMaxValue(); + } catch (Exception e) { + LOG.error(errorMessage(state), e.getMessage()); + return false; + } + return true; + } + + /** + * Validates the value when it is non-blank and rejects blank value + * - only properly formed Integer string is considered valid + * + * @param state source state + * @return true when the configuration is non-blank and valid + */ + public boolean isValidNonblank(State state) { + return !isBlank(state) && isValid(state); + } + + /** + * Retrieves property value from state object if valid and not blank + * otherwise, return default value + * + * @param state state + * @return property value if non-blank and valid, otherwise the default value + */ + protected Integer getValidNonblankWithDefault(State state) { + if (isValidNonblank(state)) { + int value = Integer.parseInt(state.getProp(getConfig())); + return value > getMaxValue() ? getMaxValue() : value < getMinValue() ? getMinValue() : value; + } + return getDefaultValue(); + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/JsonArrayProperties.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/JsonArrayProperties.java new file mode 100644 index 0000000..868755b --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/JsonArrayProperties.java @@ -0,0 +1,113 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import com.google.gson.JsonArray; +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.State; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.StaticConstants.*; + + +/** + * A JsonArray type of property has default defaultValue of "[]" + */ +public class JsonArrayProperties extends MultistageProperties { + private static final Logger LOG = LoggerFactory.getLogger(JsonArrayProperties.class); + + /** + * Constructor with implicit default value + * @param config property name + */ + JsonArrayProperties(String config) { + super(config, JsonArray.class, new JsonArray()); + } + + /** + * Constructor with explicit default value + * @param config property name + * @param defaultValue default value + */ + + JsonArrayProperties(String config, JsonArray defaultValue) { + super(config, JsonArray.class, defaultValue); + } + + /** + * Validates the value when it is blank + * - No configuration is considered blank + * - A blank string is considered blank + * - An empty array [] is considered blank + * + * @param state state + * @return true if blank + */ + @Override + public boolean isBlank(State state) { + if (!state.contains(getConfig())) { + return true; + } + + if (StringUtils.isBlank(state.getProp(getConfig()))) { + return true; + } + + try { + return GSON.fromJson(state.getProp(getConfig()), JsonArray.class).size() == 0; + } catch (Exception e) { + LOG.error(getConfig(), e.getMessage()); + return false; + } + } + + /** + * Validates the value when it is non-blank and accepts blank value + * - A blank configuration is considered valid + * - Any properly formed JSON array is considered valid + * @param state state + * @return true if blank or non-blank and valid + */ + @Override + public boolean isValid(State state) { + if (!isBlank(state)) try { + // Properly formed JsonArray string is valid + GSON.fromJson(state.getProp(getConfig()), JsonArray.class); + } catch (Exception e) { + LOG.error(errorMessage(state), e.getMessage()); + return false; + } + return true; + } + + /** + * Validates the value when it is non-blank and rejects blank value, including blank array "[]" + * - only properly formed JSON array with at least 1 item is considered valid + * + * @param state source state + * @return true when the configuration is non-blank and valid + */ + public boolean isValidNonblank(State state) { + if (!isBlank(state) && isValid(state)) { + return GSON.fromJson(state.getProp(getConfig()), JsonArray.class).size() > 0; + } + return false; + } + + /** + * Retrieves property value from state object if valid and not blank + * otherwise, return default value + * + * @param state state + * @return property value if non-blank and valid, otherwise the default value + */ + protected JsonArray getValidNonblankWithDefault(State state) { + if (isValidNonblank(state)) { + return GSON.fromJson(state.getProp(getConfig()), JsonArray.class); + } + return getDefaultValue(); + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/JsonObjectProperties.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/JsonObjectProperties.java new file mode 100644 index 0000000..f0a95ff --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/JsonObjectProperties.java @@ -0,0 +1,112 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import com.google.gson.JsonObject; +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.State; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.StaticConstants.*; + + +/** + * A JsonObject type of property has default defaultValue of "{}" + */ +public class JsonObjectProperties extends MultistageProperties { + private static final Logger LOG = LoggerFactory.getLogger(JsonObjectProperties.class); + + /** + * Constructor with implicit default value + * @param config property name + */ + JsonObjectProperties(String config) { + super(config, JsonObject.class, new JsonObject()); + } + + /** + * Constructor with explicit default value + * @param config property name + * @param defaultValue default value + */ + JsonObjectProperties(String config, JsonObject defaultValue) { + super(config, JsonObject.class, defaultValue); + } + + /** + * Validates the value when it is blank + * - No configuration is considered blank + * - A blank string is considered blank + * - An empty array [] is considered blank + * + * @param state state + * @return true if blank + */ + @Override + public boolean isBlank(State state) { + if (!state.contains(getConfig())) { + return true; + } + + if (StringUtils.isBlank(state.getProp(getConfig()))) { + return true; + } + + try { + return GSON.fromJson(state.getProp(getConfig()), JsonObject.class).entrySet().size() == 0; + } catch (Exception e) { + LOG.error(getConfig(), e.getMessage()); + return false; + } + } + + /** + * Validates the value when it is non-blank and accepts blank value + * - A blank configuration is considered valid + * - Any properly formed JSON array is considered valid + * @param state state + * @return true if blank or non-blank and valid + */ + @Override + public boolean isValid(State state) { + if (!isBlank(state)) try { + // Properly formed JsonObject string is valid + GSON.fromJson(state.getProp(getConfig()), JsonObject.class); + } catch (Exception e) { + LOG.error(errorMessage(state), e.getMessage()); + return false; + } + return true; + } + + /** + * Validates the value when it is non-blank and rejects blank value, including blank object "{}" + * - only properly formed JSON array with at least 1 item is considered valid + * + * @param state source state + * @return true when the configuration is non-blank and valid + */ + public boolean isValidNonblank(State state) { + if (!isBlank(state) && isValid(state)) { + return GSON.fromJson(state.getProp(getConfig()), JsonObject.class).entrySet().size() > 0; + } + return false; + } + + /** + * Retrieves property value from state object if valid and not blank + * otherwise, return default value + * + * @param state state + * @return property value if non-blank and valid, otherwise the default value + */ + protected JsonObject getValidNonblankWithDefault(State state) { + if (isValidNonblank(state)) { + return GSON.fromJson(state.getProp(getConfig()), JsonObject.class); + } + return getDefaultValue(); + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/LongProperties.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/LongProperties.java new file mode 100644 index 0000000..d6a42ca --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/LongProperties.java @@ -0,0 +1,127 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.State; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * A Long type of property has default defaultValue of 0L + */ +public class LongProperties extends MultistageProperties { + private static final Logger LOG = LoggerFactory.getLogger(LongProperties.class); + + /** + * Constructor with implicit default, max, and min + * @param config the property name + */ + LongProperties(String config) { + super(config, Long.class, 0L, Long.MAX_VALUE, 0L); + } + + /** + * Constructor with explicit default, and implicit max and min + * @param config the property name + * @param defaultValue default value + */ + LongProperties(String config, Long defaultValue) { + super(config, Long.class, defaultValue, Long.MAX_VALUE, 0L); + } + + /** + * Constructor with explicit default and max, and implicit min + * @param config the property name + * @param defaultValue default value + * @param maxValue max value + */ + LongProperties(String config, Long defaultValue, Long maxValue) { + super(config, Long.class, defaultValue, maxValue, 0L); + } + + /** + * Constructor with explicit default, max and min + * @param config the property name + * @param defaultValue default value + * @param maxValue max value + * @param minValue min value + */ + LongProperties(String config, Long defaultValue, Long maxValue, Long minValue) { + super(config, Long.class, defaultValue, maxValue, minValue); + } + + /** + * Validates the value when it is blank + * - No configuration is considered blank + * - A blank string is considered blank + * + * @param state state + * @return true if blank + */ + @Override + public boolean isBlank(State state) { + return !state.contains(getConfig()) + || StringUtils.isBlank(state.getProp(getConfig())); + } + + /** + * Validates the value when it is non-blank and accepts blank value + * - A blank configuration is considered valid + * - Any properly formed Long is considered valid + * @param state state + * @return true if blank or non-blank and valid + */ + @Override + public boolean isValid(State state) { + if (!isBlank(state)) try { + // Properly formed Long string is valid + long value = Long.parseLong(state.getProp(getConfig())); + return value >= getMinValue() && value <= getMaxValue(); + } catch (Exception e) { + LOG.error(errorMessage(state), e.getMessage()); + return false; + } + return true; + } + + /** + * Validates the value when it is non-blank and rejects blank value + * - only properly formed Long string is considered valid + * + * @param state source state + * @return true when the configuration is non-blank and valid + */ + public boolean isValidNonblank(State state) { + return !isBlank(state) && isValid(state); + } + + /** + * Retrieves property value from state object if valid and not blank + * otherwise, return default value + * + * @param state state + * @return property value if non-blank and valid, otherwise the default value + * @see #get(State) + */ + protected Long getValidNonblankWithDefault(State state) { + if (isValidNonblank(state)) { + long value = Long.parseLong(state.getProp(getConfig())); + return value > getMaxValue() ? getMaxValue() : value < getMinValue() ? getMinValue() : value; + } + return getDefaultValue(); + } + + /** + * The default millisecond value is the property value itself + * @param state state + * @return the configured value + */ + @Override + public Long getMillis(State state) { + return getValidNonblankWithDefault(state); + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/MultistageProperties.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/MultistageProperties.java index 410cb93..07ea1ff 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/configuration/MultistageProperties.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/MultistageProperties.java @@ -4,982 +4,150 @@ package com.linkedin.cdi.configuration; -import com.google.gson.Gson; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.factory.DefaultS3ClientFactory; + +import static com.linkedin.cdi.configuration.StaticConstants.*; /** - * Each item is associated with a Java class so that its actual configuration value - * can be validated.

- * - * The getProp() function will return values with the specified type, with default values. getProp - * can raise exceptions if the configured value mismatch the specified type. For example, if a - * string is configured for an Integer property, or an incorrect string is configured for for - * a JsonObject property.

- * - * The getValidNonblankWithDefault() function will always return a validate value. And if the configured - * value is not valid, a default value will be returned. - * - * The getMillis() function is mostly used to convert days, seconds, and other configured time values - * to milli-seconds. This function might be meaningless to some properties. The default implementation - * will return 0L. - * - * Each item can define functions like:

- * - value validation, by implementing the validate() function for each property

- * - conversion functions applicable to specific property types, like days to millis

- * - provide default values, like default status codes

- * + * Define a structure for MultistageProperties * @author chrli */ -@Slf4j -@Getter -@SuppressWarnings("unchecked") -public enum MultistageProperties { - /** - * Abstinent Period is designed to avoid re-extract a dataset repeatedly. This is particular useful - * for situations like downloading files in large quantity. - * - * Assuming we will control all data extraction through a time range, including file downloads. Further - * assuming that files were all uploaded to source on 6/30, and assuming that we can only download 100 files - * a day, and there are 1000 files. Files downloaded on 7/1 will be downloaded again on 7/2 because - * their cut off time is 7/1, which is before the new extraction time. - * - * An abstinent period 30 is thus added to the last download/extract time, allowing us move the cutoff time forward. - * Therefore, if there is an abstinent period of 30 days, the downloaded files will not be downloaded - * again in 30 days. - * - * Abstinent period can be set to a large number so that the same file will never be downloaded again. - */ - MSTAGE_ABSTINENT_PERIOD_DAYS("ms.abstinent.period.days", Integer.class) { - @Override - public Long getMillis(State state) { - return 24L * 3600L * 1000L * (Integer) this.getProp(state); - } - }, - /** - * activation.property is a row object in JsonObject string format, the activation - * property can container key value pairs to be used as filters or URI parameters - * in extractor. - * - */ - MSTAGE_ACTIVATION_PROPERTY("ms.activation.property", JsonObject.class) { - // this property is normally set from the Source and used in Extractor - // therefore, we use Work Unit State to retrieve the value, and using - // Source State to retrieve the value will always be blank. - @Override - public boolean validate(State state) { - try { - JsonObject activation = getProp(state); - return activation == null || activation.entrySet().size() >= 0; - } catch (Exception e) { - return false; - } - } - }, - MSTAGE_AUTHENTICATION("ms.authentication", JsonObject.class) { - @Override - // accepts only validly formed values - public boolean validateNonblank(State state) { - try { - JsonObject auth = getProp(state); - return auth.entrySet().size() > 0 && auth.has("method") && auth.has("encryption"); - } catch (Exception e) { - return false; - } - } - }, - /** - * to do back fill, set ms.backfill=true, and also set the watermark - * any value other than true indicates a normal load. - * - */ - MSTAGE_BACKFILL("ms.backfill", Boolean.class) { - @Override - public T getDefaultValue() { - return (T) Boolean.FALSE; - } - }, - /** - * call.interval is used in pagination and waiting/looping - * - * when used in pagination, call.interval specify how long the client should wait - * before submit a new page request. - * - * when used in waiting/looping, call.interval specify the waiting period between - * calls. - * - * this value is in milliseconds. - */ - MSTAGE_CALL_INTERVAL("ms.call.interval.millis", Long.class), - MSTAGE_CSV_COLUMN_HEADER("ms.csv.column.header", Boolean.class), - /** - * a comma-separated string, where each value is either an integer or a range - * representing the index of the field to include - * Valid values include [0 based indexing]: - * 0,1,2,3,4 - * 0,1,2,4-15 - * 0,1,3-7,10 - * 0,5,3-4,2 - * - * Note: the values need not to be ordered - */ - MSTAGE_CSV_COLUMN_PROJECTION("ms.csv.column.projection", String.class) { - @Override - // accepts only validly formed values - public boolean validateNonblank(State state) { - String columnProjections = getProp(state); - return columnProjections != null && columnProjections.split(",").length > 0; - } - }, - MSTAGE_CSV_ESCAPE_CHARACTER("ms.csv.escape.character", String.class) { - @Override - public T getDefaultValue() { - return (T) "u005C"; - } - }, - MSTAGE_CSV_QUOTE_CHARACTER("ms.csv.quote.character", String.class) { - @Override - public T getDefaultValue() { - return (T) "\""; - } - }, - MSTAGE_CSV_SEPARATOR("ms.csv.separator", String.class) { - @Override - public T getDefaultValue() { - return (T) ","; - } - }, - /** - * if csv.column.header is true, csv.skip.lines will be 1 by default, if more than 1 - * row to be skipped, then set this parameter explicitly. - * - * if csv.column.header is false, csv.skip.lines will be 0 by default, if there are - * rows to be skipped, then set this parameter explicitly. - */ - MSTAGE_CSV_SKIP_LINES("ms.csv.skip.lines", Integer.class), - MSTAGE_DATA_EXPLICIT_EOF("ms.data.explicit.eof", Boolean.class) { - @Override - public T getDefaultValue() { - return (T) Boolean.FALSE; - } - }, - MSTAGE_DATA_DEFAULT_TYPE("ms.data.default.type", JsonObject.class), - /** - * DATA_FIELD specified where true data payload is in a nested structure. - * If data.field is not specified or it is blank, then the whole response will be - * treated as the payload. If data.field is not present in the response, - * it will generate an error. - */ - MSTAGE_DATA_FIELD("ms.data.field", String.class), - /** - * derived.fields is an array of field definitions - * - * each field definition will have "name", "type", "source", and "format" - * - * Example 1: following define a derived field using regular expression to subtract part of a source field - * [{ - * "name": "surveyid", - * "formula": { - * "type": "regexp", - * "source": "survey_url", - * "format": "https.*\\/surveys\\/([0-9]+)$" - * } - * }] - * - * Example 2: following define a epoc timestamp field to meet Lumos requirement - * [{ - * "name": "callDate", - * "formula": { - * "type": "epoc", - * "source": "started", - * "format": "yyyy-MM-dd" - * } - * }] - * - */ - MSTAGE_DERIVED_FIELDS("ms.derived.fields", JsonArray.class) { - @Override - // accepts blank and validly formed values, only rejects badly formed values - public boolean validate(State state) { - JsonArray derivedFields = getProp(state); - return derivedFields == null - || derivedFields.size() == 0 - || validateNonblank(state); - } - @Override - // accepts only validly formed values - public boolean validateNonblank(State state) { - JsonArray derivedFields = getProp(state); - return derivedFields != null - && derivedFields.size() > 0 - && derivedFields.get(0).getAsJsonObject().has("name") - && derivedFields.get(0).getAsJsonObject().has("formula"); - } - }, - /** - * In this job property you can specify the fields (array of fields) which needs to be encrypted by the Gobblin - * utility. - * These fields can be of JsonPrimitive type (string/int/boolean/etc.) or JsonObject type (with nested structure) - * e.g. "ms.encryption.fields" : ["emailAddress", "settings.webConferencesRecorded"] - */ - MSTAGE_ENCRYPTION_FIELDS("ms.encryption.fields", JsonArray.class), - /** - * Limited cleansing include tasks such as standardizing element name and - * replacing null values with default ones or dummy values - * - * Limited cleansing can also be used to replace certain elements in Json data. - * Currently white spaces and $ in Json element names will be replaced with _ if - * cleansing is enabled. - * - * Default: true - * - * Default value is used when this parameter is blank. - * - * This feature should be used only on need basis in large datasets where cleansing is expensive, - * for example, where source data element names are un-conforming, such as containing spaces, - * and needed standardization. - * - */ - MSTAGE_ENABLE_CLEANSING("ms.enable.cleansing", Boolean.class) { - @Override - public T getDefaultValue() { - return (T) Boolean.TRUE; - } - }, +public abstract class MultistageProperties { + private final String config; + private final Class className; + private final T defaultValue; + private final T maxValue; + private final T minValue; - /** - * Dynamic full load will ignore extract.is.full setting and set extract.is.full based on following - * condidtions: - * 1. true if it is SNAPSHOT_ONLY extract - * 2. true if there is no pre-existing watermarks of the job - * - * To observe the extract.is.full setting, disable dynamic full load - */ - MSTAGE_ENABLE_DYNAMIC_FULL_LOAD("ms.enable.dynamic.full.load", Boolean.class) { - @Override - public T getDefaultValue() { - return (T) Boolean.TRUE; - } - }, - /** - * each Extractor will enforce a compliance filter based on given schema, currently this is - * soft enforced. Use case can turn the filter off by setting this parameter to false - */ - MSTAGE_ENABLE_SCHEMA_BASED_FILTERING("ms.enable.schema.based.filtering", Boolean.class) { - @Override - public T getDefaultValue() { - return (T) Boolean.TRUE; - } - }, - MSTAGE_ENCODING("ms.encoding", String.class) { - @Override - public T getDefaultValue() { - return (T) "UTF-8"; - } - }, - /** - * extract.preprocessors define one or more preprocessor classes, see - * {@link com.linkedin.cdi.preprocessor.GunzipProcessor} - */ - MSTAGE_EXTRACT_PREPROCESSORS("ms.extract.preprocessors", String.class), - /** - * Parameters to pass into the preprocessor along with the input. - * e.g, If a source file is encrypted, it requires additional credentials to decrypt - * For GPG based decryption/encryption, parameters follow {@link org.apache.gobblin.crypto.EncryptionConfigParser} - * A sample parameter map: - * { - * "action" : string, decrypt/encrypt - * "keystore_password" : string, some password, - * "keystore_path" : string, path to the secret keyring, - * "cipher" : string, optional, cipher algorithm to use, default to CAST5 (128 bit key, as per RFC 2144) - * "key_name" : string, optional, the key id, a long value, of the public Gpg key as a Hex string - * } - */ - MSTAGE_EXTRACT_PREPROCESSORS_PARAMETERS("ms.extract.preprocessor.parameters", JsonObject.class), + final public String getDocUrl() { + return DOC_BASE_URL + "/parameters/" + getConfig() + ".md"; + } - MSTAGE_EXTRACTOR_CLASS("ms.extractor.class", String.class), + final public String getSummaryUrl() { + return DOC_BASE_URL + "/parameters/summary.md"; + } - MSTAGE_EXTRACTOR_TARGET_FILE_NAME("ms.extractor.target.file.name", String.class), - //use this property for file dump extractor to save file with specific permission. - MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION("ms.extractor.target.file.permission", String.class) { - @Override - public T getValidNonblankWithDefault(State state) { - return (T) ((validateNonblank(state)) - ? ((String) getProp(state)).toUpperCase() : "755"); - } - }, - /** - * Grace Period is for overlapped data extraction, it assumes that the source can have late comers, - * which are older data that showed up in source after last extract. For example, a record was modified - * 2 days ago, but did not show up until today. In such case, if we extract based records' last - * update date, the last extraction would have missed that record, amd today's extraction will - * again miss it if we cut off by last extraction time (yesterday). - * - * A grace period is thus subtracted from the last extraction time, allowing us move the cut off - * time backward. Therefore, if there is grace period of 2 days, it will capture data arrived 2 days - * late in source. - */ - MSTAGE_GRACE_PERIOD_DAYS("ms.grace.period.days", Integer.class) { - @Override - public Long getMillis(State state) { - return 24L * 3600L * 1000L * (Integer) this.getProp(state); - } - }, - /** - * http.client.factory define an indirect way to specify the type of HttpClient to use. - * default = {@link com.linkedin.cdi.factory.ApacheHttpClientFactory} - */ - MSTAGE_HTTP_CLIENT_FACTORY("ms.http.client.factory", String.class) { - @Override - public T getDefaultValue() { - return (T) "com.linkedin.cdi.factory.ApacheHttpClientFactory"; - } - }, - /** - * custom headers include Content-Type are to be included in this property - */ - MSTAGE_HTTP_REQUEST_HEADERS("ms.http.request.headers", JsonObject.class), - MSTAGE_HTTP_REQUEST_METHOD("ms.http.request.method", String.class), - /** - * use this property to set custom response type explicitly. You can also have multiple accepted types for - * multi-stage jobs - * ex: {"Content-Type":"application/x-gzip","Content-Type1":"application/json"} - */ - MSTAGE_HTTP_RESPONSE_TYPE("ms.http.response.type", JsonObject.class), - /** - * http.statuses defines success codes and warnings, and optionally errors. - * By default, if this parameter is not set, 200 (OK), 201 (CREATED), and 202 (ACCEPTED) - * will be treated as success; anything else below 400 will be treated as warning; and - * anything 400 and above will be treated as error. Warnings will be logged but will not - * cause job failure. Errors will cause job failure. - * - * In cases where 4xx codes, like 404 (NOT FOUND), happened frequently, and a failure is - * not desirable, exceptions can be added to warnings. - * - * In following configuration, we make 404 an warning, and make 206 a failure indicating - * that partial content is not acceptable: - * {"success": [200], "warning": [404], "error": [206]} - */ - MSTAGE_HTTP_STATUSES("ms.http.statuses", JsonObject.class) { - @Override - public T getDefaultValue() { - return (T) GSON.fromJson("{\"success\":[200,201,202], \"pagination_error\":[401]}", JsonObject.class); - } - }, - /** - * http.status.reasons define reason codes (strings) that have special meaning in determining - * whether a request was a success or failure. - * - * for example, when status is 200, but there is a reason to indicate the request was not successful, - * then the status.reason can be set: - * {"error": ["not found"]} - * - * An Http response is considered success if and only if: - * - status code in http.statuses.success - * - reason code not in http.status.reasons.error - * - * Currently, we don't allow exceptions being made to revert errors by using reason code. - */ - MSTAGE_HTTP_STATUS_REASONS("ms.http.status.reasons", JsonObject.class), - /** - * jdbc.client.factory define an indirect way to specify the type of JDBC Client to use. - * default = {@link com.linkedin.cdi.factory.DefaultJdbcClientFactory} - */ - MSTAGE_JDBC_CLIENT_FACTORY("ms.jdbc.client.factory", String.class) { - @Override - public T getDefaultValue() { - return (T) "com.linkedin.cdi.factory.DefaultJdbcClientFactory"; - } - }, + final public String errorMessage(State state) { + return String.format(EXCEPTION_INCORRECT_CONFIGURATION, + getConfig(), state.getProp(getConfig(), StringUtils.EMPTY), getDocUrl()); + } - MSTAGE_JDBC_SCHEMA_REFACTOR("ms.jdbc.schema.refactor", String.class) { - @Override - public T getDefaultValue() { - return (T) "none"; + final public String info(State state) { + if (get(state).equals(getDefaultValue())) { + return String.format("Property %s has default value %s", getConfig(), get(state)); } - }, - /** - * JDBC statement is the query to be executed for data extraction, usually a SELECT - * statement or a store procedure. DIL doesn't explicitly restrict or support syntax - * of the statement. The source database decides whether to accept or fail the statement. - */ - MSTAGE_JDBC_STATEMENT("ms.jdbc.statement", String.class), - MSTAGE_OUTPUT_SCHEMA("ms.output.schema", JsonArray.class), - /** - * pagination is a Json object with 2 members: - * - * fields: is an array of up to 3 string elements, each denote a source key column for: - * 1. page start, or offset - * 2. page size, or limit of each page - * 3. page no, if page no is used to control instead of using page start and page size - * - * initialvalues: is an array of up to 3 integer elements, each denote a initial value for: - * 1. page start, or offset - * 2. pagesize, or limit of each page - * 3. page no, if page no is used to control instead of using page start and page size - */ - MSTAGE_PAGINATION("ms.pagination", JsonObject.class), - /** - * ms.parameter holds a list of parameters in the form of a JsonArray. - * - * Parameters are named, i.e. the name will be referenced in other places. - * - * Parameters will have either static values or dynamic values derived from a formula. - * - * Terminology: in following description, we call parameters used in URI as URI Parameters. - * - * For HTTP GET requests, parameters will be used to form the final URI. In such case, - * the parameter can be used in the URI path or as URI parameter. - * - * When used in URI path, the parameter name need to be specified in URI template - * as a variable contained in {{}}. - * - * When used as URI parameters, the parameter name and derived value will be coded as - * KV pairs; therefore, the parameter name need to be acceptable to source. - * - * For example, if a source accepts URI like http://domainname/endpoint?cursor=xxxx, - * and the "cursor" parameter is optional, then the parameter should be named as - * "cursor", and the URI template should be set as http://domainname/endpoint in pull file. - * In such case, a "?cursor=xxx" will be appended to the final URI when cursor is - * present. - * - * However, if the cursor URI parameter is not optional, the URI template could be coded as - * http://domain/endpoint?cursor={{p1}}, then the parameter can be named as "p1", and the - * parameter value will replace {{p1}} before the request is sent to the URI source. - * - * Examples of setting parameters in pull files: - * - * For one case, the URI needs 3 mandatory variables, and they can be named as p1, p2, and p3. - * And we can configure the pull file as following: - * - * ms.uri=https://domain.com/api/bulk/2.0/syncs/{{p1}}/data?offset={{p2}}&limit={{p3}} - * ms.parameter=[ - * {"name": "p1", "type": "list", "value": "3837498"}, - * {"name": "p2", "type": "pagestart"}, - * {"name": "p3", "type": "pagesize"}] - * - * - * For another case, the URI needs 1 optional variable, and the parameter has to be named as - * required by the source. And here is the configuration: - * - * ms.uri=https://domain.com/users - * ms.parameter=[{"name":"cursor","type":"session"}] - * - * For HTTP POST and HTTP PUT requests, the parameter name will be used as-is in the form of - * "parameter name": "parameter value" in the request body; therefore, the parameter name - * need to be as required by to URI source. - * - */ + return String.format("Property %s has non-default value %s", getConfig(), get(state)); + } - MSTAGE_PARAMETERS("ms.parameters", JsonArray.class) { - @Override - public boolean validateNonblank(State state) { - try { - return ((JsonArray) getProp(state)).size() > 0; - } catch (Exception e) { - return false; - } - } - }, - /** - * Payloads are a type of secondary input. Where activations generates - * work units, payloads don't. In planning phase, payloads are passed - * as is to work units without actually reading the content. Payload - * content are only being read when the transmission starts. The payload - * property, therefore, contains secondary entries, in a JsonArray form. - * And the array contains records of secondary inputs, which include - * path and fields, etc. - */ - MSTAGE_PAYLOAD_PROPERTY("ms.payload.property", JsonArray.class), - MSTAGE_RETENTION("ms.retention", JsonObject.class) { - @Override - public T getDefaultValue() { - JsonObject retention = new JsonObject(); - retention.addProperty("state.store", "P90D"); // keep 90 days state store by default - retention.addProperty("publish.dir", "P731D"); // keep 2 years published data - retention.addProperty("log", "P30D"); - return (T) retention; - } - }, - /** - * s3.client.factory define an indirect way to specify the type of S3 Client to use. - * default = {@link DefaultS3ClientFactory} - */ - MSTAGE_S3_CLIENT_FACTORY("ms.s3.client.factory", String.class) { - @Override - public T getDefaultValue() { - return (T) "com.linkedin.cdi.factory.DefaultS3ClientFactory"; - } - }, - /** - * Schema cleansing will replace special characters in the schema element names based - * on a pattern. By default it will replace all blank spaces, $, and @ to underscores. - * - * Schema cleansing parameter is a JsonObject, and it supports the following elements: - * enabled : true|false - * pattern: if enabled, it has default value "(\\s|\\$|@)" - * replacement: if enabled, it has default value "_" - * nullable: it has default value "false", allowed values are: - * 1. true: all fields will be forced to be nullable - * 2. false: the schema infer will try to detect nullability from samples - * - * This configuration has no impact on schema from metadata stores. - */ - MSTAGE_SCHEMA_CLENSING("ms.schema.cleansing", JsonObject.class), - /** - * This property is used to set:

- * 1. location from where the hdfs data will be loaded as secondary data to call the - * subsequent API

- * 2. define the field names that needs to be extracted and added into the work units. - * 3. define filters on one or more fields based on following rules - * a. if multiple fields are filtered, the relationship is AND, that means all condition must be met - * b. if a filter is defined on a field, and field value is NULL, the record is rejected - * c. if a filter is defined on a field, and the field value is not NULL, the record will be rejected if - * its value doesn't match the pattern - * d. if no filter is defined on a field, the default filter ".*" is applied to the field, and NULL values - * are accepted - * 4. define the category of the input, currently we allow these categories: - * a. activation, that means the secondary input is for creating work units - * b. authentication, that means the secondary input provide authentication information - *

- * - * Example : - * - * ms.secondary.input=[{ - * "path": "/path/to/hdfs/inputFileDir/2019/08/07/19/720", - * "fields": ["id", "tempId"], - * "filters": {"status": "(OK|Success)", "field2": "pattern2"}, - * "category" "activation" - * }] - * - * The gobblin job will read records from that location and extract the two fields and inject it into the work units. - */ - MSTAGE_SECONDARY_INPUT("ms.secondary.input", JsonArray.class) { - @Override - public boolean validate(State state) { - return getProp(state) != null; - } - }, - /** - * session.key.field specifies the key field for session and the condition for termination. - * Although Restful API is stateless, data sources can maintain a session in backend - * by a status field, a session cursor, or through pagination (see comments on PAGINATION). - * - * it takes the form a Json object with a "name", "condition", and "failCondition". - * - the name specifies the field in response that gives session info. - * - the condition, specifies when the session should stop. - * - the failCondition specifies when the session should fail. - * - * A conditions can be a regular expression or a formula. Currently, only regular expression is supported. - * - * "name" is required - * "condition" and "failCondition" are optional - * "condition" takes precedence over "failCondition" - * - * When both session and pagination are enabled, the extractor will keep consuming data from - * source until all pages are extracted. Then the extractor will check the status until - * the stop condition is met. - * - * In that regard, when the source give conflicting signal in turns of total expected rows - * and status, the data can have duplicate, and actual extracted rows in log file should - * show more rows extracted than expected. - * - */ - MSTAGE_SESSION_KEY_FIELD("ms.session.key.field", JsonObject.class), - /** - * Default source data character set is UTF-8, which should be good for most use cases. - * See StandardCharsets for other common names, such as UTF-16 - */ - MSTAGE_SOURCE_DATA_CHARACTER_SET("ms.source.data.character.set", String.class) { - @Override - public T getDefaultValue() { - return (T) "UTF-8"; - } - }, - MSTAGE_SOURCE_FILES_PATTERN("ms.source.files.pattern", String.class) { - @Override - public T getDefaultValue() { - return (T) ".*"; - } - }, + final public String getConfig() { + return config; + } - /** - * Parameters specific to the S3 source. - * { - * "region": string, aws region code: https://docs.aws.amazon.com/general/latest/gr/rande.html - * "read_timeout_seconds", integer, read time out in seconds - * "write_timeout_seconds", integer, write time out in seconds - * "connection_timeout_seconds": Sets the socket to timeout after failing to establish a connection with the server after milliseconds. - * "connection_max_idle_millis", Sets the socket to timeout after timeout milliseconds of inactivity on the socket. - * } - */ - MSTAGE_SOURCE_S3_PARAMETERS("ms.source.s3.parameters", JsonObject.class), - /** - * Source schema is a URN string of the following forms: - * 1. a dataset URN, if the source schema can be represented by a dataset. - * For example: urn:li:dataset:(urn:li:dataPlatform:hive,rightnow.incidents,PROD) - * In such case, the latest schema of the dataset will be read from metadata store, - * and then parsed to retrieve fields and types, etc - * 2. a registered schema, if the source schema is registered with metadata store in - * the form of either a pegasus (PDL) or GraphQL schema. - * For example: TODO - * 3. other forms TODO - */ - MSTAGE_SOURCE_SCHEMA_URN("ms.source.schema.urn", String.class), - /** - * Define a factory parameter to decouple proprietary code from OpenSource code - */ - MSTAGE_SOURCE_SCHEMA_READER_FACTORY("ms.source.schema.reader.factory", String.class) { - @Override - public T getDefaultValue() { - return (T) "com.linkedin.cdi.factory.reader.MetastoreReader"; - } - }, - /** - * ms.source.uri defines a data source identifier, it follows the URI format - * here: https://en.wikipedia.org/wiki/Uniform_Resource_Identifier - * - * The only exception is that authority is not supported, because all authority - * cannot be fit in the URI. - * - * source.uri also accepts variables that allow substitution in runtime - */ - MSTAGE_SOURCE_URI("ms.source.uri", String.class), + final public Class getClassName() { + return className; + } - // TODO: Merge back to @link{MSTAGE_SOURCE_S3_PARAMETERS} - MSTAGE_S3_LIST_MAX_KEYS("ms.s3.list.max.keys", Integer.class) { - @Override - public T getDefaultValue() { - return (T) Integer.valueOf(1000); - } - }, - /** - * Target schema denotes the schema to be passed to writer, this applies - * to situation where the source data are transformed through a converter - * or other processes. - * - * Target schema should be specified through target schema URN. - * An URN can point to the schema storage location on DataHub, which is - * the only supported schema storage for now. - * - * If target schema is specified through ms.target.schema, then the syntax - * of the schema string is same as ms.output.schema. - */ - MSTAGE_TARGET_SCHEMA("ms.target.schema", JsonArray.class), - MSTAGE_TARGET_SCHEMA_URN("ms.target.schema.urn", String.class), - MSTAGE_TARGET_SCHEMA_READER_FACTORY("ms.target.schema.reader.factory", String.class) { - @Override - public T getDefaultValue() { - return (T) "com.linkedin.cdi.factory.reader.MetastoreReader"; - } - }, - /** - * Total count field is a Json path. This attribute can be used in many - * types of connectors, typically with Json Extractor - * - * If response is like { "records": { "totalRecords": 10000, "pagesize": 100, "currentpage": 0}}, - * the configurations should be: ms.totalcount.field=records.totalRecords - * - */ - MSTAGE_TOTAL_COUNT_FIELD("ms.total.count.field", String.class) { - @Override - public boolean validateNonblank(State state) { - String tcField = getProp(state); - return StringUtils.isNotBlank(tcField); - } - }, - /** - * If there is not total expected row count, the session will keep looping and waiting - * until either the session completion condition is met or time out. - * - * wait.timeout control how long the job will wait before the session completion status - * is met. - * - * default is 10 minutes or 600 seconds - * - * see also call.interval - * - */ + public T getDefaultValue() { + return defaultValue; + } - MSTAGE_WAIT_TIMEOUT_SECONDS("ms.wait.timeout.seconds", Long.class) { - @Override - public Long getMillis(State state) { - return 1000L * (Long) this.getValidNonblankWithDefault(state); - } + public T getMaxValue() { + return maxValue; + } + + public T getMinValue() { + return minValue; + } - @Override - public T getDefaultValue() { - return (T) Long.valueOf(600); - } - }, /** - * ms.watermark holds a list of watermark ranges in the form of a JsonArray. - * A watermark property is a JsonObject with name, type, and range. - * For now, only datetime and unit type watermark are supported. + * Constructor with explicit default, and implicit max and min * - * For datetime watermark, a range has "from" and "to" values. - * They have to be in "yyyy-MM-dd" format; however "to" can be just "-" to present current date. - * - * For example: - * - * ms.watermark=[{"name": "system","type": "datetime","range": {"from": "2019-01-01", "to": "-"}}] + * All subclasses don't need to have a max or min * + * @param config property name + * @param defaultValue default value */ - MSTAGE_WATERMARK("ms.watermark", JsonArray.class), - MSTAGE_WATERMARK_GROUPS("ms.watermark.groups", JsonArray.class), - MSTAGE_WORK_UNIT_PARALLELISM_MAX("ms.work.unit.parallelism.max", Integer.class) { - @Override - public boolean validateNonblank(State state) { - Integer parallelMax = getProp(state); - return parallelMax > 0; - } - }, + MultistageProperties(String config, Class className, T defaultValue) { + this.config = config; + this.className = className; + this.defaultValue = defaultValue; + this.maxValue = null; + this.minValue = null; + } + /** - * Work unit partitioning scheme is either a string or a JsonObject. - * - * When it is a string, it will accept values like monthly, weekly, daily, hourly, or none, - * which can be blank or literally "none". - * - * When it is a JsonObject, there can be multiple ways to partition, either with a range. For - * example, following will break 2010-2019 by monthly partitions, and daily partitions afterwards. - * - * {"monthly": ["2010-01-01", "2020-01-01"], "daily": ["2020-01-01": "-"]} - * - * In such case, the partition is called composite. For the composite partition to work, - * the ranges should be continuous with no gaps or overlaps. In order to avoid gaps and overlaps, - * one range end should be the same as another range's start. - * - * Note the end of partition accepts "-" as current date, but it doesn't access PxD syntax, the - * reason being a partition range can be broader than watermark range. - * - * For a composite partition, if the range definition is not as specified, or not valid, then the there - * is no partitioning, equivalent to ms.work.unit.partition='' - * - * For a composite partition, a range is matched against watermark to define partitions, if a range - * is smaller than full partition range, for example {"monthly": ["2020-01-01", "2020-01-18"]}, - * it will still generate a full partition. So to avoid confusion, the range should be at minimum - * 1 partition size. That means, a range should at least 1 month for monthly, or at least 1 week for - * etc. + * Constructor with explicit default, max, and min for subclasses using min/max control * + * @param config property name + * @param defaultValue default value + * @param maxValue max value + * @param minValue min value */ - MSTAGE_WORK_UNIT_PARTITION("ms.work.unit.partition", String.class) { - @Override - public T getDefaultValue() { - return (T) "none"; - } - }, - MSTAGE_WORK_UNIT_PARTIAL_PARTITION("ms.work.unit.partial.partition", Boolean.class) { - @Override - public T getDefaultValue() { - return (T) Boolean.TRUE; - } - }, - MSTAGE_WORK_UNIT_PACING_SECONDS("ms.work.unit.pacing.seconds", Integer.class) { - @Override - public Long getMillis(State state) { - return 1000L * (Integer) this.getProp(state); - } - }, - // this is an internal property, its purpose is to pass value between Source and Extractor - MSTAGE_WORKUNIT_STARTTIME_KEY("ms.work.unit.scheduling.starttime", Long.class) { - @Override - public T getDefaultValue() { - return (T) Long.valueOf(0L); - } - }, - MSTAGE_NORMALIZER_BATCH_SIZE("ms.normalizer.batch.size", Long.class) { - @Override - public T getDefaultValue() { - return (T) Long.valueOf(500L); - } - }, - // Properties defined in Gobblin, redefine here to leverage the new features like validation - CONVERTER_CLASSES("converter.classes", String.class), - DATASET_URN_KEY("dataset.urn", String.class), - ENCRYPT_KEY_LOC("encrypt.key.loc", String.class), - EXTRACTOR_CLASSES("extractor.class", String.class), - - // add a default value of FALSE to Gobblin configuration extract.is.full - EXTRACT_IS_FULL("extract.is.full", Boolean.class) { - public T getDefaultValue() { - return (T) Boolean.FALSE; - } - }, - EXTRACT_NAMESPACE_NAME_KEY("extract.namespace", String.class), - EXTRACT_TABLE_NAME_KEY("extract.table.name", String.class), - EXTRACT_TABLE_TYPE_KEY("extract.table.type", String.class) { - @Override - public T getValidNonblankWithDefault(State state) { - return (T) ((validateNonblank(state)) - ? ((String) getProp(state)).toUpperCase() : "SNAPSHOT_ONLY"); - } - }, - SOURCE_CLASS("source.class", String.class), - SOURCE_CONN_USERNAME("source.conn.username", String.class), - SOURCE_CONN_PASSWORD("source.conn.password", String.class), - SOURCE_CONN_USE_PROXY_URL("source.conn.use.proxy.url", String.class), - SOURCE_CONN_USE_PROXY_PORT("source.conn.use.proxy.port", String.class), - STATE_STORE_ENABLED("state.store.enabled", Boolean.class) { - @Override - public T getDefaultValue() { - return (T) Boolean.TRUE; - } - }, - DATA_PUBLISHER_FINAL_DIR("data.publisher.final.dir", String.class); - - final static private Gson GSON = new Gson(); - final static private String PROPERTY_SEPARATOR = "."; - - private final String config; - private final Class className; - private final Object defaultValue; - - MultistageProperties(String config, Class className) { + MultistageProperties(String config, Class className, T defaultValue, T maxValue, T minValue) { this.config = config; this.className = className; - this.defaultValue = null; + this.defaultValue = defaultValue; + this.maxValue = maxValue; + this.minValue = minValue; } @Override public String toString() { - assert config != null; return config; } /** - * validate accepts blank entry and validates the value when it is non-blank - *

- * This version serves those Source properties - *

- * @param state source state - * @return true when the configuration is blank or non-blank and valid + * Converts configured value to a millisecond value if supported + * @param state state + * @return milliseconds value if supported */ - public boolean validate(State state) { - return true; + public Long getMillis(State state) { + throw new RuntimeException("Not Supported"); } /** - * validate rejects blank entry and only validates the value when it is non-blank - *

- * This version serves those Source properties, the duplication is not ideal, we could - * make this better by define the getProp methods on State, instead of WorkUnitState and - * SourceState separately in Gobblin core. - *

- * @param state source state - * @return true when the configuration is non-blank and valid + * Validates the value when it is non-blank and accepts blank value + * @param state state + * @return sub-classes should override */ - public boolean validateNonblank(State state) { - try { - if (className == JsonArray.class) { - return ((JsonArray) getProp(state)).size() > 0; - } + abstract public boolean isValid(State state); - if (className == JsonObject.class) { - return ((JsonObject) getProp(state)).entrySet().size() > 0; - } - - if (className == Boolean.class) { - // cannot call getPropAsBoolean to tell if a configuration exists - // as FALSE will be return from empty configuration - String prop = state.getProp(config, StringUtils.EMPTY); - return StringUtils.isNotBlank(prop) - && (prop.equalsIgnoreCase("true") || prop.equalsIgnoreCase("false")); - } - - if (className == String.class) { - return StringUtils.isNotEmpty(getProp(state)); - } - - if (className == Integer.class) { - return (Integer) getProp(state) != 0; - } - - if (className == Long.class) { - return (Long) getProp(state) != 0; - } - } catch (Exception e) { - return false; - } - return true; - } + /** + * Validates the value when it is blank + * @param state state + * @return sub-classes should override + */ + abstract public boolean isBlank(State state); - public Long getMillis(State state) { - return 0L; - } + /** + * Validates the value when it is non-blank and rejects blank value + * @param state source state + * @return true when the configuration is non-blank and valid + */ + abstract public boolean isValidNonblank(State state); /** - * get property value for a specified MultistageProperties item - *

- * This version serves those Source properties, the duplication is not ideal, we could - * make this better by define the getProp methods on State, instead of WorkUnitState and - * SourceState separately in Gobblin core. - *

- * @param state the source or work unit state - * @param the template class type - * @return the property value of the specific MultistageProperties item + * Retrieves property value from state object if valid and not blank + * otherwise, return default value of the property type + * + * @param state state + * @return subclasses should override */ - public T getProp(State state) { - if (className == Boolean.class) { - return (T) Boolean.valueOf(state.getPropAsBoolean(config)); - } else if (className == Integer.class) { - return (T) Integer.valueOf(state.getPropAsInt(config, 0)); - } else if (className == Long.class) { - return (T) Long.valueOf(state.getPropAsLong(config, 0L)); - } else if (className == String.class) { - return (T) state.getProp(config, StringUtils.EMPTY); - } else if (className == JsonArray.class) { - return (T) GSON.fromJson(state.getProp(config, new JsonArray().toString()), JsonArray.class); - } else if (className == JsonObject.class) { - return (T) GSON.fromJson(state.getProp(config, new JsonObject().toString()), JsonObject.class); - } - return null; + public T get(State state) { + return getValidNonblankWithDefault(state); } - public T getValidNonblankWithDefault(State state) { - if (className == JsonArray.class) { - return (T) (validateNonblank(state) ? getProp(state) : getDefaultValue()); - } - if (className == JsonObject.class) { - return (T) (validateNonblank(state) ? getProp(state) : getDefaultValue()); - } - if (className == Long.class) { - return (T) ((validateNonblank(state)) ? getProp(state) : getDefaultValue()); - } - if (className == String.class) { - return (T) ((validateNonblank(state)) ? getProp(state) : getDefaultValue()); - } - if (className == Integer.class) { - return (T) ((validateNonblank(state)) ? getProp(state) : getDefaultValue()); - } - if (className == Boolean.class) { - return (T) ((validateNonblank(state)) ? getProp(state) : getDefaultValue()); - } - return getProp(state); - } + /** + * Retrieves property value from state object if valid and not blank + * otherwise, return default value + * @param state state + * @return subclasses should override + */ + abstract protected T getValidNonblankWithDefault(State state); - public T getDefaultValue() { - if (className == JsonArray.class) { - return (T) new JsonArray(); - } - if (className == JsonObject.class) { - return (T) new JsonObject(); - } - if (className == String.class) { - return (T) StringUtils.EMPTY; - } - if (className == Long.class) { - return (T) Long.valueOf(0L); - } - if (className == Integer.class) { - return (T) Integer.valueOf(0); - } - return null; + public boolean isDeprecated() { + return false; } } -// END of enum MultistageProperties \ No newline at end of file diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/PropertyCollection.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/PropertyCollection.java new file mode 100644 index 0000000..c9309d8 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/PropertyCollection.java @@ -0,0 +1,428 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.gson.JsonArray; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import com.linkedin.cdi.util.SchemaUtils; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import org.apache.gobblin.configuration.State; + +import static com.linkedin.cdi.configuration.StaticConstants.*; + + +/** + * Defines all properties here. + * + * Properties can have their unique default values and validation rules through + * inline class override. If the rules become too complicated, a new class + * should be derived to avoid making this class too clumsy. + * + * Definitions are recommended to be organized in alphabetic order. + * + */ +public interface PropertyCollection { + + // default: 0, minimum: 0, maximum: - + IntegerProperties MSTAGE_ABSTINENT_PERIOD_DAYS = new IntegerProperties("ms.abstinent.period.days") { + @Override + public Long getMillis(State state) { + return 24L * 3600L * 1000L * this.get(state); + } + }; + + JsonObjectProperties MSTAGE_ACTIVATION_PROPERTY = new JsonObjectProperties("ms.activation.property"); + JsonObjectProperties MSTAGE_AUTHENTICATION = new JsonObjectProperties("ms.authentication") { + @Override + public boolean isValid(State state) { + if (super.isValid(state) && !super.isBlank(state)) { + JsonObject auth = GSON.fromJson(state.getProp(getConfig()), JsonObject.class); + return auth.entrySet().size() > 0 && auth.has("method") && auth.has("encryption"); + } + return super.isValid(state); + } + }; + + BooleanProperties MSTAGE_BACKFILL = new BooleanProperties("ms.backfill", Boolean.FALSE); + + // default: 0, minimum: 0, maximum: - + LongProperties MSTAGE_CALL_INTERVAL_MILLIS = new LongProperties("ms.call.interval.millis"); + + StringProperties MSTAGE_CONNECTION_CLIENT_FACTORY = new StringProperties("ms.connection.client.factory", + "com.linkedin.cdi.factory.DefaultConnectionClientFactory"); + + CsvProperties MSTAGE_CSV = new CsvProperties("ms.csv"); + + BooleanProperties MSTAGE_DATA_EXPLICIT_EOF = new BooleanProperties("ms.data.explicit.eof", Boolean.FALSE); + JsonObjectProperties MSTAGE_DATA_DEFAULT_TYPE = new JsonObjectProperties("ms.data.default.type"); + StringProperties MSTAGE_DATA_FIELD = new StringProperties("ms.data.field"); + JsonArrayProperties MSTAGE_DERIVED_FIELDS = new JsonArrayProperties("ms.derived.fields") { + @Override + public boolean isValid(State state) { + if (super.isValid(state) && !isBlank(state)) { + // Derived fields should meet general JsonArray configuration requirements + // and contain only JsonObject items that each has a "name" element and a "formula" element + JsonArray derivedFields = GSON.fromJson(state.getProp(getConfig()), JsonArray.class); + for (JsonElement field : derivedFields) { + if (!field.isJsonObject() + || !field.getAsJsonObject().has(KEY_WORD_NAME) + || !field.getAsJsonObject().has(KEY_WORD_FORMULA)) { + return false; + } + } + } + return super.isValid(state); + } + }; + + /** + * Deprecated + * @see #MSTAGE_SCHEMA_CLEANSING + */ + BooleanProperties MSTAGE_ENABLE_CLEANSING = new BooleanProperties("ms.enable.cleansing", Boolean.TRUE) { + @Override + public boolean isDeprecated() { + return true; + } + }; + + BooleanProperties MSTAGE_ENABLE_DYNAMIC_FULL_LOAD = new BooleanProperties("ms.enable.dynamic.full.load", Boolean.TRUE); + BooleanProperties MSTAGE_ENABLE_SCHEMA_BASED_FILTERING = new BooleanProperties("ms.enable.schema.based.filtering", Boolean.TRUE); + JsonArrayProperties MSTAGE_ENCRYPTION_FIELDS = new JsonArrayProperties("ms.encryption.fields") { + @Override + public boolean isValid(State state) { + if (super.isValid(state) && !isBlank(state)) { + // Encrypted fields cannot be nullable, required: isNullable = false + if (!MSTAGE_OUTPUT_SCHEMA.isBlank(state)) { + JsonArray encryptionFields = GSON.fromJson(state.getProp(getConfig()), JsonArray.class); + for (JsonElement field : encryptionFields) { + if (!field.isJsonPrimitive() || field.getAsString().isEmpty() || SchemaUtils.isNullable(field.getAsString(), + MSTAGE_OUTPUT_SCHEMA.get(state))) { + return false; + } + } + } + } + return super.isValid(state); + } + }; + StringProperties MSTAGE_EXTRACTOR_CLASS = new StringProperties("ms.extractor.class"); + StringProperties MSTAGE_EXTRACTOR_TARGET_FILE_NAME = new StringProperties("ms.extractor.target.file.name"); + StringProperties MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION = new StringProperties("ms.extractor.target.file.permission", "755"); + StringProperties MSTAGE_EXTRACT_PREPROCESSORS = new StringProperties("ms.extract.preprocessors"); + JsonObjectProperties MSTAGE_EXTRACT_PREPROCESSORS_PARAMETERS = new JsonObjectProperties("ms.extract.preprocessor.parameters"); + + // default: 0, minimum: 0, maximum: - + IntegerProperties MSTAGE_GRACE_PERIOD_DAYS = new IntegerProperties("ms.grace.period.days") { + @Override + public Long getMillis(State state) { + return 24L * 3600L * 1000L * this.get(state); + } + }; + + // ms.http.maxConnections has default value 50 and max value 500 + // 0 is interpreted as default + IntegerProperties MSTAGE_HTTP_CONN_MAX = + new IntegerProperties("ms.http.conn.max", 50, 500) { + @Override + protected Integer getValidNonblankWithDefault(State state) { + int value = super.getValidNonblankWithDefault(state); + return value == 0 ? getDefaultValue() : value; + } + }; + + // ms.http.maxConnectionsPerRoute has default value 20 and max value 200 + // 0 is interpreted as default + IntegerProperties MSTAGE_HTTP_CONN_PER_ROUTE_MAX = + new IntegerProperties("ms.http.conn.per.route.max", 20, 200) { + @Override + protected Integer getValidNonblankWithDefault(State state) { + int value = super.getValidNonblankWithDefault(state); + return value == 0 ? getDefaultValue() : value; + } + }; + + /** + * see org.apache.http.impl.client.HttpClientBuilder#connTimeToLive + */ + IntegerProperties MSTAGE_HTTP_CONN_TTL_SECONDS = new IntegerProperties("ms.http.conn.ttl.seconds", 10) { + @Override + public Long getMillis(State state) { + return 1000L * this.get(state); + } + }; + + JsonObjectProperties MSTAGE_HTTP_REQUEST_HEADERS = new JsonObjectProperties("ms.http.request.headers"); + StringProperties MSTAGE_HTTP_REQUEST_METHOD = new StringProperties("ms.http.request.method"); + JsonObjectProperties MSTAGE_HTTP_RESPONSE_TYPE = new JsonObjectProperties("ms.http.response.type"); + JsonObjectProperties MSTAGE_HTTP_STATUSES = new JsonObjectProperties("ms.http.statuses", + GSON.fromJson("{\"success\":[200,201,202], \"pagination_error\":[401]}", JsonObject.class)); + JsonObjectProperties MSTAGE_HTTP_STATUS_REASONS = new JsonObjectProperties("ms.http.status.reasons"); + StringProperties MSTAGE_JDBC_SCHEMA_REFACTOR = new StringProperties("ms.jdbc.schema.refactor", "none"); + StringProperties MSTAGE_JDBC_STATEMENT = new StringProperties("ms.jdbc.statement"); + StringProperties MSTAGE_KAFKA_BROKERS = new StringProperties("ms.kafka.brokers"); + StringProperties MSTAGE_KAFKA_SCHEMA_REGISTRY_URL = new StringProperties("ms.kafka.schema.registry.url"); + StringProperties MSTAGE_KAFKA_CLIENT_ID = new StringProperties("ms.kafka.clientId"); + StringProperties MSTAGE_KAFKA_TOPIC_NAME = new StringProperties("ms.kafka.audit.topic.name"); + + // default: 500, minimum: 1, maximum: - + LongProperties MSTAGE_NORMALIZER_BATCH_SIZE = new LongProperties("ms.normalizer.batch.size", 500L, Long.MAX_VALUE, 1L); + + JsonArrayProperties MSTAGE_OUTPUT_SCHEMA = new JsonArrayProperties("ms.output.schema"); + JsonObjectProperties MSTAGE_PAGINATION = new JsonObjectProperties("ms.pagination"); + JsonArrayProperties MSTAGE_PARAMETERS = new JsonArrayProperties("ms.parameters") { + @Override + public boolean isValid(State state) { + if (super.isValid(state) && !isBlank(state)) { + // Derived fields should meet general JsonArray configuration requirements + // and contain only JsonObject items that each has a "name" element and a "formula" element + JsonArray parameters = GSON.fromJson(state.getProp(getConfig()), JsonArray.class); + return parameters.size() > 0; + } + return super.isValid(state); + } + }; + + JsonArrayProperties MSTAGE_PAYLOAD_PROPERTY = new JsonArrayProperties("ms.payload.property"); + JsonObjectProperties MSTAGE_RETENTION = + new JsonObjectProperties("ms.retention") { + @Override + public JsonObject getDefaultValue() { + JsonObject retention = new JsonObject(); + retention.addProperty("state.store", "P90D"); // keep 90 days state store by default + retention.addProperty("publish.dir", "P731D"); // keep 2 years published data + retention.addProperty("log", "P30D"); + return retention; + } + }; + + // default: 1000, minimum: 1, maximum: - + IntegerProperties MSTAGE_S3_LIST_MAX_KEYS = new IntegerProperties("ms.s3.list.max.keys", 1000, Integer.MAX_VALUE, 1); + + JsonObjectProperties MSTAGE_SCHEMA_CLEANSING = new JsonObjectProperties("ms.schema.cleansing"); + JsonArrayProperties MSTAGE_SECONDARY_INPUT = new JsonArrayProperties("ms.secondary.input"); + JsonObjectProperties MSTAGE_SESSION_KEY_FIELD = new JsonObjectProperties("ms.session.key.field"); + + // default: 60 seconds, minimum: 0, maximum: - + IntegerProperties MSTAGE_SFTP_CONN_TIMEOUT_MILLIS = new IntegerProperties("ms.sftp.conn.timeout.millis", 60000); + + StringProperties MSTAGE_SOURCE_DATA_CHARACTER_SET = new StringProperties("ms.source.data.character.set", + StandardCharsets.UTF_8.toString()); + + StringProperties MSTAGE_SOURCE_FILES_PATTERN = new StringProperties("ms.source.files.pattern", REGEXP_DEFAULT_PATTERN); + JsonObjectProperties MSTAGE_SOURCE_S3_PARAMETERS = new JsonObjectProperties("ms.source.s3.parameters"); + StringProperties MSTAGE_SOURCE_SCHEMA_URN = new StringProperties("ms.source.schema.urn"); + StringProperties MSTAGE_SOURCE_URI = new StringProperties("ms.source.uri"); + + SslProperties MSTAGE_SSL = new SslProperties("ms.ssl"); + JsonArrayProperties MSTAGE_TARGET_SCHEMA = new JsonArrayProperties("ms.target.schema"); + StringProperties MSTAGE_TARGET_SCHEMA_URN = new StringProperties("ms.target.schema.urn"); + StringProperties MSTAGE_TOTAL_COUNT_FIELD = new StringProperties("ms.total.count.field"); + JsonObjectProperties MSTAGE_VALIDATION_ATTRIBUTES = + new JsonObjectProperties("ms.validation.attributes") { + @Override + public JsonObject getDefaultValue() { + JsonObject attributesJson = new JsonObject(); + attributesJson.addProperty(StaticConstants.KEY_WORD_THRESHOLD, 0); + attributesJson.addProperty(StaticConstants.KEY_WORD_CRITERIA, StaticConstants.KEY_WORD_FAIL); + return attributesJson; + } + }; + + // default: 600 second, minimum: 0 second, maximum: 24 hours + LongProperties MSTAGE_WAIT_TIMEOUT_SECONDS = new LongProperties("ms.wait.timeout.seconds", 600L, 24 * 3600L, 0L) { + @Override + public Long getMillis(State state) { + return 1000L * this.get(state); + } + }; + + WatermarkProperties MSTAGE_WATERMARK = new WatermarkProperties("ms.watermark"); + + JsonArrayProperties MSTAGE_WATERMARK_GROUPS = new JsonArrayProperties("ms.watermark.groups"); + + // default: 0, minimum: 0, maximum: - + LongProperties MSTAGE_WORK_UNIT_SCHEDULING_STARTTIME = new LongProperties("ms.work.unit.scheduling.starttime"); + + // default: 0, minimum: 0, maximum: - + LongProperties MSTAGE_WORK_UNIT_MIN_RECORDS = new LongProperties("ms.work.unit.min.records"); + + // default: 0, minimum: 0, maximum: - + LongProperties MSTAGE_WORK_UNIT_MIN_UNITS = new LongProperties("ms.work.unit.min.units"); + + // default: 0, minimum: 0, maximum: - + IntegerProperties MSTAGE_WORK_UNIT_PACING_SECONDS = new IntegerProperties("ms.work.unit.pacing.seconds") { + @Override + public Long getMillis(State state) { + return 1000L * this.get(state); + } + }; + + // default: 100, minimum: 0, maximum: 1000, 0 = default value + IntegerProperties MSTAGE_WORK_UNIT_PARALLELISM_MAX = new IntegerProperties("ms.work.unit.parallelism.max", 100, 1000, 0) { + @Override + protected Integer getValidNonblankWithDefault(State state) { + int value = super.getValidNonblankWithDefault(state); + return value == 0 ? getDefaultValue() : value; + } + }; + + BooleanProperties MSTAGE_WORK_UNIT_PARTIAL_PARTITION = + new BooleanProperties("ms.work.unit.partial.partition", Boolean.TRUE); + StringProperties MSTAGE_WORK_UNIT_PARTITION = new StringProperties("ms.work.unit.partition", "none"); + StringProperties CONVERTER_CLASSES = new StringProperties("converter.classes"); + StringProperties DATA_PUBLISHER_FINAL_DIR = new StringProperties("data.publisher.final.dir"); + StringProperties DATASET_URN = new StringProperties("dataset.urn"); + StringProperties ENCRYPT_KEY_LOC = new StringProperties("encrypt.key.loc"); + StringProperties EXTRACTOR_CLASSES = new StringProperties("extractor.class"); + // add a default value of FALSE to Gobblin configuration extract.is.full + BooleanProperties EXTRACT_IS_FULL = new BooleanProperties("extract.is.full", Boolean.FALSE); + StringProperties EXTRACT_NAMESPACE = new StringProperties("extract.namespace"); + StringProperties EXTRACT_TABLE_NAME = new StringProperties("extract.table.name"); + StringProperties EXTRACT_TABLE_TYPE = new StringProperties("extract.table.type", "SNAPSHOT_ONLY") { + @Override + protected String getValidNonblankWithDefault(State state) { + return super.getValidNonblankWithDefault(state).toUpperCase(); + } + }; + + StringProperties JOB_DIR = new StringProperties("job.dir"); + StringProperties JOB_NAME = new StringProperties("job.name"); + StringProperties SOURCE_CLASS = new StringProperties("source.class"); + StringProperties SOURCE_CONN_HOST = new StringProperties("source.conn.host"); + StringProperties SOURCE_CONN_KNOWN_HOSTS = new StringProperties("source.conn.known.hosts"); + StringProperties SOURCE_CONN_USERNAME = new StringProperties("source.conn.username"); + StringProperties SOURCE_CONN_PASSWORD = new StringProperties("source.conn.password"); + StringProperties SOURCE_CONN_PORT = new StringProperties("source.conn.port"); + StringProperties SOURCE_CONN_PRIVATE_KEY = new StringProperties("source.conn.private.key"); + StringProperties SOURCE_CONN_USE_PROXY_URL = new StringProperties("source.conn.use.proxy.url"); + StringProperties SOURCE_CONN_USE_PROXY_PORT = new StringProperties("source.conn.use.proxy.port"); + StringProperties STATE_STORE_DIR = new StringProperties("state.store.dir"); + BooleanProperties STATE_STORE_ENABLED = new BooleanProperties("state.store.enabled", Boolean.TRUE); + StringProperties STATE_STORE_TYPE = new StringProperties("state.store.type"); + IntegerProperties TASK_MAXRETRIES = new IntegerProperties("task.maxretries", 4); + IntegerProperties TASKEXECUTOR_THREADPOOL_SIZE = new IntegerProperties("taskexecutor.threadpool.size", 10); + + List> allProperties = Lists.newArrayList( + MSTAGE_ABSTINENT_PERIOD_DAYS, + MSTAGE_ACTIVATION_PROPERTY, + MSTAGE_AUTHENTICATION, + MSTAGE_BACKFILL, + MSTAGE_CALL_INTERVAL_MILLIS, + MSTAGE_CONNECTION_CLIENT_FACTORY, + MSTAGE_CSV, + MSTAGE_DATA_EXPLICIT_EOF, + MSTAGE_DATA_DEFAULT_TYPE, + MSTAGE_DATA_FIELD, + MSTAGE_DERIVED_FIELDS, + MSTAGE_ENABLE_CLEANSING, + MSTAGE_ENABLE_DYNAMIC_FULL_LOAD, + MSTAGE_ENABLE_SCHEMA_BASED_FILTERING, + MSTAGE_ENCRYPTION_FIELDS, + MSTAGE_EXTRACTOR_CLASS, + MSTAGE_EXTRACTOR_TARGET_FILE_NAME, + MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION, + MSTAGE_EXTRACT_PREPROCESSORS, + MSTAGE_EXTRACT_PREPROCESSORS_PARAMETERS, + MSTAGE_GRACE_PERIOD_DAYS, + MSTAGE_HTTP_CONN_MAX, + MSTAGE_HTTP_CONN_PER_ROUTE_MAX, + MSTAGE_HTTP_CONN_TTL_SECONDS, + MSTAGE_HTTP_REQUEST_HEADERS, + MSTAGE_HTTP_REQUEST_METHOD, + MSTAGE_HTTP_RESPONSE_TYPE, + MSTAGE_HTTP_STATUSES, + MSTAGE_HTTP_STATUS_REASONS, + MSTAGE_JDBC_SCHEMA_REFACTOR, + MSTAGE_JDBC_STATEMENT, + MSTAGE_KAFKA_BROKERS, + MSTAGE_KAFKA_SCHEMA_REGISTRY_URL, + MSTAGE_KAFKA_CLIENT_ID, + MSTAGE_KAFKA_TOPIC_NAME, + MSTAGE_NORMALIZER_BATCH_SIZE, + MSTAGE_OUTPUT_SCHEMA, + MSTAGE_PAGINATION, + MSTAGE_PARAMETERS, + MSTAGE_PAYLOAD_PROPERTY, + MSTAGE_RETENTION, + MSTAGE_S3_LIST_MAX_KEYS, + MSTAGE_SCHEMA_CLEANSING, + MSTAGE_SECONDARY_INPUT, + MSTAGE_SESSION_KEY_FIELD, + MSTAGE_SFTP_CONN_TIMEOUT_MILLIS, + MSTAGE_SOURCE_DATA_CHARACTER_SET, + MSTAGE_SOURCE_FILES_PATTERN, + MSTAGE_SOURCE_S3_PARAMETERS, + MSTAGE_SOURCE_SCHEMA_URN, + MSTAGE_SOURCE_URI, + MSTAGE_SSL, + MSTAGE_TARGET_SCHEMA, + MSTAGE_TARGET_SCHEMA_URN, + MSTAGE_TOTAL_COUNT_FIELD, + MSTAGE_VALIDATION_ATTRIBUTES, + MSTAGE_WAIT_TIMEOUT_SECONDS, + MSTAGE_WATERMARK, + MSTAGE_WATERMARK_GROUPS, + MSTAGE_WORK_UNIT_SCHEDULING_STARTTIME, + MSTAGE_WORK_UNIT_MIN_RECORDS, + MSTAGE_WORK_UNIT_MIN_UNITS, + MSTAGE_WORK_UNIT_PACING_SECONDS, + MSTAGE_WORK_UNIT_PARALLELISM_MAX, + MSTAGE_WORK_UNIT_PARTIAL_PARTITION, + MSTAGE_WORK_UNIT_PARTITION, + CONVERTER_CLASSES, + DATA_PUBLISHER_FINAL_DIR, + DATASET_URN, + ENCRYPT_KEY_LOC, + EXTRACTOR_CLASSES, + EXTRACT_IS_FULL, + EXTRACT_NAMESPACE, + EXTRACT_TABLE_NAME, + EXTRACT_TABLE_TYPE, + JOB_DIR, + JOB_NAME, + SOURCE_CLASS, + SOURCE_CONN_HOST, + SOURCE_CONN_KNOWN_HOSTS, + SOURCE_CONN_PASSWORD, + SOURCE_CONN_PORT, + SOURCE_CONN_PRIVATE_KEY, + SOURCE_CONN_USERNAME, + SOURCE_CONN_USE_PROXY_URL, + SOURCE_CONN_USE_PROXY_PORT, + STATE_STORE_DIR, + STATE_STORE_ENABLED, + STATE_STORE_TYPE, + TASK_MAXRETRIES, + TASKEXECUTOR_THREADPOOL_SIZE + ); + + Map> deprecatedProperties = + new ImmutableMap.Builder>() + .put("ms.csv.column.header", MSTAGE_CSV) + .put("ms.csv.column.header.index", MSTAGE_CSV) + .put("ms.csv.column.projection", MSTAGE_CSV) + .put("ms.csv.default.field.type", MSTAGE_CSV) + .put("ms.csv.escape.character", MSTAGE_CSV) + .put("ms.csv.quote.character", MSTAGE_CSV) + .put("ms.csv.separator", MSTAGE_CSV) + .put("ms.csv.skip.lines", MSTAGE_CSV) + .put("ms.converter.csv.max.failures", MSTAGE_CSV) + .put("ms.converter.keep.null.strings", MSTAGE_CSV) + .put("csv.max.failures", MSTAGE_CSV) + .put("sftpConn.timeout", MSTAGE_SFTP_CONN_TIMEOUT_MILLIS) + .put("gaap.http.maxConnectionsPerRoute", MSTAGE_HTTP_CONN_PER_ROUTE_MAX) + .put("gaap.http.maxConnections", MSTAGE_HTTP_CONN_MAX) + .put("gaap.trustStorePath", MSTAGE_SSL) + .put("gaap.authType", MSTAGE_CONNECTION_CLIENT_FACTORY) + .put("ms.kraken.enabled", MSTAGE_CONNECTION_CLIENT_FACTORY) + .put("gobblinGaapHttpClientFactory.authType", MSTAGE_CONNECTION_CLIENT_FACTORY) + .build(); +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/SslProperties.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/SslProperties.java new file mode 100644 index 0000000..4df665e --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/SslProperties.java @@ -0,0 +1,127 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import com.google.common.collect.Lists; +import com.google.gson.JsonObject; +import java.util.List; +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.State; + +import static com.linkedin.cdi.configuration.StaticConstants.*; + + +/** + * SSL Parameters + */ +public class SslProperties extends JsonObjectProperties{ + final private static String KEY_STORE_TYPE = "keyStoreType"; + final private static String KEY_STORE_PATH = "keyStorePath"; + final private static String KEY_STORE_PASSWORD = "keyStorePassword"; + final private static String KEY_PASSWORD = "keyPassword"; + final private static String TRUST_STORE_PATH = "trustStorePath"; + final private static String TRUST_STORE_PASSWORD = "trustStorePassword"; + final private static String CONNECTION_TIMEOUT = "connectionTimeoutSeconds"; + final private static String SOCKET_TIMEOUT = "socketTimeoutSeconds"; + final private static String VERSION = "version"; + final private static String VERSION_DEFAULT = "TLSv1.2"; + final private static String KEY_STORE_TYPE_DEFAULT = "pkcs12"; + + final private static List allAttributes = Lists.newArrayList( + KEY_STORE_TYPE, KEY_STORE_PATH, KEY_STORE_PASSWORD, KEY_PASSWORD, + TRUST_STORE_PATH, TRUST_STORE_PASSWORD, + CONNECTION_TIMEOUT, SOCKET_TIMEOUT, + VERSION + ); + + @Override + public boolean isValid(State state) { + if (super.isValid(state) && !super.isBlank(state)) { + JsonObject value = GSON.fromJson(state.getProp(getConfig()), JsonObject.class); + return value.entrySet().stream().allMatch(p -> allAttributes.contains(p.getKey())); + } + return super.isValid(state); + } + + /** + * Constructor with implicit default value + * @param config property name + */ + SslProperties(String config) { + super(config); + } + + public String getVersion(State state) { + JsonObject value = get(state); + if (value.has(VERSION)) { + return value.get(VERSION).getAsString(); + } + return VERSION_DEFAULT; + } + + public String getKeyStoreType(State state) { + JsonObject value = get(state); + if (value.has(KEY_STORE_TYPE)) { + return value.get(KEY_STORE_TYPE).getAsString(); + } + return KEY_STORE_TYPE_DEFAULT; + } + + public String getKeyStorePath(State state) { + JsonObject value = get(state); + if (value.has(KEY_STORE_PATH)) { + return get(state).getAsJsonObject().get(KEY_STORE_PATH).getAsString(); + } + return StringUtils.EMPTY; + } + + public String getKeyStorePassword(State state) { + JsonObject value = get(state); + if (value.has(KEY_STORE_PASSWORD)) { + return get(state).getAsJsonObject().get(KEY_STORE_PASSWORD).getAsString(); + } + return StringUtils.EMPTY; + } + + public String getKeyPassword(State state) { + JsonObject value = get(state); + if (value.has(KEY_PASSWORD)) { + return value.get(KEY_PASSWORD).getAsString(); + } + return StringUtils.EMPTY; + } + + public String getTrustStorePath(State state) { + JsonObject value = get(state); + if (value.has(TRUST_STORE_PATH)) { + return value.get(TRUST_STORE_PATH).getAsString(); + } + return StringUtils.EMPTY; + } + + public String getTrustStorePassword(State state) { + JsonObject value = get(state); + if (value.has(TRUST_STORE_PASSWORD)) { + return value.get(TRUST_STORE_PASSWORD).getAsString(); + } + return StringUtils.EMPTY; + } + + public Integer getConnectionTimeoutMillis(State state) { + JsonObject value = get(state); + if (value.has(CONNECTION_TIMEOUT) && value.get(CONNECTION_TIMEOUT).getAsInt() >= 0) { + return 1000 * value.get(CONNECTION_TIMEOUT).getAsInt(); + } + return 60 * 1000; + } + + public Integer getSocketTimeoutMillis(State state) { + JsonObject value = get(state); + if (value.has(SOCKET_TIMEOUT) && value.get(SOCKET_TIMEOUT).getAsInt() >= 0) { + return 1000 * value.get(SOCKET_TIMEOUT).getAsInt(); + } + return 60 * 1000; + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/StaticConstants.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/StaticConstants.java index 75d5aa7..4cc83b8 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/configuration/StaticConstants.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/StaticConstants.java @@ -15,14 +15,18 @@ public interface StaticConstants { String KEY_WORD_BOOLEAN = "boolean"; String KEY_WORD_CATEGORY = "category"; String KEY_WORD_COLUMN_NAME = "columnName"; + String KEY_WORD_COMMA = ","; String KEY_WORD_DATA_IS_NULLABLE = "isNullable"; String KEY_WORD_DATA_TYPE = "dataType"; String KEY_WORD_DATA_TYPE_TYPE = "dataType.type"; String KEY_WORD_EOF = "EOF"; + String KEY_WORD_EPOC = "epoc"; String KEY_WORD_FIELDS = "fields"; - String KEY_WORD_RANGE_FROM = "from"; + String KEY_WORD_FORMULA = "formula"; + String KEY_WORD_FROM = "from"; String KEY_WORD_HTTP_OK = "ok"; String KEY_WORD_INTEGER = "integer"; + String KEY_WORD_IS_NULLABLE = "isNullable"; String KEY_WORD_ITEMS = "items"; String KEY_WORD_MAP = "map"; String KEY_WORD_NAME = "name"; @@ -37,6 +41,7 @@ public interface StaticConstants { String KEY_WORD_PROPERTIES = "properties"; String KEY_WORD_RANGE = "range"; String KEY_WORD_RECORD = "record"; + String KEY_WORD_REGEXP = "regexp"; String KEY_WORD_RETRY = "retry"; String KEY_WORD_RETRY_COUNT = "retryCount"; String KEY_WORD_RETRY_DELAY_IN_SEC = "delayInSec"; @@ -46,13 +51,37 @@ public interface StaticConstants { String KEY_WORD_SOURCE_TYPE = "source.type"; String KEY_WORD_STRING = "string"; String KEY_WORD_SYMBOLS = "symbols"; - String KEY_WORD_RANGE_TO = "to"; String KEY_WORD_TIMESTAMP = "timestamp"; + String KEY_WORD_TO = "to"; String KEY_WORD_TYPE = "type"; String KEY_WORD_UNITS = "units"; String KEY_WORD_UNKNOWN = "unknown"; String KEY_WORD_VALUES = "values"; + String KEY_WORD_THRESHOLD = "threshold"; + String KEY_WORD_CRITERIA = "criteria"; + String KEY_WORD_FAIL = "fail"; + String KEY_WORD_SUCCESS = "success"; + String KEY_WORD_ERROR_COLUMN = "errorColumn"; + String KEY_WORD_INT = "int"; + String KEY_WORD_LONG = "long"; + String KEY_WORD_DOUBLE = "double"; + String KEY_WORD_FLOAT = "float"; + String KEY_WORD_JSON = "json"; + String KEY_WORD_CSV = "csv"; + String KEY_WORD_AVRO = "avro"; - Gson GSON = new Gson(); + String PROPERTY_DELIMINATOR = "."; + String REGEXP_DEFAULT_PATTERN = ".*"; + String DOC_BASE_URL = "https://github.com/linkedin/data-integration-library/blob/master/docs"; + + String EXCEPTION_WORK_UNIT_MINIMUM = "Job requires a minimum of %s work unit(s) to proceed because ms.work.unit.min.units = %s."; + String EXCEPTION_RECORD_MINIMUM = "Work unit requires a minimum of %s record(s) to succeed because ms.work.unit.min.records = %s."; + String EXCEPTION_INCORRECT_CONFIGURATION = "Property %s has incorrect configuration: %s, see: %s"; + String EXCEPTION_DEPRECATED_CONFIGURATION = "Property %s has been deprecated, and the replacement is: %s, see: %s"; + String MSG_ROWS_PROCESSED = "Processed %s records, work unit: %s"; + String MSG_WORK_UNIT_ALWAYS = "There should be a work unit."; + String MSG_LOW_WATER_MARK_ALWAYS = "There should be a low watermark."; + String MSG_WORK_UNIT_INFO = "Generating Work Unit: %s, watermark: %s"; + Gson GSON = new Gson(); } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/StringProperties.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/StringProperties.java new file mode 100644 index 0000000..25ea7e4 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/StringProperties.java @@ -0,0 +1,84 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.State; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * A String type of property has default defaultValue of "" + */ +public class StringProperties extends MultistageProperties { + private static final Logger LOG = LoggerFactory.getLogger(StringProperties.class); + + /** + * Constructor with implicit default value + * @param config property name + */ + StringProperties(String config) { + super(config, String.class, StringUtils.EMPTY); + } + + /** + * Constructor with explicit default value + * @param config property name + * @param defaultValue default value + */ + StringProperties(String config, String defaultValue) { + super(config, String.class, defaultValue); + } + + /** + * Validates the value when it is blank + * - No configuration is considered blank + * - A blank string is considered blank + * + * @param state state + * @return true if blank + */ + @Override + public boolean isBlank(State state) { + return !state.contains(getConfig()) + || StringUtils.isBlank(state.getProp(getConfig())); + } + + /** + * Strings blank or not are always valid + * @param state state + * @return true + */ + @Override + public boolean isValid(State state) { + return true; + } + + /** + * Validates the value when it is non-blank and rejects blank value + * + * @param state source state + * @return true when the configuration is non-blank and valid + */ + public boolean isValidNonblank(State state) { + return !isBlank(state) && isValid(state); + } + + /** + * Retrieves property value from state object if valid and not blank + * otherwise, return default value + * + * @param state state + * @return property value if non-blank and valid, otherwise the default value + * @see #get(State) + */ + protected String getValidNonblankWithDefault(State state) { + if (isValidNonblank(state)) { + return state.getProp(getConfig()); + } + return getDefaultValue(); + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/configuration/WatermarkProperties.java b/cdi-core/src/main/java/com/linkedin/cdi/configuration/WatermarkProperties.java new file mode 100644 index 0000000..0cf883b --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/configuration/WatermarkProperties.java @@ -0,0 +1,138 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import com.google.common.collect.Lists; +import com.google.gson.JsonArray; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import com.linkedin.cdi.util.JsonUtils; +import java.util.ArrayList; +import java.util.List; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.gobblin.configuration.State; + +import static com.linkedin.cdi.configuration.StaticConstants.*; + +/** + * Watermark definitions + */ +public class WatermarkProperties extends JsonArrayProperties { + final private static Pair DATETIME_WATERMARK_DEFAULT = new ImmutablePair("2020-01-01", "P0D"); + + /** + * Constructor with implicit default value + * @param config property name + */ + WatermarkProperties(String config) { + super(config); + } + + @Override + public boolean isValid(State state) { + final List types = Lists.newArrayList("unit", "datetime"); + if (super.isValid(state) && !isBlank(state)) { + // Derived fields should meet general JsonArray configuration requirements + // and contain only JsonObject items that each has a "name" element and a "formula" element + JsonArray value = GSON.fromJson(state.getProp(getConfig()), JsonArray.class); + if (value.size() == 0) { + return false; + } + int dateTimeWatermarks = 0; + int unitWatermarks = 0; + for (JsonElement def: value) { + if (!def.isJsonObject()) { + return false; + } + + if (!def.getAsJsonObject().has(KEY_WORD_NAME) || !def.getAsJsonObject().has(KEY_WORD_TYPE)) { + return false; + } + + String type = def.getAsJsonObject().get(KEY_WORD_TYPE).getAsString(); + if (types.stream().noneMatch(t -> t.equals(type))) { + return false; + } + + // a datetime watermark must have a "range" element, and the element + // must have a "from" and a "to" + if (type.equalsIgnoreCase("datetime")) { + dateTimeWatermarks++; + if (!def.getAsJsonObject().has(KEY_WORD_RANGE) || !def.getAsJsonObject().get(KEY_WORD_RANGE).isJsonObject()) { + return false; + } + JsonObject range = def.getAsJsonObject().get(KEY_WORD_RANGE).getAsJsonObject(); + if (!range.has(KEY_WORD_FROM) || ! range.has(KEY_WORD_TO)) { + return false; + } + + String from = range.get(KEY_WORD_FROM).getAsString(); + // TODO validate from + String to = range.get(KEY_WORD_TO).getAsString(); + // TODO validate to + } + + // a unit watermark must have a "units" element, and the element + // can be an array or a string of comma separated values + if (type.equalsIgnoreCase("unit")) { + unitWatermarks++; + if (!def.getAsJsonObject().has(KEY_WORD_UNITS)) { + return false; + } + + JsonElement units = def.getAsJsonObject().get(KEY_WORD_UNITS); + if (!units.isJsonPrimitive() && !units.isJsonArray()) { + return false; + } + } + + if (dateTimeWatermarks > 1 || unitWatermarks > 1) { + return false; + } + } + } + return super.isValid(state); + } + + /** + * Parse out the date range of the datetime watermark + * @param state state object + * @return date range + */ + public Pair getRanges(State state) { + JsonArray dateWatermark = JsonUtils.filter(KEY_WORD_TYPE, "datetime", get(state)); + if (dateWatermark.isJsonNull()) { + return DATETIME_WATERMARK_DEFAULT; + } + JsonObject range = dateWatermark.get(0).getAsJsonObject().get(KEY_WORD_RANGE).getAsJsonObject(); + return new ImmutablePair<>(range.get(KEY_WORD_FROM).getAsString(), range.get(KEY_WORD_TO).getAsString()); + } + + /** + * Parse out the units of unit watermark if exist + * @param state state object + * @return units + */ + public List getUnits(State state) { + JsonArray unitWatermark = JsonUtils.filter(KEY_WORD_TYPE, "unit", get(state)); + if (unitWatermark.isJsonNull()) { + return new ArrayList<>(); + } + + // units can be a comma delimited string or a Json array of strings + JsonElement units = unitWatermark.get(0).getAsJsonObject().get("units"); + if (units.isJsonArray()) { + List unitList = new ArrayList<>(); + for( JsonElement unit: units.getAsJsonArray()) { + unitList.add(unit.getAsString()); + } + return unitList; + } else if (units.isJsonPrimitive()) { + return Lists.newArrayList(units.getAsString().split(KEY_WORD_COMMA)); + } + return null; + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/connection/HdfsConnection.java b/cdi-core/src/main/java/com/linkedin/cdi/connection/HdfsConnection.java index e7eb49b..f7af7a0 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/connection/HdfsConnection.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/connection/HdfsConnection.java @@ -5,25 +5,24 @@ package com.linkedin.cdi.connection; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import java.io.InputStream; -import java.net.URI; -import java.util.List; -import java.util.stream.Collectors; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; -import org.apache.gobblin.configuration.State; import com.linkedin.cdi.exception.RetriableAuthenticationException; import com.linkedin.cdi.keys.ExtractorKeys; import com.linkedin.cdi.keys.HdfsKeys; import com.linkedin.cdi.keys.JobKeys; import com.linkedin.cdi.util.InputStreamUtils; import com.linkedin.cdi.util.WorkUnitStatus; +import java.io.InputStream; +import java.net.URI; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.gobblin.configuration.State; import org.apache.gobblin.source.extractor.filebased.FileBasedHelperException; import org.apache.gobblin.source.extractor.filebased.TimestampAwareFileBasedHelper; import org.apache.gobblin.source.extractor.hadoop.HadoopFsHelper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -32,12 +31,23 @@ * * @author Chris Li */ -@Slf4j public class HdfsConnection extends MultistageConnection { + private static final Logger LOG = LoggerFactory.getLogger(HdfsConnection.class); + + public HdfsKeys getHdfsKeys() { + return hdfsKeys; + } + + public TimestampAwareFileBasedHelper getFsHelper() { + return fsHelper; + } + + public void setFsHelper(TimestampAwareFileBasedHelper fsHelper) { + this.fsHelper = fsHelper; + } + private final static String URI_REGEXP_PATTERN = "RE="; - @Getter final private HdfsKeys hdfsKeys; - @Setter (AccessLevel.PACKAGE) private TimestampAwareFileBasedHelper fsHelper; public HdfsConnection(State state, JobKeys jobKeys, ExtractorKeys extractorKeys) { @@ -66,7 +76,7 @@ public HdfsConnection(State state, JobKeys jobKeys, ExtractorKeys extractorKeys) */ @Override public WorkUnitStatus execute(final WorkUnitStatus status) { - assert hdfsKeys.getSourceUri() != null; + Preconditions.checkNotNull(hdfsKeys.getSourceUri(), "ms.source.uri is missing or of wrong format"); URI uri = URI.create(getWorkUnitSpecificString(hdfsKeys.getSourceUri(), getExtractorKeys().getDynamicParameters())); @@ -98,7 +108,7 @@ public boolean closeAll(String message) { fsHelper = null; return true; } catch (Exception e) { - log.error("Error closing file system connection", e); + LOG.error("Error closing file system connection", e); return false; } } @@ -131,7 +141,7 @@ private List readFileList(final String path, final String pattern) { .filter(fileName -> fileName.matches(pattern)) .collect(Collectors.toList()); } catch (FileBasedHelperException e) { - log.error("Not able to run ls command due to " + e.getMessage(), e); + LOG.error("Not able to run ls command due to " + e.getMessage(), e); } return Lists.newArrayList(); } @@ -142,10 +152,11 @@ private List readFileList(final String path, final String pattern) { * @return the file content in an InputStream */ private InputStream readSingleFile(final String path) { + LOG.info("Processing file: {}", path); try { return fsHelper.getFileStream(path); } catch (FileBasedHelperException e) { - log.error("Not able to run getFileStream command due to " + e.getMessage(), e); + LOG.error("Not able to run getFileStream command due to " + e.getMessage(), e); return null; } } @@ -157,7 +168,7 @@ TimestampAwareFileBasedHelper getHdfsClient() { fsHelper.connect(); return fsHelper; } catch (Exception e) { - log.error("Failed to initialize HdfsSource", e); + LOG.error("Failed to initialize HdfsSource", e); return null; } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/connection/HttpConnection.java b/cdi-core/src/main/java/com/linkedin/cdi/connection/HttpConnection.java index 44b3e06..c0f382a 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/connection/HttpConnection.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/connection/HttpConnection.java @@ -9,34 +9,33 @@ import com.google.common.collect.Lists; import com.google.gson.JsonElement; import com.google.gson.JsonObject; +import com.linkedin.cdi.exception.RetriableAuthenticationException; +import com.linkedin.cdi.factory.ConnectionClientFactory; +import com.linkedin.cdi.keys.ExtractorKeys; +import com.linkedin.cdi.keys.HttpKeys; +import com.linkedin.cdi.keys.JobKeys; +import com.linkedin.cdi.factory.http.HttpRequestMethod; +import com.linkedin.cdi.util.JsonUtils; +import com.linkedin.cdi.util.WorkUnitStatus; import java.io.Closeable; import java.io.IOException; import java.util.List; import java.util.Map; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.SneakyThrows; -import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.exception.RetriableAuthenticationException; -import com.linkedin.cdi.factory.HttpClientFactory; -import com.linkedin.cdi.keys.ExtractorKeys; -import com.linkedin.cdi.keys.HttpKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.util.HttpRequestMethod; -import com.linkedin.cdi.util.JsonUtils; -import com.linkedin.cdi.util.WorkUnitStatus; import org.apache.http.Header; import org.apache.http.HeaderElement; import org.apache.http.HttpResponse; import org.apache.http.client.HttpClient; import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.client.protocol.HttpClientContext; import org.apache.http.util.EntityUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static com.linkedin.cdi.configuration.StaticConstants.*; @@ -46,16 +45,32 @@ * * @author Chris Li */ -@Slf4j public class HttpConnection extends MultistageConnection { - @Getter (AccessLevel.PACKAGE) + private static final Logger LOG = LoggerFactory.getLogger(HttpConnection.class); final private HttpKeys httpSourceKeys; - - @Setter (AccessLevel.PACKAGE) private HttpClient httpClient; - @Setter (AccessLevel.PACKAGE) private CloseableHttpResponse response; + public HttpKeys getHttpSourceKeys() { + return httpSourceKeys; + } + + public HttpClient getHttpClient() { + return httpClient; + } + + public void setHttpClient(HttpClient httpClient) { + this.httpClient = httpClient; + } + + public CloseableHttpResponse getResponse() { + return response; + } + + public void setResponse(CloseableHttpResponse response) { + this.response = response; + } + public HttpConnection(State state, JobKeys jobKeys, ExtractorKeys extractorKeys) { super(state, jobKeys, extractorKeys); httpClient = getHttpClient(state); @@ -77,11 +92,11 @@ synchronized HttpClient getHttpClient(State state) { if (httpClient == null) { try { Class factoryClass = Class.forName( - MultistageProperties.MSTAGE_HTTP_CLIENT_FACTORY.getValidNonblankWithDefault(state)); - HttpClientFactory factory = (HttpClientFactory) factoryClass.newInstance(); - httpClient = factory.get(state); + MSTAGE_CONNECTION_CLIENT_FACTORY.get(state)); + ConnectionClientFactory factory = (ConnectionClientFactory) factoryClass.newInstance(); + httpClient = factory.getHttpClient(state); } catch (Exception e) { - log.error("Error creating HttpClient: {}", e.getMessage()); + LOG.error("Error creating HttpClient: {}", e.getMessage()); } } return httpClient; @@ -100,7 +115,6 @@ public WorkUnitStatus executeNext(WorkUnitStatus workUnitStatus) throws Retriabl } @VisibleForTesting - @SneakyThrows WorkUnitStatus execute(HttpRequestMethod command, WorkUnitStatus status) throws RetriableAuthenticationException { Preconditions.checkNotNull(status, "WorkUnitStatus is not initialized."); try { @@ -109,7 +123,7 @@ WorkUnitStatus execute(HttpRequestMethod command, WorkUnitStatus status) throws } catch (RetriableAuthenticationException e) { throw e; } catch (Exception e) { - log.error(e.getMessage(), e); + LOG.error(e.getMessage(), e); return null; } @@ -130,21 +144,20 @@ WorkUnitStatus execute(HttpRequestMethod command, WorkUnitStatus status) throws // Log but ignore errors when getting content and content type // These errors will lead to a NULL buffer in work unit status // And that situation will be handled in extractor accordingly - log.error(e.getMessage()); + LOG.error(e.getMessage()); } return status; } - @SneakyThrows private CloseableHttpResponse retryExecuteHttpRequest( final HttpRequestMethod command, final JsonObject parameters ) throws RetriableAuthenticationException { - log.debug("Execute Http {} with parameters:", command.toString()); + LOG.debug("Execute Http {} with parameters:", command.toString()); for (Map.Entry entry: parameters.entrySet()) { if (!entry.getKey().equalsIgnoreCase(KEY_WORD_PAYLOAD)) { - log.debug("parameter: {} value: {}", entry.getKey(), entry.getValue()); + LOG.debug("parameter: {} value: {}", entry.getKey(), entry.getValue()); } } Pair response = executeHttpRequest(command, @@ -153,7 +166,7 @@ private CloseableHttpResponse retryExecuteHttpRequest( httpSourceKeys.getHttpRequestHeadersWithAuthentication()); if (response.getLeft().equalsIgnoreCase(KEY_WORD_HTTP_OK)) { - log.info("Request was successful, return HTTP response"); + LOG.info("Request was successful, return HTTP response"); return response.getRight(); } @@ -165,7 +178,7 @@ private CloseableHttpResponse retryExecuteHttpRequest( // by returning NULL, the task will complete without failure if (status < 400 && !httpSourceKeys.getHttpStatuses().getOrDefault("error", Lists.newArrayList()).contains(status) || httpSourceKeys.getHttpStatuses().getOrDefault("warning", Lists.newArrayList()).contains(status)) { - log.warn("Request was successful with warnings, return NULL response"); + LOG.warn("Request was successful with warnings, return NULL response"); return null; } @@ -174,8 +187,8 @@ private CloseableHttpResponse retryExecuteHttpRequest( List paginationErrors = httpSourceKeys.getHttpStatuses().getOrDefault( "pagination_error", Lists.newArrayList()); if (getJobKeys().getIsSecondaryAuthenticationEnabled() && paginationErrors.contains(status)) { - log.info("Request was unsuccessful, and needed retry with new authentication credentials"); - log.info("Sleep {} seconds, waiting for credentials to refresh", getJobKeys().getRetryDelayInSec()); + LOG.info("Request was unsuccessful, and needed retry with new authentication credentials"); + LOG.info("Sleep {} seconds, waiting for credentials to refresh", getJobKeys().getRetryDelayInSec()); throw new RetriableAuthenticationException("Stale authentication token."); } @@ -212,6 +225,7 @@ private Pair executeHttpRequest(final HttpRequest // trying to make a Http request, capture the client side error and // fail the task if any encoding exception or IO exception CloseableHttpResponse response; + HttpClientContext context = HttpClientContext.create(); try { JsonObject payloads = new JsonObject(); JsonObject queryParameters = new JsonObject(); @@ -222,8 +236,9 @@ private Pair executeHttpRequest(final HttpRequest queryParameters.add(entry.getKey(), entry.getValue()); } } - response = (CloseableHttpResponse) httpClient.execute( - command.getHttpRequest(httpUriTemplate, queryParameters, headers, payloads)); + HttpUriRequest request = command.getHttpRequest(httpUriTemplate, queryParameters, headers, payloads); + response = (CloseableHttpResponse) httpClient.execute(request, context); + LOG.debug(context.toString()); } catch (Exception e) { throw new RuntimeException(e.getMessage(), e); } @@ -236,10 +251,10 @@ private Pair executeHttpRequest(final HttpRequest // it will retry accessing the token by passing the response object back). Integer status = response.getStatusLine().getStatusCode(); String reason = response.getStatusLine().getReasonPhrase(); - log.info("processing status: {} and reason: {}", status, reason); + LOG.info("processing status: {} and reason: {}", status, reason); if (httpSourceKeys.getHttpStatuses().getOrDefault("success", Lists.newArrayList()).contains(status) && !httpSourceKeys.getHttpStatusReasons().getOrDefault("error", Lists.newArrayList()).contains(reason)) { - log.info("Request was successful, returning OK and HTTP response."); + LOG.info("Request was successful, returning OK and HTTP response."); return Pair.of(KEY_WORD_HTTP_OK, response); } @@ -248,13 +263,13 @@ private Pair executeHttpRequest(final HttpRequest if (null != response.getEntity()) { try { reason += StringUtils.LF + EntityUtils.toString(response.getEntity()); - log.error("Status code: {}, reason: {}", status, reason); + LOG.error("Status code: {}, reason: {}", status, reason); response.close(); } catch (IOException e) { throw new RuntimeException(e.getMessage(), e); } } - log.warn("Request was unsuccessful, returning NOTOK and HTTP response"); + LOG.warn("Request was unsuccessful, returning NOTOK and HTTP response"); return Pair.of(KEY_WORD_HTTP_NOTOK, response); } @@ -291,13 +306,13 @@ private JsonObject getResponseHeaders(HttpResponse response) { @Override public boolean closeStream() { - log.info("Closing InputStream for {}", getExtractorKeys().getSignature()); + LOG.info("Closing InputStream for {}", getExtractorKeys().getSignature()); try { if (response != null) { response.close(); } } catch (Exception e) { - log.warn("Error closing the input stream", e); + LOG.warn("Error closing the input stream", e); return false; } return true; @@ -312,7 +327,7 @@ public boolean closeAll(String message) { httpClient = null; } } catch (IOException e) { - log.error("error closing HttpSource {}", e.getMessage()); + LOG.error("error closing HttpSource {}", e.getMessage()); return false; } return true; diff --git a/cdi-core/src/main/java/com/linkedin/cdi/connection/JdbcConnection.java b/cdi-core/src/main/java/com/linkedin/cdi/connection/JdbcConnection.java index f24bc5a..4fdca08 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/connection/JdbcConnection.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/connection/JdbcConnection.java @@ -6,6 +6,15 @@ import com.google.gson.JsonArray; import com.google.gson.JsonObject; +import com.linkedin.cdi.exception.RetriableAuthenticationException; +import com.linkedin.cdi.factory.ConnectionClientFactory; +import com.linkedin.cdi.keys.ExtractorKeys; +import com.linkedin.cdi.keys.JdbcKeys; +import com.linkedin.cdi.keys.JobKeys; +import com.linkedin.cdi.util.JdbcUtils; +import com.linkedin.cdi.util.ParameterTypes; +import com.linkedin.cdi.util.SchemaBuilder; +import com.linkedin.cdi.util.WorkUnitStatus; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.ByteArrayInputStream; import java.nio.charset.StandardCharsets; @@ -17,23 +26,13 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.NonNull; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringEscapeUtils; import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.exception.RetriableAuthenticationException; -import com.linkedin.cdi.factory.JdbcClientFactory; -import com.linkedin.cdi.keys.ExtractorKeys; -import com.linkedin.cdi.keys.JdbcKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.util.JdbcUtils; -import com.linkedin.cdi.util.ParameterTypes; -import com.linkedin.cdi.util.SchemaBuilder; -import com.linkedin.cdi.util.WorkUnitStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; + /** * JdbcConnection creates transmission channel with JDBC data provider or JDBC data receiver, @@ -41,13 +40,26 @@ * * @author Chris Li */ -@Slf4j public class JdbcConnection extends MultistageConnection { - @Getter(AccessLevel.PACKAGE) - @Setter(AccessLevel.PACKAGE) + private static final Logger LOG = LoggerFactory.getLogger(JdbcConnection.class); private JdbcKeys jdbcSourceKeys; - @Getter(AccessLevel.PACKAGE) - @Setter(AccessLevel.PACKAGE) + + public JdbcKeys getJdbcSourceKeys() { + return jdbcSourceKeys; + } + + public void setJdbcSourceKeys(JdbcKeys jdbcSourceKeys) { + this.jdbcSourceKeys = jdbcSourceKeys; + } + + public Connection getJdbcConnection() { + return jdbcConnection; + } + + public void setJdbcConnection(Connection jdbcConnection) { + this.jdbcConnection = jdbcConnection; + } + private Connection jdbcConnection; public JdbcConnection(State state, JobKeys jobKeys, ExtractorKeys extractorKeys) { @@ -63,7 +75,7 @@ public WorkUnitStatus execute(WorkUnitStatus status) { getWorkUnitSpecificString(jdbcSourceKeys.getJdbcStatement(), getExtractorKeys().getDynamicParameters()), status); } catch (Exception e) { - log.error(e.getMessage(), e); + LOG.error(e.getMessage(), e); return null; } } @@ -76,7 +88,7 @@ public boolean closeAll(String message) { jdbcConnection = null; } } catch (Exception e) { - log.error("Error closing the input stream", e); + LOG.error("Error closing the input stream", e); return false; } return true; @@ -101,16 +113,16 @@ public WorkUnitStatus executeNext(WorkUnitStatus workUnitStatus) throws Retriabl */ private synchronized Connection getJdbcConnection(State state) { try { - Class factoryClass = Class.forName(MultistageProperties.MSTAGE_JDBC_CLIENT_FACTORY.getValidNonblankWithDefault(state)); - JdbcClientFactory factory = (JdbcClientFactory) factoryClass.newInstance(); + Class factoryClass = Class.forName(MSTAGE_CONNECTION_CLIENT_FACTORY.get(state)); + ConnectionClientFactory factory = (ConnectionClientFactory) factoryClass.newInstance(); - return factory.getConnection( + return factory.getJdbcConnection( jdbcSourceKeys.getSourceUri(), - MultistageProperties.SOURCE_CONN_USERNAME.getValidNonblankWithDefault(state), - MultistageProperties.SOURCE_CONN_PASSWORD.getValidNonblankWithDefault(state), + SOURCE_CONN_USERNAME.get(state), + SOURCE_CONN_PASSWORD.get(state), state); } catch (Exception e) { - log.error("Error creating Jdbc connection: {}", e.getMessage()); + LOG.error("Error creating Jdbc connection: {}", e.getMessage()); } return null; } @@ -145,25 +157,23 @@ private WorkUnitStatus executeStatement( String query, WorkUnitStatus wuStatus) throws SQLException { - log.info("Executing SQL statement: {}", query); + LOG.info("Executing SQL statement: {}", query); Statement stmt = jdbcConnection.createStatement(); if (jdbcSourceKeys.isPaginationEnabled()) { try { stmt.setFetchSize(jdbcSourceKeys.getPaginationInitValues().get(ParameterTypes.PAGESIZE).intValue()); } catch (SQLException e) { - log.warn("not able to set fetch size"); + LOG.warn("not able to set fetch size"); } } if (stmt.execute(query)) { ResultSet resultSet = stmt.getResultSet(); - if (MultistageProperties.MSTAGE_EXTRACTOR_CLASS.getValidNonblankWithDefault(getState()).toString() - .matches(".*JsonExtractor.*")) { + if (MSTAGE_EXTRACTOR_CLASS.get(getState()).matches(".*JsonExtractor.*")) { wuStatus.setBuffer(new ByteArrayInputStream(toJson(resultSet, resultSet.getMetaData()).toString().getBytes(StandardCharsets.UTF_8))); - } else if (MultistageProperties.MSTAGE_EXTRACTOR_CLASS.getValidNonblankWithDefault(getState()).toString() - .matches(".*CsvExtractor.*")) { + } else if (MSTAGE_EXTRACTOR_CLASS.get(getState()).matches(".*CsvExtractor.*")) { wuStatus.setBuffer(new ByteArrayInputStream(toCsv(resultSet, resultSet.getMetaData()).getBytes(StandardCharsets.UTF_8))); } else { @@ -212,7 +222,6 @@ private JsonArray toJson(final ResultSet resultSet, final ResultSetMetaData resu * @return a 2-dimensional string matrix representing a CSV file * @throws SQLException SQL Exception from processing ResultSet */ - @NonNull private String toCsv(final ResultSet resultSet, final ResultSetMetaData resultSetMetadata) throws SQLException { StringBuilder builder = new StringBuilder(); @@ -220,7 +229,7 @@ private String toCsv(final ResultSet resultSet, final ResultSetMetaData resultSe for (int i = 0; i < resultSetMetadata.getColumnCount(); i++) { builder.append(StringEscapeUtils.escapeCsv(JdbcUtils.parseColumnAsString(resultSet, resultSetMetadata, i + 1))); if (i < resultSetMetadata.getColumnCount() - 1) { - builder.append(jdbcSourceKeys.getSeparator()); + builder.append(MSTAGE_CSV.getFieldSeparator(getState())); } else { builder.append(System.lineSeparator()); } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/connection/MultistageConnection.java b/cdi-core/src/main/java/com/linkedin/cdi/connection/MultistageConnection.java index da9cc92..d37a9cb 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/connection/MultistageConnection.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/connection/MultistageConnection.java @@ -5,27 +5,50 @@ package com.linkedin.cdi.connection; import com.google.gson.JsonObject; -import lombok.Getter; -import lombok.Setter; -import lombok.SneakyThrows; -import lombok.extern.slf4j.Slf4j; -import org.apache.gobblin.configuration.State; import com.linkedin.cdi.exception.RetriableAuthenticationException; import com.linkedin.cdi.keys.ExtractorKeys; import com.linkedin.cdi.keys.JobKeys; import com.linkedin.cdi.util.VariableUtils; import com.linkedin.cdi.util.WorkUnitStatus; +import org.apache.gobblin.configuration.State; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * MultistageConnection is a basic implementation of Connection interface. * * @author Chris Li */ -@Slf4j public class MultistageConnection implements Connection { - @Getter @Setter private State state = null; - @Getter @Setter private JobKeys jobKeys = null; - @Getter @Setter private ExtractorKeys extractorKeys = null; + private static final Logger LOG = LoggerFactory.getLogger(MultistageConnection.class); + private State state = null; + private JobKeys jobKeys = null; + private ExtractorKeys extractorKeys = null; + + public State getState() { + return state; + } + + public void setState(State state) { + this.state = state; + } + + public JobKeys getJobKeys() { + return jobKeys; + } + + public void setJobKeys(JobKeys jobKeys) { + this.jobKeys = jobKeys; + } + + public ExtractorKeys getExtractorKeys() { + return extractorKeys; + } + + public void setExtractorKeys(ExtractorKeys extractorKeys) { + this.extractorKeys = extractorKeys; + } public MultistageConnection(State state, JobKeys jobKeys, ExtractorKeys extractorKeys) { this.setJobKeys(jobKeys); @@ -73,7 +96,6 @@ public JsonObject getWorkUnitParameters() { * @param workUnitStatus prior work unit status * @return new work unit status */ - @SneakyThrows public WorkUnitStatus executeFirst(final WorkUnitStatus workUnitStatus) throws RetriableAuthenticationException { return WorkUnitStatus.builder().build(); } @@ -82,11 +104,11 @@ public WorkUnitStatus executeNext(final WorkUnitStatus workUnitStatus) throws Re try { Thread.sleep(jobKeys.getCallInterval()); } catch (Exception e) { - log.warn(e.getMessage()); + LOG.warn(e.getMessage()); } - log.info("Starting a new request to the source, work unit = {}", extractorKeys.getSignature()); - log.debug("Prior parameters: {}", extractorKeys.getDynamicParameters().toString()); - log.debug("Prior work unit status: {}", workUnitStatus.toString()); + LOG.info("Starting a new request to the source, work unit = {}", extractorKeys.getSignature()); + LOG.debug("Prior parameters: {}", extractorKeys.getDynamicParameters().toString()); + LOG.debug("Prior work unit status: {}", workUnitStatus.toString()); return workUnitStatus; } @@ -107,9 +129,9 @@ protected String getWorkUnitSpecificString(String template, JsonObject parameter parameters, false).getKey(); } catch (Exception e) { - log.error("Error getting work unit specific string " + e); + LOG.error("Error getting work unit specific string " + e); } - log.info("Final work unit specific string: {}", finalString); + LOG.info("Final work unit specific string: {}", finalString); return finalString; } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/connection/S3Connection.java b/cdi-core/src/main/java/com/linkedin/cdi/connection/S3Connection.java index 0030ea9..f034302 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/connection/S3Connection.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/connection/S3Connection.java @@ -5,24 +5,22 @@ package com.linkedin.cdi.connection; import com.google.common.collect.Lists; -import java.net.URI; -import java.time.Duration; -import java.util.List; -import java.util.stream.Collectors; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang.StringUtils; -import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; import com.linkedin.cdi.exception.RetriableAuthenticationException; -import com.linkedin.cdi.factory.S3ClientFactory; +import com.linkedin.cdi.factory.ConnectionClientFactory; import com.linkedin.cdi.keys.ExtractorKeys; import com.linkedin.cdi.keys.JobKeys; import com.linkedin.cdi.keys.S3Keys; import com.linkedin.cdi.util.EncryptionUtils; import com.linkedin.cdi.util.InputStreamUtils; import com.linkedin.cdi.util.WorkUnitStatus; +import java.net.URI; +import java.time.Duration; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.commons.lang.StringUtils; +import org.apache.gobblin.configuration.State; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentials; @@ -37,6 +35,7 @@ import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; import software.amazon.awssdk.utils.AttributeMap; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static software.amazon.awssdk.http.SdkHttpConfigurationOption.*; /** @@ -45,10 +44,22 @@ * * @author Chris Li */ -@Slf4j public class S3Connection extends MultistageConnection { - @Getter final private S3Keys s3SourceV2Keys; - @Setter private S3Client s3Client = null; + private static final Logger LOG = LoggerFactory.getLogger(S3Connection.class); + final private S3Keys s3SourceV2Keys; + private S3Client s3Client = null; + + public S3Keys getS3SourceV2Keys() { + return s3SourceV2Keys; + } + + public S3Client getS3Client() { + return s3Client; + } + + public void setS3Client(S3Client s3Client) { + this.s3Client = s3Client; + } public S3Connection(State state, JobKeys jobKeys, ExtractorKeys extractorKeys) { super(state, jobKeys, extractorKeys); @@ -66,64 +77,48 @@ public boolean closeAll(String message) { return true; } - /* - Below is the logic of when to download a file and when to list similar files based on the uri and pattern - ms.source.file.pattern - if Is not blank: - List the S3 keys and output as CSV - - if Is blank: - ms.extract.target.file.name? - If is blank: - List the S3 keys and output as CSV - If is not blank: - If ms.source.uri prefix produces only 1 file: - dump the S3 object into the given output file name - If ms.source.uir prefix produces more than 1 file: - dump only the file where prefix = object key, and ignore all other objects - */ + /** + * @param workUnitStatus prior work unit status + * @return new work unit status + * @throws RetriableAuthenticationException + */ @Override public WorkUnitStatus executeFirst(WorkUnitStatus workUnitStatus) throws RetriableAuthenticationException { WorkUnitStatus status = super.executeFirst(workUnitStatus); s3Client = getS3HttpClient(getState()); String finalPrefix = getWorkUnitSpecificString(s3SourceV2Keys.getPrefix(), getExtractorKeys().getDynamicParameters()); - log.debug("Final Prefix to get files list: {}", finalPrefix); + LOG.debug("Final Prefix to get files list: {}", finalPrefix); try { - List files = getFilesList(finalPrefix); - boolean isObjectWithPrefixExist = files.stream().anyMatch(objectKey -> objectKey.equals(finalPrefix)); - log.debug("Number of files identified: {}", files.size()); - - if (StringUtils.isNotBlank(s3SourceV2Keys.getFilesPattern())) { - List filteredFiles = files.stream() - .filter(fileName -> fileName.matches(s3SourceV2Keys.getFilesPattern())) - .collect(Collectors.toList()); - status.setBuffer(InputStreamUtils.convertListToInputStream(filteredFiles)); + List files = getFilesList(finalPrefix).stream() + .filter(objectKey -> objectKey.matches(s3SourceV2Keys.getFilesPattern())) + .collect(Collectors.toList()); + + LOG.debug("Number of files identified: {}", files.size()); + + if (StringUtils.isBlank(s3SourceV2Keys.getTargetFilePattern())) { + status.setBuffer(InputStreamUtils.convertListToInputStream(files)); } else { - if (StringUtils.isBlank(s3SourceV2Keys.getTargetFilePattern())) { - status.setBuffer(InputStreamUtils.convertListToInputStream(files)); + // Multiple files are returned, then only process the exact match + String fileToDownload = files.size() == 0 + ? StringUtils.EMPTY : files.size() == 1 + ? files.get(0) : finalPrefix; + + if (StringUtils.isNotBlank(fileToDownload)) { + LOG.debug("Downloading file: {}", fileToDownload); + GetObjectRequest getObjectRequest = + GetObjectRequest.builder().bucket(s3SourceV2Keys.getBucket()).key(fileToDownload).build(); + ResponseInputStream response = + s3Client.getObject(getObjectRequest, ResponseTransformer.toInputStream()); + status.setBuffer(response); } else { - String fileToDownload = ""; - if (files.size() == 1) { - fileToDownload = files.get(0); - } else if (isObjectWithPrefixExist) { - fileToDownload = finalPrefix; - } - if (StringUtils.isNotBlank(fileToDownload)) { - log.debug("Downloading file: {}", files.get(0)); - GetObjectRequest getObjectRequest = - GetObjectRequest.builder().bucket(s3SourceV2Keys.getBucket()).key(files.get(0)).build(); - ResponseInputStream response = - s3Client.getObject(getObjectRequest, ResponseTransformer.toInputStream()); - status.setBuffer(response); - } else { - log.warn("Invalid set of parameters. To list down files from a bucket, pattern " - + "parameter is needed and to get object from s3 source target file name is needed."); - } + LOG.warn("Invalid set of parameters. " + + "To list down files from a bucket, pattern parameter is needed," + + ", and to get object from s3 source target file name is needed."); } } } catch (Exception e) { - log.error("Unexpected Exception", e); + LOG.error("Unexpected Exception", e); return null; } return status; @@ -135,8 +130,8 @@ public WorkUnitStatus executeFirst(WorkUnitStatus workUnitStatus) throws Retriab synchronized S3Client getS3HttpClient(State state) { if (s3Client == null) { try { - Class factoryClass = Class.forName(MultistageProperties.MSTAGE_S3_CLIENT_FACTORY.getValidNonblankWithDefault(state)); - S3ClientFactory factory = (S3ClientFactory) factoryClass.newInstance(); + Class factoryClass = Class.forName(MSTAGE_CONNECTION_CLIENT_FACTORY.get(state)); + ConnectionClientFactory factory = (ConnectionClientFactory) factoryClass.getDeclaredConstructor().newInstance(); Integer connectionTimeout = s3SourceV2Keys.getConnectionTimeout(); AttributeMap config = connectionTimeout == null ? GLOBAL_HTTP_DEFAULTS @@ -147,11 +142,11 @@ synchronized S3Client getS3HttpClient(State state) { s3Client = S3Client.builder() .region(this.s3SourceV2Keys.getRegion()) .endpointOverride(URI.create(s3SourceV2Keys.getEndpoint())) - .httpClient(factory.getHttpClient(state, config)) + .httpClient(factory.getS3Client(state, config)) .credentialsProvider(getCredentialsProvider(state)) .build(); } catch (Exception e) { - log.error("Error creating S3 Client: {}", e.getMessage()); + LOG.error("Error creating S3 Client: {}", e.getMessage()); } } return s3Client; @@ -172,7 +167,7 @@ private List getFilesList(String finalPrefix) { ListObjectsV2Request request = builder.build(); ListObjectsV2Response listObjectsV2Response = null; - log.debug("Listing object by prefix: {}", finalPrefix); + LOG.debug("Listing object by prefix: {}", finalPrefix); do { if (listObjectsV2Response != null) { request = builder.continuationToken(listObjectsV2Response.continuationToken()).build(); diff --git a/cdi-core/src/main/java/com/linkedin/cdi/connection/SftpConnection.java b/cdi-core/src/main/java/com/linkedin/cdi/connection/SftpConnection.java new file mode 100644 index 0000000..cb3a32a --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/connection/SftpConnection.java @@ -0,0 +1,164 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.connection; + +import com.linkedin.cdi.exception.RetriableAuthenticationException; +import com.linkedin.cdi.factory.ConnectionClientFactory; +import com.linkedin.cdi.factory.sftp.SftpClient; +import com.linkedin.cdi.keys.ExtractorKeys; +import com.linkedin.cdi.keys.JobKeys; +import com.linkedin.cdi.keys.SftpKeys; +import com.linkedin.cdi.util.InputStreamUtils; +import com.linkedin.cdi.util.WorkUnitStatus; +import java.io.File; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.commons.lang.StringUtils; +import org.apache.gobblin.configuration.State; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; + + +public class SftpConnection extends MultistageConnection { + private static final Logger LOG = LoggerFactory.getLogger(SftpConnection.class); + + final private SftpKeys sftpSourceKeys; + SftpClient fsClient; + + public SftpConnection(State state, JobKeys jobKeys, ExtractorKeys extractorKeys) { + super(state, jobKeys, extractorKeys); + assert jobKeys instanceof SftpKeys; + sftpSourceKeys = (SftpKeys) jobKeys; + } + + @Override + public WorkUnitStatus execute(WorkUnitStatus status) { + return null; + } + + @Override + public boolean closeAll(String message) { + if (this.fsClient != null) { + LOG.info("Shutting down FileSystem connection"); + this.fsClient.close(); + fsClient = null; + } + return true; + } + + /** + * @param workUnitStatus prior work unit status + * @return new work unit status + * @throws RetriableAuthenticationException + */ + @Override + public WorkUnitStatus executeFirst(WorkUnitStatus workUnitStatus) throws RetriableAuthenticationException { + WorkUnitStatus status = super.executeFirst(workUnitStatus); + String path = getPath(); + String finalPrefix = getWorkUnitSpecificString(path, getExtractorKeys().getDynamicParameters()); + LOG.info("File path found is: " + finalPrefix); + try { + if (getFsClient() == null) { + LOG.error("Error initializing SFTP connection"); + return null; + } + } catch (Exception e) { + LOG.error("Error initializing SFTP connection", e); + return null; + } + + //get List of files matching the pattern + List files; + try { + files = getFiles(finalPrefix).stream() + .filter(objectKey -> objectKey.matches(sftpSourceKeys.getFilesPattern())) + .collect(Collectors.toList()); + } catch (Exception e) { + LOG.error("Error reading file list", e); + return null; + } + + LOG.info("No Of Files to be processed matching the pattern: {}", files.size()); + + if (StringUtils.isBlank(sftpSourceKeys.getTargetFilePattern())) { + status.setBuffer(InputStreamUtils.convertListToInputStream(files)); + } else { + String fileToDownload = files.size() == 0 ? StringUtils.EMPTY : files.get(0); + if (StringUtils.isNotBlank(fileToDownload)) { + LOG.info("Downloading file: {}", fileToDownload); + try { + status.setBuffer(this.fsClient.getFileStream(fileToDownload)); + } catch (Exception e) { + LOG.error("Error downloading file {}", fileToDownload, e); + return null; + } + } else { + LOG.warn("Invalid set of parameters. Please make sure to set source directory, entity and file pattern"); + } + } + return status; + } + + private SftpClient getFsClient() { + if (this.fsClient == null) { + try { + Class factoryClass = Class.forName(MSTAGE_CONNECTION_CLIENT_FACTORY.get(this.getState())); + ConnectionClientFactory factory = (ConnectionClientFactory) factoryClass.getDeclaredConstructor().newInstance(); + this.fsClient = factory.getSftpChannelClient(this.getState()); + } catch (Exception e) { + LOG.error("Error initiating SFTP client", e); + } + } + return this.fsClient; + } + + /** + * //TODO: List files based on pattern on parent nodes as well. + * The current version supports pattern only on leaf node. + * Ex: file path supported "/a/b/*c*" + * file path not supported "/a/*b/*c* + * Get files list based on pattern + * @param filesPattern pattern of content to list + * @return list of content + */ + private List getFiles(String filesPattern) { + List files = new ArrayList<>(); + LOG.info("Files to be processed from input " + filesPattern); + try { + files = fsClient.ls(filesPattern); + int i = 0; + for (String file : files) { + URI uri = new URI(file); + String filepath = uri.toString(); + if (!uri.isAbsolute()) { + File f = new File(getBaseDir(filesPattern), filepath); + filepath = f.getAbsolutePath(); + } + files.set(i, filepath); + i++; + } + } catch (Exception e) { + LOG.error("Unable to list files " + e.getMessage()); + } + return files; + } + private String getPath() { + return sftpSourceKeys.getFilesPath(); + } + + private List getFilteredFiles(List files) { + return files.stream().filter(file -> file.matches(sftpSourceKeys.getFilesPattern())).collect(Collectors.toList()); + } + + private String getBaseDir(String uri) { + File file = new File(uri); + return file.getParentFile().getAbsolutePath() + sftpSourceKeys.getPathSeparator(); + } + +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/converter/AvroNormalizerConverter.java b/cdi-core/src/main/java/com/linkedin/cdi/converter/AvroNormalizerConverter.java index ccfa957..ecfffc0 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/converter/AvroNormalizerConverter.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/converter/AvroNormalizerConverter.java @@ -7,6 +7,7 @@ import com.google.common.base.Optional; import com.google.gson.JsonArray; import com.google.gson.JsonElement; +import com.linkedin.cdi.util.AvroSchemaUtils; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -18,12 +19,10 @@ import org.apache.gobblin.converter.Converter; import org.apache.gobblin.converter.SchemaConversionException; import org.apache.gobblin.converter.SingleRecordIterable; -import org.apache.gobblin.converter.avro.UnsupportedDateTypeException; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.util.AvroSchemaUtils; import org.apache.gobblin.util.AvroUtils; import org.apache.gobblin.util.EmptyIterable; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static com.linkedin.cdi.configuration.StaticConstants.*; @@ -68,12 +67,12 @@ public Converter init(WorkUnitStat // Avro Array's max capacity is max int. In case of overflow, use the default value 500. try { maxRecordsPerBatch = - Math.toIntExact(MultistageProperties.MSTAGE_NORMALIZER_BATCH_SIZE.getValidNonblankWithDefault(workUnit)); + Math.toIntExact(MSTAGE_NORMALIZER_BATCH_SIZE.get(workUnit)); } catch (ArithmeticException e) { maxRecordsPerBatch = 500; } - targetSchema = MultistageProperties.MSTAGE_TARGET_SCHEMA.getValidNonblankWithDefault(workUnit); + targetSchema = MSTAGE_TARGET_SCHEMA.get(workUnit); return this; } @@ -92,11 +91,7 @@ public Schema convertSchema(Schema schema, WorkUnitState workUnitState) throws S buildIntermediateSchemas(schema); } - try { - finalSchema = AvroSchemaUtils.fromJsonSchema(targetSchema, workUnitState); - } catch (UnsupportedDateTypeException e) { - throw new SchemaConversionException(e); - } + finalSchema = AvroSchemaUtils.fromJsonSchema(targetSchema, workUnitState); return finalSchema; } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/converter/InFlowValidationConverter.java b/cdi-core/src/main/java/com/linkedin/cdi/converter/InFlowValidationConverter.java new file mode 100644 index 0000000..aea737a --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/converter/InFlowValidationConverter.java @@ -0,0 +1,182 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.converter; + +import com.google.common.base.Optional; +import com.google.gson.JsonArray; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import com.linkedin.cdi.util.HdfsReader; +import com.linkedin.cdi.util.JsonUtils; +import java.text.DecimalFormat; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.WorkUnitState; +import org.apache.gobblin.converter.Converter; +import org.apache.gobblin.util.AvroUtils; +import org.apache.gobblin.util.EmptyIterable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; +import static com.linkedin.cdi.configuration.StaticConstants.*; + + +/** + * This converter does basic count validation based on the Failure Records or Success Records criteria. + * + * To use this converter for validation, the main source should be the dataset to be validated, + * and the secondary input should be the base dataset to validate against. + * + * The base dataset can be in a nested column of the secondary input, i.e. a field, which can be + * retrieved through a JSON path, contains the actual base records. + * + * Currently following rules are defined: + * + * fail (upper bound rule): the source should be failed records + * Job succeeds when the row count in validation set / row count in base set < threshold + * Job fails when the row count in validation set / row count in base set >= threshold + * + * success (lower bound rule): the source should be succeeded records + * Job succeeds when the row count in validation set / row count in base set >= threshold + * Job fails when the row count in validation set / row count in base set < threshold + */ +public class InFlowValidationConverter extends Converter { + private static final Logger LOG = LoggerFactory.getLogger(InFlowValidationConverter.class); + int expectedRecordsCount; + int actualRecordsCount; + private String field; + private int threshold; + private String criteria; + private String errorColumn; + + @Override + public Converter init(WorkUnitState workUnitState) { + //Load the input to memory + expectedRecordsCount = getBaseRowCount(workUnitState); + fillValidationAttributes(workUnitState); + return super.init(workUnitState); + } + + @Override + public Schema convertSchema(Schema inputSchema, WorkUnitState workUnit) { + return inputSchema; + } + + @Override + public Iterable convertRecord(Schema outputSchema, GenericRecord inputRecord, WorkUnitState workUnit) { + Optional eof = AvroUtils.getFieldValue(inputRecord, KEY_WORD_EOF); + if (eof.isPresent() && eof.get().toString().equals(KEY_WORD_EOF)) { + validateRule(); + } else { + verifyAndUpdateCount(inputRecord); + } + return new EmptyIterable<>(); + } + + private void verifyAndUpdateCount(GenericRecord inputRecord) { + List fieldList = inputRecord.getSchema().getFields(); + if (fieldList.size() == 1 && inputRecord.get(fieldList.get(0).name()) instanceof GenericData.Array) { + // Check if error column exists and is not null + if (errorColumn != null) { + GenericData.Array arrayElements = (GenericData.Array) inputRecord.get(fieldList.get(0).name()); + arrayElements.stream().iterator().forEachRemaining(this::updateFailureCount); + } else { + actualRecordsCount += ((GenericData.Array) inputRecord.get(fieldList.get(0).name())).size(); + } + } else { + actualRecordsCount += (errorColumn == null || inputRecord.get(errorColumn) != null ? 1 : 0); + } + } + + private void fillValidationAttributes(WorkUnitState workUnitState) { + JsonObject validationAttributes = + MSTAGE_VALIDATION_ATTRIBUTES.get(workUnitState); + if (validationAttributes.has(KEY_WORD_THRESHOLD)) { + threshold = validationAttributes.get(KEY_WORD_THRESHOLD).getAsInt(); + } + if (validationAttributes.has(KEY_WORD_CRITERIA)) { + criteria = validationAttributes.get(KEY_WORD_CRITERIA).getAsString(); + } + if (validationAttributes.has(KEY_WORD_ERROR_COLUMN)) { + errorColumn = validationAttributes.get(KEY_WORD_ERROR_COLUMN).getAsString(); + } + } + + /** + * Extract records from secondary input and store the expected record count. + * If field is configured in the secondary input and field column + * is of type array expected record count with array size + * else use all the input records as expected size + * @param workUnitState the work unit state object containing secondary input parameter + * @return the expected row count + */ + private int getBaseRowCount(WorkUnitState workUnitState) { + JsonArray payloads = JsonUtils.filter(KEY_WORD_CATEGORY, KEY_WORD_PAYLOAD, + MSTAGE_SECONDARY_INPUT.get(workUnitState)); + + // by default, we expect 1 record + if (payloads.size() == 0) { + return 1; + } + + // secondary input can have multiple payload entries, and each can configure a "fields" element + // but for validation purpose, only the first payload entry, and the first field is used. + JsonElement fields = JsonUtils.get(KEY_WORD_FIELDS, payloads.get(0).getAsJsonObject()); + field = StringUtils.EMPTY; + if (fields.isJsonArray() && fields.getAsJsonArray().size() > 0) { + field = fields.getAsJsonArray().get(0).getAsString(); + } + + AtomicInteger rowCount = new AtomicInteger(); + for (JsonElement entry : payloads) { + JsonObject entryJson = entry.getAsJsonObject(); + JsonArray records = new JsonArray(); + records.addAll(new HdfsReader(workUnitState).readSecondary(entryJson)); + + // No of expected records + if (records.size() > 0 + && StringUtils.isNotBlank(field) + && (records.get(0).getAsJsonObject().get(field) instanceof JsonArray)) { + records.forEach(record -> rowCount.addAndGet(record.getAsJsonObject().get(field).getAsJsonArray().size())); + } else { + rowCount.addAndGet(records.size()); + } + } return rowCount.get(); + } + + private void updateFailureCount(GenericRecord record) { + if (record.get(errorColumn) != null) { + actualRecordsCount++; + } + } + + /** + * Validate if failure/success percentage is within configured threshold + */ + private void validateRule() { + // check the threshold and throw new Runtime Exception + float actualPercentage = ((float) actualRecordsCount / expectedRecordsCount) * 100; + LOG.info("base row count: {}, actual row count: {}", expectedRecordsCount, actualRecordsCount); + + boolean failJob = criteria.equalsIgnoreCase(KEY_WORD_FAIL) && actualPercentage >= threshold + || criteria.equalsIgnoreCase(KEY_WORD_SUCCESS) && actualPercentage < threshold; + + if (failJob) { + // Fail the validation by throwing runtime exception + throw new RuntimeException("Failure Threshold exceeds more than " + threshold + "%"); + } else { + LOG.info("Validation passed with {} rate {}% {} {}%", + criteria.equalsIgnoreCase(KEY_WORD_FAIL) ? "failure" : "success", + new DecimalFormat("##.##").format(actualPercentage), + criteria.equalsIgnoreCase(KEY_WORD_FAIL) ? "less than" : "greater than or equal", + threshold); + } + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/converter/JsonNormalizerConverter.java b/cdi-core/src/main/java/com/linkedin/cdi/converter/JsonNormalizerConverter.java index c92f549..ca16440 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/converter/JsonNormalizerConverter.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/converter/JsonNormalizerConverter.java @@ -4,19 +4,20 @@ package com.linkedin.cdi.converter; +import com.google.common.base.Preconditions; import com.google.gson.JsonArray; import com.google.gson.JsonElement; import com.google.gson.JsonObject; +import com.linkedin.cdi.util.JsonUtils; import java.util.HashSet; import java.util.Map; import java.util.Set; import org.apache.gobblin.configuration.WorkUnitState; import org.apache.gobblin.converter.Converter; import org.apache.gobblin.converter.SingleRecordIterable; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.util.JsonUtils; import org.apache.gobblin.util.EmptyIterable; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static com.linkedin.cdi.configuration.StaticConstants.*; @@ -49,8 +50,8 @@ public class JsonNormalizerConverter extends Converter init(WorkUnitState workUnit) { - maxRecordsPerBatch = MultistageProperties.MSTAGE_NORMALIZER_BATCH_SIZE.getValidNonblankWithDefault(workUnit); - targetSchema = MultistageProperties.MSTAGE_TARGET_SCHEMA.getValidNonblankWithDefault(workUnit); + maxRecordsPerBatch = MSTAGE_NORMALIZER_BATCH_SIZE.get(workUnit); + targetSchema = MSTAGE_TARGET_SCHEMA.get(workUnit); return this; } @@ -58,12 +59,14 @@ public Converter init(WorkUnitStat public JsonArray convertSchema(JsonArray inputSchema, WorkUnitState workUnit) { for (JsonElement element : targetSchema) { String columnName = element.getAsJsonObject().get(KEY_WORD_COLUMN_NAME).getAsString(); + boolean isNullable = element.getAsJsonObject().get(KEY_WORD_IS_NULLABLE).getAsBoolean(); outputFields.add(columnName); - if (normalizedField == null && !schemaSearch(inputSchema, columnName)) { + if (normalizedField == null && !schemaSearch(inputSchema, columnName) && !isNullable) { normalizedField = columnName; } } - assert normalizedField != null; + + Preconditions.checkNotNull(normalizedField, "Normalized field is NULL."); JsonObject dataType = JsonUtils.get(KEY_WORD_COLUMN_NAME, normalizedField, KEY_WORD_DATA_TYPE, targetSchema).getAsJsonObject(); String trueType = JsonUtils.get(KEY_WORD_TYPE, dataType).getAsString(); @@ -87,7 +90,7 @@ public Iterable convertRecord(JsonArray outputSchema, JsonObject inp // only output when there's at least one record return outputIterable(1); } - // note: the common fields among records will have the same value, so we only need to retain one record + // note: the common fields within each batch will have the same value, so we only need to retain one record if (firstRecord == null) { firstRecord = inputRecord; } @@ -138,13 +141,17 @@ private JsonObject buildNormalizedRecord() { String columnType = JsonUtils.get(KEY_WORD_COLUMN_NAME, normalizedField, KEY_WORD_DATA_TYPE_TYPE, targetSchema).getAsString(); - if (columnType.equalsIgnoreCase(KEY_WORD_MAP) || columnType.equalsIgnoreCase(KEY_WORD_RECORD)) { + // filter out null values for map type + if (columnType.equalsIgnoreCase(KEY_WORD_MAP)) { + newRecord.add(normalizedField, JsonUtils.filterNull(normalized.get(0).getAsJsonObject())); + } else if (columnType.equalsIgnoreCase(KEY_WORD_RECORD)) { newRecord.add(normalizedField, normalized.get(0)); } else { newRecord.add(normalizedField, normalized); } - // reset the buffer + // reset the buffer and first record normalized = new JsonArray(); + firstRecord = null; return newRecord; } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/extractor/AvroExtractor.java b/cdi-core/src/main/java/com/linkedin/cdi/extractor/AvroExtractor.java index 76cf7fa..7226264 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/extractor/AvroExtractor.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/extractor/AvroExtractor.java @@ -5,15 +5,20 @@ package com.linkedin.cdi.extractor; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; import com.google.gson.JsonArray; +import com.linkedin.cdi.filter.AvroSchemaBasedFilter; +import com.linkedin.cdi.keys.AvroExtractorKeys; +import com.linkedin.cdi.keys.ExtractorKeys; +import com.linkedin.cdi.keys.JobKeys; +import com.linkedin.cdi.util.AvroSchemaUtils; +import com.linkedin.cdi.util.JsonIntermediateSchema; +import com.linkedin.cdi.util.SchemaUtils; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Set; import javax.annotation.Nullable; -import lombok.Getter; -import lombok.SneakyThrows; -import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; import org.apache.avro.file.DataFileStream; import org.apache.avro.generic.GenericData; @@ -21,21 +26,15 @@ import org.apache.avro.generic.GenericRecord; import org.apache.commons.lang3.StringUtils; import org.apache.gobblin.configuration.WorkUnitState; -import org.apache.gobblin.converter.avro.UnsupportedDateTypeException; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.filter.AvroSchemaBasedFilter; -import com.linkedin.cdi.keys.AvroExtractorKeys; -import com.linkedin.cdi.keys.ExtractorKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.util.AvroSchemaUtils; -import com.linkedin.cdi.util.JsonIntermediateSchema; -import com.linkedin.cdi.util.SchemaUtils; import org.apache.gobblin.util.AvroUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.Assert; +import static com.linkedin.cdi.configuration.PropertyCollection.*; +import static com.linkedin.cdi.configuration.StaticConstants.*; import static org.apache.avro.Schema.Type.*; - /** * AvroExtractor reads Avro formatted files from HDFS locations. * @@ -48,11 +47,14 @@ * * @author esong */ -@Slf4j public class AvroExtractor extends MultistageExtractor { - @Getter + private static final Logger LOG = LoggerFactory.getLogger(AvroExtractor.class); private AvroExtractorKeys avroExtractorKeys = new AvroExtractorKeys(); + public AvroExtractorKeys getAvroExtractorKeys() { + return avroExtractorKeys; + } + public AvroExtractor(WorkUnitState state, JobKeys jobKeys) { super(state, jobKeys); super.initialize(avroExtractorKeys); @@ -83,11 +85,10 @@ protected void setAvroExtractorKeys(AvroExtractorKeys avroExtractorKeys) { * * @return the schema of the extracted record set in AvroSchema */ - @SneakyThrows @Override public Schema getSchema() { - Schema avroSchema; - log.debug("Retrieving schema definition"); + Schema avroSchema = null; + LOG.debug("Retrieving schema definition"); if (this.jobKeys.hasOutputSchema()) { // take pre-defined fixed schema JsonArray schemaArray = jobKeys.getOutputSchema(); @@ -95,7 +96,7 @@ public Schema getSchema() { avroSchema = fromJsonSchema(schemaArray); } else { avroSchema = processInputStream(0) ? avroExtractorKeys.getAvroOutputSchema() - : fromJsonSchema(createMinimumSchema()); + : createMinimumAvroSchema(); } Assert.assertNotNull(avroSchema); return addDerivedFieldsToSchema(avroSchema); @@ -108,7 +109,7 @@ public Schema getSchema() { @Override protected void setRowFilter(JsonArray schemaArray) { if (rowFilter == null) { - if (MultistageProperties.MSTAGE_ENABLE_SCHEMA_BASED_FILTERING.getValidNonblankWithDefault(state)) { + if (MSTAGE_ENABLE_SCHEMA_BASED_FILTERING.get(state)) { rowFilter = new AvroSchemaBasedFilter(new JsonIntermediateSchema(jobKeys.getOutputSchema()), avroExtractorKeys, state); } @@ -128,19 +129,19 @@ protected void setRowFilter(JsonArray schemaArray) { @Nullable @Override public GenericRecord readRecord(GenericRecord reuse) { + super.readRecord(reuse); + if (avroExtractorKeys.getAvroRecordIterator() == null && !processInputStream(0)) { - return null; + return (GenericRecord) endProcessingAndValidateCount(); } - DataFileStream avroRecordIterator = avroExtractorKeys.getAvroRecordIterator(); - if (hasNext()) { avroExtractorKeys.incrProcessedCount(); // update work unit status along the way, since we are using iterators workUnitStatus.setPageStart(avroExtractorKeys.getProcessedCount()); workUnitStatus.setPageNumber(avroExtractorKeys.getCurrentPageNumber()); - GenericRecord row = avroRecordIterator.next(); + GenericRecord row = extractDataField(getNext()); AvroSchemaBasedFilter avroSchemaBasedFilter = (AvroSchemaBasedFilter) rowFilter; if (avroSchemaBasedFilter != null) { row = avroSchemaBasedFilter.filter(row); @@ -152,7 +153,7 @@ public GenericRecord readRecord(GenericRecord reuse) { eof = true; return AvroSchemaUtils.createEOF(state); } - return null; + return (GenericRecord) endProcessingAndValidateCount(); } /** @@ -162,19 +163,29 @@ public GenericRecord readRecord(GenericRecord reuse) { * @return true if Successful */ @Override + // Suppressing un-checked casting warning when casting GenericData.Array + // The casting is thoroughly checked by the isArrayOfRecord method, but it does not get rid of the warning protected boolean processInputStream(long starting) { if (!super.processInputStream(starting)) { return false; } + // returning false to end the work unit if the buffer is null + if (workUnitStatus.getBuffer() == null) { + return false; + } + DataFileStream avroRecordIterator; try { avroRecordIterator = new DataFileStream<>(workUnitStatus.getBuffer(), new GenericDatumReader<>()); + avroExtractorKeys.setAvroRecordIterator(avroRecordIterator); - // store the original schema for further processing + // save one record to infer the avro schema from data if (hasNext() && avroExtractorKeys.getAvroOutputSchema() == null) { - avroExtractorKeys.setAvroOutputSchema(avroRecordIterator.getSchema()); + GenericRecord sampleData = avroRecordIterator.next(); + avroExtractorKeys.setSampleData(AvroSchemaUtils.deepCopy(sampleData.getSchema(), sampleData)); + avroExtractorKeys.setAvroOutputSchema(getAvroSchemaFromData(sampleData)); } if (jobKeys.hasOutputSchema()) { List schemaColumns = new ArrayList<>(new JsonIntermediateSchema(jobKeys.getOutputSchema()) @@ -183,18 +194,16 @@ protected boolean processInputStream(long starting) { avroExtractorKeys.setIsValidOutputSchema(SchemaUtils.isValidOutputSchema(schemaColumns, fieldNames)); } } catch (Exception e) { - log.error("Source Error: {}", e.getMessage()); + LOG.error("Source Error: {}", e.getMessage()); state.setWorkingState(WorkUnitState.WorkingState.FAILED); return false; } // return false to stop the job under these situations - if (workUnitStatus.getBuffer() == null - || avroExtractorKeys.getAvroRecordIterator() == null) { + if (avroExtractorKeys.getAvroRecordIterator() == null) { return false; } avroExtractorKeys.incrCurrentPageNumber(); - avroExtractorKeys.logDebugAll(state.getWorkunit()); workUnitStatus.logDebugAll(); extractorKeys.logDebugAll(state.getWorkunit()); @@ -212,14 +221,39 @@ protected boolean isFirst(long starting) { } /** - * Helper function that indicates if there are any records left to read + * Helper function that checks if there are sample data or more records in the iterator * @return true if there are more records and false otherwise */ protected boolean hasNext() { DataFileStream avroRecordIterator = avroExtractorKeys.getAvroRecordIterator(); + return avroExtractorKeys.getSampleData() != null || hasNext(avroRecordIterator); + } + + /** + * Helper function that indicates if there are any records left to read in the iterator + * @return true if there are more records and false otherwise + */ + private boolean hasNext(DataFileStream avroRecordIterator) { return avroRecordIterator != null && avroRecordIterator.hasNext(); } + /** + * Helper function to get the next record either from sample data or the iterator + * Should only calls this after {@link #hasNext()} + * @return next avro record + */ + private GenericRecord getNext() { + GenericRecord sampleData = avroExtractorKeys.getSampleData(); + + if (sampleData != null) { + avroExtractorKeys.setSampleData(null); + return sampleData; + } else { + DataFileStream avroRecordExtractor = avroExtractorKeys.getAvroRecordIterator(); + return avroRecordExtractor.hasNext() ? avroRecordExtractor.next() : null; + } + } + /** * Append the derived field definition to the output schema * @param schema current schema @@ -235,22 +269,22 @@ private Schema addDerivedFieldsToSchema(Schema schema) { List fields = AvroUtils.deepCopySchemaFields(schema); for (Map.Entry> derivedField: derivedFields) { String name = derivedField.getKey(); - String type = derivedField.getValue().get("type"); + String type = derivedField.getValue().get(KEY_WORD_TYPE); switch (type) { - case "epoc": + case KEY_WORD_EPOC: fields.add(new Schema.Field(name, Schema.create(LONG), name, null)); break; - case "string": - case "regexp": + case KEY_WORD_STRING: + case KEY_WORD_REGEXP: fields.add(new Schema.Field(name, Schema.create(STRING), name, null)); break; - case "boolean": + case KEY_WORD_BOOLEAN: fields.add(new Schema.Field(name, Schema.create(BOOLEAN), name, null)); break; - case "integer": + case KEY_WORD_INTEGER: fields.add(new Schema.Field(name, Schema.create(INT), name, null)); break; - case "number": + case KEY_WORD_NUMBER: fields.add(new Schema.Field(name, Schema.create(DOUBLE), name, null)); break; default: @@ -287,7 +321,7 @@ private GenericRecord addDerivedFields(GenericRecord row) { for (Map.Entry> derivedField: derivedFields) { String name = derivedField.getKey(); Map derivedFieldDef = derivedField.getValue(); - String strValue = processDerivedFieldSource(row, derivedFieldDef); + String strValue = processDerivedFieldSource(row, name, derivedFieldDef); String type = derivedField.getValue().get("type"); switch (type) { case "epoc": @@ -323,7 +357,7 @@ private GenericRecord addDerivedFields(GenericRecord row) { * @param derivedFieldDef map {type: type1, source: source1, format: format1} * @return String value of the derived field */ - private String processDerivedFieldSource(GenericRecord row, Map derivedFieldDef) { + private String processDerivedFieldSource(GenericRecord row, String name, Map derivedFieldDef) { String source = derivedFieldDef.getOrDefault("source", StringUtils.EMPTY); String inputValue = derivedFieldDef.getOrDefault("value", StringUtils.EMPTY); boolean isInputValueFromSource = false; @@ -337,16 +371,112 @@ private String processDerivedFieldSource(GenericRecord row, Map } } - return generateDerivedFieldValue(derivedFieldDef, inputValue, isInputValueFromSource); + return generateDerivedFieldValue(name, derivedFieldDef, inputValue, isInputValueFromSource); } /** * Utility method to convert JsonArray schema to avro schema * @param schema of JsonArray type * @return avro schema - * @throws UnsupportedDateTypeException */ - private Schema fromJsonSchema(JsonArray schema) throws UnsupportedDateTypeException { + private Schema fromJsonSchema(JsonArray schema){ return AvroSchemaUtils.fromJsonSchema(schema, state); } + + /** + * get the avro schema of the data + * @param sampleData a single record + * @return avro schema + */ + private Schema getAvroSchemaFromData(GenericRecord sampleData) { + Schema sampleDataSchema = sampleData.getSchema(); + String dataFieldPath = jobKeys.getDataField(); + if (StringUtils.isBlank(dataFieldPath)) { + return sampleDataSchema; + } + return createDataFieldRecordSchema(sampleData, dataFieldPath); + } + + /** + * Extract the data field from the current row + * @param row the original data + * @return a GenericRecord containing the data field + */ + private GenericRecord extractDataField(GenericRecord row) { + String dataFieldPath = jobKeys.getDataField(); + if (StringUtils.isBlank(dataFieldPath)) { + return row; + } + // if the data field is not present, the schema will be null and the work unit will fail + Schema dataFieldRecordSchema = createDataFieldRecordSchema(row, dataFieldPath); + GenericRecord dataFieldRecord = new GenericData.Record(dataFieldRecordSchema); + // the value should be present here otherwise the work unit would've have failed + Object dataFieldValue = AvroUtils.getFieldValue(row, dataFieldPath).get(); + dataFieldRecord.put(extractDataFieldName(dataFieldPath), dataFieldValue); + return dataFieldRecord; + } + + /** + * create the schema of the record that wraps the data field + * @param data original record + * @param dataFieldPath path to data field + * @return avro schema of the wrapping record + */ + private Schema createDataFieldRecordSchema(GenericRecord data, String dataFieldPath) { + Schema rowSchema = data.getSchema(); + Optional fieldValue = AvroUtils.getFieldValue(data, dataFieldPath); + Schema dataFieldSchema; + if (fieldValue.isPresent()) { + Object dataFieldValue = fieldValue.get(); + if (isArrayOfRecord(dataFieldValue)) { + dataFieldSchema = ((GenericData.Array) dataFieldValue).getSchema(); + } else { + // no need for isPresent check here since the value already exists + dataFieldSchema = AvroUtils.getFieldSchema(rowSchema, dataFieldPath).get(); + } + String dataFieldName = extractDataFieldName(dataFieldPath); + Schema schema = Schema.createRecord(rowSchema.getName(), rowSchema.getDoc(), + rowSchema.getNamespace(), false); + List schemaFields = new ArrayList<>(); + schemaFields.add(new Schema.Field(dataFieldName, dataFieldSchema, dataFieldSchema.getDoc(), null)); + schema.setFields(schemaFields); + return schema; + } else { + failWorkUnit("Terminate the ingestion because the data.field cannot be found"); + return null; + } + } + + /** + * Use the last value in path as the name of the field. For example, for field1.nestedField1 + * this will return nestedField1. + * @param dataFieldPath path to the data field + * @return name of the data field + */ + private String extractDataFieldName(String dataFieldPath) { + String[] pathArray = dataFieldPath.split("\\."); + return pathArray[pathArray.length - 1]; + } + + /** + * Check if the object is of type GenericData.Array + * Also allowing UNION, as the inner records' type could be UNION + * @param payload an object + * @return true if the type is correct and false otherwise + */ + private boolean isArrayOfRecord(Object payload) { + if (payload instanceof GenericData.Array) { + Schema.Type arrayElementType = ((GenericData.Array) payload).getSchema().getElementType().getType(); + return arrayElementType == RECORD || arrayElementType == UNION; + } + return false; + } + + /** + * Create a minimum avro schema + * @return avro schema + */ + private Schema createMinimumAvroSchema() { + return fromJsonSchema(createMinimumSchema()); + } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/extractor/CsvExtractor.java b/cdi-core/src/main/java/com/linkedin/cdi/extractor/CsvExtractor.java index df5664d..edfa17b 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/extractor/CsvExtractor.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/extractor/CsvExtractor.java @@ -11,7 +11,19 @@ import com.google.common.primitives.Longs; import com.google.gson.JsonArray; import com.google.gson.JsonElement; +import com.google.gson.JsonNull; import com.google.gson.JsonObject; +import com.linkedin.cdi.configuration.StaticConstants; +import com.linkedin.cdi.filter.CsvSchemaBasedFilter; +import com.linkedin.cdi.keys.CsvExtractorKeys; +import com.linkedin.cdi.keys.ExtractorKeys; +import com.linkedin.cdi.keys.JobKeys; +import com.linkedin.cdi.preprocessor.InputStreamProcessor; +import com.linkedin.cdi.preprocessor.StreamProcessor; +import com.linkedin.cdi.util.JsonIntermediateSchema; +import com.linkedin.cdi.util.JsonUtils; +import com.linkedin.cdi.util.SchemaBuilder; +import com.linkedin.cdi.util.SchemaUtils; import com.opencsv.CSVParser; import com.opencsv.CSVParserBuilder; import com.opencsv.CSVReader; @@ -30,26 +42,15 @@ import java.util.Map; import java.util.Set; import javax.annotation.Nullable; -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.filter.CsvSchemaBasedFilter; -import com.linkedin.cdi.keys.CsvExtractorKeys; -import com.linkedin.cdi.keys.ExtractorKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.preprocessor.InputStreamProcessor; -import com.linkedin.cdi.preprocessor.StreamProcessor; -import com.linkedin.cdi.util.CsvUtils; -import com.linkedin.cdi.util.JsonIntermediateSchema; -import com.linkedin.cdi.util.SchemaBuilder; -import com.linkedin.cdi.util.SchemaUtils; -import com.linkedin.cdi.util.VariableUtils; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Period; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.Assert; +import static com.linkedin.cdi.configuration.PropertyCollection.*; +import static com.linkedin.cdi.configuration.StaticConstants.*; + /** * CSV Extractor extracts CSV formatted data from an InputStream passed from a Source. @@ -59,12 +60,15 @@ * * @author chrli, esong */ -@Slf4j public class CsvExtractor extends MultistageExtractor { + private static final Logger LOG = LoggerFactory.getLogger(CsvExtractor.class); private final static Long SCHEMA_INFER_MAX_SAMPLE_SIZE = 100L; - @Getter private CsvExtractorKeys csvExtractorKeys = new CsvExtractorKeys(); + public CsvExtractorKeys getCsvExtractorKeys() { + return csvExtractorKeys; + } + public CsvExtractor(WorkUnitState state, JobKeys jobKeys) { super(state, jobKeys); super.initialize(csvExtractorKeys); @@ -74,26 +78,15 @@ public CsvExtractor(WorkUnitState state, JobKeys jobKeys) { @Override protected void initialize(ExtractorKeys keys) { csvExtractorKeys.logUsage(state); - csvExtractorKeys.setColumnHeader( - MultistageProperties.MSTAGE_CSV_COLUMN_HEADER.validateNonblank(state) ? MultistageProperties.MSTAGE_CSV_COLUMN_HEADER.getProp( - state) : false); - csvExtractorKeys.setRowsToSkip(MultistageProperties.MSTAGE_CSV_SKIP_LINES.getValidNonblankWithDefault(state)); - if (csvExtractorKeys.getColumnHeader() && csvExtractorKeys.getRowsToSkip() == 0) { - csvExtractorKeys.setRowsToSkip(1); - } - csvExtractorKeys.setSeparator( - CsvUtils.unescape(MultistageProperties.MSTAGE_CSV_SEPARATOR.getValidNonblankWithDefault(state))); - csvExtractorKeys.setQuoteCharacter( - CsvUtils.unescape(MultistageProperties.MSTAGE_CSV_QUOTE_CHARACTER.getValidNonblankWithDefault(state))); - csvExtractorKeys.setEscapeCharacter( - CsvUtils.unescape(MultistageProperties.MSTAGE_CSV_ESCAPE_CHARACTER.getValidNonblankWithDefault(state))); + csvExtractorKeys.setColumnHeader(MSTAGE_CSV.getColumnHeaderIndex(state) >= 0); + csvExtractorKeys.setDefaultFieldType(MSTAGE_CSV.getDefaultFieldType(state).toLowerCase()); csvExtractorKeys.setSampleRows(new ArrayDeque<>()); // check if user has defined the output schema if (jobKeys.hasOutputSchema()) { JsonArray outputSchema = jobKeys.getOutputSchema(); - csvExtractorKeys.setColumnProjection(expandColumnProjection(MultistageProperties.MSTAGE_CSV_COLUMN_PROJECTION - .getValidNonblankWithDefault(state), outputSchema.size())); + csvExtractorKeys.setColumnProjection(expandColumnProjection(MSTAGE_CSV.getColumnProjection(state), + outputSchema.size())); // initialize the column name to index map based on the schema when derived fields are present if (jobKeys.getDerivedFields().entrySet().size() > 0) { buildColumnToIndexMap(outputSchema); @@ -120,10 +113,11 @@ protected void setCsvExtractorKeys(CsvExtractorKeys csvExtractorKeys) { */ @Override public String getSchema() { - log.debug("Retrieving schema definition"); + LOG.debug("Retrieving schema definition"); JsonArray schemaArray = super.getOrInferSchema(); Assert.assertNotNull(schemaArray); - if (jobKeys.getDerivedFields().size() > 0) { + if (jobKeys.getDerivedFields().size() > 0 && JsonUtils.get(StaticConstants.KEY_WORD_COLUMN_NAME, + jobKeys.getDerivedFields().keySet().iterator().next(), StaticConstants.KEY_WORD_COLUMN_NAME, schemaArray) == JsonNull.INSTANCE) { schemaArray.addAll(addDerivedFieldsToAltSchema()); } return schemaArray.toString(); @@ -142,8 +136,10 @@ public String getSchema() { @Nullable @Override public String[] readRecord(String[] reuse) { + super.readRecord(reuse); + if (csvExtractorKeys.getCsvIterator() == null && !processInputStream(0)) { - return null; + return (String[]) endProcessingAndValidateCount(); } Iterator readerIterator = csvExtractorKeys.getCsvIterator(); @@ -177,7 +173,7 @@ public String[] readRecord(String[] reuse) { return readRecord(reuse); } } - return null; + return (String[]) endProcessingAndValidateCount(); } /** @@ -194,7 +190,7 @@ protected boolean processInputStream(long starting) { // if Content-Type is provided, but not text/csv, the response can have // useful error information - JsonObject expectedContentType = MultistageProperties.MSTAGE_HTTP_RESPONSE_TYPE.getValidNonblankWithDefault(state); + JsonObject expectedContentType = MSTAGE_HTTP_RESPONSE_TYPE.get(state); HashSet expectedContentTypeSet = new LinkedHashSet<>(Arrays.asList("text/csv", "application/gzip")); if (expectedContentType.has(CONTENT_TYPE_KEY) || expectedContentType.has(CONTENT_TYPE_KEY.toLowerCase())) { for (Map.Entry entry: expectedContentType.entrySet()) { @@ -215,29 +211,32 @@ protected boolean processInputStream(long starting) { } } - CSVParser parser = new CSVParserBuilder().withSeparator(csvExtractorKeys.getSeparator().charAt(0)) - .withQuoteChar(csvExtractorKeys.getQuoteCharacter().charAt(0)) - .withEscapeChar(csvExtractorKeys.getEscapeCharacter().charAt(0)) + CSVParser parser = new CSVParserBuilder().withSeparator(MSTAGE_CSV.getFieldSeparator(state).charAt(0)) + .withQuoteChar(MSTAGE_CSV.getQuoteCharacter(state).charAt(0)) + .withEscapeChar(MSTAGE_CSV.getEscapeCharacter(state).charAt(0)) .build(); CSVReader reader = new CSVReaderBuilder(new InputStreamReader(input, Charset.forName( - MultistageProperties.MSTAGE_SOURCE_DATA_CHARACTER_SET.getValidNonblankWithDefault(state)))).withCSVParser(parser) + MSTAGE_SOURCE_DATA_CHARACTER_SET.get(state)))).withCSVParser(parser) .build(); Iterator readerIterator = reader.iterator(); + + // header row can be in the front of informational rows or after them + skipRowAndSaveHeader(readerIterator); + // convert some sample data to json to infer the schema if (!jobKeys.hasOutputSchema() && starting == 0) { // initialize a reader without skipping lines since header might be used JsonArray inferredSchema = inferSchemaWithSample(readerIterator); extractorKeys.setInferredSchema(inferredSchema); + // build the columnToIndexMap for derived fields based on the inferred schema if (jobKeys.getDerivedFields().entrySet().size() != 0) { buildColumnToIndexMap(inferredSchema); } - } else { - skipRowAndSaveHeader(readerIterator); } csvExtractorKeys.setCsvIterator(readerIterator); } catch (Exception e) { - log.error("Error reading the input stream: {}", e.getMessage()); + LOG.error("Error reading the input stream: {}", e.getMessage()); return false; } } @@ -262,7 +261,7 @@ protected boolean processInputStream(long starting) { @Override protected void setRowFilter(JsonArray schemaArray) { if (rowFilter == null) { - if (MultistageProperties.MSTAGE_ENABLE_SCHEMA_BASED_FILTERING.getValidNonblankWithDefault(state)) { + if (MSTAGE_ENABLE_SCHEMA_BASED_FILTERING.get(state)) { rowFilter = new CsvSchemaBasedFilter(new JsonIntermediateSchema(schemaArray), csvExtractorKeys); } } @@ -330,34 +329,18 @@ private void buildColumnToIndexMap(JsonArray schema) { * @param derivedFieldDef map {type: type1, source: source1, format: format1} * @return String value of the derived field */ - private String processDerivedFieldSource(String[] row, Map derivedFieldDef) { - String source = derivedFieldDef.get("source"); - String strValue = ""; - DateTimeZone timeZone = DateTimeZone.forID(timezone.isEmpty() ? DEFAULT_TIMEZONE : timezone); - - // get the base value from various sources - if (source.equalsIgnoreCase("currentdate")) { - strValue = String.valueOf(DateTime.now().getMillis()); - } else if (source.matches("P\\d+D")) { - Period period = Period.parse(source); - strValue = - String.valueOf(DateTime.now().withZone(timeZone).minus(period).dayOfMonth().roundFloorCopy().getMillis()); - } else if (csvExtractorKeys.getColumnToIndexMap().containsKey(source)) { + private String processDerivedFieldSource(String[] row, String name, Map derivedFieldDef) { + String source = derivedFieldDef.getOrDefault("source", StringUtils.EMPTY); + String inputValue = derivedFieldDef.getOrDefault("value", StringUtils.EMPTY); + boolean isInputValueFromSource = false; + + if (csvExtractorKeys.getColumnToIndexMap().containsKey(source)) { int sourceIndex = csvExtractorKeys.getColumnToIndexMap().get(source); - strValue = row[sourceIndex]; - } else if (VariableUtils.PATTERN.matcher(source).matches()) { - strValue = replaceVariable(source); - } else { - failWorkUnit("Unsupported source for derived fields: " + source); + inputValue = row[sourceIndex]; + isInputValueFromSource = true; } - // further processing required for specific types - String type = derivedFieldDef.get("type"); - if (type.equals("epoc") && !(source.equalsIgnoreCase(CURRENT_DATE) || source.matches(PXD)) - && derivedFieldDef.containsKey("format")) { - strValue = deriveEpoc(derivedFieldDef.get("format"), strValue); - } - return strValue; + return generateDerivedFieldValue(name, derivedFieldDef, inputValue, isInputValueFromSource); } /** @@ -378,8 +361,9 @@ private String[] addDerivedFields(String[] row) { int index = originalLength; for (Map.Entry> derivedField : derivedFields) { + String name = derivedField.getKey(); Map derivedFieldDef = derivedField.getValue(); - String strValue = processDerivedFieldSource(row, derivedFieldDef); + String strValue = processDerivedFieldSource(row, name, derivedFieldDef); String type = derivedFieldDef.get("type"); if (SUPPORTED_DERIVED_FIELD_TYPES.contains(type)) { row[index] = strValue; @@ -397,20 +381,20 @@ private String[] addDerivedFields(String[] row) { */ private void skipRowAndSaveHeader(Iterator readerIterator) { int linesRead = 0; - while (readerIterator.hasNext() && linesRead < csvExtractorKeys.getRowsToSkip()) { + while (readerIterator.hasNext() && linesRead < MSTAGE_CSV.getLinesToSkip(state)) { String[] line = getNextLineWithCleansing(readerIterator); - if (linesRead == 0 && csvExtractorKeys.getColumnHeader()) { - // if header is present, the first row will be used as header + // save the column header + if (linesRead == MSTAGE_CSV.getColumnHeaderIndex(state) && csvExtractorKeys.getColumnHeader()) { csvExtractorKeys.setHeaderRow(line); // check if header has all columns in schema if (jobKeys.hasOutputSchema()) { - List schemaColumns = new ArrayList<>(new JsonIntermediateSchema(jobKeys.getOutputSchema()) - .getColumns().keySet()); - List headerRow = Arrays.asList(csvExtractorKeys.getHeaderRow()); - csvExtractorKeys.setIsValidOutputSchema(SchemaUtils.isValidOutputSchema(schemaColumns, headerRow)); + List schemaColumns = + new ArrayList<>(new JsonIntermediateSchema(jobKeys.getOutputSchema()).getColumns().keySet()); + List headerRow = Arrays.asList(csvExtractorKeys.getHeaderRow()); + csvExtractorKeys.setIsValidOutputSchema(SchemaUtils.isValidOutputSchema(schemaColumns, headerRow)); } - linesRead++; } + linesRead++; } } @@ -418,7 +402,6 @@ private void skipRowAndSaveHeader(Iterator readerIterator) { * Perform limited cleansing so that data can be processed by converters * * @param input the input data to be cleansed - * @return the cleansed data */ private void limitedCleanse(String[] input) { for (int i = 0; i < input.length; i++) { @@ -445,7 +428,6 @@ private String[] getNextLineWithCleansing(Iterator readerIterator) { * @return inferred schema */ private JsonArray inferSchemaWithSample(Iterator readerIterator) { - skipRowAndSaveHeader(readerIterator); String[] header = csvExtractorKeys.getHeaderRow(); JsonArray sample = new JsonArray(); int linesRead = 0; @@ -464,30 +446,35 @@ private JsonArray inferSchemaWithSample(Iterator readerIterator) { sample.add(row); linesRead++; } - return SchemaBuilder.fromJsonData(sample).buildAltSchema(jobKeys.getDefaultFieldTypes(), - jobKeys.isEnableCleansing(), - jobKeys.getSchemaCleansingPattern(), - jobKeys.getSchemaCleansingReplacement(), - jobKeys.getSchemaCleansingNullable()).getAsJsonArray(); + return SchemaBuilder.fromJsonData(sample) + .buildAltSchema(jobKeys.getDefaultFieldTypes(), jobKeys.isEnableCleansing(), + jobKeys.getSchemaCleansingPattern(), jobKeys.getSchemaCleansingReplacement(), + jobKeys.getSchemaCleansingNullable()) + .getAsJsonArray(); } /** * Helper function for creating sample json data for schema inference * Type conversion is required as all data will be parsed as string otherwise + * Users can specify the default type for all fields using ms.csv.default.field.type or + * specify the default for a specific field using ms.default.data.type. * @param key name of the column * @param data original data from a column * @param row json form of the row */ private void addParsedCSVData(String key, String data, JsonObject row) { - if (Ints.tryParse(data) != null) { + String defaultFieldType = csvExtractorKeys.getDefaultFieldType(); + if (defaultFieldType.equals(KEY_WORD_STRING)) { + row.addProperty(key, data); + } else if (defaultFieldType.equals(KEY_WORD_INT) || Ints.tryParse(data) != null) { row.addProperty(key, Integer.valueOf(data)); - } else if (Longs.tryParse(data) != null) { + } else if (defaultFieldType.equals(KEY_WORD_LONG) || Longs.tryParse(data) != null) { row.addProperty(key, Long.valueOf(data)); - } else if (Doubles.tryParse(data) != null) { + } else if (defaultFieldType.equals(KEY_WORD_DOUBLE) || Doubles.tryParse(data) != null) { row.addProperty(key, Double.valueOf(data)); - } else if (data.toLowerCase().matches("(true|false)")) { + } else if (defaultFieldType.equals(KEY_WORD_BOOLEAN) || data.toLowerCase().matches("(true|false)")) { row.addProperty(key, Boolean.valueOf(data)); - } else if (Floats.tryParse(data) != null) { + } else if (defaultFieldType.equals(KEY_WORD_FLOAT) || Floats.tryParse(data) != null) { row.addProperty(key, Float.valueOf(data)); } else { row.addProperty(key, data); diff --git a/cdi-core/src/main/java/com/linkedin/cdi/extractor/FileDumpExtractor.java b/cdi-core/src/main/java/com/linkedin/cdi/extractor/FileDumpExtractor.java index 81941e6..016473f 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/extractor/FileDumpExtractor.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/extractor/FileDumpExtractor.java @@ -9,16 +9,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.gson.JsonObject; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; -import javax.annotation.Nullable; -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; -import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; import com.linkedin.cdi.keys.ExtractorKeys; import com.linkedin.cdi.keys.FileDumpExtractorKeys; import com.linkedin.cdi.keys.JobKeys; @@ -27,22 +17,36 @@ import com.linkedin.cdi.preprocessor.StreamProcessor; import com.linkedin.cdi.util.ParameterTypes; import com.linkedin.cdi.util.VariableUtils; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.WorkUnitState; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; /** * FileDumpExtractor takes an InputStream, applies proper preprocessors, and saves the InputStream * to a file. */ -@Slf4j public class FileDumpExtractor extends MultistageExtractor { + private static final Logger LOG = LoggerFactory.getLogger(FileDumpExtractor.class); private final static int HADOOP_DEFAULT_FILE_LENGTH_LIMIT = 255; - @Getter private FileDumpExtractorKeys fileDumpExtractorKeys = new FileDumpExtractorKeys(); + public FileDumpExtractorKeys getFileDumpExtractorKeys() { + return fileDumpExtractorKeys; + } + public FileDumpExtractor(WorkUnitState state, JobKeys jobKeys) { super(state, jobKeys); super.initialize(fileDumpExtractorKeys); @@ -55,15 +59,15 @@ protected void initialize(ExtractorKeys keys) { // initialize FileDumpExtractor keys // Extractors follow the pattern of initializing in constructor to avoid forgetting initialization // in sub-classes - if (MultistageProperties.DATA_PUBLISHER_FINAL_DIR.validateNonblank(state)) { - fileDumpExtractorKeys.setFileDumpLocation(MultistageProperties.DATA_PUBLISHER_FINAL_DIR.getProp(state)); + if (DATA_PUBLISHER_FINAL_DIR.isValidNonblank(state)) { + fileDumpExtractorKeys.setFileDumpLocation(DATA_PUBLISHER_FINAL_DIR.get(state)); } else { throw new RuntimeException("data publisher final dir is empty or null"); } - // file permission is required, but a default value is given in MultistageProperties + // file permission is required, but a default value is given in PropertyCollection fileDumpExtractorKeys.setFileWritePermissions( - MultistageProperties.MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION.getValidNonblankWithDefault(state)); + MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION.get(state)); // work unit file name is based on a template that is defined by ms.extractor.target.file.name // and then substituted with activation parameters @@ -128,13 +132,13 @@ protected boolean processInputStream(long starting) { } if (StringUtils.isBlank(fileDumpExtractorKeys.getFileName())) { - log.error("File name is empty so cannot dump onto the file system."); + LOG.error("File name is empty so cannot dump onto the file system."); this.state.setWorkingState(WorkUnitState.WorkingState.FAILED); return false; } if (workUnitStatus.getBuffer() == null) { - log.info("Received a NULL InputStream, end the work unit"); + LOG.info("Received a NULL InputStream, end the work unit"); return false; } @@ -156,7 +160,7 @@ protected boolean processInputStream(long starting) { } writeToFileSystem(input, fileName); } catch (Exception e) { - log.error("Error while extracting from source or writing to target", e); + LOG.error("Error while extracting from source or writing to target", e); this.state.setWorkingState(WorkUnitState.WorkingState.FAILED); return false; } @@ -199,7 +203,7 @@ private void writeToFileSystem(InputStream is, String dumplocation) { is.close(); os.flush(); os.close(); - log.info("FileDumpExtractor: written {} bytes to file {}", totalBytes, dumplocation); + LOG.info("FileDumpExtractor: written {} bytes to file {}", totalBytes, dumplocation); } catch (IOException e) { throw new RuntimeException("Unable to dump file at specified location from FileDumpExtractor", e); } @@ -213,22 +217,22 @@ private void writeToFileSystem(InputStream is, String dumplocation) { * @return the file name */ private String getFileName(WorkUnitState state) { - String fileNameTemplate = MultistageProperties.MSTAGE_EXTRACTOR_TARGET_FILE_NAME.getValidNonblankWithDefault(state); + String fileNameTemplate = MSTAGE_EXTRACTOR_TARGET_FILE_NAME.get(state); JsonObject activationParameters = extractorKeys.getActivationParameters(); try { String filePath = VariableUtils.replaceWithTracking(fileNameTemplate, activationParameters).getKey(); List segments = Lists.newArrayList(filePath.split(Path.SEPARATOR)); String fileName = segments.get(segments.size() - 1); if (fileName.length() > HADOOP_DEFAULT_FILE_LENGTH_LIMIT) { - log.warn("File name is truncated to {} characters", HADOOP_DEFAULT_FILE_LENGTH_LIMIT); + LOG.warn("File name is truncated to {} characters", HADOOP_DEFAULT_FILE_LENGTH_LIMIT); fileName = fileName.substring(0, HADOOP_DEFAULT_FILE_LENGTH_LIMIT - 1); } segments.remove(segments.size() - 1); segments.add(fileName); return Joiner.on(Path.SEPARATOR_CHAR).join(segments); } catch (Exception e) { - log.error("Error resolving placeholders in {}", MultistageProperties.MSTAGE_EXTRACTOR_TARGET_FILE_NAME.toString()); - log.error("The value \"{}\" will be used as is", fileNameTemplate); + LOG.error("Error resolving placeholders in {}", MSTAGE_EXTRACTOR_TARGET_FILE_NAME.toString()); + LOG.error("The value \"{}\" will be used as is", fileNameTemplate); return fileNameTemplate; } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/extractor/JsonExtractor.java b/cdi-core/src/main/java/com/linkedin/cdi/extractor/JsonExtractor.java index 94fd0cf..cbcf5ab 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/extractor/JsonExtractor.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/extractor/JsonExtractor.java @@ -9,35 +9,39 @@ import com.google.gson.Gson; import com.google.gson.JsonArray; import com.google.gson.JsonElement; +import com.google.gson.JsonNull; import com.google.gson.JsonObject; import com.google.gson.JsonParser; import com.google.gson.JsonPrimitive; +import com.linkedin.cdi.configuration.StaticConstants; +import com.linkedin.cdi.filter.JsonSchemaBasedFilter; +import com.linkedin.cdi.keys.ExtractorKeys; +import com.linkedin.cdi.keys.JobKeys; +import com.linkedin.cdi.keys.JsonExtractorKeys; +import com.linkedin.cdi.util.EncryptionUtils; +import com.linkedin.cdi.util.JsonUtils; +import com.linkedin.cdi.util.ParameterTypes; +import com.linkedin.cdi.util.SchemaBuilder; import java.io.InputStream; import java.io.InputStreamReader; import java.nio.charset.Charset; import java.nio.charset.UnsupportedCharsetException; -import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashSet; import java.util.Map; import javax.annotation.Nullable; -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.filter.JsonSchemaBasedFilter; -import com.linkedin.cdi.keys.ExtractorKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.keys.JsonExtractorKeys; -import com.linkedin.cdi.util.EncryptionUtils; -import com.linkedin.cdi.util.JsonUtils; -import com.linkedin.cdi.util.ParameterTypes; -import com.linkedin.cdi.util.SchemaBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.Assert; +import static com.linkedin.cdi.configuration.PropertyCollection.*; +import static com.linkedin.cdi.configuration.StaticConstants.*; + /** * JsonExtractor reads Json formatted responses from HTTP sources, like Rest API source. @@ -52,16 +56,18 @@ * * @author chrli */ -@Slf4j public class JsonExtractor extends MultistageExtractor { + private static final Logger LOG = LoggerFactory.getLogger(JsonExtractor.class); final private static JsonObject EOF = new Gson().fromJson("{\"EOF\": \"EOF\"}", JsonObject.class); private final static String JSON_MEMBER_SEPARATOR = "."; private final static Long SCHEMA_INFER_MAX_SAMPLE_SIZE = 100L; - - @Getter private JsonExtractorKeys jsonExtractorKeys = new JsonExtractorKeys(); + public JsonExtractorKeys getJsonExtractorKeys() { + return jsonExtractorKeys; + } + public JsonExtractor(WorkUnitState state, JobKeys jobKeys) { super(state, jobKeys); super.initialize(jsonExtractorKeys); @@ -138,10 +144,11 @@ protected void setJsonExtractorKeys(JsonExtractorKeys jsonExtractorKeys) { */ @Override public JsonArray getSchema() { - log.debug("Retrieving schema definition"); + LOG.debug("Retrieving schema definition"); JsonArray schemaArray = super.getOrInferSchema(); Assert.assertNotNull(schemaArray); - if (jobKeys.getDerivedFields().size() > 0) { + if (jobKeys.getDerivedFields().size() > 0 && JsonUtils.get(StaticConstants.KEY_WORD_COLUMN_NAME, + jobKeys.getDerivedFields().keySet().iterator().next(), StaticConstants.KEY_WORD_COLUMN_NAME, schemaArray) == JsonNull.INSTANCE) { schemaArray.addAll(addDerivedFieldsToAltSchema()); } return schemaArray; @@ -150,8 +157,10 @@ public JsonArray getSchema() { @Nullable @Override public JsonObject readRecord(JsonObject reuse) { + super.readRecord(reuse); + if (jsonExtractorKeys.getJsonElementIterator() == null && !processInputStream(0)) { - return null; + return (JsonObject) endProcessingAndValidateCount(); } if (jsonExtractorKeys.getJsonElementIterator().hasNext()) { @@ -175,7 +184,7 @@ public JsonObject readRecord(JsonObject reuse) { eof = true; return EOF; } - return null; + return (JsonObject) endProcessingAndValidateCount(); } /** @@ -192,8 +201,8 @@ protected boolean processInputStream(long starting) { // if Content-Type is provided, but not application/json, the response can have // useful error information - JsonObject expectedContentType = MultistageProperties.MSTAGE_HTTP_RESPONSE_TYPE.getValidNonblankWithDefault(state); - HashSet expectedContentTypeSet = new LinkedHashSet<>(Arrays.asList("application/json")); + JsonObject expectedContentType = MSTAGE_HTTP_RESPONSE_TYPE.get(state); + HashSet expectedContentTypeSet = new LinkedHashSet<>(Collections.singletonList("application/json")); if (expectedContentType.has(CONTENT_TYPE_KEY)) { for (Map.Entry entry: expectedContentType.entrySet()) { expectedContentTypeSet.add(entry.getValue().getAsString()); @@ -211,19 +220,19 @@ protected boolean processInputStream(long starting) { return false; } } catch (Exception e) { - log.error("Source Error: {}", e.getMessage()); + LOG.error("Source Error: {}", e.getMessage()); state.setWorkingState(WorkUnitState.WorkingState.FAILED); return false; } - log.debug("Checking parsed Json object"); + LOG.debug("Checking parsed Json object"); JsonArray coreData = new JsonArray(); JsonElement payload; if (StringUtils.isNotBlank(jobKeys.getDataField())) { payload = JsonUtils.get(data.getAsJsonObject(), jobKeys.getDataField()); if (payload.isJsonNull()) { - log.info("Terminate the ingestion because no actual payload in the response"); + LOG.info("Terminate the ingestion because no actual payload in the response"); return false; } } else { @@ -233,7 +242,7 @@ protected boolean processInputStream(long starting) { if (payload.isJsonArray()) { coreData = payload.getAsJsonArray(); } else { - log.info("Payload is not a Json Array, therefore add the whole payload a one single entry"); + LOG.info("Payload is not a Json Array, therefore add the whole payload a one single entry"); coreData.add(payload); } @@ -295,7 +304,7 @@ private String processDerivedFieldSource(JsonObject row, String name, Map 0) { row.addProperty(name, Long.parseLong(strValue)); } break; - case "string": - case "regexp": + case KEY_WORD_STRING: + case KEY_WORD_REGEXP: row.addProperty(name, strValue); break; - case "boolean": + case KEY_WORD_BOOLEAN: row.addProperty(name, Boolean.parseBoolean(strValue)); break; - case "integer": + case KEY_WORD_INTEGER: row.addProperty(name, Integer.parseInt(strValue)); break; - case "number": + case KEY_WORD_NUMBER: row.addProperty(name, Double.parseDouble(strValue)); break; default: @@ -436,12 +445,14 @@ private Long getTotalCountValue(JsonElement data) { if (StringUtils.isNotBlank(jobKeys.getDataField())) { JsonElement payload = JsonUtils.get(data.getAsJsonObject(), jobKeys.getDataField()); if (payload.isJsonNull()) { - log.info("Expected payload at JsonPath={} doesn't exist", jobKeys.getDataField()); + LOG.info("Expected payload at JsonPath={} doesn't exist", jobKeys.getDataField()); return jsonExtractorKeys.getTotalCount(); } else if (payload.isJsonArray()) { return jsonExtractorKeys.getTotalCount() + payload.getAsJsonArray().size(); + } else if (payload.isJsonObject()) { + return jsonExtractorKeys.getTotalCount() + 1; } else { - throw new RuntimeException("Payload is not a JsonArray, only array payload is supported"); + throw new RuntimeException("Unsupported payload type: only JsonArray or JsonObject is supported"); } } else { // no total count field and no data field @@ -557,9 +568,9 @@ private JsonObject retrievePushDowns(JsonElement response, Map> entry : derivedFields.entrySet()) { String source = entry.getValue().get("source"); - if (data.has(source)) { + if (JsonUtils.has(data, source)) { pushDowns.addProperty(entry.getKey(), data.get(source).getAsString()); - log.info("Identified push down value: {}", pushDowns); + LOG.info("Identified push down value: {}", pushDowns); } } return pushDowns; @@ -571,11 +582,11 @@ private JsonObject retrievePushDowns(JsonElement response, Map The schema class * @param The data class */ -@Slf4j public class MultistageExtractor implements Extractor { + private static final Logger LOG = LoggerFactory.getLogger(MultistageExtractor.class); protected final static String CURRENT_DATE = "currentdate"; protected final static String PXD = "P\\d+D"; protected final static String CONTENT_TYPE_KEY = "Content-Type"; protected final static List SUPPORTED_DERIVED_FIELD_TYPES = - Arrays.asList("epoc", "string", "integer", "number"); + Arrays.asList(KEY_WORD_EPOC, KEY_WORD_STRING, KEY_WORD_REGEXP, KEY_WORD_BOOLEAN, KEY_WORD_INTEGER, KEY_WORD_NUMBER); protected static final String COMMA_STR = ","; protected final static String DEFAULT_TIMEZONE = "America/Los_Angeles"; - @Setter protected String timezone = ""; - @Getter(AccessLevel.PUBLIC) - @Setter(AccessLevel.PACKAGE) protected WorkUnitStatus workUnitStatus = WorkUnitStatus.builder().build(); - - @Getter(AccessLevel.PUBLIC) protected WorkUnitState state = null; protected MultistageSchemaBasedFilter rowFilter = null; protected Boolean eof = false; // subclass might override this to decide whether to do record // level pagination protected Iterator payloadIterator = null; - @Getter ExtractorKeys extractorKeys = new ExtractorKeys(); - @Getter JsonObject currentParameters = null; - @Getter - @Setter MultistageConnection connection = null; - @Getter - @Setter JobKeys jobKeys; + public String getTimezone() { + return timezone; + } + + public void setTimezone(String timezone) { + this.timezone = timezone; + } + + public WorkUnitStatus getWorkUnitStatus() { + return workUnitStatus; + } + + public void setWorkUnitStatus(WorkUnitStatus workUnitStatus) { + this.workUnitStatus = workUnitStatus; + } + + public WorkUnitState getState() { + return state; + } + + public void setState(WorkUnitState state) { + this.state = state; + } + + public MultistageSchemaBasedFilter getRowFilter() { + return rowFilter; + } + + public void setRowFilter(MultistageSchemaBasedFilter rowFilter) { + this.rowFilter = rowFilter; + } + + public Boolean getEof() { + return eof; + } + + public void setEof(Boolean eof) { + this.eof = eof; + } + + public Iterator getPayloadIterator() { + return payloadIterator; + } + + public void setPayloadIterator(Iterator payloadIterator) { + this.payloadIterator = payloadIterator; + } + + public ExtractorKeys getExtractorKeys() { + return extractorKeys; + } + + public void setExtractorKeys(ExtractorKeys extractorKeys) { + this.extractorKeys = extractorKeys; + } + + public JsonObject getCurrentParameters() { + return currentParameters; + } + + public void setCurrentParameters(JsonObject currentParameters) { + this.currentParameters = currentParameters; + } + + public MultistageConnection getConnection() { + return connection; + } + + public void setConnection(MultistageConnection connection) { + this.connection = connection; + } + + public JobKeys getJobKeys() { + return jobKeys; + } + + public void setJobKeys(JobKeys jobKeys) { + this.jobKeys = jobKeys; + } + public MultistageExtractor(WorkUnitState state, JobKeys jobKeys) { this.state = state; this.jobKeys = jobKeys; @@ -110,10 +178,10 @@ public MultistageExtractor(WorkUnitState state, JobKeys jobKeys) { protected void initialize(ExtractorKeys keys) { extractorKeys = keys; - extractorKeys.setActivationParameters(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.getValidNonblankWithDefault(state)); - extractorKeys.setDelayStartTime(MultistageProperties.MSTAGE_WORKUNIT_STARTTIME_KEY.getProp(state)); - extractorKeys.setExplictEof(MultistageProperties.MSTAGE_DATA_EXPLICIT_EOF.getValidNonblankWithDefault(state)); - extractorKeys.setSignature(MultistageProperties.DATASET_URN_KEY.getProp(state)); + extractorKeys.setActivationParameters(MSTAGE_ACTIVATION_PROPERTY.get(state)); + extractorKeys.setDelayStartTime(MSTAGE_WORK_UNIT_SCHEDULING_STARTTIME.get(state)); + extractorKeys.setExplictEof(MSTAGE_DATA_EXPLICIT_EOF.get(state)); + extractorKeys.setSignature(DATASET_URN.get(state)); extractorKeys.setPreprocessors(getPreprocessors(state)); extractorKeys.setPayloads(getPayloads(state)); payloadIterator = extractorKeys.getPayloads().iterator(); @@ -138,14 +206,29 @@ public long getHighWatermark() { @Nullable @Override public D readRecord(D reuse) { + if (extractorKeys.getProcessedCount() % (100 * 1000) == 0) { + LOG.debug(String.format(MSG_ROWS_PROCESSED, + extractorKeys.getProcessedCount(), + extractorKeys.getSignature())); + } return null; } @Override public void close() { + LOG.info("Closing the work unit: {}", this.extractorKeys.getSignature()); + + Preconditions.checkNotNull(state.getWorkunit(), MSG_WORK_UNIT_ALWAYS); + Preconditions.checkNotNull(state.getWorkunit().getLowWatermark(), MSG_LOW_WATER_MARK_ALWAYS); if (state.getWorkingState().equals(WorkUnitState.WorkingState.SUCCESSFUL)) { state.setActualHighWatermark(state.getWorkunit().getExpectedHighWatermark(LongWatermark.class)); + } else if (state.getActualHighWatermark() == null) { + // Set the actual high watermark to low watermark explicitly, + // replacing the implicit behavior in state.getActualHighWatermark(LongWatermark.class) + // avoiding different returns from the two versions of getActualHighWatermark() + state.setActualHighWatermark(state.getWorkunit().getLowWatermark(LongWatermark.class)); } + if (connection != null) { connection.closeAll(StringUtils.EMPTY); } @@ -208,7 +291,7 @@ protected boolean processInputStream(long starting) { */ protected void setRowFilter(JsonArray schemaArray) { if (rowFilter == null) { - if (MultistageProperties.MSTAGE_ENABLE_SCHEMA_BASED_FILTERING.getValidNonblankWithDefault(state)) { + if (MSTAGE_ENABLE_SCHEMA_BASED_FILTERING.get(state)) { rowFilter = new JsonSchemaBasedFilter(new JsonIntermediateSchema(schemaArray)); } } @@ -235,10 +318,10 @@ JsonArray getOrInferSchema() { if (this.jobKeys.hasSourceSchema()) { schemaArray = this.jobKeys.getSourceSchema(); schemaArray = JsonUtils.deepCopy(schemaArray).getAsJsonArray(); - log.info("Source provided schema: {}", schemaArray.toString()); + LOG.info("Source provided schema: {}", schemaArray.toString()); } else if (extractorKeys.getInferredSchema() != null) { schemaArray = JsonUtils.deepCopy(extractorKeys.getInferredSchema()).getAsJsonArray(); - log.info("Inferred schema: {}", schemaArray.toString()); + LOG.info("Inferred schema: {}", schemaArray.toString()); } } @@ -272,7 +355,7 @@ protected void holdExecutionUnitPresetStartTime() { try { Thread.sleep(100L); } catch (Exception e) { - log.warn(e.getMessage()); + LOG.warn(e.getMessage()); } } } @@ -288,8 +371,8 @@ protected void holdExecutionUnitPresetStartTime() { List> getPreprocessors(State state) { ImmutableList.Builder> builder = ImmutableList.builder(); JsonObject preprocessorsParams = - MultistageProperties.MSTAGE_EXTRACT_PREPROCESSORS_PARAMETERS.getValidNonblankWithDefault(state); - String preprocessors = MultistageProperties.MSTAGE_EXTRACT_PREPROCESSORS.getValidNonblankWithDefault(state); + MSTAGE_EXTRACT_PREPROCESSORS_PARAMETERS.get(state); + String preprocessors = MSTAGE_EXTRACT_PREPROCESSORS.get(state); JsonObject preprocessorParams; for (String preprocessor : preprocessors.split(COMMA_STR)) { String p = preprocessor.trim(); @@ -324,7 +407,7 @@ List> getPreprocessors(State state) { (StreamProcessor) clazz.getConstructor(JsonObject.class).newInstance(preprocessorParams); builder.add(instance); } catch (Exception e) { - log.error("Error creating preprocessor: {}, Exception: {}", p, e.getMessage()); + LOG.error("Error creating preprocessor: {}, Exception: {}", p, e.getMessage()); } } } @@ -337,7 +420,7 @@ List> getPreprocessors(State state) { */ protected void failWorkUnit(String error) { if (!StringUtils.isEmpty(error)) { - log.error(error); + LOG.error(error); } this.state.setWorkingState(WorkUnitState.WorkingState.FAILED); } @@ -393,8 +476,8 @@ protected JsonArray addDerivedFieldsToAltSchema() { default: // by default take the source types JsonElement source = JsonUtils.get(entry.getValue().get(KEY_WORD_SOURCE), jobKeys.getOutputSchema()); - dataType.addProperty(KEY_WORD_TYPE, source.isJsonNull() ? KEY_WORD_STRING - : source.getAsJsonObject().get(KEY_WORD_TYPE).getAsString()); + dataType.addProperty(KEY_WORD_TYPE, + source.isJsonNull() ? KEY_WORD_STRING : source.getAsJsonObject().get(KEY_WORD_TYPE).getAsString()); break; } column.add("dataType", dataType); @@ -408,21 +491,21 @@ protected boolean isInputValueFromSource(String source) { || VariableUtils.PATTERN.matcher(source).matches()); } - protected String generateDerivedFieldValue(Map derivedFieldDef, + protected String generateDerivedFieldValue(String name, Map derivedFieldDef, final String inputValue, boolean isStrValueFromSource) { - String strValue = StringUtils.EMPTY; + String strValue = inputValue; long longValue = Long.MIN_VALUE; String source = derivedFieldDef.getOrDefault("source", StringUtils.EMPTY); String type = derivedFieldDef.get("type"); String format = derivedFieldDef.getOrDefault("format", StringUtils.EMPTY); DateTimeZone timeZone = DateTimeZone.forID(timezone.isEmpty() ? DEFAULT_TIMEZONE : timezone); - // get the base value from various sources + // get the base value from date times or variables if (source.equalsIgnoreCase(CURRENT_DATE)) { longValue = DateTime.now().getMillis(); } else if (source.matches(PXD)) { Period period = Period.parse(source); - longValue = DateTime.now().withZone(timeZone).minus(period).dayOfMonth().roundFloorCopy().getMillis(); + longValue = DateTime.now().withZone(timeZone).minus(period).dayOfMonth().roundFloorCopy().getMillis(); } else if (VariableUtils.PATTERN.matcher(source).matches()) { strValue = replaceVariable(source); } else if (!StringUtils.isEmpty(source) && !isStrValueFromSource) { @@ -435,30 +518,35 @@ protected String generateDerivedFieldValue(Map derivedFieldDef, if (longValue != Long.MIN_VALUE) { strValue = String.valueOf(longValue); } else if (!format.equals(StringUtils.EMPTY)) { - strValue = deriveEpoc(format, inputValue); + strValue = deriveEpoc(format, strValue); } else { // Otherwise, the strValue should be a LONG string derived from a dynamic variable source - Assert.assertNotNull(LongValidator.getInstance().validate(strValue)); + Assert.assertNotNull(LongValidator.getInstance().isValid(strValue)); } break; case "regexp": Pattern pattern = Pattern.compile(!format.equals(StringUtils.EMPTY) ? format : "(.*)"); - Matcher matcher = pattern.matcher(inputValue); + Matcher matcher = pattern.matcher(strValue); if (matcher.find()) { strValue = matcher.group(1); } else { - log.error("Regular expression finds no match!"); + LOG.error("Regular expression finds no match!"); strValue = "no match"; } break; case "boolean": - if (!StringUtils.isEmpty(inputValue)) { - strValue = inputValue; + if (StringUtils.isEmpty(strValue)) { + LOG.error("Input value of a boolean derived field should not be empty!"); } break; default: break; } + + if (StringUtils.isEmpty(strValue)) { + failWorkUnit(String.format("Could not extract the value for the derived field %s from %s", + name, StringUtils.join(derivedFieldDef))); + } return strValue; } @@ -469,14 +557,14 @@ protected String generateDerivedFieldValue(Map derivedFieldDef, * then an exception should be logged in debug mode, and an empty string returned. */ protected String extractText(InputStream input) { - log.debug("Parsing response InputStream as Text"); + LOG.debug("Parsing response InputStream as Text"); String data = ""; if (input != null) { try { data = InputStreamUtils.extractText(input, - MultistageProperties.MSTAGE_SOURCE_DATA_CHARACTER_SET.getValidNonblankWithDefault(state)); + MSTAGE_SOURCE_DATA_CHARACTER_SET.get(state)); } catch (Exception e) { - log.debug(e.toString()); + LOG.debug(e.toString()); } } return data; @@ -495,8 +583,8 @@ protected boolean checkContentType(WorkUnitStatus wuStatus, String expectedConte if (wuStatus.getMessages() != null && wuStatus.getMessages().containsKey("contentType")) { String contentType = wuStatus.getMessages().get("contentType"); if (!contentType.equalsIgnoreCase(expectedContentType)) { - log.info("Content is {}, expecting {}", contentType, expectedContentType); - log.debug(extractText(wuStatus.getBuffer())); + LOG.info("Content is {}, expecting {}", contentType, expectedContentType); + LOG.debug(extractText(wuStatus.getBuffer())); return false; } } @@ -507,8 +595,8 @@ protected boolean checkContentType(WorkUnitStatus wuStatus, HashSet expe if (wuStatus.getMessages() != null && wuStatus.getMessages().containsKey("contentType")) { String contentType = wuStatus.getMessages().get("contentType"); if (!expectedContentType.contains(contentType.toLowerCase())) { - log.info("Content is {}, expecting {}", contentType, expectedContentType.toString()); - log.debug(extractText(wuStatus.getBuffer())); + LOG.info("Content is {}, expecting {}", contentType, expectedContentType.toString()); + LOG.debug(extractText(wuStatus.getBuffer())); return false; } } @@ -582,14 +670,14 @@ protected boolean waitingBySessionKeyWithTimeout() { // Fail if the session failCondition is met if (isSessionStateFailed()) { String message = String.format("Session fail condition is met: %s", jobKeys.getSessionStateFailCondition()); - log.warn(message); + LOG.warn(message); throw new RuntimeException(message); } // if stop condition is present but the condition has not been met, we // will check if the session should time out if (DateTime.now().getMillis() > extractorKeys.getStartTime() + jobKeys.getSessionTimeout()) { - log.warn("Session time out after {} seconds", jobKeys.getSessionTimeout() / 1000); + LOG.warn("Session time out after {} seconds", jobKeys.getSessionTimeout() / 1000); throw new RuntimeException("Session timed out before ending condition is met"); } @@ -671,23 +759,21 @@ protected JsonArray createMinimumSchema() { if (state.contains(ConfigurationKeys.EXTRACT_PRIMARY_KEY_FIELDS_KEY)) { String[] primaryKeys = - state.getProp(ConfigurationKeys.EXTRACT_PRIMARY_KEY_FIELDS_KEY, - StringUtils.EMPTY).split(COMMA_STR); + state.getProp(ConfigurationKeys.EXTRACT_PRIMARY_KEY_FIELDS_KEY, StringUtils.EMPTY).split(COMMA_STR); for (String key: primaryKeys) { if (!key.isEmpty()) { - elements.add(new SchemaBuilder(key, SchemaBuilder.PRIMITIVE, true, new ArrayList<>()) - .setPrimitiveType(KEY_WORD_STRING)); + elements.add(new SchemaBuilder(key, SchemaBuilder.PRIMITIVE, true, new ArrayList<>()).setPrimitiveType( + KEY_WORD_STRING)); } } } if (state.contains(ConfigurationKeys.EXTRACT_DELTA_FIELDS_KEY)) { String[] deltaKeys = - state.getProp(ConfigurationKeys.EXTRACT_DELTA_FIELDS_KEY, - StringUtils.EMPTY).split(COMMA_STR); + state.getProp(ConfigurationKeys.EXTRACT_DELTA_FIELDS_KEY, StringUtils.EMPTY).split(COMMA_STR); for (String key: deltaKeys) { if (!key.isEmpty()) { - elements.add(new SchemaBuilder(key, SchemaBuilder.PRIMITIVE, true, new ArrayList<>()) - .setPrimitiveType(KEY_WORD_TIMESTAMP)); + elements.add(new SchemaBuilder(key, SchemaBuilder.PRIMITIVE, true, new ArrayList<>()).setPrimitiveType( + KEY_WORD_TIMESTAMP)); } } } @@ -760,9 +846,9 @@ JsonObject replaceVariablesInParameters(final JsonObject parameters) { } } } catch (Exception e) { - log.error("Encoding error is not expected, but : {}", e.getMessage()); + LOG.error("Encoding error is not expected, but : {}", e.getMessage()); } - log.debug("Final parameters: {}", finalParameter.toString()); + LOG.debug("Final parameters: {}", finalParameter.toString()); return finalParameter; } @@ -850,14 +936,6 @@ private JsonObject getUpdatedWorkUnitVariableValues(JsonObject initialVariableVa return updatedVariableValues; } - protected void logUsage(State state) { - log.info("Checking essential (not always mandatory) parameters..."); - log.info("Values can be default values for the specific type if the property is not configured"); - for (MultistageProperties p : JobKeys.ESSENTIAL_PARAMETERS) { - log.info("Property {} ({}) has value {} ", p.toString(), p.getClassName(), p.getValidNonblankWithDefault(state)); - } - } - /** * Read payload records from secondary input location. Subclasses might * override this to process payload differently. @@ -866,11 +944,24 @@ protected void logUsage(State state) { * @return the payload records */ protected JsonArray getPayloads(State state) { - JsonArray payloads = MultistageProperties.MSTAGE_PAYLOAD_PROPERTY.getValidNonblankWithDefault(state); + JsonArray payloads = MSTAGE_PAYLOAD_PROPERTY.get(state); JsonArray records = new JsonArray(); for (JsonElement entry : payloads) { records.addAll(new HdfsReader(state).readSecondary(entry.getAsJsonObject())); } return records; } + + /** + * Validate the minimum work unit records threshold is met, otherwise raise + * exception, which will faile the task + */ + protected Object endProcessingAndValidateCount() { + if (extractorKeys.getProcessedCount() < jobKeys.getMinWorkUnitRecords()) { + throw new RuntimeException(String.format(EXCEPTION_RECORD_MINIMUM, + jobKeys.getMinWorkUnitRecords(), + jobKeys.getMinWorkUnitRecords())); + } + return null; + } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/ApacheHttpClientFactory.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/ApacheHttpClientFactory.java deleted file mode 100644 index 06c67e4..0000000 --- a/cdi-core/src/main/java/com/linkedin/cdi/factory/ApacheHttpClientFactory.java +++ /dev/null @@ -1,19 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.factory; - -import org.apache.gobblin.configuration.State; -import org.apache.http.client.HttpClient; -import org.apache.http.impl.client.HttpClientBuilder; - - -/** - * A vehicle to produce an Apache HttpClient - */ -public class ApacheHttpClientFactory implements HttpClientFactory { - public HttpClient get(State state) { - return HttpClientBuilder.create().build(); - } -} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/ConnectionClientFactory.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/ConnectionClientFactory.java new file mode 100644 index 0000000..3781e59 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/ConnectionClientFactory.java @@ -0,0 +1,60 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory; + +import com.jcraft.jsch.Session; +import com.linkedin.cdi.factory.reader.SchemaReader; +import com.linkedin.cdi.factory.sftp.SftpChannelClient; +import com.linkedin.cdi.factory.sftp.SftpClient; +import java.sql.Connection; +import org.apache.gobblin.configuration.State; +import org.apache.http.client.HttpClient; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.utils.AttributeMap; + + +/** + * The interface for dynamic client creation based on environment + */ +public interface ConnectionClientFactory { + /** + * Initiate an HTTP client + * @param state the State of execution environment + * @return an HTTP client object + */ + HttpClient getHttpClient(State state); + + /** + * Initiate an S3 HTTP client + * @param state the state of execution environment + * @param config S3 parameters + * @return an S3 HTTP client object + */ + SdkHttpClient getS3Client(State state, AttributeMap config); + + /** + * Initiate a JDBC Connection + * @param jdbcUrl plain or encrypted URL + * @param userId plain or encrypted user name + * @param cryptedPassword plain or encrypted password + * @param state the state of execution environment + * @return a JDBC connection + */ + Connection getJdbcConnection(String jdbcUrl, String userId, String cryptedPassword, State state); + + /** + * Initiate a Secure Channel client for SFTP Connection + * @param state the state of execution environment + * @return a SFTP channel client + */ + SftpClient getSftpChannelClient(State state); + + /** + * Initiate a SchemaReader + * @param state the state of execution environment + * @return a SchemaReader + */ + SchemaReader getSchemaReader(State state); +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/DefaultConnectionClientFactory.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/DefaultConnectionClientFactory.java new file mode 100644 index 0000000..cb928a4 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/DefaultConnectionClientFactory.java @@ -0,0 +1,94 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory; + +import com.linkedin.cdi.factory.reader.JsonFileReader; +import com.linkedin.cdi.factory.reader.SchemaReader; +import com.linkedin.cdi.factory.sftp.SftpChannelClient; +import com.linkedin.cdi.factory.sftp.SftpClient; +import com.linkedin.cdi.util.EncryptionUtils; +import java.sql.Connection; +import java.sql.DriverManager; +import org.apache.gobblin.configuration.State; +import org.apache.http.client.HttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.utils.AttributeMap; + +import static software.amazon.awssdk.http.SdkHttpConfigurationOption.*; + + +/** + * This is the default implementation + */ +public class DefaultConnectionClientFactory implements ConnectionClientFactory { + private static final Logger LOG = LoggerFactory.getLogger(DefaultConnectionClientFactory.class); + + /** + * Initiate an HTTP client + * @param state the State of execution environment + * @return an HTTP client object + */ + @Override + public HttpClient getHttpClient(State state) { + return HttpClientBuilder.create().build(); + } + + /** + * Initiate an S3 HTTP client + * @param state the state of execution environment + * @param config S3 parameters + * @return an S3 HTTP client object + */ + @Override + public SdkHttpClient getS3Client(State state, AttributeMap config) { + return ApacheHttpClient.builder() + .connectionTimeout(config.get(CONNECTION_TIMEOUT)) + .build(); + } + /** + * Initiate a JDBC Connection + * @param jdbcUrl plain or encrypted URL + * @param userId plain or encrypted user name + * @param cryptedPassword plain or encrypted password + * @param state source or work unit state that can provide the encryption master key location + * @return a JDBC connection + */ + @Override + public Connection getJdbcConnection(String jdbcUrl, String userId, String cryptedPassword, State state) { + try { + return DriverManager.getConnection( + EncryptionUtils.decryptGobblin(jdbcUrl, state), + EncryptionUtils.decryptGobblin(userId, state), + EncryptionUtils.decryptGobblin(cryptedPassword, state)); + } catch (Exception e) { + LOG.error("Error creating JDBC connection", e); + throw new RuntimeException(e); + } + } + + /** + * Initiate a Secure Channel client for SFTP Connection + * @param state the state of execution environment + * @return a SFTP channel client + */ + @Override + public SftpClient getSftpChannelClient(State state) { + return new SftpChannelClient(state); + } + + /** + * Initiate a SchemaReader + * @param state the state of execution environment + * @return a SchemaReader + */ + @Override + public SchemaReader getSchemaReader(State state) { + return new JsonFileReader(); + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/DefaultJdbcClientFactory.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/DefaultJdbcClientFactory.java deleted file mode 100644 index 61321fd..0000000 --- a/cdi-core/src/main/java/com/linkedin/cdi/factory/DefaultJdbcClientFactory.java +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.factory; - -import java.sql.Connection; -import java.sql.DriverManager; -import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.util.EncryptionUtils; - - -/** - * An implementation to create an default JDBC connection - */ -public class DefaultJdbcClientFactory implements JdbcClientFactory { - public Connection getConnection(String jdbcUrl, String userId, String cryptedPassword, State state) { - try { - return DriverManager.getConnection( - EncryptionUtils.decryptGobblin(jdbcUrl, state), - EncryptionUtils.decryptGobblin(userId, state), - EncryptionUtils.decryptGobblin(cryptedPassword, state)); - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/DefaultS3ClientFactory.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/DefaultS3ClientFactory.java deleted file mode 100644 index 93db465..0000000 --- a/cdi-core/src/main/java/com/linkedin/cdi/factory/DefaultS3ClientFactory.java +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.factory; - -import org.apache.gobblin.configuration.State; -import software.amazon.awssdk.http.SdkHttpClient; -import software.amazon.awssdk.http.apache.ApacheHttpClient; -import software.amazon.awssdk.utils.AttributeMap; - -import static software.amazon.awssdk.http.SdkHttpConfigurationOption.*; - - -/** - * An implementation to produce an Apache HttpClient - */ -public class DefaultS3ClientFactory implements S3ClientFactory { - public SdkHttpClient getHttpClient(State state, AttributeMap config) { - return ApacheHttpClient.builder() - .connectionTimeout(config.get(CONNECTION_TIMEOUT)) - .build(); - } -} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/HttpClientFactory.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/HttpClientFactory.java deleted file mode 100644 index 775736c..0000000 --- a/cdi-core/src/main/java/com/linkedin/cdi/factory/HttpClientFactory.java +++ /dev/null @@ -1,16 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.factory; - -import org.apache.gobblin.configuration.State; -import org.apache.http.client.HttpClient; - - -/** - * The interface for dynamic HttpClient creation based on environment - */ -public interface HttpClientFactory { - HttpClient get(State state); -} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/JdbcClientFactory.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/JdbcClientFactory.java deleted file mode 100644 index b308815..0000000 --- a/cdi-core/src/main/java/com/linkedin/cdi/factory/JdbcClientFactory.java +++ /dev/null @@ -1,23 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.factory; - -import java.sql.Connection; -import org.apache.gobblin.configuration.State; - - -/** - * The interface for dynamic JDBC Connection creation based on environment. - */ -public interface JdbcClientFactory { - /** - * @param jdbcUrl plain or encrypted URL - * @param userId plain or encrypted user name - * @param cryptedPassword plain or encrypted password - * @param state source or work unit state that can provide the encryption master key location - * @return a JDBC connection - */ - Connection getConnection(String jdbcUrl, String userId, String cryptedPassword, State state); -} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/S3ClientFactory.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/S3ClientFactory.java deleted file mode 100644 index 3e5de55..0000000 --- a/cdi-core/src/main/java/com/linkedin/cdi/factory/S3ClientFactory.java +++ /dev/null @@ -1,17 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.factory; - -import org.apache.gobblin.configuration.State; -import software.amazon.awssdk.http.SdkHttpClient; -import software.amazon.awssdk.utils.AttributeMap; - - -/** - * The interface for dynamic S3Client creation based on environment - */ -public interface S3ClientFactory { - SdkHttpClient getHttpClient(State state, AttributeMap config); -} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/SchemaReaderFactory.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/SchemaReaderFactory.java deleted file mode 100644 index c1facdb..0000000 --- a/cdi-core/src/main/java/com/linkedin/cdi/factory/SchemaReaderFactory.java +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.factory; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.factory.reader.SchemaReader; - - -/** - * The factory to create SchemaReader - */ -public interface SchemaReaderFactory { - /** - * Creating a schema reader, default reads from TMS - * @param state Gobblin configuration - * @return the reader factory - */ - @VisibleForTesting - static SchemaReader create(State state) { - try { - Class readerClass = Class.forName( - MultistageProperties.MSTAGE_SOURCE_SCHEMA_READER_FACTORY.getValidNonblankWithDefault(state)); - return (SchemaReader) readerClass.newInstance(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/SecureConnectionClientFactory.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/SecureConnectionClientFactory.java new file mode 100644 index 0000000..73f20a6 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/SecureConnectionClientFactory.java @@ -0,0 +1,94 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory; + +import com.linkedin.cdi.factory.network.KeyCertRetriever; +import com.linkedin.cdi.factory.network.SecureNetworkUtil; +import com.linkedin.cdi.factory.reader.JsonFileReader; +import com.linkedin.cdi.factory.reader.SchemaReader; +import com.linkedin.cdi.factory.sftp.SftpChannelClient; +import com.linkedin.cdi.factory.sftp.SftpClient; +import java.sql.Connection; +import org.apache.gobblin.configuration.State; +import org.apache.http.client.HttpClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.utils.AttributeMap; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; + + +/** + * This is the secure implementation that supports secure network (SSL, TLS, HTTPS etc). + * + * This is built for integrations with systems inside LinkedIn that doesn't require GaaP proxying + * + */ +public class SecureConnectionClientFactory extends DefaultConnectionClientFactory { + private static final Logger LOG = LoggerFactory.getLogger(SecureConnectionClientFactory.class); + + /** + * Initiate an HTTP client + * @param state the State of execution environment + * @return an HTTP client object + */ + @Override + public HttpClient getHttpClient(State state) { + return SecureNetworkUtil.createSecureHttpClientBuilder( + new KeyCertRetriever(state), + MSTAGE_HTTP_CONN_TTL_SECONDS.get(state), + MSTAGE_HTTP_CONN_PER_ROUTE_MAX.get(state), + MSTAGE_HTTP_CONN_MAX.get(state), + MSTAGE_SSL.getConnectionTimeoutMillis(state), + MSTAGE_SSL.getSocketTimeoutMillis(state), + MSTAGE_SSL.getVersion(state), + null, -1).build(); + } + + /** + * Initiate an S3 HTTP client + * @param state the state of execution environment + * @param config S3 parameters + * @return an S3 HTTP client object + */ + @Override + public SdkHttpClient getS3Client(State state, AttributeMap config) { + return super.getS3Client(state, config); + } + /** + * Initiate a JDBC Connection + * @param jdbcUrl plain or encrypted URL + * @param userId plain or encrypted user name + * @param cryptedPassword plain or encrypted password + * @param state source or work unit state that can provide the encryption master key location + * @return a JDBC connection + */ + @Override + public Connection getJdbcConnection(String jdbcUrl, String userId, String cryptedPassword, State state) { + return super.getJdbcConnection(jdbcUrl, userId, cryptedPassword, state); + } + + /** + * Initiate a Secure Channel client for SFTP Connection + * @param state the state of execution environment + * @return a SFTP channel client + */ + @Override + public SftpClient getSftpChannelClient(State state) { + //return new SftpChannelClient(state, createSslContext(state)); + return new SftpChannelClient(state); + } + + /** + * Initiate a SchemaReader + * @param state the state of execution environment + * @return a SchemaReader + */ + @Override + public SchemaReader getSchemaReader(State state) { + return new JsonFileReader(); + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/HttpRequestMethod.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/http/HttpRequestMethod.java similarity index 82% rename from cdi-core/src/main/java/com/linkedin/cdi/util/HttpRequestMethod.java rename to cdi-core/src/main/java/com/linkedin/cdi/factory/http/HttpRequestMethod.java index 75ac7b1..27bd286 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/HttpRequestMethod.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/http/HttpRequestMethod.java @@ -2,23 +2,33 @@ // Licensed under the BSD-2 Clause license. // See LICENSE in the project root for license information. -package com.linkedin.cdi.util; +package com.linkedin.cdi.factory.http; import com.google.gson.JsonElement; import com.google.gson.JsonObject; +import com.linkedin.cdi.util.JsonUtils; +import com.linkedin.cdi.util.VariableUtils; import java.io.UnsupportedEncodingException; +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; -import lombok.extern.slf4j.Slf4j; -import okhttp3.HttpUrl; import org.apache.commons.lang3.tuple.Pair; +import org.apache.http.NameValuePair; +import org.apache.http.client.entity.UrlEncodedFormEntity; import org.apache.http.client.methods.HttpDelete; import org.apache.http.client.methods.HttpEntityEnclosingRequestBase; import org.apache.http.client.methods.HttpGet; import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpPut; import org.apache.http.client.methods.HttpUriRequest; +import org.apache.http.client.utils.URIBuilder; import org.apache.http.entity.StringEntity; +import org.apache.http.message.BasicNameValuePair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -44,7 +54,6 @@ * @author chrli */ -@Slf4j public enum HttpRequestMethod { GET("GET") { @Override @@ -79,8 +88,7 @@ protected HttpUriRequest getHttpRequestContentJson(String uriTemplate, protected HttpUriRequest getHttpRequestContentUrlEncoded(String uriTemplate, JsonObject parameters) throws UnsupportedEncodingException { Pair replaced = replaceVariables(uriTemplate, parameters); - String urlEncoded = jsonToUrlEncoded(replaced.getValue()); - return setEntity(new HttpPost(replaced.getKey()), urlEncoded); + return setEntity(new HttpPost(replaced.getKey()), jsonToUrlEncodedEntity(replaced.getValue())); } }, @@ -100,7 +108,7 @@ protected HttpUriRequest getHttpRequestContentJson(String uriTemplate, protected HttpUriRequest getHttpRequestContentUrlEncoded(String uriTemplate, JsonObject parameters) throws UnsupportedEncodingException { Pair replaced = replaceVariables(uriTemplate, parameters); - return setEntity(new HttpPut(replaced.getKey()), jsonToUrlEncoded(replaced.getValue())); + return setEntity(new HttpPut(replaced.getKey()), jsonToUrlEncodedEntity(replaced.getValue())); } }, @@ -124,6 +132,7 @@ protected HttpUriRequest getHttpRequestContentUrlEncoded(String uriTemplate, Jso } }; + private static final Logger LOG = LoggerFactory.getLogger(HttpRequestMethod.class); private final String name; HttpRequestMethod(String name) { @@ -170,13 +179,13 @@ public HttpUriRequest getHttpRequest(final String uriTemplate, if (!replaced.getLeft().equals(entry.getValue())) { parametersCopy = JsonUtils.deepCopy(replaced.getRight()).getAsJsonObject(); headersCopy.put(entry.getKey(), replaced.getLeft()); - log.info("Substituted header string: {} = {}", entry.getKey(), replaced.getLeft()); + LOG.info("Substituted header string: {} = {}", entry.getKey(), replaced.getLeft()); } else { headersCopy.put(entry.getKey(), entry.getValue()); } } - log.info("Final parameters for HttpRequest: {}", parametersCopy.toString()); + LOG.info("Final parameters for HttpRequest: {}", parametersCopy.toString()); if (headersCopy.containsKey("Content-Type") && headersCopy.get("Content-Type").equals("application/x-www-form-urlencoded")) { request = getHttpRequestContentUrlEncoded(uriTemplate, parametersCopy); @@ -216,20 +225,21 @@ protected Pair replaceVariables(String uriTemplate, JsonObje } protected String appendParameters(String uri, JsonObject parameters) { - HttpUrl url = HttpUrl.parse(uri); - if (url != null) { - HttpUrl.Builder builder = url.newBuilder(); + try { + URIBuilder builder = new URIBuilder(new URI(uri)); for (Map.Entry entry : parameters.entrySet()) { - String key = entry.getKey(); - builder.addQueryParameter(key, parameters.get(key).getAsString()); + if (!entry.getKey().matches("tmp.*")) { + builder.addParameter(entry.getKey(), entry.getValue().getAsString()); + } } - url = builder.build(); + return builder.build().toString(); + } catch (Exception e) { + throw new RuntimeException(e); } - return url != null ? url.toString() : uri; } /** - * Convert Json formatted parameter set to Url Encoded format as requested by + * Convert Json formatted parameter set to Url Encoded Entity as requested by * Content-Type: application/x-www-form-urlencoded * Json Example: * {"param1": "value1", "param2": "value2"} @@ -238,20 +248,27 @@ protected String appendParameters(String uri, JsonObject parameters) { * param1=value1¶m2=value2 * * @param parameters Json structured parameters - * @return URL encoded parameters + * @return URL encoded entity */ - protected String jsonToUrlEncoded(JsonObject parameters) { - HttpUrl.Builder builder = new HttpUrl.Builder().scheme("https").host("www.dummy.com"); - for (Map.Entry entry : parameters.entrySet()) { - String key = entry.getKey(); - builder.addQueryParameter(key, parameters.get(key).getAsString()); + protected UrlEncodedFormEntity jsonToUrlEncodedEntity(JsonObject parameters) { + try { + List nameValuePairs = new ArrayList<>(); + for (Map.Entry entry : parameters.entrySet()) { + nameValuePairs.add(new BasicNameValuePair(entry.getKey(),entry.getValue().getAsString())); + } + return new UrlEncodedFormEntity(nameValuePairs, StandardCharsets.UTF_8); + } catch (Exception e) { + throw new RuntimeException(e); } - return builder.build().encodedQuery(); } protected HttpUriRequest setEntity(HttpEntityEnclosingRequestBase requestBase, String stringEntity) throws UnsupportedEncodingException { - requestBase.setEntity(new StringEntity(stringEntity)); + return setEntity(requestBase, new StringEntity(stringEntity)); + } + + protected HttpUriRequest setEntity(HttpEntityEnclosingRequestBase requestBase, StringEntity stringEntity) { + requestBase.setEntity(stringEntity); return requestBase; } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/network/KeyCertRetriever.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/network/KeyCertRetriever.java new file mode 100644 index 0000000..cb6ba11 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/network/KeyCertRetriever.java @@ -0,0 +1,155 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.network; + +import java.io.FileInputStream; +import java.io.IOException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableEntryException; +import java.security.cert.CertificateException; +import java.util.Base64; +import org.apache.gobblin.configuration.State; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; + + +/** + * This is replicated from li-gobblin for cleaner dependency + * + * A {@link KeyCertRetriever} provides private key and cert that can be found from job configurations + * + *

It supports following scenarios: + *

  • Find key and cert loaded by Azkaban. The key and cert are extracted from the key store file + */ +public class KeyCertRetriever { + private static final Logger LOG = LoggerFactory.getLogger(KeyCertRetriever.class); + private String keyAlias; + private String privateKey; + private String cert; + private String keyStoreFilePath; + private String trustStoreFilePath; + private String trustStorePassword; + private String keyStorePassword; + private String keyStoreType; + private String keyPassword; + + public KeyCertRetriever(State state) { + try { + keyStoreFilePath = MSTAGE_SSL.getKeyStorePath(state); + keyStorePassword = MSTAGE_SSL.getKeyStorePassword(state); + keyStoreType = MSTAGE_SSL.getKeyStoreType(state); + keyPassword = MSTAGE_SSL.getKeyPassword(state); + trustStoreFilePath = MSTAGE_SSL.getTrustStorePath(state); + trustStorePassword = MSTAGE_SSL.getTrustStorePassword(state); + + KeyStore keyStore = loadKeyStore(keyStoreFilePath, keyStoreType, keyStorePassword.toCharArray()); + keyAlias = keyStore.aliases().nextElement(); + LOG.info("Loading key and cert of alias {}", keyAlias); + + KeyStore.ProtectionParameter protectionParameter = new KeyStore.PasswordProtection(keyStorePassword.toCharArray()); + KeyStore.PrivateKeyEntry entry = + (KeyStore.PrivateKeyEntry) keyStore.getEntry(keyAlias, protectionParameter); + privateKey = Base64.getEncoder().encodeToString(entry.getPrivateKey().getEncoded()); + cert = Base64.getEncoder().encodeToString(entry.getCertificate().getEncoded()); + } catch (KeyStoreException | NoSuchAlgorithmException | CertificateException | UnrecoverableEntryException e) { + throw new RuntimeException(e); + } + } + + private static KeyStore loadKeyStore(String keyStoreFile, String keyStoreType, char[] keyStorePassword) { + try (FileInputStream inputStream = new FileInputStream(keyStoreFile)) { + KeyStore keyStore = KeyStore.getInstance(keyStoreType); + keyStore.load(inputStream, keyStorePassword); + return keyStore; + } catch (KeyStoreException | NoSuchAlgorithmException | CertificateException | IOException e) { + throw new RuntimeException(e); + } + } + + public String getKeyAlias() { + return keyAlias; + } + + public KeyCertRetriever setKeyAlias(String keyAlias) { + this.keyAlias = keyAlias; + return this; + } + + public String getPrivateKey() { + return privateKey; + } + + public KeyCertRetriever setPrivateKey(String privateKey) { + this.privateKey = privateKey; + return this; + } + + public String getCert() { + return cert; + } + + public KeyCertRetriever setCert(String cert) { + this.cert = cert; + return this; + } + + public String getKeyStoreFilePath() { + return keyStoreFilePath; + } + + public KeyCertRetriever setKeyStoreFilePath(String keyStoreFilePath) { + this.keyStoreFilePath = keyStoreFilePath; + return this; + } + + public String getTrustStoreFilePath() { + return trustStoreFilePath; + } + + public KeyCertRetriever setTrustStoreFilePath(String trustStoreFilePath) { + this.trustStoreFilePath = trustStoreFilePath; + return this; + } + + public String getTrustStorePassword() { + return trustStorePassword; + } + + public KeyCertRetriever setTrustStorePassword(String trustStorePassword) { + this.trustStorePassword = trustStorePassword; + return this; + } + + public String getKeyStorePassword() { + return keyStorePassword; + } + + public KeyCertRetriever setKeyStorePassword(String keyStorePassword) { + this.keyStorePassword = keyStorePassword; + return this; + } + + public String getKeyStoreType() { + return keyStoreType; + } + + public KeyCertRetriever setKeyStoreType(String keyStoreType) { + this.keyStoreType = keyStoreType; + return this; + } + + public String getKeyPassword() { + return keyPassword; + } + + public KeyCertRetriever setKeyPassword(String keyPassword) { + this.keyPassword = keyPassword; + return this; + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/network/SecureNetworkUtil.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/network/SecureNetworkUtil.java new file mode 100644 index 0000000..a135773 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/network/SecureNetworkUtil.java @@ -0,0 +1,143 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.network; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.security.KeyStore; +import java.util.concurrent.TimeUnit; +import javax.net.ssl.SSLContext; +import org.apache.commons.lang.StringUtils; +import org.apache.http.HttpHost; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.config.Registry; +import org.apache.http.config.RegistryBuilder; +import org.apache.http.conn.socket.ConnectionSocketFactory; +import org.apache.http.conn.socket.PlainConnectionSocketFactory; +import org.apache.http.conn.ssl.NoopHostnameVerifier; +import org.apache.http.conn.ssl.SSLConnectionSocketFactory; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; +import org.apache.http.ssl.SSLContexts; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public abstract class SecureNetworkUtil { + private static final Logger LOG = LoggerFactory.getLogger(SecureNetworkUtil.class); + + private SecureNetworkUtil() { + } + + static public SSLContext createSSLContext( + KeyCertRetriever keyCertRetriever, + String sslVersion + ) throws IOException { + InputStream keystoreInput = null; + try { + keystoreInput = new FileInputStream(keyCertRetriever.getKeyStoreFilePath()); + final KeyStore keystore = KeyStore.getInstance(keyCertRetriever.getKeyStoreType()); + keystore.load(keystoreInput, keyCertRetriever.getKeyStorePassword().toCharArray()); + return SSLContexts.custom() + .setProtocol(sslVersion) + .loadKeyMaterial(keystore, keyCertRetriever.getKeyPassword().toCharArray()) + .loadTrustMaterial(new File(keyCertRetriever.getTrustStoreFilePath()), keyCertRetriever.getTrustStorePassword().toCharArray()) + .build(); + } catch (Exception e) { + LOG.error("Fatal error: couldn't create SSLContext.", e); + if (keystoreInput != null) { + keystoreInput.close(); + } + throw new RuntimeException(e); + } finally { + if (keystoreInput != null) { + keystoreInput.close(); + } + } + } + + static private HttpClientBuilder createHttpClientBuilder( + SSLContext sslContext, + int secondConnTTL, + int connectionRouteMax, + int connectionTotalMax, + int connectionTimeoutMillis, + int socketTimeoutMillis) { + return createHttpClientBuilder(sslContext, + secondConnTTL, + connectionRouteMax, + connectionTotalMax, + connectionTimeoutMillis, + socketTimeoutMillis, + null, -1); + } + + static private HttpClientBuilder createHttpClientBuilder( + SSLContext sslContext, + int secondConnTTL, + int connectionRouteMax, + int connectionTotalMax, + int connectionTimeoutMillis, + int socketTimeoutMillis, + String proxyUrl, + int proxyPort) { + Registry registry = RegistryBuilder.create() + .register("https", new SSLConnectionSocketFactory(sslContext, new NoopHostnameVerifier())) + .register("http", new PlainConnectionSocketFactory()).build(); + HttpClientBuilder builder = HttpClientBuilder.create(); + builder.setSSLSocketFactory(new SSLConnectionSocketFactory(sslContext, new NoopHostnameVerifier())); + PoolingHttpClientConnectionManager connectionManager = new PoolingHttpClientConnectionManager(registry); + connectionManager.setDefaultMaxPerRoute(connectionRouteMax); + connectionManager.setMaxTotal(connectionTotalMax); + builder.setConnectionManager(connectionManager); + builder.setConnectionTimeToLive(secondConnTTL, TimeUnit.SECONDS); + + if (proxyPort > 0 && !StringUtils.isEmpty(proxyUrl)) { + builder.setProxy(new HttpHost(proxyUrl, proxyPort)); + } + + // add default connection timeout and socket timeout + RequestConfig requestConfig = RequestConfig.custom() + .setConnectTimeout(connectionTimeoutMillis) + .setSocketTimeout(socketTimeoutMillis) + .build(); + builder.setDefaultRequestConfig(requestConfig); + return builder; + } + + static public HttpClientBuilder createSecureHttpClientBuilder( + KeyCertRetriever keyCertRetriever, + int secondConnTTL, + int connectionRouteMax, + int connectionTotalMax, + int connectionTimeoutMillis, + int socketTimeoutMillis, + String sslVersion, + String proxyUrl, + int proxyPort + ) { + try { + SSLContext sslContext = createSSLContext(keyCertRetriever, sslVersion); + return StringUtils.isEmpty(proxyUrl) && proxyPort <= 0 + ? createHttpClientBuilder(sslContext, + secondConnTTL, + connectionRouteMax, + connectionTotalMax, + connectionTimeoutMillis, + socketTimeoutMillis) + : createHttpClientBuilder(sslContext, + secondConnTTL, + connectionRouteMax, + connectionTotalMax, + connectionTimeoutMillis, + socketTimeoutMillis, + proxyUrl, proxyPort); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/DistributedCacheIdentityStrategy.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/DistributedCacheIdentityStrategy.java new file mode 100644 index 0000000..1ddf0b6 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/DistributedCacheIdentityStrategy.java @@ -0,0 +1,19 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.sftp; + +import com.jcraft.jsch.JSch; +import java.io.File; + + +/** + * Sets identity using a file on distributed cache + */ +public class DistributedCacheIdentityStrategy extends LocalFileIdentityStrategy { + @Override + public boolean setIdentity(String privateKey, JSch jsch) { + return super.setIdentity(new File(privateKey).getName(), jsch); + } +} \ No newline at end of file diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/HdfsIdentityStrategy.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/HdfsIdentityStrategy.java new file mode 100644 index 0000000..0abd7ac --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/HdfsIdentityStrategy.java @@ -0,0 +1,46 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.sftp; + +import com.google.common.base.Preconditions; +import com.jcraft.jsch.JSch; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Sets identity using a file on HDFS + */ +public class HdfsIdentityStrategy implements IdentityStrategy { + private static final Logger LOG = LoggerFactory.getLogger(HdfsIdentityStrategy.class); + + @Override + public boolean setIdentity(String privateKey, JSch jsch) { + + FileSystem fs; + try { + fs = FileSystem.get(new Configuration()); + } catch (Exception e) { + LOG.warn("Failed to set identity using HDFS file. Will attempt next strategy. " + e.getMessage()); + return false; + } + + Preconditions.checkNotNull(fs, "FileSystem cannot be null"); + try (FSDataInputStream privateKeyStream = fs.open(new Path(privateKey))) { + byte[] bytes = IOUtils.toByteArray(privateKeyStream); + jsch.addIdentity("sftpIdentityKey", bytes, (byte[]) null, (byte[]) null); + LOG.info("Successfully set identity using HDFS file"); + return true; + } catch (Exception e) { + LOG.warn("Failed to set identity using HDFS file. Will attempt next strategy. " + e.getMessage()); + return false; + } + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/IdentityStrategy.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/IdentityStrategy.java new file mode 100644 index 0000000..2a5ebb4 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/IdentityStrategy.java @@ -0,0 +1,15 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.sftp; + +import com.jcraft.jsch.JSch; + + +/** + * Interface for multiple identity setter strategies + */ +interface IdentityStrategy { + public boolean setIdentity(String privateKey, JSch jsch); +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/LocalFileIdentityStrategy.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/LocalFileIdentityStrategy.java new file mode 100644 index 0000000..3f674fb --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/LocalFileIdentityStrategy.java @@ -0,0 +1,28 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.sftp; + +import com.jcraft.jsch.JSch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Sets identity using a local file + */ +public class LocalFileIdentityStrategy implements IdentityStrategy { + private static final Logger LOG = LoggerFactory.getLogger(LocalFileIdentityStrategy.class); + @Override + public boolean setIdentity(String privateKey, JSch jsch) { + try { + jsch.addIdentity(privateKey); + LOG.info("Successfully set identity using local file " + privateKey); + return true; + } catch (Exception e) { + LOG.warn("Failed to set identity using local file. Will attempt next strategy. " + e.getMessage()); + } + return false; + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/MyUserInfo.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/MyUserInfo.java new file mode 100644 index 0000000..b5435f8 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/MyUserInfo.java @@ -0,0 +1,50 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.sftp; + +import com.jcraft.jsch.UserInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Implementation of UserInfo class for JSch which allows for password-less login via keys + * @author stakiar + */ +public class MyUserInfo implements UserInfo { + private static final Logger LOG = LoggerFactory.getLogger(MyUserInfo.class); + + // The passphrase used to access the private key + @Override + public String getPassphrase() { + return null; + } + + // The password to login to the client server + @Override + public String getPassword() { + return null; + } + + @Override + public boolean promptPassword(String message) { + return true; + } + + @Override + public boolean promptPassphrase(String message) { + return true; + } + + @Override + public boolean promptYesNo(String message) { + return true; + } + + @Override + public void showMessage(String message) { + LOG.info(message); + } +} \ No newline at end of file diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpChannelClient.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpChannelClient.java new file mode 100644 index 0000000..a3e9398 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpChannelClient.java @@ -0,0 +1,201 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.sftp; + +import com.google.common.collect.ImmutableList; +import com.jcraft.jsch.ChannelSftp; +import com.jcraft.jsch.JSch; +import com.jcraft.jsch.JSchException; +import com.jcraft.jsch.Session; +import com.jcraft.jsch.SftpException; +import com.linkedin.cdi.util.Credentials; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.Vector; +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.State; +import org.apache.gobblin.source.extractor.extract.sftp.SftpFsHelper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; + + +public class SftpChannelClient implements SftpClient { + private static final Logger LOG = LoggerFactory.getLogger(SftpChannelClient.class); + + private static final List STRATEGIES = ImmutableList.of( + new LocalFileIdentityStrategy(), + new DistributedCacheIdentityStrategy(), + new HdfsIdentityStrategy()); + + protected State state; + protected Session session = null; + protected JSch jsch = new JSch(); +// protected SSLContext sslContext = null; + + public SftpChannelClient(State state) { + this.state = state; + initializeConnection(state); + } +// +// public SftpChannelClient(State state, SSLContext sslContext) { +// this(state); +// this.sslContext = sslContext; +// initializeConnection(state); +// } + + protected void initializeConnection(State state) { + JSch.setLogger(new SftpFsHelper.JSchLogger()); + try { + setIdentityStrategy(this.jsch); + if (StringUtils.isNotEmpty(Credentials.getKnownHosts(state))) { + jsch.setKnownHosts(Credentials.getKnownHosts(state)); + } + session = jsch.getSession(Credentials.getUserName(state), + Credentials.getHostName(state), + Credentials.getPort(state)); + configSessionProperties(); + if (!session.isConnected()) { + this.session.connect(); + } + LOG.info("Finished connecting to source"); + } catch (JSchException e) { + if (session != null) { + session.disconnect(); + } + LOG.error("Cannot connect to SFTP source", e); + } + } + + /** + * Create new channel every time a command needs to be executed. This is required to support execution of multiple + * commands in parallel. All created channels are cleaned up when the session is closed. + * + * @return a new {@link ChannelSftp} + * @throws SftpException An SftpException + */ + @Override + public ChannelSftp getSftpChannel() throws SftpException { + try { + ChannelSftp channelSftp = (ChannelSftp) this.session.openChannel("sftp"); + channelSftp.connect(MSTAGE_SFTP_CONN_TIMEOUT_MILLIS.get(state)); + return channelSftp; + } catch (JSchException e) { + throw new SftpException(0, "Cannot open a channel to SFTP server", e); + } + } + + /** + * Close the session and therefore its channels + */ + @Override + public void close() { + if (this.session != null) { + this.session.disconnect(); + } + } + + /** + * Executes a get SftpCommand and returns an input stream to the file + */ + @Override + public InputStream getFileStream(String file) { + SftpMonitor monitor = new SftpMonitor(); + try { + ChannelSftp channel = getSftpChannel(); + return new SftpChannelFileInputStream(channel.get(file, monitor), channel); + } catch (SftpException e) { + throw new RuntimeException("Cannot download file " + file + " due to " + e.getMessage(), e); + } + } + + /** + * Execute an FTP ls command + * @param path the target path to list content + * @return the list of files and directories + */ + @Override + public List ls(String path) { + try { + List list = new ArrayList<>(); + ChannelSftp channel = getSftpChannel(); + Vector vector = channel.ls(path); + for (ChannelSftp.LsEntry entry : vector) { + list.add(entry.getFilename()); + } + channel.disconnect(); + return list; + } catch (SftpException e) { + throw new RuntimeException("Cannot execute ls command on sftp connection", e); + } + } + + /** + * Get file modification time + * @param path file path on target to be checked + * @return the modification time in long format + */ + @Override + public long getFileMTime(String path) { + ChannelSftp channelSftp = null; + try { + channelSftp = getSftpChannel(); + return channelSftp.lstat(path).getMTime(); + } catch (SftpException e) { + throw new RuntimeException( + String.format("Failed to get modified timestamp for file at path %s due to error %s", path, + e.getMessage()), e); + } finally { + if (channelSftp != null) { + channelSftp.disconnect(); + } + } + } + + /** + * Get file size + * @param path file path on target to be checked + * @return the file size + */ + @Override + public long getFileSize(String path) { + try { + ChannelSftp channelSftp = getSftpChannel(); + long fileSize = channelSftp.lstat(path).getSize(); + channelSftp.disconnect(); + return fileSize; + } catch (SftpException e) { + throw new RuntimeException( + String.format("Failed to get size for file at path %s due to error %s", path, e.getMessage()), e); + } + } + + protected void setIdentityStrategy(JSch jsch) { + String privateKey = Credentials.getPrivateKey(state); + if (StringUtils.isNotEmpty(privateKey)) { + for (IdentityStrategy strategy : STRATEGIES) { + if (strategy.setIdentity(privateKey, jsch)) { + break; + } + } + } + } + + protected void configSessionProperties() throws JSchException { + session.setUserInfo(new MyUserInfo()); + session.setDaemonThread(true); + session.setTimeout(MSTAGE_SFTP_CONN_TIMEOUT_MILLIS.get(state)); + session.setConfig("PreferredAuthentications", "publickey,password"); + if (StringUtils.isEmpty(SOURCE_CONN_KNOWN_HOSTS.get(state))) { + LOG.info("Known hosts path is not set, StrictHostKeyChecking will be turned off"); + session.setConfig("StrictHostKeyChecking", "no"); + } + if (!StringUtils.isEmpty(Credentials.getPassword(state))) { + session.setPassword(Credentials.getPassword(state)); + } + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpChannelFileInputStream.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpChannelFileInputStream.java new file mode 100644 index 0000000..66f1273 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpChannelFileInputStream.java @@ -0,0 +1,29 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.sftp; + +import com.jcraft.jsch.Channel; +import java.io.IOException; +import java.io.InputStream; +import org.apache.gobblin.util.io.SeekableFSInputStream; + +/** + * A {@link SeekableFSInputStream} that holds a handle on the Sftp {@link Channel} used to open the + * {@link InputStream}. The {@link Channel} is disconnected when {@link InputStream#close()} is called. + */ +public class SftpChannelFileInputStream extends SeekableFSInputStream { + private final Channel channel; + + public SftpChannelFileInputStream(InputStream in, Channel channel) { + super(in); + this.channel = channel; + } + + @Override + public void close() throws IOException { + super.close(); + this.channel.disconnect(); + } +} \ No newline at end of file diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpClient.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpClient.java new file mode 100644 index 0000000..09df9c8 --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpClient.java @@ -0,0 +1,54 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.sftp; + +import com.jcraft.jsch.ChannelSftp; +import com.jcraft.jsch.SftpException; +import java.io.InputStream; +import java.util.List; + + +/** + * The base class for dynamic schema reader based on environment. + */ +public interface SftpClient { + /** + * Establish a secure channel + * @return a new secure channel + * @throws SftpException An SftpException + */ + ChannelSftp getSftpChannel() throws SftpException; + + /** + * Close the session and therefore its channels + */ + void close(); + + /** + * Executes a get SftpCommand and returns an input stream to the file + */ + InputStream getFileStream(String file); + + /** + * Execute an FTP ls command + * @param path path on target host to be listed + * @return the list of files and directories + */ + List ls(String path); + + /** + * Get file modification time + * @param path file path on target to be checked + * @return the modification time in long format + */ + long getFileMTime(String path); + + /** + * Get file size + * @param path file path on target to be checked + * @return the file size + */ + long getFileSize(String path); +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpMonitor.java b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpMonitor.java new file mode 100644 index 0000000..a49ea1c --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/factory/sftp/SftpMonitor.java @@ -0,0 +1,121 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory.sftp; + +import com.jcraft.jsch.SftpProgressMonitor; +import com.linkedin.cdi.factory.DefaultConnectionClientFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Implementation of an SftpProgressMonitor to monitor the progress of file transfers using the ChannelSftp.GET + * and ChannelSftp.PUT methods + */ +public class SftpMonitor implements SftpProgressMonitor { + private static final Logger LOG = LoggerFactory.getLogger(DefaultConnectionClientFactory.class); + + private int op; + private String src; + private String dest; + private long totalCount; + private long logFrequency; + private long startTime; + + public int getOp() { + return op; + } + + public void setOp(int op) { + this.op = op; + } + + public String getSrc() { + return src; + } + + public void setSrc(String src) { + this.src = src; + } + + public String getDest() { + return dest; + } + + public void setDest(String dest) { + this.dest = dest; + } + + public long getTotalCount() { + return totalCount; + } + + public void setTotalCount(long totalCount) { + this.totalCount = totalCount; + } + + public long getLogFrequency() { + return logFrequency; + } + + public void setLogFrequency(long logFrequency) { + this.logFrequency = logFrequency; + } + + public long getStartTime() { + return startTime; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + @Override + public void init(int op, String src, String dest, long max) { + this.op = op; + this.src = src; + this.dest = dest; + this.startTime = System.currentTimeMillis(); + this.logFrequency = 0L; + if (op == SftpProgressMonitor.GET) { + LOG.info("DOWNLOAD operation has started with src: " + src + " dest: " + dest + " and file length: " + (max + / 1000000L) + " mb"); + } else { + LOG.info("UPLOAD operation has started with src: " + src + " dest: " + dest); + } + } + + @Override + public boolean count(long count) { + this.totalCount += count; + + if (this.logFrequency == 0L) { + this.logFrequency = 1000L; + if (op == SftpProgressMonitor.GET) { + LOG.info("Transfer is in progress for file: " + this.src + ". Finished transferring " + this.totalCount + " bytes "); + } else { + LOG.info("Upload in progress for file " + this.dest + ". Finished uploading" + this.totalCount + " bytes"); + } + long mb = this.totalCount / 1000000L; + LOG.info("Transferred " + mb + " Mb. Speed " + getMbps() + " Mbps"); + } + this.logFrequency--; + return true; + } + + @Override + public void end() { + long secs = (System.currentTimeMillis() - this.startTime) / 1000L; + LOG.info("Transfer finished " + this.op + " src: " + this.src + " dest: " + this.dest + " in " + secs + " at " + + getMbps()); + } + + private String getMbps() { + long mb = this.totalCount / 1000000L; + long secs = (System.currentTimeMillis() - this.startTime) / 1000L; + double mbps = secs == 0L ? 0.0D : mb * 1.0D / secs; + return String.format("%.2f", mbps); + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/filter/AvroSchemaBasedFilter.java b/cdi-core/src/main/java/com/linkedin/cdi/filter/AvroSchemaBasedFilter.java index a332d0d..e4d7a9e 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/filter/AvroSchemaBasedFilter.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/filter/AvroSchemaBasedFilter.java @@ -5,20 +5,19 @@ package com.linkedin.cdi.filter; import com.google.common.base.Optional; -import lombok.SneakyThrows; -import lombok.extern.slf4j.Slf4j; +import com.linkedin.cdi.keys.AvroExtractorKeys; +import com.linkedin.cdi.util.AvroSchemaUtils; +import com.linkedin.cdi.util.JsonIntermediateSchema; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.keys.AvroExtractorKeys; -import com.linkedin.cdi.util.AvroSchemaUtils; -import com.linkedin.cdi.util.JsonIntermediateSchema; import org.apache.gobblin.util.AvroUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; - -@Slf4j public class AvroSchemaBasedFilter extends MultistageSchemaBasedFilter { + private static final Logger LOG = LoggerFactory.getLogger(AvroSchemaBasedFilter.class); private AvroExtractorKeys avroExtractorKeys; private WorkUnitState state; @@ -29,14 +28,10 @@ public AvroSchemaBasedFilter(JsonIntermediateSchema schema, AvroExtractorKeys av this.state = state; } - @SneakyThrows @Override public GenericRecord filter(GenericRecord input) { Schema outputSchema = AvroSchemaUtils.fromJsonSchema(schema.toJson(), state); GenericRecord filteredRow = new GenericData.Record(outputSchema); - if (avroExtractorKeys.getIsValidOutputSchema()) { - log.warn("Some columns from the schema are not present at source, padding with null value."); - } for (String fieldName : AvroSchemaUtils.getSchemaFieldNames(outputSchema)) { Optional fieldValue = AvroUtils.getFieldValue(input, fieldName); filteredRow.put(fieldName, fieldValue.isPresent() ? fieldValue.get() : null); diff --git a/cdi-core/src/main/java/com/linkedin/cdi/filter/CsvSchemaBasedFilter.java b/cdi-core/src/main/java/com/linkedin/cdi/filter/CsvSchemaBasedFilter.java index a52d6f2..2106110 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/filter/CsvSchemaBasedFilter.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/filter/CsvSchemaBasedFilter.java @@ -4,11 +4,12 @@ package com.linkedin.cdi.filter; -import java.util.Arrays; -import java.util.Set; -import lombok.extern.slf4j.Slf4j; import com.linkedin.cdi.keys.CsvExtractorKeys; import com.linkedin.cdi.util.JsonIntermediateSchema; +import java.util.Arrays; +import java.util.Set; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -17,8 +18,8 @@ * @author esong * */ -@Slf4j public class CsvSchemaBasedFilter extends MultistageSchemaBasedFilter { + private static final Logger LOG = LoggerFactory.getLogger(CsvSchemaBasedFilter.class); private CsvExtractorKeys csvExtractorKeys; public CsvSchemaBasedFilter(JsonIntermediateSchema schema, CsvExtractorKeys csvExtractorKeys) { @@ -43,7 +44,7 @@ public String[] filter(String[] input) { csvExtractorKeys.setColumnProjection(columnProjection); return filter(input, columnProjection); } else { - log.debug("Defaulting to project first N columns"); + LOG.debug("Defaulting to project first N columns"); // take first N column, where N is the number of columns in the schema // if the schema's size larger than input, then the extra columns will be padded with null return Arrays.copyOf(input, schema.getColumns().size()); diff --git a/cdi-core/src/main/java/com/linkedin/cdi/filter/SchemaBasedFilter.java b/cdi-core/src/main/java/com/linkedin/cdi/filter/SchemaBasedFilter.java index bc37a5f..a1c146c 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/filter/SchemaBasedFilter.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/filter/SchemaBasedFilter.java @@ -4,7 +4,6 @@ package com.linkedin.cdi.filter; - /** * Base filter interface * diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/AvroExtractorKeys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/AvroExtractorKeys.java index 0958832..5175633 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/keys/AvroExtractorKeys.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/AvroExtractorKeys.java @@ -4,18 +4,14 @@ package com.linkedin.cdi.keys; -import com.google.common.collect.Lists; -import java.util.List; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; import org.apache.avro.file.DataFileStream; import org.apache.avro.generic.GenericRecord; -import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; import org.apache.gobblin.source.workunit.WorkUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; /** @@ -23,45 +19,75 @@ * * @author esong */ -@Slf4j -@Getter(AccessLevel.PUBLIC) -@Setter public class AvroExtractorKeys extends ExtractorKeys { - final private static List ESSENTIAL_PARAMETERS = Lists.newArrayList( - MultistageProperties.MSTAGE_DATA_FIELD, - MultistageProperties.MSTAGE_TOTAL_COUNT_FIELD); + public DataFileStream getAvroRecordIterator() { + return avroRecordIterator; + } + + public void setAvroRecordIterator(DataFileStream avroRecordIterator) { + this.avroRecordIterator = avroRecordIterator; + } + + public long getTotalCount() { + return totalCount; + } + + public void setTotalCount(long totalCount) { + this.totalCount = totalCount; + } + + public long getCurrentPageNumber() { + return currentPageNumber; + } + + public void setCurrentPageNumber(long currentPageNumber) { + this.currentPageNumber = currentPageNumber; + } + + public Schema getAvroOutputSchema() { + return avroOutputSchema; + } + public void setAvroOutputSchema(Schema avroOutputSchema) { + this.avroOutputSchema = avroOutputSchema; + } + + public Boolean getIsValidOutputSchema() { + return isValidOutputSchema; + } + + public void setIsValidOutputSchema(Boolean validOutputSchema) { + isValidOutputSchema = validOutputSchema; + } + + public GenericRecord getSampleData() { + return sampleData; + } + + public void setSampleData(GenericRecord sampleData) { + this.sampleData = sampleData; + } + + private static final Logger LOG = LoggerFactory.getLogger(AvroExtractorKeys.class); private DataFileStream avroRecordIterator = null; - private long processedCount; private long totalCount; // TODO: move this to ExtractorKeys if pagination is needed private long currentPageNumber = 0; private Schema avroOutputSchema = null; private Boolean isValidOutputSchema = true; + private GenericRecord sampleData = null; public void incrCurrentPageNumber() { currentPageNumber++; } - public void incrProcessedCount() { - processedCount++; - } @Override public void logDebugAll(WorkUnit workUnit) { super.logDebugAll(workUnit); - log.debug("These are values of JsonExtractor regarding to Work Unit: {}", - workUnit == null ? "testing" : workUnit.getProp(MultistageProperties.DATASET_URN_KEY.toString())); - log.debug("Total rows expected or processed: {}", totalCount); - log.debug("Total rows processed: {}", processedCount); - } - - @Override - public void logUsage(State state) { - super.logUsage(state); - for (MultistageProperties p: ESSENTIAL_PARAMETERS) { - log.info("Property {} ({}) has value {} ", p.toString(), p.getClassName(), p.getValidNonblankWithDefault(state)); - } + LOG.debug("These are values of JsonExtractor regarding to Work Unit: {}", + workUnit == null ? "testing" : workUnit.getProp(DATASET_URN.toString())); + LOG.debug("Total rows expected or processed: {}", totalCount); } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/CsvExtractorKeys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/CsvExtractorKeys.java index ba77212..3d839df 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/keys/CsvExtractorKeys.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/CsvExtractorKeys.java @@ -4,22 +4,19 @@ package com.linkedin.cdi.keys; -import com.google.common.collect.Lists; import java.util.ArrayDeque; import java.util.Deque; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Set; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; -import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; +import org.apache.commons.lang3.StringUtils; import org.apache.gobblin.source.workunit.WorkUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; /** @@ -27,25 +24,11 @@ * * @author chrli */ -@Slf4j -@Getter(AccessLevel.PUBLIC) -@Setter public class CsvExtractorKeys extends ExtractorKeys { - final private static List ESSENTIAL_PARAMETERS = Lists.newArrayList( - MultistageProperties.MSTAGE_CSV_COLUMN_HEADER, - MultistageProperties.MSTAGE_CSV_SEPARATOR, - MultistageProperties.MSTAGE_CSV_SKIP_LINES, - MultistageProperties.MSTAGE_CSV_QUOTE_CHARACTER, - MultistageProperties.MSTAGE_CSV_ESCAPE_CHARACTER); - + private static final Logger LOG = LoggerFactory.getLogger(CsvExtractorKeys.class); private Iterator csvIterator = null; - private long processedCount = 0; private long currentPageNumber = 0; private Boolean columnHeader = false; - private int rowsToSkip = 0; - private String separator = MultistageProperties.MSTAGE_CSV_SEPARATOR.getDefaultValue(); - private String quoteCharacter = MultistageProperties.MSTAGE_CSV_QUOTE_CHARACTER.getDefaultValue(); - private String escapeCharacter = MultistageProperties.MSTAGE_CSV_ESCAPE_CHARACTER.getDefaultValue(); // column name --> index mapping created based on the output or inferred schema private Map columnToIndexMap = new HashMap<>(); // A queue that stores sample rows read in during schema inference @@ -54,28 +37,89 @@ public class CsvExtractorKeys extends ExtractorKeys { private String[] headerRow; private Set columnProjection = new HashSet<>(); private Boolean isValidOutputSchema = true; + private String defaultFieldType = StringUtils.EMPTY; public void incrCurrentPageNumber() { currentPageNumber++; } - public void incrProcessedCount() { - processedCount++; - } @Override public void logDebugAll(WorkUnit workUnit) { super.logDebugAll(workUnit); - log.debug("These are values of CsvExtractor regarding to Work Unit: {}", - workUnit == null ? "testing" : workUnit.getProp(MultistageProperties.DATASET_URN_KEY.toString())); - log.debug("Is column header present: {}", columnHeader); - log.debug("Total rows to skip: {}", rowsToSkip); + LOG.debug("These are values of CsvExtractor regarding to Work Unit: {}", + workUnit == null ? "testing" : workUnit.getProp(DATASET_URN.toString())); + LOG.debug("Is column header present: {}", columnHeader); } - @Override - public void logUsage(State state) { - super.logUsage(state); - for (MultistageProperties p: ESSENTIAL_PARAMETERS) { - log.info("Property {} ({}) has value {} ", p.toString(), p.getClassName(), p.getValidNonblankWithDefault(state)); - } + public Iterator getCsvIterator() { + return csvIterator; + } + + public void setCsvIterator(Iterator csvIterator) { + this.csvIterator = csvIterator; + } + + public long getCurrentPageNumber() { + return currentPageNumber; + } + + public void setCurrentPageNumber(long currentPageNumber) { + this.currentPageNumber = currentPageNumber; + } + + public Boolean getColumnHeader() { + return columnHeader; + } + + public void setColumnHeader(Boolean columnHeader) { + this.columnHeader = columnHeader; + } + + public Map getColumnToIndexMap() { + return columnToIndexMap; + } + + public void setColumnToIndexMap(Map columnToIndexMap) { + this.columnToIndexMap = columnToIndexMap; + } + + public Deque getSampleRows() { + return sampleRows; + } + + public void setSampleRows(Deque sampleRows) { + this.sampleRows = sampleRows; + } + + public String[] getHeaderRow() { + return headerRow; + } + + public void setHeaderRow(String[] headerRow) { + this.headerRow = headerRow; + } + + public Set getColumnProjection() { + return columnProjection; + } + + public void setColumnProjection(Set columnProjection) { + this.columnProjection = columnProjection; + } + + public Boolean getIsValidOutputSchema() { + return isValidOutputSchema; + } + + public void setIsValidOutputSchema(Boolean validOutputSchema) { + isValidOutputSchema = validOutputSchema; + } + + public String getDefaultFieldType() { + return defaultFieldType; + } + + public void setDefaultFieldType(String defaultFieldType) { + this.defaultFieldType = defaultFieldType; } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/ExtractorKeys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/ExtractorKeys.java index 5ee5a35..5f11fd0 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/keys/ExtractorKeys.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/ExtractorKeys.java @@ -7,17 +7,17 @@ import com.google.common.collect.Lists; import com.google.gson.JsonArray; import com.google.gson.JsonObject; +import com.linkedin.cdi.configuration.MultistageProperties; +import com.linkedin.cdi.preprocessor.StreamProcessor; import java.util.ArrayList; import java.util.List; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.preprocessor.StreamProcessor; import org.apache.gobblin.source.workunit.WorkUnit; import org.joda.time.DateTime; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; /** @@ -28,15 +28,13 @@ * * @author chrli */ -@Slf4j -@Getter (AccessLevel.PUBLIC) -@Setter public class ExtractorKeys { - final static private List ESSENTIAL_PARAMETERS = Lists.newArrayList( - MultistageProperties.EXTRACT_TABLE_NAME_KEY, - MultistageProperties.MSTAGE_ACTIVATION_PROPERTY, - MultistageProperties.MSTAGE_PARAMETERS - ); + private static final Logger LOG = LoggerFactory.getLogger(ExtractorKeys.class); + final static private List> WORK_UNIT_PARAMETERS = Lists.newArrayList( + MSTAGE_ACTIVATION_PROPERTY, + MSTAGE_PAYLOAD_PROPERTY, + MSTAGE_WATERMARK_GROUPS, + MSTAGE_WORK_UNIT_SCHEDULING_STARTTIME); private JsonObject activationParameters = new JsonObject(); private long startTime = DateTime.now().getMillis(); @@ -48,24 +46,122 @@ public class ExtractorKeys { private JsonObject dynamicParameters = new JsonObject(); private Boolean explictEof; private JsonArray payloads = new JsonArray(); + private long processedCount = 0; + + public void incrProcessedCount() { + processedCount++; + } public void logDebugAll(WorkUnit workUnit) { - log.debug("These are values in MultistageExtractor regarding to Work Unit: {}", - workUnit == null ? "testing" : workUnit.getProp(MultistageProperties.DATASET_URN_KEY.toString())); - log.debug("Activation parameters: {}", activationParameters); - log.debug("Payload size: {}", payloads.size()); - log.debug("Starting time: {}", startTime); - log.debug("Signature of the work unit: {}", signature); + LOG.debug("These are values in MultistageExtractor regarding to Work Unit: {}", + workUnit == null ? "testing" : workUnit.getProp(DATASET_URN.toString())); + LOG.debug("Activation parameters: {}", activationParameters); + LOG.debug("Payload size: {}", payloads.size()); + LOG.debug("Starting time: {}", startTime); + LOG.debug("Signature of the work unit: {}", signature); if (inferredSchema != null) { - log.info("Inferred schema: {}", inferredSchema.toString()); - log.info("Avro-flavor schema: {}", inferredSchema.toString()); + LOG.info("Inferred schema: {}", inferredSchema.toString()); + LOG.info("Avro-flavor schema: {}", inferredSchema.toString()); } - log.debug("Session Status: {}", sessionKeyValue); + LOG.debug("Session Status: {}", sessionKeyValue); + LOG.debug("Total rows processed: {}", processedCount); } + /** + * Log work unit specific property values + * @param state work unit states + */ public void logUsage(State state) { - for (MultistageProperties p: ESSENTIAL_PARAMETERS) { - log.info("Property {} ({}) has value {} ", p.toString(), p.getClassName(), p.getValidNonblankWithDefault(state)); + for (MultistageProperties p: WORK_UNIT_PARAMETERS) { + LOG.info(p.info(state)); } } + + public JsonObject getActivationParameters() { + return activationParameters; + } + + public void setActivationParameters(JsonObject activationParameters) { + this.activationParameters = activationParameters; + } + + public long getStartTime() { + return startTime; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + public long getDelayStartTime() { + return delayStartTime; + } + + public void setDelayStartTime(long delayStartTime) { + this.delayStartTime = delayStartTime; + } + + public String getSignature() { + return signature; + } + + public void setSignature(String signature) { + this.signature = signature; + } + + public JsonArray getInferredSchema() { + return inferredSchema; + } + + public void setInferredSchema(JsonArray inferredSchema) { + this.inferredSchema = inferredSchema; + } + + public String getSessionKeyValue() { + return sessionKeyValue; + } + + public void setSessionKeyValue(String sessionKeyValue) { + this.sessionKeyValue = sessionKeyValue; + } + + public List> getPreprocessors() { + return preprocessors; + } + + public void setPreprocessors(List> preprocessors) { + this.preprocessors = preprocessors; + } + + public JsonObject getDynamicParameters() { + return dynamicParameters; + } + + public void setDynamicParameters(JsonObject dynamicParameters) { + this.dynamicParameters = dynamicParameters; + } + + public Boolean getExplictEof() { + return explictEof; + } + + public void setExplictEof(Boolean explictEof) { + this.explictEof = explictEof; + } + + public JsonArray getPayloads() { + return payloads; + } + + public void setPayloads(JsonArray payloads) { + this.payloads = payloads; + } + + public long getProcessedCount() { + return processedCount; + } + + public void setProcessedCount(long processedCount) { + this.processedCount = processedCount; + } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/FileDumpExtractorKeys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/FileDumpExtractorKeys.java index c5b0c17..9347841 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/keys/FileDumpExtractorKeys.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/FileDumpExtractorKeys.java @@ -4,23 +4,49 @@ package com.linkedin.cdi.keys; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; import org.apache.gobblin.source.workunit.WorkUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; - -@Slf4j -@Getter(AccessLevel.PUBLIC) -@Setter public class FileDumpExtractorKeys extends ExtractorKeys { + private static final Logger LOG = LoggerFactory.getLogger(FileDumpExtractorKeys.class); String fileName; String fileWritePermissions; String fileDumpLocation; - @Getter private long currentFileNumber = 0; + public String getFileName() { + return fileName; + } + + public void setFileName(String fileName) { + this.fileName = fileName; + } + + public String getFileWritePermissions() { + return fileWritePermissions; + } + + public void setFileWritePermissions(String fileWritePermissions) { + this.fileWritePermissions = fileWritePermissions; + } + + public String getFileDumpLocation() { + return fileDumpLocation; + } + + public void setFileDumpLocation(String fileDumpLocation) { + this.fileDumpLocation = fileDumpLocation; + } + + public long getCurrentFileNumber() { + return currentFileNumber; + } + + public void setCurrentFileNumber(long currentFileNumber) { + this.currentFileNumber = currentFileNumber; + } + public long incrCurrentFileNumber() { return currentFileNumber++; } @@ -28,10 +54,10 @@ public long incrCurrentFileNumber() { @Override public void logDebugAll(WorkUnit workUnit) { super.logDebugAll(workUnit); - log.debug("These are values in FileDumpExtractor:"); - log.debug("Dumping data with file name - " + fileName); - log.debug("Dumping data with permissions - " + fileWritePermissions); - log.debug("Dumping data at location - " + fileDumpLocation); - log.debug("Current file number - {}", currentFileNumber); + LOG.debug("These are values in FileDumpExtractor:"); + LOG.debug("Dumping data with file name - " + fileName); + LOG.debug("Dumping data with permissions - " + fileWritePermissions); + LOG.debug("Dumping data at location - " + fileDumpLocation); + LOG.debug("Current file number - {}", currentFileNumber); } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/HdfsKeys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/HdfsKeys.java index 16b9955..0b7be64 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/keys/HdfsKeys.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/HdfsKeys.java @@ -4,14 +4,8 @@ package com.linkedin.cdi.keys; -import com.google.common.collect.Lists; -import java.util.List; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; -import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -20,20 +14,6 @@ * * @author chrli */ - -@Slf4j -@Getter(AccessLevel.PUBLIC) -@Setter(AccessLevel.PUBLIC) public class HdfsKeys extends JobKeys { - final private static List ESSENTIAL_PARAMETERS = Lists.newArrayList( - // HDFS essential parameters - ); - - @Override - public void logUsage(State state) { - super.logUsage(state); - for (MultistageProperties p : ESSENTIAL_PARAMETERS) { - log.info("Property {} ({}) has value {} ", p.toString(), p.getClassName(), p.getValidNonblankWithDefault(state)); - } - } + private static final Logger LOG = LoggerFactory.getLogger(HdfsKeys.class); } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/HttpKeys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/HttpKeys.java index 3180542..8575942 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/keys/HttpKeys.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/HttpKeys.java @@ -4,18 +4,13 @@ package com.linkedin.cdi.keys; -import com.google.common.collect.Lists; import com.google.gson.JsonObject; +import com.linkedin.cdi.factory.http.HttpRequestMethod; import java.util.HashMap; import java.util.List; import java.util.Map; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; -import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.util.HttpRequestMethod; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -23,18 +18,8 @@ * * @author chrli */ -@Slf4j -@Getter (AccessLevel.PUBLIC) -@Setter(AccessLevel.PUBLIC) public class HttpKeys extends JobKeys { - final private static List ESSENTIAL_PARAMETERS = Lists.newArrayList( - MultistageProperties.SOURCE_CONN_USERNAME, - MultistageProperties.SOURCE_CONN_PASSWORD, - MultistageProperties.MSTAGE_AUTHENTICATION, - MultistageProperties.MSTAGE_HTTP_REQUEST_METHOD, - MultistageProperties.MSTAGE_HTTP_REQUEST_HEADERS, - MultistageProperties.MSTAGE_SESSION_KEY_FIELD); - + private static final Logger LOG = LoggerFactory.getLogger(HttpKeys.class); private JsonObject authentication = new JsonObject(); private JsonObject httpRequestHeaders = new JsonObject(); private Map httpRequestHeadersWithAuthentication = new HashMap<>(); @@ -46,19 +31,67 @@ public class HttpKeys extends JobKeys { @Override public void logDebugAll() { super.logDebugAll(); - log.debug("These are values in HttpSource"); - log.debug("Http Request Headers: {}", httpRequestHeaders); - //log.debug("Http Request Headers with Authentication: {}", httpRequestHeadersWithAuthentication.toString()); - log.debug("Http Request Method: {}", httpRequestMethod); - log.debug("Http Statuses: {}", httpStatuses); - log.debug("Initial values of dynamic parameters: {}", initialParameters); + LOG.debug("These are values in HttpSource"); + LOG.debug("Http Request Headers: {}", httpRequestHeaders); + //LOG.debug("Http Request Headers with Authentication: {}", httpRequestHeadersWithAuthentication.toString()); + LOG.debug("Http Request Method: {}", httpRequestMethod); + LOG.debug("Http Statuses: {}", httpStatuses); + LOG.debug("Initial values of dynamic parameters: {}", initialParameters); } - @Override - public void logUsage(State state) { - super.logUsage(state); - for (MultistageProperties p: ESSENTIAL_PARAMETERS) { - log.info("Property {} ({}) has value {} ", p.toString(), p.getClassName(), p.getValidNonblankWithDefault(state)); - } + public JsonObject getAuthentication() { + return authentication; + } + + public void setAuthentication(JsonObject authentication) { + this.authentication = authentication; + } + + public JsonObject getHttpRequestHeaders() { + return httpRequestHeaders; + } + + public void setHttpRequestHeaders(JsonObject httpRequestHeaders) { + this.httpRequestHeaders = httpRequestHeaders; + } + + public Map getHttpRequestHeadersWithAuthentication() { + return httpRequestHeadersWithAuthentication; + } + + public void setHttpRequestHeadersWithAuthentication(Map httpRequestHeadersWithAuthentication) { + this.httpRequestHeadersWithAuthentication = httpRequestHeadersWithAuthentication; + } + + public String getHttpRequestMethod() { + return httpRequestMethod; + } + + public void setHttpRequestMethod(String httpRequestMethod) { + this.httpRequestMethod = httpRequestMethod; + } + + public JsonObject getInitialParameters() { + return initialParameters; + } + + public void setInitialParameters(JsonObject initialParameters) { + this.initialParameters = initialParameters; + } + + public Map> getHttpStatuses() { + return httpStatuses; + } + + public void setHttpStatuses(Map> httpStatuses) { + this.httpStatuses = httpStatuses; + } + + public Map> getHttpStatusReasons() { + return httpStatusReasons; + } + + public void setHttpStatusReasons(Map> httpStatusReasons) { + this.httpStatusReasons = httpStatusReasons; } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/JdbcKeys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/JdbcKeys.java index 3fd8495..f058d93 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/keys/JdbcKeys.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/JdbcKeys.java @@ -4,15 +4,11 @@ package com.linkedin.cdi.keys; -import com.google.common.collect.Lists; import com.google.gson.JsonObject; -import java.util.List; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; -import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; /** @@ -20,35 +16,41 @@ * * @author chrli */ -@Slf4j -@Getter (AccessLevel.PUBLIC) -@Setter(AccessLevel.PUBLIC) public class JdbcKeys extends JobKeys { - final private static List ESSENTIAL_PARAMETERS = Lists.newArrayList( - MultistageProperties.MSTAGE_JDBC_STATEMENT, - MultistageProperties.SOURCE_CONN_USERNAME, - MultistageProperties.SOURCE_CONN_PASSWORD); - + private static final Logger LOG = LoggerFactory.getLogger(JdbcKeys.class); private String jdbcStatement = null; private JsonObject initialParameterValues = new JsonObject(); - private String separator = MultistageProperties.MSTAGE_CSV_SEPARATOR.getDefaultValue(); - private String quoteCharacter = MultistageProperties.MSTAGE_CSV_QUOTE_CHARACTER.getDefaultValue(); - private String escapeCharacter = MultistageProperties.MSTAGE_CSV_ESCAPE_CHARACTER.getDefaultValue(); - private String schemaRefactorFunction = MultistageProperties.MSTAGE_JDBC_SCHEMA_REFACTOR.getDefaultValue(); + private String schemaRefactorFunction = MSTAGE_JDBC_SCHEMA_REFACTOR.getDefaultValue(); @Override public void logDebugAll() { super.logDebugAll(); - log.debug("These are values in JdbcSource"); - log.debug("JDBC statement: {}", jdbcStatement); - log.debug("Initial values of dynamic parameters: {}", initialParameterValues); + LOG.debug("These are values in JdbcSource"); + LOG.debug("JDBC statement: {}", jdbcStatement); + LOG.debug("Initial values of dynamic parameters: {}", initialParameterValues); } - @Override - public void logUsage(State state) { - super.logUsage(state); - for (MultistageProperties p: ESSENTIAL_PARAMETERS) { - log.info("Property {} ({}) has value {} ", p.toString(), p.getClassName(), p.getValidNonblankWithDefault(state)); - } + public String getJdbcStatement() { + return jdbcStatement; + } + + public void setJdbcStatement(String jdbcStatement) { + this.jdbcStatement = jdbcStatement; + } + + public JsonObject getInitialParameterValues() { + return initialParameterValues; + } + + public void setInitialParameterValues(JsonObject initialParameterValues) { + this.initialParameterValues = initialParameterValues; + } + + public String getSchemaRefactorFunction() { + return schemaRefactorFunction; + } + + public void setSchemaRefactorFunction(String schemaRefactorFunction) { + this.schemaRefactorFunction = schemaRefactorFunction; } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/JobKeys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/JobKeys.java index a4a6027..6a51877 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/keys/JobKeys.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/JobKeys.java @@ -11,27 +11,26 @@ import com.google.gson.JsonElement; import com.google.gson.JsonObject; import com.google.gson.reflect.TypeToken; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; -import org.apache.gobblin.configuration.State; import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.factory.SchemaReaderFactory; +import com.linkedin.cdi.factory.ConnectionClientFactory; import com.linkedin.cdi.factory.reader.SchemaReader; import com.linkedin.cdi.util.DateTimeUtils; import com.linkedin.cdi.util.HdfsReader; import com.linkedin.cdi.util.JsonUtils; import com.linkedin.cdi.util.ParameterTypes; import com.linkedin.cdi.util.WorkUnitPartitionTypes; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.StringUtils; +import org.apache.gobblin.configuration.State; import org.joda.time.DateTime; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static com.linkedin.cdi.configuration.StaticConstants.*; @@ -49,42 +48,9 @@ * * @author chrli */ - -@Slf4j -@Getter(AccessLevel.PUBLIC) -@Setter(AccessLevel.PUBLIC) public class JobKeys { + private static final Logger LOG = LoggerFactory.getLogger(JobKeys.class); final static public Gson GSON = new Gson(); - final static public List ESSENTIAL_PARAMETERS = Lists.newArrayList( - MultistageProperties.SOURCE_CLASS, - MultistageProperties.EXTRACTOR_CLASSES, - MultistageProperties.CONVERTER_CLASSES, - MultistageProperties.EXTRACT_IS_FULL, - MultistageProperties.EXTRACT_TABLE_TYPE_KEY, - MultistageProperties.STATE_STORE_ENABLED, - MultistageProperties.MSTAGE_ABSTINENT_PERIOD_DAYS, - MultistageProperties.MSTAGE_DERIVED_FIELDS, - MultistageProperties.MSTAGE_ENABLE_CLEANSING, - MultistageProperties.MSTAGE_ENABLE_DYNAMIC_FULL_LOAD, - MultistageProperties.MSTAGE_ENABLE_SCHEMA_BASED_FILTERING, - MultistageProperties.MSTAGE_ENCODING, - MultistageProperties.MSTAGE_ENCRYPTION_FIELDS, - MultistageProperties.MSTAGE_GRACE_PERIOD_DAYS, - MultistageProperties.MSTAGE_OUTPUT_SCHEMA, - MultistageProperties.MSTAGE_PAGINATION, - MultistageProperties.MSTAGE_PARAMETERS, - MultistageProperties.MSTAGE_RETENTION, - MultistageProperties.MSTAGE_SECONDARY_INPUT, - MultistageProperties.MSTAGE_SESSION_KEY_FIELD, - MultistageProperties.MSTAGE_SOURCE_DATA_CHARACTER_SET, - MultistageProperties.MSTAGE_SOURCE_SCHEMA_URN, - MultistageProperties.MSTAGE_SOURCE_URI, - MultistageProperties.MSTAGE_TOTAL_COUNT_FIELD, - MultistageProperties.MSTAGE_WAIT_TIMEOUT_SECONDS, - MultistageProperties.MSTAGE_WORK_UNIT_PACING_SECONDS, - MultistageProperties.MSTAGE_WORK_UNIT_PARALLELISM_MAX, - MultistageProperties.MSTAGE_WORK_UNIT_PARTIAL_PARTITION, - MultistageProperties.MSTAGE_WATERMARK); final private static int RETRY_DELAY_IN_SEC_DEFAULT = 300; final private static int RETRY_COUNT_DEFAULT = 3; final private static String ITEMS_KEY = "items"; @@ -123,25 +89,29 @@ public class JobKeys { private String schemaCleansingPattern = "(\\s|\\$|@)"; private String schemaCleansingReplacement = "_"; private Boolean schemaCleansingNullable = false; + private long minWorkUnits = 0; + private long minWorkUnitRecords = 0; public void initialize(State state) { parsePaginationFields(state); parsePaginationInitialValues(state); - setSessionKeyField(MultistageProperties.MSTAGE_SESSION_KEY_FIELD.getValidNonblankWithDefault(state)); - setTotalCountField(MultistageProperties.MSTAGE_TOTAL_COUNT_FIELD.getValidNonblankWithDefault(state)); - setSourceParameters(MultistageProperties.MSTAGE_PARAMETERS.getValidNonblankWithDefault(state)); - setSourceUri(MultistageProperties.MSTAGE_SOURCE_URI.getValidNonblankWithDefault(state)); + setSessionKeyField(MSTAGE_SESSION_KEY_FIELD.get(state)); + setTotalCountField(MSTAGE_TOTAL_COUNT_FIELD.get(state)); + setSourceParameters(MSTAGE_PARAMETERS.get(state)); + setSourceUri(MSTAGE_SOURCE_URI.get(state)); setDefaultFieldTypes(parseDefaultFieldTypes(state)); setDerivedFields(parseDerivedFields(state)); setOutputSchema(parseOutputSchema(state)); - setTargetSchema(MultistageProperties.MSTAGE_TARGET_SCHEMA.getValidNonblankWithDefault(state)); - setEncryptionField(MultistageProperties.MSTAGE_ENCRYPTION_FIELDS.getValidNonblankWithDefault(state)); - setDataField(MultistageProperties.MSTAGE_DATA_FIELD.getValidNonblankWithDefault(state)); - setCallInterval(MultistageProperties.MSTAGE_CALL_INTERVAL.getProp(state)); - setSessionTimeout(MultistageProperties.MSTAGE_WAIT_TIMEOUT_SECONDS.getMillis(state)); - - setEnableCleansing(MultistageProperties.MSTAGE_ENABLE_CLEANSING.getValidNonblankWithDefault(state)); - JsonObject schemaCleansing = MultistageProperties.MSTAGE_SCHEMA_CLENSING.getValidNonblankWithDefault(state); + setTargetSchema(MSTAGE_TARGET_SCHEMA.get(state)); + setEncryptionField(MSTAGE_ENCRYPTION_FIELDS.get(state)); + setDataField(MSTAGE_DATA_FIELD.get(state)); + setCallInterval(MSTAGE_CALL_INTERVAL_MILLIS.get(state)); + setSessionTimeout(MSTAGE_WAIT_TIMEOUT_SECONDS.getMillis(state)); + setMinWorkUnitRecords(MSTAGE_WORK_UNIT_MIN_RECORDS.get(state)); + setMinWorkUnits(MSTAGE_WORK_UNIT_MIN_UNITS.get(state)); + + setEnableCleansing(MSTAGE_ENABLE_CLEANSING.get(state)); + JsonObject schemaCleansing = MSTAGE_SCHEMA_CLEANSING.get(state); if (schemaCleansing.has("enabled")) { setEnableCleansing(Boolean.parseBoolean(schemaCleansing.get("enabled").getAsString())); if (enableCleansing && schemaCleansing.has("pattern")) { @@ -155,20 +125,17 @@ public void initialize(State state) { } } - setIsPartialPartition(MultistageProperties.MSTAGE_WORK_UNIT_PARTIAL_PARTITION.getValidNonblankWithDefault(state)); + setIsPartialPartition(MSTAGE_WORK_UNIT_PARTIAL_PARTITION.get(state)); setWorkUnitPartitionType(parsePartitionType(state)); - setWatermarkDefinition(MultistageProperties.MSTAGE_WATERMARK.getValidNonblankWithDefault(state)); - Map retry = parseSecondaryInputRetry( - MultistageProperties.MSTAGE_SECONDARY_INPUT.getValidNonblankWithDefault(state)); + setWatermarkDefinition(MSTAGE_WATERMARK.get(state)); + Map retry = parseSecondaryInputRetry(MSTAGE_SECONDARY_INPUT.get(state)); setRetryDelayInSec(retry.get(KEY_WORD_RETRY_DELAY_IN_SEC)); setRetryCount(retry.get(KEY_WORD_RETRY_COUNT)); - setSecondaryInputs(MultistageProperties.MSTAGE_SECONDARY_INPUT.getValidNonblankWithDefault(state)); + setSecondaryInputs(MSTAGE_SECONDARY_INPUT.get(state)); setIsSecondaryAuthenticationEnabled(checkSecondaryAuthenticationEnabled()); - setSourceSchema(readSourceSchemaFromUrn(state, - MultistageProperties.MSTAGE_SOURCE_SCHEMA_URN.getValidNonblankWithDefault(state))); - setTargetSchema(readTargetSchemaFromUrn(state, - MultistageProperties.MSTAGE_TARGET_SCHEMA_URN.getValidNonblankWithDefault(state))); + setSourceSchema(readSourceSchemaFromUrn(state, MSTAGE_SOURCE_SCHEMA_URN.get(state))); + setTargetSchema(readTargetSchemaFromUrn(state, MSTAGE_TARGET_SCHEMA_URN.get(state))); // closing out schema reader if it was created because of reading // output schema or target schema. @@ -210,7 +177,7 @@ public String getSessionStateFailCondition() { try { retValue = sessionKeyField.get("failCondition").getAsJsonObject().get("regexp").getAsString(); } catch (Exception e) { - log.debug("failCondition is not defined: {}", sessionKeyField); + LOG.debug("failCondition is not defined: {}", sessionKeyField); } } return retValue; @@ -237,8 +204,8 @@ public JobKeys setSourceSchema(JsonArray sourceSchema) { if (!this.hasOutputSchema()) { setOutputSchema(JsonUtils.deepCopy(sourceSchema).getAsJsonArray()); } - log.debug("Source Schema: {}", sourceSchema.toString()); - log.debug("Output Schema: {}", outputSchema.toString()); + LOG.debug("Source Schema: {}", sourceSchema.toString()); + LOG.debug("Output Schema: {}", outputSchema.toString()); return this; } @@ -248,6 +215,30 @@ public JobKeys setSourceSchema(JsonArray sourceSchema) { * @return true if validation was successful, otherwise false */ public boolean validate(State state) { + + // Validate all job parameters + boolean allValid = true; + for (MultistageProperties p: allProperties) { + if (!p.isBlank(state) && !p.isValid(state)) { + LOG.error(p.errorMessage(state)); + allValid = false; + } + } + + for (String deprecatedKey: deprecatedProperties.keySet()) { + if (state.contains(deprecatedKey) && + StringUtils.isNotBlank(state.getProp(deprecatedKey, StringUtils.EMPTY))) { + LOG.error(String.format(EXCEPTION_DEPRECATED_CONFIGURATION, deprecatedKey, + deprecatedProperties.get(deprecatedKey).getConfig(), + deprecatedProperties.get(deprecatedKey).getDocUrl())); + allValid = false; + } + } + + if(!allValid) { + return false; + } + /** * If pagination is enabled, we need one of following ways to stop pagination * 1. through a total count field, i.e. ms.total.count.field = data. @@ -264,7 +255,7 @@ public boolean validate(State state) { */ if (isPaginationEnabled()) { if (totalCountField == null && !isSessionStateEnabled()) { - log.warn("Pagination is enabled, but there is no total count field or session \n" + LOG.warn("Pagination is enabled, but there is no total count field or session \n" + "control to stop it. Pagination will stop only when a blank page is returned from source. \n" + "Please check the configuration of essential parameters if such condition can happen."); } @@ -276,8 +267,8 @@ public boolean validate(State state) { * It is OK if output schema is intentionally left blank. */ if (!hasOutputSchema()) { - if (!state.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), StringUtils.EMPTY).isEmpty()) { - log.error("Output schema is specified but it is an invalid or empty JsonArray"); + if (!state.getProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), StringUtils.EMPTY).isEmpty()) { + LOG.error("Output schema is specified but it is an invalid or empty JsonArray"); return false; } } @@ -286,9 +277,9 @@ public boolean validate(State state) { * Check if partitioning property is correct */ if (getWorkUnitPartitionType() == null) { - String partTypeString = state.getProp(MultistageProperties.MSTAGE_WORK_UNIT_PARTITION.getConfig()); + String partTypeString = state.getProp(MSTAGE_WORK_UNIT_PARTITION.getConfig()); if (!StringUtils.isBlank(partTypeString)) { - log.error("ms.work.unit.partition has a unaccepted value: {}", partTypeString); + LOG.error("ms.work.unit.partition has a unaccepted value: {}", partTypeString); return false; } } else if (getWorkUnitPartitionType() == WorkUnitPartitionTypes.COMPOSITE) { @@ -299,7 +290,7 @@ public boolean validate(State state) { if (WorkUnitPartitionTypes.COMPOSITE.getRanges( DateTime.parse("2001-01-01"), DateTime.now(), true).size() < 1) { - log.error("ms.work.unit.partition has incorrect or non-ISO-formatted date time values"); + LOG.error("ms.work.unit.partition has incorrect or non-ISO-formatted date time values"); return false; } } @@ -314,24 +305,24 @@ public boolean validate(State state) { } public void logDebugAll() { - log.debug("These are values in MultistageSource"); - log.debug("Source Uri: {}", sourceUri); - log.debug("Total count field: {}", totalCountField); - log.debug("Pagination: fields {}, initial values {}", paginationFields.toString(), paginationInitValues.toString()); - log.debug("Session key field definition: {}", sessionKeyField.toString()); - log.debug("Call interval in milliseconds: {}", callInterval); - log.debug("Session timeout: {}", sessionTimeout); - log.debug("Derived fields definition: {}", derivedFields.toString()); - log.debug("Output schema definition: {}", outputSchema.toString()); - log.debug("Watermark definition: {}", watermarkDefinition.toString()); - log.debug("Encrypted fields: {}", encryptionField); - log.debug("Retry Delay: {}", retryDelayInSec); - log.debug("Retry Count: {}", retryCount); + LOG.debug("These are values in MultistageSource"); + LOG.debug("Source Uri: {}", sourceUri); + LOG.debug("Total count field: {}", totalCountField); + LOG.debug("Pagination: fields {}, initial values {}", paginationFields.toString(), paginationInitValues.toString()); + LOG.debug("Session key field definition: {}", sessionKeyField.toString()); + LOG.debug("Call interval in milliseconds: {}", callInterval); + LOG.debug("Session timeout: {}", sessionTimeout); + LOG.debug("Derived fields definition: {}", derivedFields.toString()); + LOG.debug("Output schema definition: {}", outputSchema.toString()); + LOG.debug("Watermark definition: {}", watermarkDefinition.toString()); + LOG.debug("Encrypted fields: {}", encryptionField); + LOG.debug("Retry Delay: {}", retryDelayInSec); + LOG.debug("Retry Count: {}", retryCount); } public void logUsage(State state) { - for (MultistageProperties p: ESSENTIAL_PARAMETERS) { - log.info("Property {} ({}) has value {} ", p.toString(), p.getClassName(), p.getValidNonblankWithDefault(state)); + for (MultistageProperties p: allProperties) { + LOG.info(p.info(state)); } } @@ -341,8 +332,8 @@ private void parsePaginationFields(State state) { ParameterTypes.PAGESIZE, ParameterTypes.PAGENO ); - if (MultistageProperties.MSTAGE_PAGINATION.validateNonblank(state)) { - JsonObject p = MultistageProperties.MSTAGE_PAGINATION.getProp(state); + if (MSTAGE_PAGINATION.isValidNonblank(state)) { + JsonObject p = MSTAGE_PAGINATION.get(state); if (p.has("fields")) { JsonArray fields = p.get("fields").getAsJsonArray(); for (int i = 0; i < fields.size(); i++) { @@ -360,8 +351,8 @@ private void parsePaginationInitialValues(State state) { ParameterTypes.PAGESIZE, ParameterTypes.PAGENO ); - if (MultistageProperties.MSTAGE_PAGINATION.validateNonblank(state)) { - JsonObject p = MultistageProperties.MSTAGE_PAGINATION.getProp(state); + if (MSTAGE_PAGINATION.isValidNonblank(state)) { + JsonObject p = MSTAGE_PAGINATION.get(state); if (p.has("initialvalues")) { JsonArray values = p.get("initialvalues").getAsJsonArray(); for (int i = 0; i < values.size(); i++) { @@ -380,8 +371,8 @@ private void parsePaginationInitialValues(State state) { * @return A map of fields and their default types */ private Map parseDefaultFieldTypes(State state) { - if (MultistageProperties.MSTAGE_DATA_DEFAULT_TYPE.validateNonblank(state)) { - return GSON.fromJson(MultistageProperties.MSTAGE_DATA_DEFAULT_TYPE.getProp(state).toString(), + if (MSTAGE_DATA_DEFAULT_TYPE.isValidNonblank(state)) { + return GSON.fromJson(MSTAGE_DATA_DEFAULT_TYPE.get(state).toString(), new TypeToken>() { }.getType()); } @@ -403,12 +394,12 @@ private Map parseDefaultFieldTypes(State state) { */ @VisibleForTesting Map> parseDerivedFields(State state) { - if (!MultistageProperties.MSTAGE_DERIVED_FIELDS.validateNonblank(state)) { + if (!MSTAGE_DERIVED_FIELDS.isValidNonblank(state)) { return new HashMap<>(); } Map> derivedFields = new HashMap<>(); - JsonArray jsonArray = MultistageProperties.MSTAGE_DERIVED_FIELDS.getProp(state); + JsonArray jsonArray = MSTAGE_DERIVED_FIELDS.get(state); for (JsonElement field: jsonArray) { // change the formula part, which is JsonObject, into map @@ -428,7 +419,7 @@ Map> parseDerivedFields(State state) { * @return the output schema */ public JsonArray parseOutputSchema(State state) { - return JsonUtils.deepCopy(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getValidNonblankWithDefault(state)).getAsJsonArray(); + return JsonUtils.deepCopy(MSTAGE_OUTPUT_SCHEMA.get(state)).getAsJsonArray(); } @@ -439,7 +430,7 @@ public JsonArray parseOutputSchema(State state) { */ WorkUnitPartitionTypes parsePartitionType(State state) { WorkUnitPartitionTypes partitionType = WorkUnitPartitionTypes.fromString( - MultistageProperties.MSTAGE_WORK_UNIT_PARTITION.getValidNonblankWithDefault(state)); + MSTAGE_WORK_UNIT_PARTITION.get(state)); if (partitionType != WorkUnitPartitionTypes.COMPOSITE) { return partitionType; @@ -449,7 +440,7 @@ WorkUnitPartitionTypes parsePartitionType(State state) { WorkUnitPartitionTypes.COMPOSITE.resetSubRange(); try { JsonObject jsonObject = GSON.fromJson( - MultistageProperties.MSTAGE_WORK_UNIT_PARTITION.getValidNonblankWithDefault(state).toString(), + MSTAGE_WORK_UNIT_PARTITION.get(state).toString(), JsonObject.class); for (Map.Entry entry : jsonObject.entrySet()) { @@ -465,8 +456,8 @@ WorkUnitPartitionTypes parsePartitionType(State state) { partitionType.addSubRange(start, end, WorkUnitPartitionTypes.fromString(partitionTypeString)); } } catch (Exception e) { - log.error("Error parsing composite partition string: " - + MultistageProperties.MSTAGE_WORK_UNIT_PARTITION.getValidNonblankWithDefault(state).toString() + LOG.error("Error parsing composite partition string: " + + MSTAGE_WORK_UNIT_PARTITION.get(state).toString() + "\n partitions may not be generated properly.", e); } @@ -520,19 +511,19 @@ protected boolean checkSecondaryAuthenticationEnabled() { } public Map readSecondaryInputs(State state, final long retries) throws InterruptedException { - log.info("Trying to read secondary input with retry = {}", retries); + LOG.info("Trying to read secondary input with retry = {}", retries); Map secondaryInputs = readContext(state); // Check if authentication is ready, and if not, whether retry is required JsonArray authentications = secondaryInputs.get(KEY_WORD_AUTHENTICATION); if ((authentications == null || authentications.size() == 0) && this.getIsSecondaryAuthenticationEnabled() && retries > 0) { - log.info("Authentication tokens are expected from secondary input, but not ready"); - log.info("Will wait for {} seconds and then retry reading the secondary input", this.getRetryDelayInSec()); + LOG.info("Authentication tokens are expected from secondary input, but not ready"); + LOG.info("Will wait for {} seconds and then retry reading the secondary input", this.getRetryDelayInSec()); TimeUnit.SECONDS.sleep(this.getRetryDelayInSec()); return readSecondaryInputs(state, retries - 1); } - log.info("Successfully read secondary input, no more retry"); + LOG.info("Successfully read secondary input, no more retry"); return secondaryInputs; } @@ -551,11 +542,14 @@ public JsonArray readSchemaFromUrn(State state, String urn) { try { // Schema Reader could be plugged in before the initialization on JobKeys if (schemaReader == null) { - schemaReader = SchemaReaderFactory.create(state); + Class factoryClass = Class.forName( + MSTAGE_CONNECTION_CLIENT_FACTORY.get(state)); + ConnectionClientFactory factory = (ConnectionClientFactory) factoryClass.newInstance(); + schemaReader = factory.getSchemaReader(state); } return schemaReader.read(state, urn).getAsJsonArray(); } catch (Exception e) { - log.error("Error reading schema based on urn: {}", urn); + LOG.error("Error reading schema based on urn: {}", urn); throw new RuntimeException(e); } } @@ -606,4 +600,232 @@ public JsonArray readTargetSchemaFromUrn(State state, String urn) { ? readSchemaFromUrn(state, urn) : getTargetSchema(); } + + public Map> getDerivedFields() { + return derivedFields; + } + + public void setDerivedFields(Map> derivedFields) { + this.derivedFields = derivedFields; + } + + public Map getDefaultFieldTypes() { + return defaultFieldTypes; + } + + public void setDefaultFieldTypes(Map defaultFieldTypes) { + this.defaultFieldTypes = defaultFieldTypes; + } + + public JsonArray getSourceSchema() { + return sourceSchema; + } + + public JsonArray getOutputSchema() { + return outputSchema; + } + + public void setOutputSchema(JsonArray outputSchema) { + this.outputSchema = outputSchema; + } + + public JsonArray getTargetSchema() { + return targetSchema; + } + + public void setTargetSchema(JsonArray targetSchema) { + this.targetSchema = targetSchema; + } + + public JsonObject getSessionKeyField() { + return sessionKeyField; + } + + public void setSessionKeyField(JsonObject sessionKeyField) { + this.sessionKeyField = sessionKeyField; + } + + public String getTotalCountField() { + return totalCountField; + } + + public void setTotalCountField(String totalCountField) { + this.totalCountField = totalCountField; + } + + public JsonArray getSourceParameters() { + return sourceParameters; + } + + public void setSourceParameters(JsonArray sourceParameters) { + this.sourceParameters = sourceParameters; + } + + public Map getPaginationFields() { + return paginationFields; + } + + public void setPaginationFields(Map paginationFields) { + this.paginationFields = paginationFields; + } + + public Map getPaginationInitValues() { + return paginationInitValues; + } + + public void setPaginationInitValues(Map paginationInitValues) { + this.paginationInitValues = paginationInitValues; + } + + public long getSessionTimeout() { + return sessionTimeout; + } + + public void setSessionTimeout(long sessionTimeout) { + this.sessionTimeout = sessionTimeout; + } + + public long getCallInterval() { + return callInterval; + } + + public void setCallInterval(long callInterval) { + this.callInterval = callInterval; + } + + public JsonArray getEncryptionField() { + return encryptionField; + } + + public void setEncryptionField(JsonArray encryptionField) { + this.encryptionField = encryptionField; + } + + public boolean isEnableCleansing() { + return enableCleansing; + } + + public void setEnableCleansing(boolean enableCleansing) { + this.enableCleansing = enableCleansing; + } + + public String getDataField() { + return dataField; + } + + public void setDataField(String dataField) { + this.dataField = dataField; + } + + public JsonArray getWatermarkDefinition() { + return watermarkDefinition; + } + + public void setWatermarkDefinition(JsonArray watermarkDefinition) { + this.watermarkDefinition = watermarkDefinition; + } + + public long getRetryDelayInSec() { + return retryDelayInSec; + } + + public void setRetryDelayInSec(long retryDelayInSec) { + this.retryDelayInSec = retryDelayInSec; + } + + public long getRetryCount() { + return retryCount; + } + + public void setRetryCount(long retryCount) { + this.retryCount = retryCount; + } + + public Boolean getIsPartialPartition() { + return isPartialPartition; + } + + public void setIsPartialPartition(Boolean partialPartition) { + isPartialPartition = partialPartition; + } + + public JsonArray getSecondaryInputs() { + return secondaryInputs; + } + + public void setSecondaryInputs(JsonArray secondaryInputs) { + this.secondaryInputs = secondaryInputs; + } + + public WorkUnitPartitionTypes getWorkUnitPartitionType() { + return workUnitPartitionType; + } + + public void setWorkUnitPartitionType(WorkUnitPartitionTypes workUnitPartitionType) { + this.workUnitPartitionType = workUnitPartitionType; + } + + public Boolean getIsSecondaryAuthenticationEnabled() { + return isSecondaryAuthenticationEnabled; + } + + public void setIsSecondaryAuthenticationEnabled(Boolean secondaryAuthenticationEnabled) { + isSecondaryAuthenticationEnabled = secondaryAuthenticationEnabled; + } + + public String getSourceUri() { + return sourceUri; + } + + public void setSourceUri(String sourceUri) { + this.sourceUri = sourceUri; + } + + public SchemaReader getSchemaReader() { + return schemaReader; + } + + public void setSchemaReader(SchemaReader schemaReader) { + this.schemaReader = schemaReader; + } + + public String getSchemaCleansingPattern() { + return schemaCleansingPattern; + } + + public void setSchemaCleansingPattern(String schemaCleansingPattern) { + this.schemaCleansingPattern = schemaCleansingPattern; + } + + public String getSchemaCleansingReplacement() { + return schemaCleansingReplacement; + } + + public void setSchemaCleansingReplacement(String schemaCleansingReplacement) { + this.schemaCleansingReplacement = schemaCleansingReplacement; + } + + public Boolean getSchemaCleansingNullable() { + return schemaCleansingNullable; + } + + public void setSchemaCleansingNullable(Boolean schemaCleansingNullable) { + this.schemaCleansingNullable = schemaCleansingNullable; + } + + public long getMinWorkUnits() { + return minWorkUnits; + } + + public void setMinWorkUnits(long minWorkUnits) { + this.minWorkUnits = minWorkUnits; + } + + public long getMinWorkUnitRecords() { + return minWorkUnitRecords; + } + + public void setMinWorkUnitRecords(long minWorkUnitRecords) { + this.minWorkUnitRecords = minWorkUnitRecords; + } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/JsonExtractorKeys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/JsonExtractorKeys.java index 1b3a7ba..8649938 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/keys/JsonExtractorKeys.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/JsonExtractorKeys.java @@ -4,18 +4,14 @@ package com.linkedin.cdi.keys; -import com.google.common.collect.Lists; import com.google.gson.JsonElement; import com.google.gson.JsonObject; import java.util.Iterator; -import java.util.List; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; -import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; import org.apache.gobblin.source.workunit.WorkUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; /** @@ -23,16 +19,9 @@ * * @author chrli */ -@Slf4j -@Getter(AccessLevel.PUBLIC) -@Setter public class JsonExtractorKeys extends ExtractorKeys { - final private static List ESSENTIAL_PARAMETERS = Lists.newArrayList( - MultistageProperties.MSTAGE_DATA_FIELD, - MultistageProperties.MSTAGE_TOTAL_COUNT_FIELD); - + private static final Logger LOG = LoggerFactory.getLogger(JsonExtractorKeys.class); private Iterator jsonElementIterator = null; - private long processedCount; private long totalCount; private long currentPageNumber = 0; private JsonObject pushDowns = new JsonObject(); @@ -40,17 +29,40 @@ public class JsonExtractorKeys extends ExtractorKeys { @Override public void logDebugAll(WorkUnit workUnit) { super.logDebugAll(workUnit); - log.debug("These are values of JsonExtractor regarding to Work Unit: {}", - workUnit == null ? "testing" : workUnit.getProp(MultistageProperties.DATASET_URN_KEY.toString())); - log.debug("Total rows expected or processed: {}", totalCount); - log.debug("Total rows processed: {}", processedCount); + LOG.debug("These are values of JsonExtractor regarding to Work Unit: {}", + workUnit == null ? "testing" : workUnit.getProp(DATASET_URN.toString())); + LOG.debug("Total rows expected or processed: {}", totalCount); } - @Override - public void logUsage(State state) { - super.logUsage(state); - for (MultistageProperties p: ESSENTIAL_PARAMETERS) { - log.info("Property {} ({}) has value {} ", p.toString(), p.getClassName(), p.getValidNonblankWithDefault(state)); - } + public Iterator getJsonElementIterator() { + return jsonElementIterator; + } + + public void setJsonElementIterator(Iterator jsonElementIterator) { + this.jsonElementIterator = jsonElementIterator; + } + + public long getTotalCount() { + return totalCount; + } + + public void setTotalCount(long totalCount) { + this.totalCount = totalCount; + } + + public long getCurrentPageNumber() { + return currentPageNumber; + } + + public void setCurrentPageNumber(long currentPageNumber) { + this.currentPageNumber = currentPageNumber; + } + + public JsonObject getPushDowns() { + return pushDowns; + } + + public void setPushDowns(JsonObject pushDowns) { + this.pushDowns = pushDowns; } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/S3Keys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/S3Keys.java index 0c89614..8527302 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/keys/S3Keys.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/S3Keys.java @@ -4,17 +4,12 @@ package com.linkedin.cdi.keys; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import software.amazon.awssdk.regions.Region; - -@Getter (AccessLevel.PUBLIC) -@Setter (AccessLevel.PUBLIC) -@Slf4j public class S3Keys extends JobKeys { + private static final Logger LOG = LoggerFactory.getLogger(S3Keys.class); private String bucket = ""; private String endpoint = ""; private String prefix = ""; @@ -29,10 +24,90 @@ public class S3Keys extends JobKeys { @Override public void logDebugAll() { super.logDebugAll(); - log.debug("These are values in S3SourceV2:"); - log.debug("S3 Bucket: {}", bucket); - log.debug("S3 endpoint: {}", endpoint); - log.debug("S3 prefix: {}", prefix); - log.debug("S3 files pattern: {}", filesPattern); + LOG.debug("These are values in S3SourceV2:"); + LOG.debug("S3 Bucket: {}", bucket); + LOG.debug("S3 endpoint: {}", endpoint); + LOG.debug("S3 prefix: {}", prefix); + LOG.debug("S3 files pattern: {}", filesPattern); + } + + public String getBucket() { + return bucket; + } + + public void setBucket(String bucket) { + this.bucket = bucket; + } + + public String getEndpoint() { + return endpoint; + } + + public void setEndpoint(String endpoint) { + this.endpoint = endpoint; + } + + public String getPrefix() { + return prefix; + } + + public void setPrefix(String prefix) { + this.prefix = prefix; + } + + public String getFilesPattern() { + return filesPattern; + } + + public void setFilesPattern(String filesPattern) { + this.filesPattern = filesPattern; + } + + public Region getRegion() { + return region; + } + + public void setRegion(Region region) { + this.region = region; + } + + public Integer getMaxKeys() { + return maxKeys; + } + + public void setMaxKeys(Integer maxKeys) { + this.maxKeys = maxKeys; + } + + public String getAccessKey() { + return accessKey; + } + + public void setAccessKey(String accessKey) { + this.accessKey = accessKey; + } + + public String getSecretId() { + return secretId; + } + + public void setSecretId(String secretId) { + this.secretId = secretId; + } + + public Integer getConnectionTimeout() { + return connectionTimeout; + } + + public void setConnectionTimeout(Integer connectionTimeout) { + this.connectionTimeout = connectionTimeout; + } + + public String getTargetFilePattern() { + return targetFilePattern; + } + + public void setTargetFilePattern(String targetFilePattern) { + this.targetFilePattern = targetFilePattern; } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/keys/SftpKeys.java b/cdi-core/src/main/java/com/linkedin/cdi/keys/SftpKeys.java new file mode 100644 index 0000000..cd5164e --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/keys/SftpKeys.java @@ -0,0 +1,77 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.keys; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SftpKeys extends JobKeys { + private static final Logger LOG = LoggerFactory.getLogger(SftpKeys.class); + private String filesPattern = ".*"; + private String splitPattern = ":::"; + private String filesPath = ""; + private String baseDirectory = ""; + private String pathSeparator = "/"; + private String targetFilePattern; + + @Override + public void logDebugAll() { + super.logDebugAll(); + LOG.debug("These are values in SftpSource:"); + LOG.debug("sftp source path: {}", filesPath); + LOG.debug("path separator: {}", pathSeparator); + LOG.debug("split pattern: {}", splitPattern); + LOG.debug("files pattern: {}", filesPattern); + LOG.debug("Base directory: {}", baseDirectory); + } + + public String getFilesPattern() { + return filesPattern; + } + + public void setFilesPattern(String filesPattern) { + this.filesPattern = filesPattern; + } + + public String getSplitPattern() { + return splitPattern; + } + + public void setSplitPattern(String splitPattern) { + this.splitPattern = splitPattern; + } + + public String getFilesPath() { + return filesPath; + } + + public void setFilesPath(String filesPath) { + this.filesPath = filesPath; + } + + public String getBaseDirectory() { + return baseDirectory; + } + + public void setBaseDirectory(String baseDirectory) { + this.baseDirectory = baseDirectory; + } + + public String getPathSeparator() { + return pathSeparator; + } + + public void setPathSeparator(String pathSeparator) { + this.pathSeparator = pathSeparator; + } + + public String getTargetFilePattern() { + return targetFilePattern; + } + + public void setTargetFilePattern(String targetFilePattern) { + this.targetFilePattern = targetFilePattern; + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/preprocessor/GpgDecryptProcessor.java b/cdi-core/src/main/java/com/linkedin/cdi/preprocessor/GpgDecryptProcessor.java index 25a9537..d6998db 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/preprocessor/GpgDecryptProcessor.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/preprocessor/GpgDecryptProcessor.java @@ -5,16 +5,15 @@ package com.linkedin.cdi.preprocessor; import com.google.gson.JsonObject; +import com.linkedin.cdi.configuration.PropertyCollection; +import com.linkedin.cdi.util.EncryptionUtils; import java.io.IOException; import java.io.InputStream; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.FilenameUtils; import org.apache.gobblin.annotation.Alias; import org.apache.gobblin.codec.StreamCodec; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.util.EncryptionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -23,15 +22,22 @@ * * This is backwards compatible with PGP algorithms */ -@Slf4j @Alias("GpgProcessor") public class GpgDecryptProcessor extends InputStreamProcessor { - @Getter - @Setter + private static final Logger LOG = LoggerFactory.getLogger(GpgDecryptProcessor.class); + + public StreamCodec getCodec() { + return codec; + } + + public void setCodec(StreamCodec codec) { + this.codec = codec; + } + private StreamCodec codec; /** - * @param params See {@link MultistageProperties} + * @param params See {@link PropertyCollection} */ public GpgDecryptProcessor(JsonObject params) { super(params); diff --git a/cdi-core/src/main/java/com/linkedin/cdi/preprocessor/GpgEncryptProcessor.java b/cdi-core/src/main/java/com/linkedin/cdi/preprocessor/GpgEncryptProcessor.java index b74bf9e..6d24f7c 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/preprocessor/GpgEncryptProcessor.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/preprocessor/GpgEncryptProcessor.java @@ -5,13 +5,11 @@ package com.linkedin.cdi.preprocessor; import com.google.gson.JsonObject; +import com.linkedin.cdi.util.EncryptionUtils; import java.io.IOException; import java.io.OutputStream; -import lombok.Getter; -import lombok.Setter; import org.apache.commons.io.FilenameUtils; import org.apache.gobblin.codec.StreamCodec; -import com.linkedin.cdi.util.EncryptionUtils; /** @@ -22,8 +20,15 @@ */ public class GpgEncryptProcessor extends OutputStreamProcessor { private static final String FILE_EXT = "gpg"; - @Getter - @Setter + + public StreamCodec getCodec() { + return codec; + } + + public void setCodec(StreamCodec codec) { + this.codec = codec; + } + private StreamCodec codec; public GpgEncryptProcessor(JsonObject params) { diff --git a/cdi-core/src/main/java/com/linkedin/cdi/source/HdfsSource.java b/cdi-core/src/main/java/com/linkedin/cdi/source/HdfsSource.java index 08f278c..71c3158 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/source/HdfsSource.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/source/HdfsSource.java @@ -4,26 +4,33 @@ package com.linkedin.cdi.source; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; +import com.linkedin.cdi.connection.HdfsConnection; +import com.linkedin.cdi.extractor.MultistageExtractor; +import com.linkedin.cdi.keys.HdfsKeys; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.gobblin.configuration.State; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.connection.HdfsConnection; -import com.linkedin.cdi.extractor.MultistageExtractor; -import com.linkedin.cdi.keys.HdfsKeys; import org.apache.gobblin.source.extractor.Extractor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class supports HDFS as just another protocol. The main function * of it is to launch a proper extractor with a HdfsConnection */ -@Slf4j public class HdfsSource extends MultistageSource { - @Getter @Setter + private static final Logger LOG = LoggerFactory.getLogger(HdfsSource.class); + + public HdfsKeys getHdfsKeys() { + return hdfsKeys; + } + + public void setHdfsKeys(HdfsKeys hdfsKeys) { + this.hdfsKeys = hdfsKeys; + } + private HdfsKeys hdfsKeys; public HdfsSource() { @@ -33,7 +40,6 @@ public HdfsSource() { protected void initialize(State state) { super.initialize(state); - hdfsKeys.logUsage(state); hdfsKeys.logDebugAll(); } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/source/HttpSource.java b/cdi-core/src/main/java/com/linkedin/cdi/source/HttpSource.java index 1f8fb24..b7b2439 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/source/HttpSource.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/source/HttpSource.java @@ -10,24 +10,24 @@ import com.google.gson.JsonElement; import com.google.gson.JsonObject; import com.google.gson.reflect.TypeToken; +import com.linkedin.cdi.connection.HttpConnection; +import com.linkedin.cdi.extractor.MultistageExtractor; +import com.linkedin.cdi.keys.HttpKeys; +import com.linkedin.cdi.util.EncryptionUtils; import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.List; import java.util.Map; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.commons.codec.binary.Base64; import org.apache.gobblin.configuration.State; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.connection.HttpConnection; -import com.linkedin.cdi.extractor.MultistageExtractor; -import com.linkedin.cdi.keys.HttpKeys; -import com.linkedin.cdi.util.EncryptionUtils; import org.apache.gobblin.source.extractor.Extractor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; /** @@ -37,19 +37,24 @@ * * @author chrli */ -@Slf4j @SuppressWarnings("unchecked") public class HttpSource extends MultistageSource { + private static final Logger LOG = LoggerFactory.getLogger(HttpSource.class); private final static Gson GSON = new Gson(); private final static String BASIC_TOKEN_PREFIX = "Basic"; private final static String BEARER_TOKEN_PREFIX = "Bearer"; final static String OAUTH_TOKEN_PREFIX = "OAuth"; final static String TOKEN_PREFIX_SEPARATOR = " "; - @VisibleForTesting - - @Getter @Setter private HttpKeys httpSourceKeys; + public HttpKeys getHttpSourceKeys() { + return httpSourceKeys; + } + + public void setHttpSourceKeys(HttpKeys httpSourceKeys) { + this.httpSourceKeys = httpSourceKeys; + } + public HttpSource() { httpSourceKeys = new HttpKeys(); jobKeys = httpSourceKeys; @@ -57,10 +62,9 @@ public HttpSource() { protected void initialize(State state) { super.initialize(state); - httpSourceKeys.logUsage(state); httpSourceKeys.setHttpRequestHeaders(getRequestHeader(state)); - httpSourceKeys.setHttpRequestMethod(MultistageProperties.MSTAGE_HTTP_REQUEST_METHOD.getProp(state)); - httpSourceKeys.setAuthentication(MultistageProperties.MSTAGE_AUTHENTICATION.getValidNonblankWithDefault(state)); + httpSourceKeys.setHttpRequestMethod(MSTAGE_HTTP_REQUEST_METHOD.get(state)); + httpSourceKeys.setAuthentication(MSTAGE_AUTHENTICATION.get(state)); httpSourceKeys.setHttpRequestHeadersWithAuthentication(getHeadersWithAuthentication(state)); httpSourceKeys.setHttpStatuses(getHttpStatuses(state)); httpSourceKeys.setHttpStatusReasons(getHttpStatusReasons(state)); @@ -101,7 +105,7 @@ Map getAuthenticationHeader(State state) { String authMethod = httpSourceKeys.getAuthentication().get("method").getAsString(); if (!authMethod.toLowerCase().matches("basic|bearer|oauth|custom")) { - log.warn("Unsupported authentication type: " + authMethod); + LOG.warn("Unsupported authentication type: " + authMethod); return new HashMap<>(); } @@ -109,8 +113,8 @@ Map getAuthenticationHeader(State state) { if (httpSourceKeys.getAuthentication().has("token")) { token = EncryptionUtils.decryptGobblin(httpSourceKeys.getAuthentication().get("token").getAsString(), state); } else { - String u = EncryptionUtils.decryptGobblin(MultistageProperties.SOURCE_CONN_USERNAME.getProp(state), state); - String p = EncryptionUtils.decryptGobblin(MultistageProperties.SOURCE_CONN_PASSWORD.getProp(state), state); + String u = EncryptionUtils.decryptGobblin(SOURCE_CONN_USERNAME.get(state), state); + String p = EncryptionUtils.decryptGobblin(SOURCE_CONN_PASSWORD.get(state), state); token = u + ":" + p; } @@ -145,7 +149,7 @@ private Map toStringStringMap(JsonObject json) { private Map> getHttpStatuses(State state) { Map> statuses = new HashMap<>(); - JsonObject jsonObject = MultistageProperties.MSTAGE_HTTP_STATUSES.getValidNonblankWithDefault(state); + JsonObject jsonObject = MSTAGE_HTTP_STATUSES.get(state); for (Map.Entry entry: jsonObject.entrySet()) { String key = entry.getKey(); JsonElement value = jsonObject.get(key); @@ -158,7 +162,7 @@ private Map> getHttpStatuses(State state) { private Map> getHttpStatusReasons(State state) { Map> reasons = new HashMap<>(); - JsonObject jsonObject = MultistageProperties.MSTAGE_HTTP_STATUS_REASONS.getValidNonblankWithDefault(state); + JsonObject jsonObject = MSTAGE_HTTP_STATUS_REASONS.get(state); for (Map.Entry entry: jsonObject.entrySet()) { String key = entry.getKey(); JsonElement value = jsonObject.get(key); @@ -175,7 +179,7 @@ private Map> getHttpStatusReasons(State state) { * @return the decrypted http request headers */ private JsonObject getRequestHeader(State state) { - JsonObject headers = MultistageProperties.MSTAGE_HTTP_REQUEST_HEADERS.getValidNonblankWithDefault(state); + JsonObject headers = MSTAGE_HTTP_REQUEST_HEADERS.get(state); JsonObject decrypted = new JsonObject(); for (Map.Entry entry: headers.entrySet()) { String key = entry.getKey(); diff --git a/cdi-core/src/main/java/com/linkedin/cdi/source/JdbcSource.java b/cdi-core/src/main/java/com/linkedin/cdi/source/JdbcSource.java index 0a4082f..4c10e4f 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/source/JdbcSource.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/source/JdbcSource.java @@ -4,31 +4,46 @@ package com.linkedin.cdi.source; +import com.linkedin.cdi.connection.JdbcConnection; +import com.linkedin.cdi.extractor.MultistageExtractor; +import com.linkedin.cdi.keys.JdbcKeys; import java.sql.Connection; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.gobblin.configuration.State; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.connection.JdbcConnection; -import com.linkedin.cdi.extractor.MultistageExtractor; -import com.linkedin.cdi.keys.JdbcKeys; -import com.linkedin.cdi.util.CsvUtils; import org.apache.gobblin.source.extractor.Extractor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; /*** * JdbcSource handles JDBC protocol * */ - -@Slf4j public class JdbcSource extends MultistageSource { - @Setter + private static final Logger LOG = LoggerFactory.getLogger(JdbcSource.class); + + public ConcurrentMap getMemberConnections() { + return memberConnections; + } + + public void setMemberConnections(ConcurrentMap memberConnections) { + this.memberConnections = memberConnections; + } + + public JdbcKeys getJdbcSourceKeys() { + return jdbcSourceKeys; + } + + public void setJdbcSourceKeys(JdbcKeys jdbcSourceKeys) { + this.jdbcSourceKeys = jdbcSourceKeys; + } + private ConcurrentMap memberConnections = new ConcurrentHashMap<>(); private JdbcKeys jdbcSourceKeys = null; @@ -39,16 +54,9 @@ public JdbcSource() { protected void initialize(State state) { super.initialize(state); - jdbcSourceKeys.logUsage(state); - jdbcSourceKeys.setJdbcStatement(MultistageProperties.MSTAGE_JDBC_STATEMENT.getValidNonblankWithDefault(state)); - jdbcSourceKeys.setSeparator(CsvUtils.unescape(MultistageProperties.MSTAGE_CSV_SEPARATOR - .getValidNonblankWithDefault(state))); - jdbcSourceKeys.setQuoteCharacter(CsvUtils.unescape(MultistageProperties.MSTAGE_CSV_QUOTE_CHARACTER - .getValidNonblankWithDefault(state))); - jdbcSourceKeys.setEscapeCharacter(CsvUtils.unescape(MultistageProperties.MSTAGE_CSV_ESCAPE_CHARACTER - .getValidNonblankWithDefault(state))); - jdbcSourceKeys.setSchemaRefactorFunction(MultistageProperties.MSTAGE_JDBC_SCHEMA_REFACTOR - .getValidNonblankWithDefault(state)); + jdbcSourceKeys.setJdbcStatement(MSTAGE_JDBC_STATEMENT.get(state)); + jdbcSourceKeys.setSchemaRefactorFunction(MSTAGE_JDBC_SCHEMA_REFACTOR + .get(state)); jdbcSourceKeys.logDebugAll(); } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/source/MultistageSource.java b/cdi-core/src/main/java/com/linkedin/cdi/source/MultistageSource.java index 540c9af..350d69d 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/source/MultistageSource.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/source/MultistageSource.java @@ -11,6 +11,11 @@ import com.google.gson.JsonArray; import com.google.gson.JsonElement; import com.google.gson.JsonObject; +import com.linkedin.cdi.extractor.MultistageExtractor; +import com.linkedin.cdi.keys.JobKeys; +import com.linkedin.cdi.util.EndecoUtils; +import com.linkedin.cdi.util.HdfsReader; +import com.linkedin.cdi.util.WatermarkDefinition; import java.lang.reflect.Constructor; import java.util.ArrayList; import java.util.Collections; @@ -20,23 +25,12 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import lombok.AccessLevel; -import lombok.Getter; -import lombok.Setter; -import lombok.SneakyThrows; -import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.gobblin.configuration.ConfigurationKeys; import org.apache.gobblin.configuration.SourceState; import org.apache.gobblin.configuration.State; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.extractor.MultistageExtractor; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.util.EndecoUtils; -import com.linkedin.cdi.util.HdfsReader; -import com.linkedin.cdi.util.WatermarkDefinition; import org.apache.gobblin.source.extractor.Extractor; import org.apache.gobblin.source.extractor.WatermarkInterval; import org.apache.gobblin.source.extractor.extract.AbstractSource; @@ -44,8 +38,11 @@ import org.apache.gobblin.source.workunit.Extract; import org.apache.gobblin.source.workunit.WorkUnit; import org.joda.time.DateTime; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.Assert; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static com.linkedin.cdi.configuration.StaticConstants.*; @@ -70,10 +67,9 @@ * @param The schema class * @param The data class */ - -@Slf4j @SuppressWarnings("unchecked") public class MultistageSource extends AbstractSource { + private static final Logger LOG = LoggerFactory.getLogger(MultistageSource.class); final static private Gson GSON = new Gson(); final static private String PROPERTY_SEPARATOR = "."; final static private String DUMMY_DATETIME_WATERMARK_START = "2019-01-01"; @@ -82,14 +78,24 @@ public class MultistageSource extends AbstractSource { // Avoid too many partition created from misconfiguration, Months * Days * Hours final private static int MAX_DATETIME_PARTITION = 3 * 30 * 24; - @Getter(AccessLevel.PUBLIC) - @Setter(AccessLevel.MODULE) protected SourceState sourceState = null; - @Getter(AccessLevel.PUBLIC) - @Setter(AccessLevel.PUBLIC) JobKeys jobKeys = new JobKeys(); - @Getter(AccessLevel.PUBLIC) - @Setter(AccessLevel.MODULE) + + public SourceState getSourceState() { + return sourceState; + } + + public void setSourceState(SourceState sourceState) { + this.sourceState = sourceState; + } + + public JobKeys getJobKeys() { + return jobKeys; + } + + public void setJobKeys(JobKeys jobKeys) { + this.jobKeys = jobKeys; + } final private ConcurrentHashMap, WorkUnitState> extractorState = new ConcurrentHashMap<>(); @@ -102,13 +108,17 @@ protected void initialize(State state) { * getWorkUnits() is the first place to receive the Source State object, therefore * initialization of parameters cannot be complete in constructor. */ - @SneakyThrows @Override public List getWorkunits(SourceState state) { sourceState = state; initialize(state); + jobKeys.logUsage(state); if (!jobKeys.validate(state)) { + LOG.error("Some parameters are invalid, job will do nothing until they are fixed."); + if (MSTAGE_WORK_UNIT_MIN_UNITS.get(state) > 0) { + throw new RuntimeException(String.format(EXCEPTION_WORK_UNIT_MINIMUM, MSTAGE_WORK_UNIT_MIN_UNITS.get(state), 0)); + } return new ArrayList<>(); } @@ -143,14 +153,22 @@ public List getWorkunits(SourceState state) { // generated work units based on watermarks defined and previous high watermarks List wuList = generateWorkUnits(definedWatermarks, previousHighWatermarks); + + // abort (fail) the job when the number of work units is below require threshold + if (wuList.size() < jobKeys.getMinWorkUnits()) { + throw new RuntimeException(String.format(EXCEPTION_WORK_UNIT_MINIMUM, + jobKeys.getMinWorkUnits(), + jobKeys.getMinWorkUnits())); + } + if (authentications != null && authentications.size() == 1) { for (WorkUnit wu : wuList) { - wu.setProp(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.toString(), + wu.setProp(MSTAGE_ACTIVATION_PROPERTY.toString(), getUpdatedWorkUnitActivation(wu, authentications.get(0).getAsJsonObject())); // unlike activation secondary inputs, payloads will be processed in each work unit // and payloads will not be loaded until the Connection executes the command - wu.setProp(MultistageProperties.MSTAGE_PAYLOAD_PROPERTY.toString(), payloads); + wu.setProp(MSTAGE_PAYLOAD_PROPERTY.toString(), payloads); } } return wuList; @@ -161,21 +179,24 @@ public List getWorkunits(SourceState state) { * In case the token is missing, it will retry accessing the tokens as per the retry parameters * ("delayInSec", "retryCount") */ - private Map readSecondaryInputs(State state, final long retries) - throws InterruptedException { - log.info("Trying to read secondary input with retry = {}", retries); + private Map readSecondaryInputs(State state, final long retries) { + LOG.info("Trying to read secondary input with retry = {}", retries); Map secondaryInputs = readContext(state); // Check if authentication is ready, and if not, whether retry is required JsonArray authentications = secondaryInputs.get(KEY_WORD_AUTHENTICATION); if ((authentications == null || authentications.size() == 0) && jobKeys.getIsSecondaryAuthenticationEnabled() && retries > 0) { - log.info("Authentication tokens are expected from secondary input, but not ready"); - log.info("Will wait for {} seconds and then retry reading the secondary input", jobKeys.getRetryDelayInSec()); - TimeUnit.SECONDS.sleep(jobKeys.getRetryDelayInSec()); + LOG.info("Authentication tokens are expected from secondary input, but not ready"); + LOG.info("Will wait for {} seconds and then retry reading the secondary input", jobKeys.getRetryDelayInSec()); + try { + TimeUnit.SECONDS.sleep(jobKeys.getRetryDelayInSec()); + } catch (Exception e) { + throw new RuntimeException("Sleep() interrupted", e); + } return readSecondaryInputs(state, retries - 1); } - log.info("Successfully read secondary input, no more retry"); + LOG.info("Successfully read secondary input, no more retry"); return secondaryInputs; } @@ -188,7 +209,7 @@ private Map readSecondaryInputs(State state, final long retri public Extractor getExtractor(WorkUnitState state) { try { ClassLoader loader = this.getClass().getClassLoader(); - Class extractorClass = loader.loadClass(MultistageProperties.MSTAGE_EXTRACTOR_CLASS.getValidNonblankWithDefault(state)); + Class extractorClass = loader.loadClass(MSTAGE_EXTRACTOR_CLASS.get(state)); Constructor> constructor = (Constructor>) extractorClass.getConstructor(WorkUnitState.class, JobKeys.class); MultistageExtractor extractor = (MultistageExtractor) constructor.newInstance(state, this.jobKeys); @@ -206,7 +227,7 @@ public Extractor getExtractor(WorkUnitState state) { */ @Override public void shutdown(SourceState state) { - log.info("MultistageSource Shutdown() called, instructing extractors to close connections"); + LOG.info("MultistageSource Shutdown() called, instructing extractors to close connections"); for (MultistageExtractor extractor: extractorState.keySet()) { extractor.closeConnection(); } @@ -234,6 +255,13 @@ List generateWorkUnits(List definitions, Map 0) { + throw new RuntimeException(String.format(EXCEPTION_WORK_UNIT_MINIMUM, + jobKeys.getMinWorkUnits(), + jobKeys.getMinWorkUnits())); + } + JsonArray unitArray = new JsonArray(); unitArray.add(new JsonObject()); unitWatermark = new WatermarkDefinition("unit", unitArray); @@ -258,19 +286,19 @@ List generateWorkUnits(List definitions, Map dtPartition : datetimePartitions) { - log.debug("dtPartition: {}", dtPartition); + LOG.debug("dtPartition: {}", dtPartition); for (String unitPartition: unitPartitions) { // adding the date time partition and unit partition combination to work units until // it reaches ms.work.unit.parallelism.max. a combination is not added if its prior // watermark doesn't require a rerun. // a work unit signature is a date time partition and unit partition combination. - if (MultistageProperties.MSTAGE_WORK_UNIT_PARALLELISM_MAX.validateNonblank(sourceState) - && workUnits.size() >= (Integer) MultistageProperties.MSTAGE_WORK_UNIT_PARALLELISM_MAX.getProp(sourceState)) { + if (MSTAGE_WORK_UNIT_PARALLELISM_MAX.isValidNonblank(sourceState) + && workUnits.size() >= (Integer) MSTAGE_WORK_UNIT_PARALLELISM_MAX.get(sourceState)) { break; } @@ -279,16 +307,16 @@ List generateWorkUnits(List definitions, Map generateWorkUnits(List definitions, Map= Longs.max(unitCutoffTime, cutoffTime)) { - // prune the date range only if it is not partitioned + // prune the date range only if the unit is not in first execution // note the nominal date range low boundary had been saved in signature - ImmutablePair dtPartitionModified = dtPartition; - if (datetimePartitions.size() == 1 && dtPartition.left < cutoffTime) { - dtPartitionModified = new ImmutablePair<>(cutoffTime, dtPartition.right); - } - log.debug("dtPartitionModified: {}", dtPartitionModified); + ImmutablePair dtPartitionModified = unitCutoffTime == -1L + ? dtPartition : previousHighWatermarks.get(wuSignature).equals(dtPartition.left) + ? dtPartition : new ImmutablePair<>(Long.max(unitCutoffTime, dtPartition.left), dtPartition.right); - log.info("Generating Work Unit: {}, watermark: {}", wuSignature, dtPartitionModified); + LOG.info(String.format(MSG_WORK_UNIT_INFO, wuSignature, dtPartitionModified)); WorkUnit workUnit = WorkUnit.create(extract, new WatermarkInterval( new LongWatermark(dtPartitionModified.getLeft()), @@ -313,33 +339,33 @@ List generateWorkUnits(List definitions, Map> getDatetimePartitions(ImmutablePair> partitions = Lists.newArrayList(); if (jobKeys.getWorkUnitPartitionType() != null) { partitions = jobKeys.getWorkUnitPartitionType().getRanges(datetimeRange, - MultistageProperties.MSTAGE_WORK_UNIT_PARTIAL_PARTITION.getValidNonblankWithDefault(sourceState)); + MSTAGE_WORK_UNIT_PARTIAL_PARTITION.get(sourceState)); } else { partitions.add(new ImmutablePair<>(datetimeRange.getLeft().getMillis(), datetimeRange.getRight().getMillis())); } @@ -367,7 +393,7 @@ private List> getDatetimePartitions(ImmutablePair MAX_DATETIME_PARTITION) { // Preserve the last N partitions partitions = partitions.subList(partitions.size() - MAX_DATETIME_PARTITION, partitions.size()); - log.warn("Too many partitions, created {}, only processing the last {}", partitions.size(), MAX_DATETIME_PARTITION); + LOG.warn("Too many partitions, created {}, only processing the last {}", partitions.size(), MAX_DATETIME_PARTITION); } return partitions; } @@ -428,7 +454,7 @@ private Map getPreviousHighWatermarks() { // Unit watermarks might contain encoded file separator, // in such case, we will decode the watermark name so that it can be compared with // work unit signatures - log.debug("Dataset Signature: {}, High Watermark: {}", EndecoUtils.getHadoopFsDecoded(entry.getKey()), highWatermark); + LOG.debug("Dataset Signature: {}, High Watermark: {}", EndecoUtils.getHadoopFsDecoded(entry.getKey()), highWatermark); watermarks.put(EndecoUtils.getHadoopFsDecoded(entry.getKey()), highWatermark); } return ImmutableMap.copyOf(watermarks); @@ -436,9 +462,9 @@ private Map getPreviousHighWatermarks() { Extract createExtractObject(final boolean isFull) { Extract extract = createExtract( - Extract.TableType.valueOf(MultistageProperties.EXTRACT_TABLE_TYPE_KEY.getValidNonblankWithDefault(sourceState)), - MultistageProperties.EXTRACT_NAMESPACE_NAME_KEY.getProp(sourceState), - MultistageProperties.EXTRACT_TABLE_NAME_KEY.getProp(sourceState)); + Extract.TableType.valueOf(EXTRACT_TABLE_TYPE.get(sourceState)), + EXTRACT_NAMESPACE.get(sourceState), + EXTRACT_TABLE_NAME.get(sourceState)); extract.setProp(ConfigurationKeys.EXTRACT_IS_FULL_KEY, isFull); return extract; } @@ -459,8 +485,7 @@ private String getWorkUnitSignature( * @param retries number of retries remaining * @return the authentication JsonObject */ - @SneakyThrows - protected JsonObject readSecondaryAuthentication(State state, final long retries) { + protected JsonObject readSecondaryAuthentication(State state, final long retries) throws InterruptedException { Map secondaryInputs = readSecondaryInputs(state, retries); if (secondaryInputs.containsKey(KEY_WORD_ACTIVATION) && secondaryInputs.get(KEY_WORD_AUTHENTICATION).isJsonArray() @@ -478,17 +503,17 @@ protected JsonObject readSecondaryAuthentication(State state, final long retries * @return the updated work unit configuration */ protected String getUpdatedWorkUnitActivation(WorkUnit wu, JsonObject authentication) { - log.debug("Activation property (origin): {}", wu.getProp(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.toString(), "")); - if (!wu.getProp(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.toString(), StringUtils.EMPTY).isEmpty()) { - JsonObject existing = GSON.fromJson(wu.getProp(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.toString()), JsonObject.class); + LOG.debug("Activation property (origin): {}", wu.getProp(MSTAGE_ACTIVATION_PROPERTY.toString(), "")); + if (!wu.getProp(MSTAGE_ACTIVATION_PROPERTY.toString(), StringUtils.EMPTY).isEmpty()) { + JsonObject existing = GSON.fromJson(wu.getProp(MSTAGE_ACTIVATION_PROPERTY.toString()), JsonObject.class); for (Map.Entry entry: authentication.entrySet()) { existing.remove(entry.getKey()); existing.addProperty(entry.getKey(), entry.getValue().getAsString()); } - log.debug("Activation property (modified): {}", existing.toString()); + LOG.debug("Activation property (modified): {}", existing.toString()); return existing.toString(); } - log.debug("Activation property (new): {}", authentication.toString()); + LOG.debug("Activation property (new): {}", authentication.toString()); return authentication.toString(); } @@ -499,12 +524,12 @@ protected String getUpdatedWorkUnitActivation(WorkUnit wu, JsonObject authentica * @return true if all conditions met for a full extract, otherwise false */ private boolean checkFullExtractState(final State state, final Map previousHighWatermarks) { - if (MultistageProperties.EXTRACT_TABLE_TYPE_KEY.getValidNonblankWithDefault(state).toString() + if (EXTRACT_TABLE_TYPE.get(state).toString() .equalsIgnoreCase(KEY_WORD_SNAPSHOT_ONLY)) { return true; } - if (MultistageProperties.MSTAGE_ENABLE_DYNAMIC_FULL_LOAD.getValidNonblankWithDefault(state)) { + if (MSTAGE_ENABLE_DYNAMIC_FULL_LOAD.get(state)) { if (previousHighWatermarks.isEmpty()) { return true; } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/source/S3SourceV2.java b/cdi-core/src/main/java/com/linkedin/cdi/source/S3SourceV2.java index 2a3405b..7b3796a 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/source/S3SourceV2.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/source/S3SourceV2.java @@ -8,50 +8,64 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.gson.JsonObject; +import com.linkedin.cdi.connection.S3Connection; +import com.linkedin.cdi.extractor.MultistageExtractor; +import com.linkedin.cdi.keys.S3Keys; +import java.net.URL; import java.util.HashSet; import java.util.List; import java.util.stream.Collectors; -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; -import okhttp3.HttpUrl; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.gobblin.configuration.State; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.connection.S3Connection; -import com.linkedin.cdi.extractor.MultistageExtractor; -import com.linkedin.cdi.keys.S3Keys; -import com.linkedin.cdi.util.EndecoUtils; import org.apache.gobblin.source.extractor.Extractor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import software.amazon.awssdk.regions.Region; +import static com.linkedin.cdi.configuration.PropertyCollection.*; + -@Slf4j public class S3SourceV2 extends MultistageSource { + private static final Logger LOG = LoggerFactory.getLogger(S3SourceV2.class); private static final String KEY_REGION = "region"; private static final String KEY_CONNECTION_TIMEOUT = "connection_timeout"; private static final HashSet S3_REGIONS_SET = Region.regions().stream().map(region -> region.toString()).collect(Collectors.toCollection(HashSet::new)); - @Getter private S3Keys s3SourceV2Keys = new S3Keys(); + public S3Keys getS3SourceV2Keys() { + return s3SourceV2Keys; + } + + public void setS3SourceV2Keys(S3Keys s3SourceV2Keys) { + this.s3SourceV2Keys = s3SourceV2Keys; + } + public S3SourceV2() { s3SourceV2Keys = new S3Keys(); jobKeys = s3SourceV2Keys; } protected void initialize(State state) { super.initialize(state); - s3SourceV2Keys.logUsage(state); - HttpUrl url = HttpUrl.parse(MultistageProperties.MSTAGE_SOURCE_URI.getValidNonblankWithDefault(state)); - if (url == null || url.host().isEmpty()) { + + URL url = null; + try { + String sourceUri = MSTAGE_SOURCE_URI.get(state); + url = new URL(sourceUri.replaceAll("(s3|S3)://", "https://")); + } catch (Exception e) { + throw new RuntimeException(e); + } + + if (url == null || url.getHost().isEmpty()) { throw new RuntimeException("Incorrect configuration in " + - MultistageProperties.MSTAGE_SOURCE_URI.toString()); + MSTAGE_SOURCE_URI.toString()); } // set region, note that aws SDK won't raise an error here if region is invalid, // later on, an exception will be raised when the actual request is issued - JsonObject parameters = MultistageProperties.MSTAGE_SOURCE_S3_PARAMETERS.getValidNonblankWithDefault(state); + JsonObject parameters = MSTAGE_SOURCE_S3_PARAMETERS.get(state); if (parameters.has(KEY_REGION)) { String region = parameters.get(KEY_REGION).getAsString(); if (!S3_REGIONS_SET.contains(region)) { @@ -73,21 +87,18 @@ protected void initialize(State state) { } // separate the endpoint, which should be a URL without bucket name, from the domain name - s3SourceV2Keys.setEndpoint("https://" + getEndpointFromHost(url.host())); - - // URL path might have variables, by default HttpUrl will encode '{' and '}' - // Here we decode those back to their plain form - s3SourceV2Keys.setPrefix(EndecoUtils.decode(url.encodedPath().substring(1))); + s3SourceV2Keys.setEndpoint("https://" + getEndpointFromHost(url.getHost())); + s3SourceV2Keys.setPrefix(url.getPath().substring(1)); // separate the bucket name from URI domain name - s3SourceV2Keys.setBucket(url.host().split("\\.")[0]); + s3SourceV2Keys.setBucket(url.getHost().split("\\.")[0]); - s3SourceV2Keys.setFilesPattern(MultistageProperties.MSTAGE_SOURCE_FILES_PATTERN.getProp(state)); - s3SourceV2Keys.setMaxKeys(MultistageProperties.MSTAGE_S3_LIST_MAX_KEYS.getValidNonblankWithDefault(state)); - s3SourceV2Keys.setAccessKey(MultistageProperties.SOURCE_CONN_USERNAME.getValidNonblankWithDefault(state)); - s3SourceV2Keys.setSecretId(MultistageProperties.SOURCE_CONN_PASSWORD.getValidNonblankWithDefault(state)); + s3SourceV2Keys.setFilesPattern(MSTAGE_SOURCE_FILES_PATTERN.get(state)); + s3SourceV2Keys.setMaxKeys(MSTAGE_S3_LIST_MAX_KEYS.get(state)); + s3SourceV2Keys.setAccessKey(SOURCE_CONN_USERNAME.get(state)); + s3SourceV2Keys.setSecretId(SOURCE_CONN_PASSWORD.get(state)); s3SourceV2Keys.setTargetFilePattern( - MultistageProperties.MSTAGE_EXTRACTOR_TARGET_FILE_NAME.getValidNonblankWithDefault(state)); + MSTAGE_EXTRACTOR_TARGET_FILE_NAME.get(state)); s3SourceV2Keys.logDebugAll(); } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/source/SftpSource.java b/cdi-core/src/main/java/com/linkedin/cdi/source/SftpSource.java new file mode 100644 index 0000000..927fdaf --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/source/SftpSource.java @@ -0,0 +1,94 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.source; + +import com.linkedin.cdi.connection.SftpConnection; +import com.linkedin.cdi.extractor.MultistageExtractor; +import com.linkedin.cdi.keys.SftpKeys; +import com.linkedin.cdi.util.VariableUtils; +import java.net.URI; +import java.net.URISyntaxException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.configuration.State; +import org.apache.gobblin.configuration.WorkUnitState; +import org.apache.gobblin.source.extractor.Extractor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; + + +/** + * Source class to handle sftp protocol + */ +public class SftpSource extends MultistageSource { + private static final Logger LOG = LoggerFactory.getLogger(SftpSource.class); + SftpKeys sftpSourceKeys; + + public SftpKeys getSftpSourceKeys() { + return sftpSourceKeys; + } + + public void setSftpSourceKeys(SftpKeys sftpSourceKeys) { + this.sftpSourceKeys = sftpSourceKeys; + } + + public SftpSource() { + sftpSourceKeys = new SftpKeys(); + jobKeys = sftpSourceKeys; + } + + protected void initialize(State state) { + super.initialize(state); + this.parseUri(state); + sftpSourceKeys.setFilesPattern(MSTAGE_SOURCE_FILES_PATTERN.get(state)); + sftpSourceKeys.setTargetFilePattern( + MSTAGE_EXTRACTOR_TARGET_FILE_NAME.get(state)); + sftpSourceKeys.logDebugAll(); + } + + /** + * Create extractor based on the input WorkUnitState, the extractor.class + * configuration, and a new SftpConnection + * + * @param state WorkUnitState passed in from Gobblin framework + * @return the MultistageExtractor object + */ + @Override + public Extractor getExtractor(WorkUnitState state) { + initialize(state); + MultistageExtractor extractor = + (MultistageExtractor) super.getExtractor(state); + extractor.setConnection(new SftpConnection(state, this.sftpSourceKeys, extractor.getExtractorKeys())); + return extractor; + } + + /** + * This method parses ms.source.uri, following are the examples of Valid URIs + * Valid[recommended] sftp://somehost.com:22/a/b/*c*.csv + * Valid[Supported for backward compatibility] : /a/b/*c*.csv + */ + private void parseUri(State state) { + String sourceUri = MSTAGE_SOURCE_URI.get(state); + if (VariableUtils.hasVariable(sourceUri)) { + sftpSourceKeys.setFilesPath(sourceUri); + } else { + try { + URI uri = new URI(sourceUri); + if (uri.getHost() != null) { + state.setProp(ConfigurationKeys.SOURCE_CONN_HOST_NAME, uri.getHost()); + } + if (uri.getPort() != -1) { + state.setProp(ConfigurationKeys.SOURCE_CONN_PORT, uri.getPort()); + } + sftpSourceKeys.setFilesPath(uri.getPath()); + } catch (URISyntaxException e) { + LOG.warn("Invalid URI format in ms.source.uri", e); + } + } + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/AvroSchemaUtils.java b/cdi-core/src/main/java/com/linkedin/cdi/util/AvroSchemaUtils.java index 35a16c9..33e7d80 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/AvroSchemaUtils.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/AvroSchemaUtils.java @@ -22,14 +22,18 @@ public interface AvroSchemaUtils { * Utility method to convert JsonArray schema to avro schema * @param schema of JsonArray type * @return avro schema - * @throws UnsupportedDateTypeException + * @throws UnsupportedDateTypeException unsupported type */ - static Schema fromJsonSchema(JsonArray schema, WorkUnitState state) throws UnsupportedDateTypeException { + static Schema fromJsonSchema(JsonArray schema, WorkUnitState state) { JsonSchema jsonSchema = new JsonSchema(schema); jsonSchema.setColumnName(state.getExtract().getTable()); - JsonElementConversionFactory.RecordConverter recordConverter = - new JsonElementConversionFactory.RecordConverter(jsonSchema, state, state.getExtract().getNamespace()); - return recordConverter.schema(); + try { + JsonElementConversionFactory.RecordConverter recordConverter = + new JsonElementConversionFactory.RecordConverter(jsonSchema, state, state.getExtract().getNamespace()); + return recordConverter.schema(); + } catch (Exception e) { + throw new RuntimeException(e.getMessage(), e); + } } /** @@ -62,15 +66,20 @@ static GenericRecord createEOF(WorkUnitState state) { JsonArray eofSchema = new Gson() .fromJson("[{\"columnName\":\"EOF\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}]", JsonArray.class); - Schema schema = null; - try { - schema = fromJsonSchema(eofSchema, state); - } catch (UnsupportedDateTypeException e) { - // impossible, since the schema is fixed here and string type is supported - } + Schema schema = fromJsonSchema(eofSchema, state); assert (schema != null); GenericRecord eofRecord = new GenericData.Record(schema); eofRecord.put("EOF", "EOF"); return eofRecord; } + + /** + * Makes a deep copy of a value given its schema. + * @param schema the schema of the value to deep copy. + * @param value the value to deep copy. + * @return a deep copy of the given value. + */ + static T deepCopy(Schema schema, T value) { + return GenericData.get().deepCopy(schema, value); + } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/Credentials.java b/cdi-core/src/main/java/com/linkedin/cdi/util/Credentials.java new file mode 100644 index 0000000..a02136d --- /dev/null +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/Credentials.java @@ -0,0 +1,60 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.util; + +import org.apache.gobblin.configuration.ConfigurationKeys; +import org.apache.gobblin.configuration.State; +import org.apache.gobblin.password.PasswordManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class Credentials { + private static final Logger LOG = LoggerFactory.getLogger(Credentials.class); + + public static String getPrivateKey(State state) { + return PasswordManager.getInstance(state) + .readPassword(state.getProp(ConfigurationKeys.SOURCE_CONN_PRIVATE_KEY)); + } + + public static String getPassword(State state) { + return PasswordManager.getInstance(state) + .readPassword(state.getProp(ConfigurationKeys.SOURCE_CONN_PASSWORD)); + } + + public static String getKnownHosts(State state) { + return state.getProp(ConfigurationKeys.SOURCE_CONN_KNOWN_HOSTS); + } + + public static String getUserName(State state) { + return state.getProp(ConfigurationKeys.SOURCE_CONN_USERNAME); + } + + public static String getHostName(State state) { + return state.getProp(ConfigurationKeys.SOURCE_CONN_HOST_NAME); + } + + public static int getPort(State state) { + return state.getPropAsInt(ConfigurationKeys.SOURCE_CONN_PORT, ConfigurationKeys.SOURCE_CONN_DEFAULT_PORT); + } + + public static String getProxyHost(State state) { + return state.getProp(ConfigurationKeys.SOURCE_CONN_USE_PROXY_URL); + } + + public static int getProxyPort(State state) { + return state.getPropAsInt(ConfigurationKeys.SOURCE_CONN_USE_PROXY_PORT, -1); + } + + public static void log(State state) { + LOG.info("privateKey: {}", getPrivateKey(state)); + LOG.info("knownHosts: {}", getKnownHosts(state)); + LOG.info("userName: {}", getUserName(state)); + LOG.info("hostName: {}", getHostName(state)); + LOG.info("port: {}", getPort(state)); + LOG.info("proxyHost: {}" , getProxyHost(state)); + LOG.info("proxyPort: {}", getProxyPort(state)); + } +} diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/Database.java b/cdi-core/src/main/java/com/linkedin/cdi/util/Database.java index 7100eba..c2368e3 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/Database.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/Database.java @@ -6,25 +6,37 @@ import com.google.common.base.Preconditions; import java.net.URI; -import lombok.Getter; -import lombok.extern.slf4j.Slf4j; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * a list of databases */ -@Slf4j public enum Database { MYSQL("MySql", "com.mysql.cj.jdbc.Driver"), SQLSERVER("SqlServer", "com.microsoft.sqlserver.jdbc.SQLServerDriver"), ORACLE("Oracle", "oracle.jdbc.driver.OracleDriver"), HSQLDB("HSqlDb", "org.hsqldb.jdbcDriver"); + private static final Logger LOG = LoggerFactory.getLogger(Database.class); final static String PROTOCOL_PREFIX = "jdbc:"; - @Getter private String name; - @Getter private String dbType; - @Getter private String defaultDriver; + private String name; + private String dbType; + private String defaultDriver; + + public String getName() { + return name; + } + + public String getDbType() { + return dbType; + } + + public String getDefaultDriver() { + return defaultDriver; + } Database(String name, String driver) { this.name = name; diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/EndecoUtils.java b/cdi-core/src/main/java/com/linkedin/cdi/util/EndecoUtils.java index 0105285..64679cf 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/EndecoUtils.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/EndecoUtils.java @@ -7,15 +7,16 @@ import java.net.URLDecoder; import java.net.URLEncoder; import java.nio.charset.StandardCharsets; -import lombok.extern.slf4j.Slf4j; import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * a collection of encoding and decoding functions */ -@Slf4j public class EndecoUtils { + private static final Logger LOG = LoggerFactory.getLogger(EndecoUtils.class); private EndecoUtils() { // hide constructor } @@ -33,7 +34,7 @@ static public String decode(String encoded, String enc) { try { return URLDecoder.decode(encoded, enc); } catch (Exception e) { - log.error("URL decoding error: " + e); + LOG.error("URL decoding error: " + e); return encoded; } } @@ -51,7 +52,7 @@ static public String getEncodedUtf8(String plainUrl, String enc) { try { return URLEncoder.encode(plainUrl, enc); } catch (Exception e) { - log.error("URL encoding error: " + e); + LOG.error("URL encoding error: " + e); return plainUrl; } } @@ -71,7 +72,7 @@ static public String getHadoopFsEncoded(String fileName, String enc) { // we don't encode the whole string intentionally so that the state file name is more readable return fileName.replace(Path.SEPARATOR, encodedSeparator); } catch (Exception e) { - log.error("Hadoop FS encoding error: " + e); + LOG.error("Hadoop FS encoding error: " + e); return fileName; } } @@ -90,7 +91,7 @@ static public String getHadoopFsDecoded(String encodedFileName, String enc) { String encodedSeparator = URLEncoder.encode(Path.SEPARATOR, enc); return encodedFileName.replace(encodedSeparator, Path.SEPARATOR); } catch (Exception e) { - log.error("Hadoop FS decoding error: " + e); + LOG.error("Hadoop FS decoding error: " + e); return encodedFileName; } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/JsonIntermediateSchema.java b/cdi-core/src/main/java/com/linkedin/cdi/util/JsonIntermediateSchema.java index 453cf64..6e0cbbd 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/JsonIntermediateSchema.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/JsonIntermediateSchema.java @@ -12,9 +12,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import lombok.Getter; -import lombok.NonNull; -import lombok.Setter; import static com.linkedin.cdi.configuration.StaticConstants.*; @@ -44,23 +41,55 @@ public class JsonIntermediateSchema { public static final String CHILD_RECORD_COLUMN_NAME = "child"; // LinkedHashMap maintains insertion order, so the key set will be in the same order as the output schema - @Getter Map columns = new LinkedHashMap<>(); - - @Getter @Setter String schemaName; + public Map getColumns() { + return columns; + } + + public void setColumns(Map columns) { + this.columns = columns; + } + + public String getSchemaName() { + return schemaName; + } + + public void setSchemaName(String schemaName) { + this.schemaName = schemaName; + } + // a JIS schema contains JIS columns public class JisColumn { - @NonNull @Getter @Setter String columnName; - - @Getter @Setter Boolean isNullable; - - @Getter @Setter JisDataType dataType; + public String getColumnName() { + return columnName; + } + + public void setColumnName(String columnName) { + this.columnName = columnName; + } + + public Boolean getIsNullable() { + return isNullable; + } + + public void setIsNullable(Boolean nullable) { + isNullable = nullable; + } + + public JisDataType getDataType() { + return dataType; + } + + public void setDataType(JisDataType dataType) { + this.dataType = dataType; + } + // define a simple column JisColumn(String name, Boolean isNullable, String type) { this.setColumnName(name); @@ -107,28 +136,70 @@ public JsonObject toJson() { // a JIS Column has a JIS Data Type public class JisDataType { - @NonNull @Getter @Setter JsonElementTypes type; // data type name is optional - @Getter @Setter String name; // values have the array of field definitions when the type is record - @Getter @Setter JsonIntermediateSchema childRecord; // items have the item definition - @Getter @Setter JisDataType itemType; // unions have item types - @Getter @Setter List itemTypes = Lists.newArrayList(); - @Getter @Setter JsonArray symbols; + public JsonElementTypes getType() { + return type; + } + + public void setType(JsonElementTypes type) { + this.type = type; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public JsonIntermediateSchema getChildRecord() { + return childRecord; + } + + public void setChildRecord(JsonIntermediateSchema childRecord) { + this.childRecord = childRecord; + } + + public JisDataType getItemType() { + return itemType; + } + + public void setItemType(JisDataType itemType) { + this.itemType = itemType; + } + + public List getItemTypes() { + return itemTypes; + } + + public void setItemTypes(List itemTypes) { + this.itemTypes = itemTypes; + } + + public JsonArray getSymbols() { + return symbols; + } + + public void setSymbols(JsonArray symbols) { + this.symbols = symbols; + } + // this defines primitive data type JisDataType(String type) { this.setType(JsonElementTypes.forType(type)); diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/JsonParameter.java b/cdi-core/src/main/java/com/linkedin/cdi/util/JsonParameter.java index aba18a6..c2b2a83 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/JsonParameter.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/JsonParameter.java @@ -11,13 +11,15 @@ import com.google.gson.JsonObject; import java.util.HashMap; import java.util.Map; -import lombok.extern.slf4j.Slf4j; import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; /** @@ -27,9 +29,8 @@ * * @author chrli */ -@Slf4j public class JsonParameter { - + private static final Logger LOG = LoggerFactory.getLogger(JsonParameter.class); private JsonObject paramJson; private State state; final private static String DEFAULT_TIMEZONE = "America/Los_Angeles"; @@ -180,7 +181,7 @@ private JsonObject parseParameter(JsonObject paramObject, JsonObject values) { Long watermarkLow = values.get("watermark").getAsJsonObject().get("low").getAsLong(); Long watermarkHigh = values.get("watermark").getAsJsonObject().get("high").getAsLong(); - log.debug("found watermark pair: {}, {} in replacement values.", watermarkLow, watermarkHigh); + LOG.debug("found watermark pair: {}, {} in replacement values.", watermarkLow, watermarkHigh); // ignore default watermarks if (watermarkLow < 0) { @@ -289,7 +290,7 @@ private String parseListParameter(JsonElement listValue, State state) { if (listValue.isJsonPrimitive()) { listValueString = listValue.getAsString(); } else if (listValue.isJsonArray() && listValue.getAsJsonArray().size() > 0) { - if (MultistageProperties.EXTRACT_IS_FULL.getValidNonblankWithDefault(state)) { + if (EXTRACT_IS_FULL.get(state)) { listValueString = listValue.getAsJsonArray().get(0).getAsString(); } else { listValueString = listValue.getAsJsonArray().size() > 1 @@ -298,7 +299,7 @@ private String parseListParameter(JsonElement listValue, State state) { } } else { listValueString = ""; - log.warn("Unable to parse LIST parameter {}, will use a BLANK string", listValue.toString()); + LOG.warn("Unable to parse LIST parameter {}, will use a BLANK string", listValue.toString()); } return listValueString; } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/JsonUtils.java b/cdi-core/src/main/java/com/linkedin/cdi/util/JsonUtils.java index be7f9cb..8744a82 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/JsonUtils.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/JsonUtils.java @@ -13,6 +13,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.commons.lang.StringUtils; import org.testng.Assert; @@ -158,6 +159,16 @@ static JsonElement get(JsonObject row, String jsonPath) { return get(jsonPath, row); } + /** + * Check if a JsonElement is available in a JsonObject given a JsonPath + * @param row the record + * @param jsonPath the JsonPath (string) how to get the data element + * @return true if present, false otherwise + */ + static boolean has(JsonObject row, String jsonPath) { + return !get(jsonPath, row).isJsonNull(); + } + /** * Get a JsonElement from a JsonObject based on the given JsonPath * @@ -229,4 +240,63 @@ static JsonElement get(Iterator jsonPath, JsonArray nested) { return JsonNull.INSTANCE; } } + + /** + * From an array of JsonObjects, filter by searching by key-value pair, and + * once the JsonObject is found, it returns the element located by the JsonPath, + * specified by returnKey, or the whole record if no returnKey is specified + * + * This function doesn't deepCopy the returned elements to avoid allocating extra + * spaces. + * + * @param searchKey key name to search in order to identify the JsonObject + * @param value value to match in order to identify the JsonObject + * @param objArray the array of JsonObjects + * @param returnKey the Json path to identify the return value + * @return the filtered elements within the array + */ + static JsonArray filter(final String searchKey, final String value, final JsonArray objArray, final String returnKey) { + JsonArray output = new JsonArray(); + for (JsonElement element: objArray) { + if (element.isJsonObject() + && element.getAsJsonObject().has(searchKey) + && element.getAsJsonObject().get(searchKey).getAsString().equalsIgnoreCase(value)) { + if (StringUtils.isEmpty(returnKey)) { + output.add(element); + } else { + output.add(get(returnKey, element.getAsJsonObject())); + } + } + } + return output; + } + + /** + * From an array of JsonObjects, filter by searching by key-value pair. + * + * @param searchKey key name to search in order to identify the JsonObject + * @param value value to match in order to identify the JsonObject + * @param objArray the array of JsonObjects + * @return the filtered elements within the array + */ + static JsonArray filter(final String searchKey, final String value, final JsonArray objArray) { + return filter(searchKey, value, objArray, null); + } + + /** + * Filter out any null values in a JsonObject + * @param input JsonObject with nulls + * @return the filtered jsonObject + */ + static JsonObject filterNull(final JsonObject input) { + JsonObject output = new JsonObject(); + for (Map.Entry entry : input.entrySet()) { + String key = entry.getKey(); + JsonElement value = entry.getValue(); + if (!value.isJsonNull()) { + output.add(key, value); + } + } + return output; + } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/SchemaUtils.java b/cdi-core/src/main/java/com/linkedin/cdi/util/SchemaUtils.java index f707f6c..ca35f03 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/SchemaUtils.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/SchemaUtils.java @@ -4,13 +4,18 @@ package com.linkedin.cdi.util; +import com.google.common.collect.Lists; +import com.google.gson.JsonArray; +import com.google.gson.JsonNull; import java.util.List; -import lombok.extern.slf4j.Slf4j; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static com.linkedin.cdi.configuration.StaticConstants.*; -@Slf4j -public class SchemaUtils { +public class SchemaUtils { + private static final Logger LOG = LoggerFactory.getLogger(SchemaUtils.class); // Following best practices for utility classes to have a private constructor private SchemaUtils() { throw new IllegalStateException("Utility class"); @@ -39,11 +44,62 @@ public static boolean isValidOutputSchema(List schemaColumns, List jsonPath, JsonArray schemaArray) { + if (jsonPath.size() == 0 || schemaArray == null || schemaArray.size() == 0) { + return false; + } + JsonArray isNullable = JsonUtils.filter(KEY_WORD_COLUMN_NAME, jsonPath.get(0), schemaArray, KEY_WORD_IS_NULLABLE); + if (isNullable.size() == 0 || isNullable.get(0) == JsonNull.INSTANCE || isNullable.get(0).getAsBoolean()) { + return true; + } + List subPath = jsonPath.subList(1, jsonPath.size()); + JsonArray dataType = JsonUtils.filter(KEY_WORD_COLUMN_NAME, jsonPath.get(0), schemaArray, KEY_WORD_DATA_TYPE); + if (dataType.size() > 0 && dataType.get(0) != JsonNull.INSTANCE) { + // try sub record + JsonArray typeDef = JsonUtils.filter(KEY_WORD_TYPE, KEY_WORD_RECORD, dataType, KEY_WORD_VALUES); + if (typeDef.size() > 0 && typeDef.get(0) != JsonNull.INSTANCE) { + return isNullable(subPath, typeDef.get(0).getAsJsonArray()); + } + + // try sub array + typeDef = JsonUtils.filter(KEY_WORD_TYPE, KEY_WORD_ARRAY, dataType, KEY_WORD_ITEMS + "." + KEY_WORD_DATA_TYPE + "." + KEY_WORD_VALUES); + if (typeDef.size() > 0 && typeDef.get(0) != JsonNull.INSTANCE) { + return isNullable(subPath, typeDef.get(0).getAsJsonArray()); + } + } + // no more sub element in schema definition + return subPath.size() > 0; + } + + /** + * Check if the field definition at the JSON path is nullable. For nested structures, + * if the top level is nullable, the field is nullable. + * + * Any Json path not exist in the schema is nullable + * + * @param jsonPath a JSON path string separated by "." + * @param schemaArray a JSON schema array + * @return true if nullable + */ + public static boolean isNullable(String jsonPath, JsonArray schemaArray) { + return isNullable(Lists.newArrayList(jsonPath.split("\\.")), schemaArray); + } } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/WatermarkDefinition.java b/cdi-core/src/main/java/com/linkedin/cdi/util/WatermarkDefinition.java index 7ee1bf4..b6d5a59 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/WatermarkDefinition.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/WatermarkDefinition.java @@ -10,15 +10,14 @@ import com.google.gson.JsonElement; import com.google.gson.JsonObject; import java.util.List; -import lombok.Getter; -import lombok.Setter; -import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.util.StringUtils; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Period; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.Assert; import static com.linkedin.cdi.configuration.StaticConstants.*; @@ -30,10 +29,62 @@ * * @author chrli */ -@Slf4j @Getter @Setter public class WatermarkDefinition { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkDefinition.class); final private static String DEFAULT_TIMEZONE = "America/Los_Angeles"; + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public WatermarkTypes getType() { + return type; + } + + public void setType(WatermarkTypes type) { + this.type = type; + } + + public Pair getRange() { + return range; + } + + public void setRange(Pair range) { + this.range = range; + } + + public Boolean getIsPartialPartition() { + return isPartialPartition; + } + + public void setIsPartialPartition(Boolean partialPartition) { + isPartialPartition = partialPartition; + } + + public WorkUnitPartitionTypes getWorkUnitPartitionType() { + return workUnitPartitionType; + } + + public void setWorkUnitPartitionType(WorkUnitPartitionTypes workUnitPartitionType) { + this.workUnitPartitionType = workUnitPartitionType; + } + + public void setUnits(String units) { + this.units = units; + } + + public String getTimezone() { + return timezone; + } + + public void setTimezone(String timezone) { + this.timezone = timezone; + } + public enum WatermarkTypes { DATETIME("datetime"), UNIT("unit"); @@ -101,8 +152,8 @@ public WatermarkDefinition setUnits(String name, String commaSeparatedUnits) { try { GSON.fromJson(commaSeparatedUnits, JsonArray.class); } catch (Exception e) { - log.info("Assuming simple Unit Watermark definition as the unit watermark cannot be converted to JsonArray"); - log.info("Origin unit watermark definition: {} : {}", name, commaSeparatedUnits); + LOG.info("Assuming simple Unit Watermark definition as the unit watermark cannot be converted to JsonArray"); + LOG.info("Origin unit watermark definition: {} : {}", name, commaSeparatedUnits); isJsonArrayUnits = false; } @@ -154,8 +205,8 @@ public WatermarkDefinition(JsonObject definition, boolean isPartialPartition, if (definition.get(KEY_WORD_TYPE).getAsString().equalsIgnoreCase(WatermarkTypes.DATETIME.name)) { this.setType(WatermarkTypes.DATETIME); this.setRange(new ImmutablePair<>( - definition.get(KEY_WORD_RANGE).getAsJsonObject().get(KEY_WORD_RANGE_FROM).getAsString(), - definition.get(KEY_WORD_RANGE).getAsJsonObject().get(KEY_WORD_RANGE_TO).getAsString())); + definition.get(KEY_WORD_RANGE).getAsJsonObject().get(KEY_WORD_FROM).getAsString(), + definition.get(KEY_WORD_RANGE).getAsJsonObject().get(KEY_WORD_TO).getAsString())); this.setWorkUnitPartitionType(workUnitPartitionType); } else if (definition.get(KEY_WORD_TYPE).getAsString().equalsIgnoreCase(WatermarkTypes.UNIT.name)) { this.setType(WatermarkTypes.UNIT); diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/WorkUnitPartitionTypes.java b/cdi-core/src/main/java/com/linkedin/cdi/util/WorkUnitPartitionTypes.java index e1b49e9..c1a0036 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/WorkUnitPartitionTypes.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/WorkUnitPartitionTypes.java @@ -10,12 +10,13 @@ import java.util.List; import java.util.SortedSet; import java.util.TreeSet; -import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; -import com.linkedin.cdi.configuration.MultistageProperties; import org.joda.time.DateTime; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static com.linkedin.cdi.configuration.StaticConstants.*; @@ -40,7 +41,6 @@ * @author chrli * */ -@Slf4j public enum WorkUnitPartitionTypes { NONE("none", 0) { @Override @@ -85,7 +85,7 @@ protected DateTime getNext(DateTime start, DateTime end) { } /** - * Refer to the specifications and constraints for "ms.work.unit.partition" in MultistageProperties. + * Refer to the specifications and constraints for "ms.work.unit.partition" in * The ranges should be continuous with no gaps or overlaps. * */ @@ -113,6 +113,7 @@ protected Pair getNext(DateTime start, DateTime end, boolean } }; + private static final Logger LOG = LoggerFactory.getLogger(WorkUnitPartitionTypes.class); final private String partitionType; final protected Integer interval; final private Boolean isMultiDayPartitioned; @@ -153,8 +154,8 @@ public static WorkUnitPartitionTypes fromString(String partitionType) { return WorkUnitPartitionTypes.COMPOSITE; } } catch (Exception e) { - log.error("Error parsing the partition type string, please check job property: " - + MultistageProperties.MSTAGE_WORK_UNIT_PARTITION.toString(), e); + LOG.error("Error parsing the partition type string, please check job property: " + + MSTAGE_WORK_UNIT_PARTITION.toString(), e); } return null; } diff --git a/cdi-core/src/main/java/com/linkedin/cdi/util/WorkUnitStatus.java b/cdi-core/src/main/java/com/linkedin/cdi/util/WorkUnitStatus.java index 3db3be8..0d409ed 100644 --- a/cdi-core/src/main/java/com/linkedin/cdi/util/WorkUnitStatus.java +++ b/cdi-core/src/main/java/com/linkedin/cdi/util/WorkUnitStatus.java @@ -19,7 +19,7 @@ * @author chrli */ public class WorkUnitStatus { - private static final Logger log = org.slf4j.LoggerFactory.getLogger(WorkUnitStatus.class); + private static final Logger LOG = org.slf4j.LoggerFactory.getLogger(WorkUnitStatus.class); private long totalCount; private long setCount; private long pageNumber = 0; @@ -38,7 +38,7 @@ public class WorkUnitStatus { this.pageStart = pageStart; this.pageSize = pageSize; this.buffer = buffer; - this.messages = messages; + this.messages = messages == null ? new HashMap<>() : messages; this.sessionKey = sessionKey; } @@ -56,18 +56,19 @@ public JsonArray getSchema() { try { return new Gson().fromJson(messages.get("schema"), JsonArray.class); } catch (Exception e) { - log.warn("Error reading source schema", e); + LOG.warn("Error reading source schema", e); } } return new JsonArray(); } public void logDebugAll() { - log.debug("These are values in WorkUnitStatus"); - log.debug("Total count: {}", totalCount); - log.debug("Chunk count: {}", setCount); - log.debug("Pagination: {},{},{}", pageStart, pageSize, pageNumber); - log.debug("Session Status: {}", sessionKey); + LOG.debug("These are values in WorkUnitStatus"); + LOG.debug("Total count: {}", totalCount); + LOG.debug("Chunk count: {}", setCount); + LOG.debug("Pagination: {},{},{}", pageStart, pageSize, pageNumber); + LOG.debug("Session Status: {}", sessionKey); + LOG.debug("Messages: {}", messages.toString()); } public long getTotalCount() { @@ -95,7 +96,7 @@ public InputStream getBuffer() { } public Map getMessages() { - return this.messages == null ? new HashMap<>() : messages; + return messages; } public String getSessionKey() { @@ -133,7 +134,7 @@ public WorkUnitStatus setBuffer(InputStream buffer) { } public WorkUnitStatus setMessages(Map messages) { - this.messages = messages; + this.messages = messages == null ? new HashMap<>() : messages; return this; } @@ -203,7 +204,7 @@ public WorkUnitStatus.WorkUnitStatusBuilder buffer(InputStream buffer) { } public WorkUnitStatus.WorkUnitStatusBuilder messages(Map messages) { - this.messages = messages; + this.messages = messages == null ? new HashMap<>() : messages; return this; } diff --git a/cdi-core/src/test/java/com/linkedin/cdi/configuration/MultistagePropertiesClassTest.java b/cdi-core/src/test/java/com/linkedin/cdi/configuration/MultistagePropertiesClassTest.java new file mode 100644 index 0000000..05714a3 --- /dev/null +++ b/cdi-core/src/test/java/com/linkedin/cdi/configuration/MultistagePropertiesClassTest.java @@ -0,0 +1,76 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import com.google.gson.JsonArray; +import com.google.gson.JsonObject; +import org.apache.gobblin.configuration.SourceState; +import org.testng.Assert; +import org.testng.annotations.Test; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; + +public class MultistagePropertiesClassTest { + @Test + public void testBaseClass() { + SourceState state = new SourceState(); + Assert.assertEquals(MSTAGE_CALL_INTERVAL_MILLIS.getDocUrl(), + "https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.call.interval.millis.md"); + } + + @Test + public void testJsonArrayProperties() { + SourceState state = new SourceState(); + Assert.assertEquals(MSTAGE_DERIVED_FIELDS.get(state), new JsonArray()); + + state.setProp("ms.derived.fields", "[0, 1, 2]"); + Assert.assertFalse(MSTAGE_DERIVED_FIELDS.isValid(state)); + Assert.assertEquals(MSTAGE_DERIVED_FIELDS.get(state), new JsonArray()); + + state.setProp("ms.derived.fields", "[{}]"); + Assert.assertFalse(MSTAGE_DERIVED_FIELDS.isValid(state)); + Assert.assertEquals(MSTAGE_DERIVED_FIELDS.get(state), new JsonArray()); + + state.setProp("ms.derived.fields", "[{\"name\": \"dummy\"}]"); + Assert.assertFalse(MSTAGE_DERIVED_FIELDS.isValid(state)); + Assert.assertFalse(MSTAGE_DERIVED_FIELDS.isValidNonblank(state)); + Assert.assertEquals(MSTAGE_DERIVED_FIELDS.get(state), new JsonArray()); + + state.setProp("ms.derived.fields", "[{\"name\": \"dummy\", \"formula\": \"dummy\"}]"); + Assert.assertTrue(MSTAGE_DERIVED_FIELDS.isValid(state)); + } + + @Test + public void testJsonObjectProperties() { + SourceState state = new SourceState(); + Assert.assertEquals(MSTAGE_ACTIVATION_PROPERTY.get(state), new JsonObject()); + + state.setProp("ms.activation.property", "[0, 1, 2]"); + Assert.assertFalse(MSTAGE_ACTIVATION_PROPERTY.isValid(state)); + Assert.assertEquals(MSTAGE_ACTIVATION_PROPERTY.get(state), new JsonObject()); + + state.setProp("ms.activation.property", "{\"name\": \"value\""); + Assert.assertFalse(MSTAGE_ACTIVATION_PROPERTY.isValid(state)); + Assert.assertEquals(MSTAGE_ACTIVATION_PROPERTY.get(state), new JsonObject()); + + state.setProp("ms.activation.property", "{\"name\": \"value\"}"); + Assert.assertTrue(MSTAGE_ACTIVATION_PROPERTY.isValid(state)); + } + + @Test + public void testIntegerProperties() { + SourceState state = new SourceState(); + Assert.assertEquals(MSTAGE_ABSTINENT_PERIOD_DAYS.get(state), (Integer) 0); + + state.setProp("ms.abstinent.period.days", "abc"); + Assert.assertFalse(MSTAGE_ABSTINENT_PERIOD_DAYS.isValid(state)); + Assert.assertEquals(MSTAGE_ABSTINENT_PERIOD_DAYS.get(state), (Integer) 0); + + state.setProp("ms.abstinent.period.days", "99"); + Assert.assertTrue(MSTAGE_ABSTINENT_PERIOD_DAYS.isValid(state)); + Assert.assertEquals(MSTAGE_ABSTINENT_PERIOD_DAYS.get(state), (Integer) 99); + } + +} diff --git a/cdi-core/src/test/java/com/linkedin/cdi/configuration/MultistagePropertiesIndividualTest.java b/cdi-core/src/test/java/com/linkedin/cdi/configuration/MultistagePropertiesIndividualTest.java new file mode 100644 index 0000000..1b46b9c --- /dev/null +++ b/cdi-core/src/test/java/com/linkedin/cdi/configuration/MultistagePropertiesIndividualTest.java @@ -0,0 +1,241 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.configuration; + +import com.google.common.collect.Lists; +import com.google.gson.JsonArray; +import com.google.gson.JsonObject; +import com.linkedin.cdi.keys.JobKeys; +import org.apache.gobblin.configuration.SourceState; +import org.testng.Assert; +import org.testng.annotations.Test; + +import static com.linkedin.cdi.configuration.PropertyCollection.*; +import static com.linkedin.cdi.configuration.StaticConstants.*; + + +public class MultistagePropertiesIndividualTest { + @Test + public void testAllKeys() { + SourceState state = new SourceState(); + Assert.assertTrue(new JobKeys().validate(state)); + + state.setProp("ms.csv.column.header", "xxx"); + Assert.assertFalse(new JobKeys().validate(state)); + } + + @Test + public void testMsAuthentication() { + SourceState state = new SourceState(); + Assert.assertEquals(MSTAGE_AUTHENTICATION.get(state), new JsonObject()); + + state.setProp("ms.authentication", "[0, 1, 2]"); + Assert.assertFalse(MSTAGE_AUTHENTICATION.isValid(state)); + Assert.assertEquals(MSTAGE_AUTHENTICATION.get(state), new JsonObject()); + + state.setProp("ms.authentication", "{\"name\": \"value\""); + Assert.assertFalse(MSTAGE_AUTHENTICATION.isValid(state)); + Assert.assertFalse(MSTAGE_AUTHENTICATION.isValidNonblank(state)); + Assert.assertEquals(MSTAGE_AUTHENTICATION.get(state), new JsonObject()); + + state.setProp("ms.authentication", "{\"method\": \"bearer\"}"); + Assert.assertFalse(MSTAGE_AUTHENTICATION.isValid(state)); + Assert.assertFalse(MSTAGE_AUTHENTICATION.isValidNonblank(state)); + Assert.assertEquals(MSTAGE_AUTHENTICATION.get(state), new JsonObject()); + + state.setProp("ms.authentication", "{\"method\": \"bearer\", \"encryption\": \"base64\", \"header\": \"Authorization\"}"); + Assert.assertTrue(MSTAGE_AUTHENTICATION.isValid(state)); + } + + @Test + public void testDefaultValues() { + SourceState state = new SourceState(); + Assert.assertEquals(EXTRACT_IS_FULL.get(state), Boolean.FALSE); + Assert.assertEquals(MSTAGE_BACKFILL.get(state), Boolean.FALSE); + Assert.assertEquals(MSTAGE_DATA_EXPLICIT_EOF.get(state), Boolean.FALSE); + Assert.assertEquals(MSTAGE_ENABLE_CLEANSING.get(state), Boolean.TRUE); + Assert.assertEquals(MSTAGE_ENABLE_DYNAMIC_FULL_LOAD.get(state), Boolean.TRUE); + Assert.assertEquals(MSTAGE_ENABLE_SCHEMA_BASED_FILTERING.get(state), Boolean.TRUE); + Assert.assertEquals(MSTAGE_WORK_UNIT_PARTIAL_PARTITION.get(state), Boolean.TRUE); + Assert.assertEquals(STATE_STORE_ENABLED.get(state), Boolean.TRUE); + Assert.assertEquals(MSTAGE_ABSTINENT_PERIOD_DAYS.get(state).longValue(), 0L); + Assert.assertEquals(MSTAGE_S3_LIST_MAX_KEYS.get(state).intValue(), 1000); + Assert.assertEquals(MSTAGE_NORMALIZER_BATCH_SIZE.get(state).longValue(), 500L); + Assert.assertEquals(MSTAGE_WAIT_TIMEOUT_SECONDS.get(state).longValue(), 600L); + Assert.assertEquals(MSTAGE_JDBC_SCHEMA_REFACTOR.get(state), "none"); + Assert.assertEquals(MSTAGE_SOURCE_DATA_CHARACTER_SET.get(state), "UTF-8"); + Assert.assertEquals(MSTAGE_SOURCE_FILES_PATTERN.get(state), REGEXP_DEFAULT_PATTERN); + Assert.assertEquals(MSTAGE_WORK_UNIT_PARTITION.get(state), "none"); + } + + + @Test + public void testCsv() { + SourceState state = new SourceState(); + JsonObject csv; + + Assert.assertTrue(MSTAGE_CSV.isValid(state)); + Assert.assertEquals(MSTAGE_CSV.getEscapeCharacter(state), "\\"); + Assert.assertEquals(MSTAGE_CSV.getQuoteCharacter(state), "\""); + Assert.assertEquals(MSTAGE_CSV.getFieldSeparator(state), ","); + Assert.assertEquals(MSTAGE_CSV.getRecordSeparator(state), System.lineSeparator()); + + csv = new JsonObject(); + csv.addProperty("columnHeaderIndex", -1); + csv.addProperty("linesToSkip", 0); + csv.addProperty("escapeCharacter", "u0003"); + csv.addProperty("quoteCharacter", "u0003"); + csv.addProperty("defaultFieldType", "xxx"); + csv.addProperty("fieldSeparator", "u0003"); + csv.addProperty("recordSeparator", "u0003"); + csv.addProperty("columnProjection", "xxx"); + csv.addProperty("maxFailures", 1); + csv.addProperty("keepNullString", true); + state.setProp("ms.csv", csv.toString()); + Assert.assertTrue(MSTAGE_CSV.isValid(state)); + Assert.assertEquals(MSTAGE_CSV.getEscapeCharacter(state), "\u0003"); + Assert.assertEquals(MSTAGE_CSV.getQuoteCharacter(state), "\u0003"); + Assert.assertEquals(MSTAGE_CSV.getFieldSeparator(state), "\u0003"); + Assert.assertEquals(MSTAGE_CSV.getRecordSeparator(state), "\u0003"); + + csv = new JsonObject(); + csv.addProperty("columnHeaderIndex", -1); + state.setProp("ms.csv", csv.toString()); + Assert.assertTrue(MSTAGE_CSV.isValid(state)); + + csv = new JsonObject(); + csv.addProperty("columnHeaderIndex", 0); + state.setProp("ms.csv", csv.toString()); + Assert.assertTrue(MSTAGE_CSV.isValid(state)); + + csv = new JsonObject(); + csv.addProperty("columnHeaderIndex", 0); + csv.addProperty("linesToSkip", 0); + state.setProp("ms.csv", csv.toString()); + Assert.assertFalse(MSTAGE_CSV.isValid(state)); + + } + + @Test + public void testEncryptionFields() { + SourceState state = new SourceState(); + Assert.assertTrue(MSTAGE_ENCRYPTION_FIELDS.isValid(state)); + + JsonArray fields = new JsonArray(); + fields.add("access_token"); + state.setProp(MSTAGE_ENCRYPTION_FIELDS.getConfig(), fields.toString()); + + JsonObject schemaColumn = new JsonObject(); + schemaColumn.addProperty("columnName", "access_token"); + schemaColumn.addProperty("isNullable", "true"); + JsonArray schema = new JsonArray(); + schema.add(schemaColumn); + state.setProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), schema.toString()); + + Assert.assertFalse(MSTAGE_ENCRYPTION_FIELDS.isValid(state)); + + schemaColumn = new JsonObject(); + schemaColumn.addProperty("columnName", "access_token"); + schema = new JsonArray(); + schema.add(schemaColumn); + state.setProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), schema.toString()); + + Assert.assertFalse(MSTAGE_ENCRYPTION_FIELDS.isValid(state)); + + schemaColumn = new JsonObject(); + schemaColumn.addProperty("columnName", "access_token"); + schemaColumn.addProperty("isNullable", "false"); + schema = new JsonArray(); + schema.add(schemaColumn); + state.setProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), schema.toString()); + + Assert.assertTrue(MSTAGE_ENCRYPTION_FIELDS.isValid(state)); + + + } + + + @Test + public void testSSL() { + SourceState state = new SourceState(); + Assert.assertTrue(MSTAGE_SSL.isValid(state)); + + JsonObject ssl = new JsonObject(); + ssl.addProperty("keyStoreType", "xxx"); + ssl.addProperty("keyStorePath", "xxx"); + ssl.addProperty("keyStorePassword", "xxx"); + ssl.addProperty("keyPassword", "xxx"); + ssl.addProperty("trustStorePath", "xxx"); + ssl.addProperty("trustStorePassword", "xxx"); + ssl.addProperty("connectionTimeoutSeconds", "1"); + ssl.addProperty("socketTimeoutSeconds", "1"); + ssl.addProperty("version", "xxx"); + state.setProp(MSTAGE_SSL.getConfig(), ssl.toString()); + Assert.assertTrue(MSTAGE_SSL.isValid(state)); + + ssl.addProperty("keystorePassword", "xxx"); + state.setProp(MSTAGE_SSL.getConfig(), ssl.toString()); + Assert.assertFalse(MSTAGE_SSL.isValid(state)); + } + + @Test + public void testWatermark() { + SourceState state = new SourceState(); + Assert.assertTrue(MSTAGE_WATERMARK.isValid(state)); + + // not a JsonArray + state.setProp("ms.watermark", "string"); + Assert.assertFalse(MSTAGE_WATERMARK.isValid(state)); + + // array item is not a JsonObject + state.setProp("ms.watermark", "[\"string\"]"); + Assert.assertFalse(MSTAGE_WATERMARK.isValid(state)); + + // no "name" + state.setProp("ms.watermark", "[{\"type\": \"datetime\",\"range\": {\"from\": \"2019-01-01\", \"to\": \"-\"}}]"); + Assert.assertFalse(MSTAGE_WATERMARK.isValid(state)); + + // unknown type + state.setProp("ms.watermark", "[{\"name\": \"system\",\"type\": \"unknown\"}]"); + Assert.assertFalse(MSTAGE_WATERMARK.isValid(state)); + + // no "range" + state.setProp("ms.watermark", "[{\"name\": \"system\",\"type\": \"datetime\"}]"); + Assert.assertFalse(MSTAGE_WATERMARK.isValid(state)); + + // no "units" + state.setProp("ms.watermark", "[{\"name\": \"system\",\"type\": \"unit\"}]"); + Assert.assertFalse(MSTAGE_WATERMARK.isValid(state)); + + // normal datetime watermark + state.setProp("ms.watermark", "[{\"name\": \"system\",\"type\": \"datetime\",\"range\": {\"from\": \"2019-01-01\", \"to\": \"-\"}}]"); + Assert.assertTrue(MSTAGE_WATERMARK.isValid(state)); + Assert.assertEquals(MSTAGE_WATERMARK.getRanges(state).getRight(), "-"); + + // normal datetime watermark and normal unit watermark + state.setProp("ms.watermark", "[{\"name\": \"system\",\"type\": \"datetime\", \"range\": {\"from\": \"2021-08-21\", \"to\": \"-\"}}, {\"name\": \"bucketId\", \"type\": \"unit\", \"units\": \"null,0,1,2,3,4,5,6,7,8,9\"}]"); + Assert.assertTrue(MSTAGE_WATERMARK.isValid(state)); + Assert.assertEquals(MSTAGE_WATERMARK.getRanges(state).getLeft(), "2021-08-21"); + Assert.assertEquals(MSTAGE_WATERMARK.getUnits(state), Lists.newArrayList("null,0,1,2,3,4,5,6,7,8,9".split(","))); + } + + + @Test + public void testWorkUnitParallelismMax() { + SourceState state = new SourceState(); + Assert.assertTrue(MSTAGE_WORK_UNIT_PARALLELISM_MAX.isValid(state)); + + state.setProp("ms.work.unit.parallelism.max", "0"); + Assert.assertTrue(MSTAGE_WORK_UNIT_PARALLELISM_MAX.isValid(state)); + Assert.assertEquals(MSTAGE_WORK_UNIT_PARALLELISM_MAX.get(state).intValue(), 100); + + state.setProp("ms.work.unit.parallelism.max", "0L"); + Assert.assertFalse(MSTAGE_WORK_UNIT_PARALLELISM_MAX.isValid(state)); + + state.setProp("ms.work.unit.parallelism.max", "20000"); + Assert.assertFalse(MSTAGE_WORK_UNIT_PARALLELISM_MAX.isValid(state)); + } + +} diff --git a/cdi-core/src/test/java/com/linkedin/cdi/configuration/MultistagePropertiesTest.java b/cdi-core/src/test/java/com/linkedin/cdi/configuration/MultistagePropertiesTest.java deleted file mode 100644 index fd951b4..0000000 --- a/cdi-core/src/test/java/com/linkedin/cdi/configuration/MultistagePropertiesTest.java +++ /dev/null @@ -1,283 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.configuration; - -import com.google.gson.Gson; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import org.apache.commons.lang.StringUtils; -import org.apache.gobblin.configuration.SourceState; -import org.apache.gobblin.configuration.State; -import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.Test; - -import static org.mockito.Mockito.*; - - -@Test -public class MultistagePropertiesTest { - private final Gson gson = new Gson(); - - @Test - void validateNonblankWithDefault() { - SourceState state = new SourceState(); - Assert.assertEquals(MultistageProperties.MSTAGE_PARAMETERS.getValidNonblankWithDefault(state), new JsonArray()); - Assert.assertEquals(MultistageProperties.MSTAGE_DATA_FIELD.getValidNonblankWithDefault(state), ""); - Assert.assertEquals(MultistageProperties.MSTAGE_ABSTINENT_PERIOD_DAYS.getValidNonblankWithDefault(state), new Integer(0)); - Assert.assertEquals(MultistageProperties.MSTAGE_AUTHENTICATION.getValidNonblankWithDefault(state), new JsonObject()); - Assert.assertEquals(MultistageProperties.MSTAGE_HTTP_STATUSES.getValidNonblankWithDefault(state).toString(), - "{\"success\":[200,201,202],\"pagination_error\":[401]}"); - Assert.assertEquals(MultistageProperties.MSTAGE_PAGINATION.getValidNonblankWithDefault(state), new JsonObject()); - Assert.assertFalse(MultistageProperties.MSTAGE_PAGINATION.validateNonblank(state)); - state.setProp(MultistageProperties.MSTAGE_PAGINATION.getConfig(), "[]"); - Assert.assertFalse(MultistageProperties.MSTAGE_PAGINATION.validateNonblank(state)); - state.setProp(MultistageProperties.MSTAGE_PAGINATION.getConfig(), "{}"); - Assert.assertFalse(MultistageProperties.MSTAGE_PAGINATION.validateNonblank(state)); - state.setProp(MultistageProperties.MSTAGE_PAGINATION.getConfig(), "{null}}"); - Assert.assertFalse(MultistageProperties.MSTAGE_PAGINATION.validateNonblank(state)); - } - - /** - * Test ms.wait.timeout.seconds under 2 scenarios - * Scenario 1: test default value - * Scenario 2: test user defined value - */ - @Test - void validateWaitTimeoutProperty() { - SourceState state = new SourceState(); - - // Scenario 1: test default value - // - // Input: State object without setting ms.wait.time.seconds - // Output: 600 seconds, or 10 minutes, or 600,000 milli-seconds - - Assert.assertEquals(MultistageProperties.MSTAGE_WAIT_TIMEOUT_SECONDS.getMillis(state).longValue(), 600000L); - - // Scenario 2: test user defined value - // - // Input: State object by setting ms.wait.time.seconds = 1000 - // Output: 1000 seconds, or 1,000,000 milli-seconds - state.setProp(MultistageProperties.MSTAGE_WAIT_TIMEOUT_SECONDS.toString(), 1000); - Assert.assertEquals(MultistageProperties.MSTAGE_WAIT_TIMEOUT_SECONDS.getMillis(state).longValue(), 1000000L); - } - - /** - * Test getDefaultValue for MSTAGE_RETENTION - */ - @Test - public void testGetDefaultValue1() { - JsonObject expected = gson.fromJson("{\"state.store\":\"P90D\",\"publish.dir\":\"P731D\",\"log\":\"P30D\"}", JsonObject.class); - Assert.assertEquals(MultistageProperties.MSTAGE_RETENTION.getDefaultValue(), expected); - } - - /** - * Test getDefaultValue for MSTAGE_ENABLE_DYNAMIC_FULL_LOAD - */ - @Test - public void testGetDefaultValue2() { - Assert.assertEquals(MultistageProperties.MSTAGE_ENABLE_DYNAMIC_FULL_LOAD.getDefaultValue(), Boolean.TRUE); - } - - /** - * Test getDefaultValue for MSTAGE_ENABLE_SCHEMA_BASED_FILTERING - */ - @Test - public void testGetDefaultValue3() { - Assert.assertEquals(MultistageProperties.MSTAGE_ENABLE_SCHEMA_BASED_FILTERING.getDefaultValue(), Boolean.TRUE); - } - - /** - * Test getDefaultValue for MSTAGE_SOURCE_FILES_PATTERN - */ - @Test - public void testGetDefaultValue4() { - Assert.assertEquals(MultistageProperties.MSTAGE_SOURCE_FILES_PATTERN.getDefaultValue(), ".*"); - } - - /** - * Test getDefaultValue for EXTRACT_IS_FULL - */ - @Test - public void testGetDefaultValue5() { - Assert.assertEquals(MultistageProperties.EXTRACT_IS_FULL.getDefaultValue(), (Boolean) false); - } - - /** - * Test getDefaultValue for MSTAGE_ENDCODING - */ - @Test - public void testGetDefaultValue6() { - Assert.assertEquals(MultistageProperties.MSTAGE_ENCODING.getDefaultValue(), "UTF-8"); - } - - /** - * Test getDefaultValue - */ - @Test - public void testGetDefaultValue7() { - Assert.assertEquals(MultistageProperties.MSTAGE_WORKUNIT_STARTTIME_KEY.getDefaultValue(), new Long(0L)); - } - - /** - * Test getValidNonblankWithDefault - */ - @Test - public void testGetValidNonblankWithDefault1() { - State state = Mockito.mock(State.class); - when(state.getPropAsInt(MultistageProperties.MSTAGE_ABSTINENT_PERIOD_DAYS.getConfig(), 0)).thenReturn(0); - Assert.assertEquals(MultistageProperties.MSTAGE_ABSTINENT_PERIOD_DAYS.getValidNonblankWithDefault(state), new Integer(0)); - - when(state.getPropAsInt(MultistageProperties.MSTAGE_ABSTINENT_PERIOD_DAYS.getConfig(), 0)).thenReturn(1); - Assert.assertEquals(MultistageProperties.MSTAGE_ABSTINENT_PERIOD_DAYS.getValidNonblankWithDefault(state), new Integer(1)); - } - - /** - * Test getValidNonblankWithDefault for MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION - */ - @Test - public void testGetValidNonblankWithDefault2() { - State state = Mockito.mock(State.class); - String expected = "input"; - when(state.getProp(MultistageProperties.MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION.getConfig(), StringUtils.EMPTY)).thenReturn(expected); - Assert.assertEquals(MultistageProperties.MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION.getValidNonblankWithDefault(state), expected.toUpperCase()); - - when(state.getProp(MultistageProperties.MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION.getConfig(), StringUtils.EMPTY)).thenReturn(""); - Assert.assertEquals(MultistageProperties.MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION.getValidNonblankWithDefault(state), "755"); - } - - /** - * Test getMillis for MSTAGE_GRACE_PERIOD_DAYS - */ - @Test - public void testGetMillis1() { - State state = new State(); - Long expected = 24L * 3600L * 1000L * (Integer) MultistageProperties.MSTAGE_GRACE_PERIOD_DAYS.getProp(state); - Assert.assertEquals(MultistageProperties.MSTAGE_GRACE_PERIOD_DAYS.getMillis(state), expected); - - Assert.assertEquals(MultistageProperties.MSTAGE_SOURCE_FILES_PATTERN.getMillis(state), (Long) 0L); - } - - /** - * Test getMillis for MSTAGE_ABSTINENT_PERIOD_DAYS - */ - @Test - public void testGetMillis2() { - State state = new State(); - Long expected = 24L * 3600L * 1000L * (Integer) MultistageProperties.MSTAGE_ABSTINENT_PERIOD_DAYS.getProp(new State()); - Assert.assertEquals(MultistageProperties.MSTAGE_ABSTINENT_PERIOD_DAYS.getMillis(state), expected); - } - - /** - * Test validate for MSTAGE_ACTIVATION_PROPERTY - */ - @Test - public void testValidate1() { - State state = Mockito.mock(State.class); - when(state.getProp(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.getConfig(), new JsonObject().toString())).thenReturn(""); - Assert.assertTrue(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.validate(state)); - - when(state.getProp(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.getConfig(), new JsonObject().toString())).thenReturn("{\"state.store\":\"P90D\"}"); - Assert.assertTrue(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.validate(state)); - } - - /** - * Test validate for MSTAGE_DERIVED_FIELDS - */ - @Test - public void testValidate2() { - State state = Mockito.mock(State.class); - when(state.getProp(MultistageProperties.MSTAGE_DERIVED_FIELDS.getConfig(), new JsonArray().toString())).thenReturn(""); - Assert.assertTrue(MultistageProperties.MSTAGE_DERIVED_FIELDS.validate(state)); - - when(state.getProp(MultistageProperties.MSTAGE_DERIVED_FIELDS.getConfig(), new JsonArray().toString())).thenReturn("[]"); - Assert.assertTrue(MultistageProperties.MSTAGE_DERIVED_FIELDS.validate(state)); - - when(state.getProp(MultistageProperties.MSTAGE_DERIVED_FIELDS.getConfig(), new JsonArray().toString())).thenReturn("[{\"random\":\"value\"}]"); - Assert.assertFalse(MultistageProperties.MSTAGE_DERIVED_FIELDS.validate(state)); - - when(state.getProp(MultistageProperties.MSTAGE_DERIVED_FIELDS.getConfig(), new JsonArray().toString())).thenReturn("[{\"name\":\"value\"}]"); - Assert.assertFalse(MultistageProperties.MSTAGE_DERIVED_FIELDS.validate(state)); - - when(state.getProp(MultistageProperties.MSTAGE_DERIVED_FIELDS.getConfig(), new JsonArray().toString())).thenReturn("[{\"name\":\"value\", \"formula\":\"formulaValue\"}]"); - Assert.assertTrue(MultistageProperties.MSTAGE_DERIVED_FIELDS.validate(state)); - } - - /** - * Test validate for MSTAGE_SECONDARY_INPUT - */ - @Test - public void testValidate3() { - State state = Mockito.mock(State.class); - when(state.getProp(MultistageProperties.MSTAGE_SECONDARY_INPUT.getConfig(), new JsonArray().toString())).thenReturn(null); - Assert.assertFalse(MultistageProperties.MSTAGE_SECONDARY_INPUT.validate(state)); - - when(state.getProp(MultistageProperties.MSTAGE_SECONDARY_INPUT.getConfig(), new JsonArray().toString())).thenReturn("[{\"name\":\"value\"}]"); - Assert.assertTrue(MultistageProperties.MSTAGE_SECONDARY_INPUT.validate(state)); - } - - /** - * Test validate for MSTAGE_SECONDARY_INPUT - */ - @Test - public void testValidate4() { - Assert.assertTrue(MultistageProperties.MSTAGE_ABSTINENT_PERIOD_DAYS.validate(new State())); - } - - /** - * Test validateNonblank for MSTAGE_AUTHENTICATION - */ - @Test - public void testValidateNonblank1() { - State state = Mockito.mock(State.class); - JsonObject obj = new JsonObject(); - when(state.getProp(MultistageProperties.MSTAGE_AUTHENTICATION.getConfig(), new JsonObject().toString())).thenReturn(obj.toString()); - Assert.assertFalse(MultistageProperties.MSTAGE_AUTHENTICATION.validateNonblank(state)); - - obj.addProperty("test", "testValue"); - when(state.getProp(MultistageProperties.MSTAGE_AUTHENTICATION.getConfig(), new JsonObject().toString())).thenReturn(obj.toString()); - Assert.assertFalse(MultistageProperties.MSTAGE_AUTHENTICATION.validateNonblank(state)); - - obj.addProperty("method", "testMethodValue"); - when(state.getProp(MultistageProperties.MSTAGE_AUTHENTICATION.getConfig(), new JsonObject().toString())).thenReturn(obj.toString()); - Assert.assertFalse(MultistageProperties.MSTAGE_AUTHENTICATION.validateNonblank(state)); - - obj.addProperty("encryption", "testEncryptionValue"); - when(state.getProp(MultistageProperties.MSTAGE_AUTHENTICATION.getConfig(), new JsonObject().toString())).thenReturn(obj.toString()); - Assert.assertTrue(MultistageProperties.MSTAGE_AUTHENTICATION.validateNonblank(state)); - } - - /** - * Test validateNonblank for MSTAGE_CSV_COLUMN_PROJECTION - */ - @Test - public void testValidateNonblank2() { - State state = Mockito.mock(State.class); - when(state.getProp(MultistageProperties.MSTAGE_CSV_COLUMN_PROJECTION.getConfig(), StringUtils.EMPTY)).thenReturn(null); - Assert.assertFalse(MultistageProperties.MSTAGE_CSV_COLUMN_PROJECTION.validateNonblank(state)); - - when(state.getProp(MultistageProperties.MSTAGE_CSV_COLUMN_PROJECTION.getConfig(), StringUtils.EMPTY)).thenReturn("test"); - Assert.assertTrue(MultistageProperties.MSTAGE_CSV_COLUMN_PROJECTION.validateNonblank(state)); - - when(state.getProp(MultistageProperties.MSTAGE_CSV_COLUMN_PROJECTION.getConfig(), StringUtils.EMPTY)).thenReturn("test1,test2"); - Assert.assertTrue(MultistageProperties.MSTAGE_CSV_COLUMN_PROJECTION.validateNonblank(state)); - } - - /** - * Test validateNonblank - */ - @Test - public void testValidateNonblank3() { - State state = Mockito.mock(State.class); - when(state.getProp(MultistageProperties.MSTAGE_BACKFILL.getConfig(), StringUtils.EMPTY)).thenReturn("non-validate"); - Assert.assertFalse(MultistageProperties.MSTAGE_BACKFILL.validateNonblank(state)); - - when(state.getProp(MultistageProperties.MSTAGE_BACKFILL.getConfig(), StringUtils.EMPTY)).thenReturn("false"); - Assert.assertTrue(MultistageProperties.MSTAGE_BACKFILL.validateNonblank(state)); - - when(state.getProp(MultistageProperties.MSTAGE_BACKFILL.getConfig(), StringUtils.EMPTY)).thenReturn("true"); - Assert.assertTrue(MultistageProperties.MSTAGE_BACKFILL.validateNonblank(state)); - } -} diff --git a/cdi-core/src/test/java/com/linkedin/cdi/connection/HttpConnectionTest.java b/cdi-core/src/test/java/com/linkedin/cdi/connection/HttpConnectionTest.java deleted file mode 100644 index b678441..0000000 --- a/cdi-core/src/test/java/com/linkedin/cdi/connection/HttpConnectionTest.java +++ /dev/null @@ -1,378 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.connection; - -import com.google.gson.Gson; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import gobblin.runtime.JobState; -import java.io.IOException; -import java.io.InputStreamReader; -import java.util.List; -import org.apache.commons.lang.StringUtils; -import org.apache.gobblin.configuration.SourceState; -import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.exception.RetriableAuthenticationException; -import com.linkedin.cdi.extractor.JsonExtractor; -import com.linkedin.cdi.extractor.MultistageExtractor; -import com.linkedin.cdi.keys.ExtractorKeys; -import com.linkedin.cdi.keys.HttpKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.source.HttpSource; -import com.linkedin.cdi.util.HttpRequestMethod; -import com.linkedin.cdi.util.WorkUnitStatus; -import org.apache.gobblin.source.extractor.extract.LongWatermark; -import org.apache.gobblin.source.workunit.WorkUnit; -import org.apache.http.Header; -import org.apache.http.HeaderElement; -import org.apache.http.HttpEntity; -import org.apache.http.HttpResponse; -import org.apache.http.StatusLine; -import org.apache.http.client.HttpClient; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpUriRequest; -import org.apache.http.impl.client.AutoRetryHttpClient; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.util.EntityUtils; -import org.mockito.Mockito; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.testng.PowerMockTestCase; -import org.powermock.reflect.Whitebox; -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import static com.linkedin.cdi.configuration.MultistageProperties.*; -import static org.mockito.Mockito.*; - - -@Test -@PrepareForTest({EntityUtils.class, CloseableHttpClient.class}) -public class HttpConnectionTest extends PowerMockTestCase { - private Gson gson; - private WorkUnitState state; - private JobKeys jobKeys; - private SourceState sourceState; - private String token; - private JsonObject pagination; - private JsonObject sessionKeyField; - private String totalCountField; - private JsonArray parameters; - private JsonArray encryptionFields; - private String dataField; - private Long callInterval; - private Long waitTimeoutSeconds; - private Boolean enableCleansing; - private Boolean workUnitPartialPartition; - private JsonArray watermark; - private JsonArray secondaryInput; - private String httpClientFactory; - private JsonObject httpRequestHeaders; - private String sourceUri; - private String httpRequestMethod; - private String extractorClass; - private JsonObject authentication; - private JsonObject httpStatus; - private JsonObject httpStatusReasons; - - @BeforeMethod - public void setUp() { - gson = new Gson(); - state = Mockito.mock(WorkUnitState.class); - jobKeys = Mockito.mock(JobKeys.class); - sourceState = Mockito.mock(SourceState.class); - } - - /** - * Test Execute - * @throws IOException - */ - @Test(expectedExceptions = RetriableAuthenticationException.class) - public void testExecute() throws IOException, RetriableAuthenticationException { - initializeHelper(); - - // the source getExtractor() method will initialize source Keys - HttpSource source = new HttpSource(); - HttpConnection conn = new HttpConnection(state, source.getHttpSourceKeys(), - ((MultistageExtractor)source.getExtractor(state)).getExtractorKeys()); - - CloseableHttpClient client = mock(CloseableHttpClient.class); - CloseableHttpResponse response = mock(CloseableHttpResponse.class); - - conn.setHttpClient(client); - when(client.execute(any())).thenReturn(response); - - WorkUnit workUnit = mock(WorkUnit.class); - LongWatermark lowWatermark = mock(LongWatermark.class); - LongWatermark highWatermark = mock(LongWatermark.class); - - long lowWaterMark = 1590994800000L; //2020-06-01 - long highWaterMark = 1591513200000L; //2020-06-07 - when(workUnit.getLowWatermark(LongWatermark.class)).thenReturn(lowWatermark); - when(lowWatermark.getValue()).thenReturn(lowWaterMark); - when(workUnit.getExpectedHighWatermark(LongWatermark.class)).thenReturn(highWatermark); - when(highWatermark.getValue()).thenReturn(highWaterMark); - when(state.getWorkunit()).thenReturn(workUnit); - - HttpRequestMethod command = mock(HttpRequestMethod.class); - WorkUnitStatus status = mock(WorkUnitStatus.class); - - JsonObject parameters = new JsonObject(); - parameters.addProperty("param1", "dummy"); - parameters.add("payload", new JsonObject()); - - when(command.toString()).thenReturn("Some http method"); - conn.getExtractorKeys().setDynamicParameters(parameters); - - StatusLine statusLine = mock(StatusLine.class); - when(response.getStatusLine()).thenReturn(statusLine); - when(statusLine.getStatusCode()).thenReturn(200); - when(statusLine.getReasonPhrase()).thenReturn("reason1 for success"); - Assert.assertNotNull(conn.execute(command, status)); - - HttpEntity entity = mock(HttpEntity.class); - Header header = mock(Header.class); - when(response.getEntity()).thenReturn(entity); - when(entity.getContentType()).thenReturn(header); - - HeaderElement element = mock(HeaderElement.class); - when(header.getElements()).thenReturn(new HeaderElement[]{element}); - when(element.getName()).thenReturn("application/json"); - PowerMockito.mockStatic(EntityUtils.class); - when(EntityUtils.toString(entity)).thenReturn("dummy error reason"); - Assert.assertNotNull(conn.execute(command, status)); - - when(response.getEntity()).thenReturn(null); - - when(statusLine.getStatusCode()).thenReturn(204); - Assert.assertNotNull(conn.execute(command, status)); - - when(statusLine.getStatusCode()).thenReturn(302); - when(statusLine.getReasonPhrase()).thenReturn("reason1 for warning"); - Assert.assertNull(conn.execute(command, status)); - - when(statusLine.getStatusCode()).thenReturn(405); - Assert.assertNull(conn.execute(command, status)); - - when(statusLine.getReasonPhrase()).thenReturn("reason1 for error"); - Assert.assertNull(conn.execute(command, status)); - - when(statusLine.getStatusCode()).thenReturn(408); - Assert.assertNull(conn.execute(command, status)); - - when(response.getEntity()).thenReturn(entity); - doThrow(new RuntimeException()).when(entity).getContentType(); - Assert.assertNull(conn.execute(command, status)); - } - - - /** - * Test getNext - */ - @Test - public void testGetNext() throws RetriableAuthenticationException { - HttpKeys httpSourceKeys = Mockito.mock(HttpKeys.class); - when(httpSourceKeys.getCallInterval()).thenReturn(1L); - ExtractorKeys extractorKeys = new ExtractorKeys(); - WorkUnitStatus workUnitStatus = Mockito.mock(WorkUnitStatus.class); - WorkUnitStatus.WorkUnitStatusBuilder builder = Mockito.mock(WorkUnitStatus.WorkUnitStatusBuilder.class); - HttpConnection conn = new HttpConnection(null, httpSourceKeys, extractorKeys); - - extractorKeys.setSignature("testSignature"); - extractorKeys.setActivationParameters(new JsonObject()); - when(builder.build()).thenReturn(workUnitStatus); - when(workUnitStatus.toBuilder()).thenReturn(builder); - when(httpSourceKeys.getHttpRequestMethod()).thenReturn("GET"); - - Assert.assertNull(conn.executeNext(workUnitStatus)); - } - - /** - * Test closeStream - */ - @Test - public void testCloseStream() throws IOException { - HttpConnection conn = new HttpConnection(null, new HttpKeys(), new ExtractorKeys()); - MultistageExtractor extractor = mock(MultistageExtractor.class); - ExtractorKeys keys = mock(ExtractorKeys.class); - String testSignature = "test_signature"; - when(extractor.getExtractorKeys()).thenReturn(keys); - when(keys.getSignature()).thenReturn(testSignature); - conn.closeStream(); - - CloseableHttpResponse httpResponse = mock(CloseableHttpResponse.class); - conn.setResponse(httpResponse); - doThrow(new RuntimeException()).when(httpResponse).close(); - conn.closeStream(); - } - - /** - * Test shutdown - */ - @Test - public void testShutdown() throws IOException { - HttpConnection conn = new HttpConnection(null, new HttpKeys(), null); - CloseableHttpClient client = mock(CloseableHttpClient.class); - conn.setHttpClient(client); - - doNothing().when(client).close(); - conn.closeAll(""); - - client = mock(CloseableHttpClient.class); - conn.setHttpClient(client); - doThrow(IOException.class).when(client).close(); - conn.closeAll(""); - - client = mock(CloseableHttpClient.class); - conn.setHttpClient(client); - AutoRetryHttpClient retryHttpClient = mock(AutoRetryHttpClient.class); - conn.setHttpClient(retryHttpClient); - conn.closeAll(""); - } - - @Test(enabled=true) - public void retriesTest() throws IOException { - - HttpClient mockHttpClient = mock(CloseableHttpClient.class); - HttpResponse httpResponse = mock(CloseableHttpResponse.class); - StatusLine statusLine = mock(StatusLine.class); - HttpEntity entity = mock(HttpEntity.class); - SourceState state = mock(SourceState.class); - - when(entity.getContent()).thenReturn(null); - when(httpResponse.getEntity()).thenReturn(entity); - when(statusLine.getStatusCode()).thenReturn(401); - when(statusLine.getReasonPhrase()).thenReturn("pagination error"); - when(httpResponse.getStatusLine()).thenReturn(statusLine); - when(mockHttpClient.execute(any(HttpUriRequest.class))).thenReturn(httpResponse); - - when(state.getProp("ms.watermark", "")).thenReturn("[{\"name\": \"system\",\"type\": \"datetime\", \"range\": {\"from\": \"2017-01-01\", \"to\": \"-\"}}]"); - when(state.getProp("extract.table.type", "SNAPSHOT_ONLY")).thenReturn("SNAPSHOT_ONLY"); - when(state.getProp("extract.namespace", "")).thenReturn("test"); - when(state.getProp("extract.table.name", "")).thenReturn("table1"); - when(state.getProp("source.conn.username", "")).thenReturn("X7CWBD5V4T6DR77WY23YSHACH55K2OXA"); - when(state.getProp("source.conn.password", "")).thenReturn(""); - when(state.getProp("ms.source.uri", "")).thenReturn("https://host/v2/users"); - when(state.getProp("ms.authentication", new JsonObject().toString())).thenReturn("{\"method\":\"basic\",\"encryption\":\"base64\", \"header\": \"Authorization\"}"); - when(state.getProp("ms.http.request.headers", new JsonObject().toString())).thenReturn("{\"Content-Type\": \"application/json\"}"); - when(state.getProp("ms.http.request.method", "")).thenReturn("GET"); - when(state.getProp("ms.session.key.field", new JsonObject().toString())).thenReturn("{\"name\": \"records.cursor\"}"); - when(state.getProp("ms.parameters", new JsonArray().toString())).thenReturn("[{\"name\":\"cursor\",\"type\":\"session\"}]"); - when(state.getProp("ms.data.field", "")).thenReturn("users"); - when(state.getProp("ms.total.count.field", "")).thenReturn("records.totalRecords"); - when(state.getProp("ms.work.unit.partition", "")).thenReturn(""); - when(state.getProp("ms.pagination", new JsonObject().toString())).thenReturn("{}"); - when(state.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); - HttpSource httpSource = new HttpSource(); - List workUnits = httpSource.getWorkunits(state); - WorkUnitState unitState = new WorkUnitState(workUnits.get(0), new JobState()); - - HttpConnection conn = new HttpConnection(null, httpSource.getJobKeys(), new ExtractorKeys()); - conn.setHttpClient(mockHttpClient); - JsonExtractor extractor = new JsonExtractor(unitState, httpSource.getHttpSourceKeys()); - extractor.setConnection(conn); - - JsonObject record = extractor.readRecord(new JsonObject()); - // since we are setting the buffer to null, the final record object will be null - Assert.assertEquals(null, record); - } - - /** - * Test getResponseContentType - */ - @Test - public void testGetResponseContentType() throws Exception { - HttpConnection conn = new HttpConnection(null, new HttpKeys(), null); - HttpResponse response = mock(HttpResponse.class); - String methodName = "getResponseContentType"; - when(response.getEntity()).thenReturn(null); - Assert.assertEquals(Whitebox.invokeMethod(conn, methodName, response), StringUtils.EMPTY); - - HttpEntity entity = mock(HttpEntity.class); - when(response.getEntity()).thenReturn(entity); - when(entity.getContentType()).thenReturn(null); - Assert.assertEquals(Whitebox.invokeMethod(conn, methodName, response), StringUtils.EMPTY); - - Header contentType = mock(Header.class); - when(entity.getContentType()).thenReturn(contentType); - - HeaderElement[] headerElements = new HeaderElement[]{}; - when(contentType.getElements()).thenReturn(headerElements); - Assert.assertEquals(Whitebox.invokeMethod(conn, methodName, response), StringUtils.EMPTY); - - String type = "some_type"; - HeaderElement element = mock(HeaderElement.class); - when(element.getName()).thenReturn(type); - headerElements = new HeaderElement[]{element}; - when(contentType.getElements()).thenReturn(headerElements); - Assert.assertEquals(Whitebox.invokeMethod(conn, methodName, response), type); - } - - private void initializeHelper() { - JsonObject allKeys = gson.fromJson(new InputStreamReader(this.getClass().getResourceAsStream("/json/sample-data-for-source.json")), JsonObject.class); - pagination = allKeys.get(MSTAGE_PAGINATION.getConfig()).getAsJsonObject(); - when(state.getProp(MSTAGE_PAGINATION.getConfig(), new JsonObject().toString())).thenReturn(pagination.toString()); - - sessionKeyField = allKeys.get(MSTAGE_SESSION_KEY_FIELD.getConfig()).getAsJsonObject(); - when(state.getProp(MSTAGE_SESSION_KEY_FIELD.getConfig(), new JsonObject().toString())).thenReturn(sessionKeyField.toString()); - - totalCountField = allKeys.get(MSTAGE_TOTAL_COUNT_FIELD.getConfig()).getAsString(); - when(state.getProp(MSTAGE_TOTAL_COUNT_FIELD.getConfig(), StringUtils.EMPTY)).thenReturn(totalCountField); - - parameters = allKeys.get(MSTAGE_PARAMETERS.getConfig()).getAsJsonArray(); - when(state.getProp(MSTAGE_PARAMETERS.getConfig(), new JsonArray().toString())).thenReturn(parameters.toString()); - - encryptionFields = allKeys.get(MSTAGE_ENCRYPTION_FIELDS.getConfig()).getAsJsonArray(); - when(state.getProp(MSTAGE_ENCRYPTION_FIELDS.getConfig(), new JsonArray().toString())).thenReturn(encryptionFields.toString()); - - dataField = allKeys.get(MSTAGE_DATA_FIELD.getConfig()).getAsString(); - when(state.getProp(MSTAGE_DATA_FIELD.getConfig(), StringUtils.EMPTY)).thenReturn(dataField); - - callInterval = allKeys.get(MSTAGE_CALL_INTERVAL.getConfig()).getAsLong(); - when(state.getPropAsLong(MSTAGE_CALL_INTERVAL.getConfig(), 0L)).thenReturn(callInterval); - - waitTimeoutSeconds = allKeys.get(MSTAGE_WAIT_TIMEOUT_SECONDS.getConfig()).getAsLong(); - when(state.getPropAsLong(MSTAGE_WAIT_TIMEOUT_SECONDS.getConfig(), 0L)).thenReturn(waitTimeoutSeconds); - - enableCleansing = allKeys.get(MSTAGE_ENABLE_CLEANSING.getConfig()).getAsBoolean(); - when(state.getPropAsBoolean(MSTAGE_ENABLE_CLEANSING.getConfig())).thenReturn(enableCleansing); - - workUnitPartialPartition = allKeys.get(MSTAGE_WORK_UNIT_PARTIAL_PARTITION.getConfig()).getAsBoolean(); - when(state.getPropAsBoolean(MSTAGE_WORK_UNIT_PARTIAL_PARTITION.getConfig())).thenReturn(workUnitPartialPartition); - - watermark = allKeys.get(MSTAGE_WATERMARK.getConfig()).getAsJsonArray(); - when(state.getProp(MSTAGE_WATERMARK.getConfig(), new JsonArray().toString())).thenReturn(watermark.toString()); - - secondaryInput = allKeys.get(MSTAGE_SECONDARY_INPUT.getConfig()).getAsJsonArray(); - when(state.getProp(MSTAGE_SECONDARY_INPUT.getConfig(), new JsonArray().toString())).thenReturn(secondaryInput.toString()); - - httpClientFactory = allKeys.get(MSTAGE_HTTP_CLIENT_FACTORY.getConfig()).getAsString(); - when(state.getProp(MSTAGE_HTTP_CLIENT_FACTORY.getConfig(), StringUtils.EMPTY)).thenReturn(httpClientFactory); - - httpRequestHeaders = allKeys.get(MSTAGE_HTTP_REQUEST_HEADERS.getConfig()).getAsJsonObject(); - when(state.getProp(MSTAGE_HTTP_REQUEST_HEADERS.getConfig(), new JsonObject().toString())).thenReturn(httpRequestHeaders.toString()); - - sourceUri = allKeys.get(MSTAGE_SOURCE_URI.getConfig()).getAsString(); - when(state.getProp(MSTAGE_SOURCE_URI.getConfig(), StringUtils.EMPTY)).thenReturn(sourceUri); - - httpRequestMethod = allKeys.get(MSTAGE_HTTP_REQUEST_METHOD.getConfig()).getAsString(); - when(state.getProp(MSTAGE_HTTP_REQUEST_METHOD.getConfig(), StringUtils.EMPTY)).thenReturn(httpRequestMethod); - - extractorClass = allKeys.get(MSTAGE_EXTRACTOR_CLASS.getConfig()).getAsString(); - when(state.getProp(MSTAGE_EXTRACTOR_CLASS.getConfig(), StringUtils.EMPTY)).thenReturn(extractorClass); - - authentication = allKeys.get(MSTAGE_AUTHENTICATION.getConfig()).getAsJsonObject(); - token = authentication.get("token").getAsString(); - when(state.getProp(MSTAGE_AUTHENTICATION.getConfig(), new JsonObject().toString())).thenReturn(authentication.toString()); - - httpStatus = allKeys.get(MSTAGE_HTTP_STATUSES.getConfig()).getAsJsonObject(); - when(state.getProp(MSTAGE_HTTP_STATUSES.getConfig(), new JsonObject().toString())).thenReturn(httpStatus.toString()); - - httpStatusReasons = allKeys.get(MSTAGE_HTTP_STATUS_REASONS.getConfig()).getAsJsonObject(); - when(state.getProp(MSTAGE_HTTP_STATUS_REASONS.getConfig(), new JsonObject().toString())).thenReturn(httpStatusReasons.toString()); - } -} diff --git a/cdi-core/src/test/java/com/linkedin/cdi/connection/JdbcReadConnectionTest.java b/cdi-core/src/test/java/com/linkedin/cdi/connection/JdbcReadConnectionTest.java deleted file mode 100644 index d5d3fa8..0000000 --- a/cdi-core/src/test/java/com/linkedin/cdi/connection/JdbcReadConnectionTest.java +++ /dev/null @@ -1,212 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.connection; - -import com.google.common.collect.ImmutableMap; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import java.io.UnsupportedEncodingException; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import javax.sql.rowset.JdbcRowSet; -import javax.sql.rowset.RowSetMetaDataImpl; -import org.apache.commons.lang3.StringEscapeUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.tuple.MutablePair; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.exception.RetriableAuthenticationException; -import com.linkedin.cdi.extractor.MultistageExtractor; -import com.linkedin.cdi.util.Database; -import com.linkedin.cdi.keys.ExtractorKeys; -import com.linkedin.cdi.keys.JdbcKeys; -import com.linkedin.cdi.util.JsonParameter; -import com.linkedin.cdi.util.ParameterTypes; -import com.linkedin.cdi.util.VariableUtils; -import com.linkedin.cdi.util.WorkUnitStatus; -import org.mockito.Mockito; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.testng.PowerMockTestCase; -import org.powermock.reflect.Whitebox; -import org.testng.Assert; -import org.testng.annotations.Test; - -import static com.linkedin.cdi.configuration.MultistageProperties.*; -import static org.mockito.Matchers.*; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.anyInt; -import static org.mockito.Mockito.*; - - -@PrepareForTest({Database.class, JsonParameter.class, StringEscapeUtils.class, VariableUtils.class}) -public class JdbcReadConnectionTest extends PowerMockTestCase { - /** - * Test getFirst: - * Scenario 1: Fail to get jdbcConnection - * Scenario 2: Fail to execute statement - * @throws UnsupportedEncodingException - */ - @Test - public void testGetFirst() throws UnsupportedEncodingException, RetriableAuthenticationException { - PowerMockito.mockStatic(JsonParameter.class); - PowerMockito.mockStatic(VariableUtils.class); - PowerMockito.mockStatic(Database.class); - when(JsonParameter.getParametersAsJson(any(), any(), any())).thenReturn(new JsonObject()); - MultistageExtractor extractor = mock(MultistageExtractor.class); - ExtractorKeys extractorKeys = mock(ExtractorKeys.class); - when(extractor.getExtractorKeys()).thenReturn(extractorKeys); - when(extractorKeys.getActivationParameters()).thenReturn(new JsonObject()); - when(VariableUtils.replace(any(), any(), any())).thenReturn(new JsonObject()); - - WorkUnitState state = mock(WorkUnitState.class); - when(extractor.getState()).thenReturn(state); - JdbcKeys jdbcSourceKeys = Mockito.mock(JdbcKeys.class); - when(jdbcSourceKeys.getSourceParameters()).thenReturn(new JsonArray()); - - JdbcConnection conn = new JdbcConnection(state, jdbcSourceKeys, extractorKeys); - Assert.assertNull(conn.executeFirst(WorkUnitStatus.builder().build())); - } - - /** - * Test getNext - */ - @Test - public void testGetNext() throws UnsupportedEncodingException, SQLException, RetriableAuthenticationException { - PowerMockito.mockStatic(Database.class); - PowerMockito.mockStatic(VariableUtils.class); - PowerMockito.mockStatic(JsonParameter.class); - - JdbcKeys jdbcSourceKeys = mock(JdbcKeys.class); - when(jdbcSourceKeys.getCallInterval()).thenReturn(1L); - ExtractorKeys extractorKeys = new ExtractorKeys(); - WorkUnitState state = Mockito.mock(WorkUnitState.class); - WorkUnitStatus workUnitStatus = Mockito.mock(WorkUnitStatus.class); - WorkUnitStatus.WorkUnitStatusBuilder builder = Mockito.mock(WorkUnitStatus.WorkUnitStatusBuilder.class); - - JdbcConnection conn = new JdbcConnection(state, jdbcSourceKeys, extractorKeys); - - String jdbcStatement = "select * from linkedin.someTable limit 1000"; - when(jdbcSourceKeys.getJdbcStatement()).thenReturn(jdbcStatement); - extractorKeys.setSignature("testSignature"); - extractorKeys.setActivationParameters(new JsonObject()); - when(builder.build()).thenReturn(workUnitStatus); - when(workUnitStatus.toBuilder()).thenReturn(builder); - - String uri = "jdbc:mysql://odbcva01.clientx.com:3630/linkedin?useSSL=true"; - String username = "username"; - String password = "password"; - when(jdbcSourceKeys.getSourceUri()).thenReturn(uri); - when(state.getProp(SOURCE_CONN_USERNAME.getConfig(), StringUtils.EMPTY)).thenReturn(username); - when(state.getProp(SOURCE_CONN_PASSWORD.getConfig(), StringUtils.EMPTY)).thenReturn(password); - - Pair res = new MutablePair<>(jdbcStatement, new JsonObject()); - when(VariableUtils.replaceWithTracking(any(), any(), any())).thenReturn(res); - when(VariableUtils.replace(any(), any())).thenReturn(new JsonObject()); - when(VariableUtils.replace(any(), any(), any())).thenReturn(new JsonObject()); - - when(jdbcSourceKeys.isPaginationEnabled()).thenReturn(true); - when(jdbcSourceKeys.getSourceParameters()).thenReturn(new JsonArray()); - java.sql.Connection jdbcConnection = PowerMockito.mock(java.sql.Connection.class); - Statement statement = PowerMockito.mock(Statement.class); - PowerMockito.when(jdbcConnection.createStatement()).thenReturn(statement); - ResultSet resultSet = PowerMockito.mock(ResultSet.class); - when(statement.getResultSet()).thenReturn(resultSet); - when(statement.execute(any())).thenReturn(true); - doNothing().when(statement).setFetchSize(anyInt()); - - String unSupportedExtractor = "com.linkedin.cdi.extractor.SomeExtractor"; - when(state.getProp(MSTAGE_EXTRACTOR_CLASS.getConfig(), StringUtils.EMPTY)).thenReturn(unSupportedExtractor); - when(JsonParameter.getParametersAsJson(any(), any(), any())).thenReturn(new JsonObject()); - Assert.assertNull(conn.executeNext(workUnitStatus)); - - when(jdbcSourceKeys.getPaginationInitValues()).thenReturn(ImmutableMap.of(ParameterTypes.PAGESIZE, 100L)); - conn.setJdbcConnection(jdbcConnection); - Assert.assertNull(conn.executeNext(workUnitStatus)); - - String supportedExtractor = "com.linkedin.cdi.extractor.CsvExtractor"; - when(state.getProp(MSTAGE_EXTRACTOR_CLASS.getConfig(), StringUtils.EMPTY)).thenReturn(supportedExtractor); - when(jdbcSourceKeys.hasSourceSchema()).thenReturn(true); - Assert.assertEquals(conn.executeNext(workUnitStatus), workUnitStatus); - - when(statement.execute(any())).thenReturn(false); - Assert.assertEquals(conn.executeNext(workUnitStatus), workUnitStatus); - } - - /** - * Test closeAll - * Scenario: throw an exception - */ - @Test - public void testCloseAll() throws SQLException { - ExtractorKeys extractorKeys = mock(ExtractorKeys.class); - String testSignature = "test_signature"; - when(extractorKeys.getSignature()).thenReturn(testSignature); - JdbcConnection conn = new JdbcConnection(null, new JdbcKeys(), extractorKeys); - conn.closeAll(""); - - java.sql.Connection jdbcConnection = mock(java.sql.Connection.class); - conn.setJdbcConnection(jdbcConnection); - doThrow(new RuntimeException()).when(jdbcConnection).close(); - conn.closeAll(""); - } - - @Test - public void testToCsv() throws Exception { - PowerMockito.mockStatic(StringEscapeUtils.class); - when(StringEscapeUtils.escapeCsv(anyString())).thenReturn("test_data"); - RowSetMetaDataImpl rowSetMetaData = mock(RowSetMetaDataImpl.class); - JdbcRowSet jdbcRowSet = PowerMockito.mock(JdbcRowSet.class); - PowerMockito.when(jdbcRowSet.next()).thenReturn(true).thenReturn(false); - when(rowSetMetaData.getColumnCount()).thenReturn(2); - JdbcConnection conn = new JdbcConnection(null, new JdbcKeys(), null); - Assert.assertEquals(Whitebox.invokeMethod(conn, "toCsv", jdbcRowSet, rowSetMetaData).toString(), - "test_data,test_data" + System.lineSeparator()); - } - - @Test - public void testToJson() throws Exception { - PowerMockito.mockStatic(StringEscapeUtils.class); - when(StringEscapeUtils.escapeCsv(anyString())).thenReturn("test_data"); - RowSetMetaDataImpl rowSetMetaData = mock(RowSetMetaDataImpl.class); - JdbcRowSet jdbcRowSet = PowerMockito.mock(JdbcRowSet.class); - PowerMockito.when(jdbcRowSet.next()) - .thenReturn(true).thenReturn(false) - .thenReturn(true).thenReturn(false) - .thenReturn(true).thenReturn(false); - - when(rowSetMetaData.getColumnCount()).thenReturn(2); - when(rowSetMetaData.getColumnName(1)).thenReturn("column0"); - when(rowSetMetaData.getColumnName(2)).thenReturn("column1"); - JdbcConnection conn = new JdbcConnection(null, new JdbcKeys(), null); - - Assert.assertEquals(Whitebox.invokeMethod(conn, "toJson", jdbcRowSet, rowSetMetaData).toString(), - "[{\"column0\":null,\"column1\":null}]"); - - conn.getJdbcSourceKeys().setSchemaRefactorFunction("toupper"); - Assert.assertEquals(Whitebox.invokeMethod(conn, "toJson", jdbcRowSet, rowSetMetaData).toString(), - "[{\"COLUMN0\":null,\"COLUMN1\":null}]"); - - conn.getJdbcSourceKeys().setSchemaRefactorFunction("tolower"); - Assert.assertEquals(Whitebox.invokeMethod(conn, "toJson", jdbcRowSet, rowSetMetaData).toString(), - "[{\"column0\":null,\"column1\":null}]"); - } - - /** - * Test retrieveSchema - */ - @Test - public void testRetrieveSchema() throws Exception { - RowSetMetaDataImpl rowSetMetaData = mock(RowSetMetaDataImpl.class); - when(rowSetMetaData.getColumnCount()).thenReturn(1); - when(rowSetMetaData.isNullable(1)).thenReturn(2); - when(rowSetMetaData.getColumnName(1)).thenReturn("columnValue"); - when(rowSetMetaData.getColumnType(1)).thenReturn(1); - JdbcConnection conn = new JdbcConnection(null, new JdbcKeys(), null); - Assert.assertEquals(Whitebox.invokeMethod(conn, "retrieveSchema", rowSetMetaData).toString(), - "[{\"columnName\":\"columnValue\",\"isNullable\":false,\"dataType\":{\"type\":\"string\"}}]"); - } -} diff --git a/cdi-core/src/test/java/com/linkedin/cdi/connection/S3ReadConnectionTest.java b/cdi-core/src/test/java/com/linkedin/cdi/connection/S3ReadConnectionTest.java index faa73ad..5964618 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/connection/S3ReadConnectionTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/connection/S3ReadConnectionTest.java @@ -4,18 +4,19 @@ package com.linkedin.cdi.connection; +import com.linkedin.cdi.keys.ExtractorKeys; +import com.linkedin.cdi.source.MultistageSource; +import com.linkedin.cdi.source.S3SourceV2; import gobblin.runtime.JobState; import java.util.List; import org.apache.gobblin.configuration.SourceState; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.keys.ExtractorKeys; -import com.linkedin.cdi.source.MultistageSource; -import com.linkedin.cdi.source.S3SourceV2; import org.apache.gobblin.source.workunit.WorkUnit; import org.testng.Assert; import org.testng.annotations.Test; +import static com.linkedin.cdi.configuration.PropertyCollection.*; + @Test public class S3ReadConnectionTest { @@ -23,11 +24,11 @@ public class S3ReadConnectionTest { public void testGetS3HttpClient() { List wus = new MultistageSource().getWorkunits(new SourceState()); WorkUnitState wuState = new WorkUnitState(wus.get(0), new JobState()); - wuState.setProp(MultistageProperties.MSTAGE_HTTP_CLIENT_FACTORY.getConfig(), "com.linkedin.cdi.factory.ApacheHttpClientFactory"); + wuState.setProp(MSTAGE_CONNECTION_CLIENT_FACTORY.getConfig(), "com.linkedin.cdi.factory.DefaultConnectionClientFactory"); S3SourceV2 source = new S3SourceV2(); SourceState sourceState = new SourceState(); - sourceState.setProp(MultistageProperties.MSTAGE_SOURCE_URI.getConfig(), "https://nonexist.s3.amazonaws.com/data"); + sourceState.setProp(MSTAGE_SOURCE_URI.getConfig(), "https://nonexist.s3.amazonaws.com/data"); source.getWorkunits(sourceState); S3Connection conn = new S3Connection(wuState, source.getS3SourceV2Keys(), new ExtractorKeys()); diff --git a/cdi-core/src/test/java/com/linkedin/cdi/converter/AvroNormalizerConverterTest.java b/cdi-core/src/test/java/com/linkedin/cdi/converter/AvroNormalizerConverterTest.java deleted file mode 100644 index 359b60e..0000000 --- a/cdi-core/src/test/java/com/linkedin/cdi/converter/AvroNormalizerConverterTest.java +++ /dev/null @@ -1,81 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.converter; - -import com.google.gson.Gson; -import com.google.gson.JsonArray; -import lombok.SneakyThrows; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.gobblin.configuration.WorkUnitState; -import org.apache.gobblin.converter.DataConversionException; -import org.apache.gobblin.converter.SchemaConversionException; -import org.apache.gobblin.converter.avro.UnsupportedDateTypeException; -import com.linkedin.cdi.util.AvroSchemaUtils; -import org.apache.gobblin.source.workunit.Extract; -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import static org.mockito.Mockito.*; - - -public class AvroNormalizerConverterTest { - AvroNormalizerConverter _avroNormalizerConverter; - String sourceSchema = "[{\"columnName\":\"asIs\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}," - + "{\"columnName\":\"toBeNormalized1\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}," - + "{\"columnName\":\"toBeNormalized2\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}]"; - String targetSchema = "[{\"columnName\":\"asIs\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}," - + "{\"columnName\":\"normalized\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}]"; - Schema inputSchema; - Schema outputSchema; - WorkUnitState state; - - @SneakyThrows - @BeforeMethod - public void beforeMethod() { - _avroNormalizerConverter = new AvroNormalizerConverter(); - state = mock(WorkUnitState.class); - Extract extract = new Extract(Extract.TableType.SNAPSHOT_ONLY, "com.linkedin.test", "test"); - when(state.getProp("ms.target.schema", new JsonArray().toString())).thenReturn(targetSchema); - when(state.getPropAsLong("ms.normalizer.batch.size", 0L)).thenReturn(2L); - when(state.getExtract()).thenReturn(extract); - _avroNormalizerConverter.init(state); - inputSchema = AvroSchemaUtils.fromJsonSchema(new Gson().fromJson(sourceSchema, JsonArray.class), state); - outputSchema = AvroSchemaUtils.fromJsonSchema(new Gson().fromJson(targetSchema, JsonArray.class), state); - } - - @Test - public void testConvertSchema() throws SchemaConversionException, UnsupportedDateTypeException { - Schema schema = _avroNormalizerConverter.convertSchema(inputSchema, state); - - Assert.assertEquals(schema, - AvroSchemaUtils.fromJsonSchema(new Gson().fromJson(targetSchema, JsonArray.class), state)); - } - - @Test - public void testConvertRecord() throws SchemaConversionException, DataConversionException { - _avroNormalizerConverter.convertSchema(inputSchema, state); - GenericRecord inputRecord = new GenericData.Record(inputSchema); - inputRecord.put("asIs", "dummy"); - inputRecord.put("toBeNormalized1", "dummy"); - inputRecord.put("toBeNormalized2", "dummy"); - // Call twice to make sure the resulting record gives JsonArray size 2 - _avroNormalizerConverter.convertRecord(outputSchema, inputRecord, state); - Iterable recordIterable = _avroNormalizerConverter.convertRecord(outputSchema, inputRecord, state); - GenericRecord record = recordIterable.iterator().next(); - GenericData.Array normalized = (GenericData.Array) record.get("normalized"); - Assert.assertEquals(normalized.size(), 2); - // There's 1 record in the buffer before before passing eof - _avroNormalizerConverter.convertRecord(outputSchema, record, state); - GenericRecord eof = AvroSchemaUtils.createEOF(state); - record = _avroNormalizerConverter.convertRecord(outputSchema, eof, state).iterator().next(); - normalized = (GenericData.Array) record.get("normalized"); - Assert.assertEquals(normalized.size(), 1); - // When there are no records in the buffer calling before eof - Assert.assertFalse(_avroNormalizerConverter.convertRecord(outputSchema, eof, state).iterator().hasNext()); - } -} \ No newline at end of file diff --git a/cdi-core/src/test/java/com/linkedin/cdi/converter/JsonNormalizerConverterTest.java b/cdi-core/src/test/java/com/linkedin/cdi/converter/JsonNormalizerConverterTest.java index ecc2186..743a035 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/converter/JsonNormalizerConverterTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/converter/JsonNormalizerConverterTest.java @@ -6,6 +6,7 @@ import com.google.gson.Gson; import com.google.gson.JsonArray; +import com.google.gson.JsonNull; import com.google.gson.JsonObject; import org.apache.gobblin.configuration.WorkUnitState; import org.testng.Assert; @@ -16,34 +17,40 @@ @Test public class JsonNormalizerConverterTest { JsonNormalizerConverter underTest; - String sourceSchema = "[{\"columnName\":\"asIs\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}," + String sourceSchemaOrdinary = "[{\"columnName\":\"asIs\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}," + "{\"columnName\":\"toBeNormalized1\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}," + "{\"columnName\":\"toBeNormalized2\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}]"; - String targetSchema = "[{\"columnName\":\"asIs\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}," + String targetSchemaOrdinary = "[{\"columnName\":\"asIs\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}," + "{\"columnName\":\"normalized\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}]"; + String sourceSchemaWithNullableFields = "[{\"columnName\":\"toBeNormalized0\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}," + + "{\"columnName\":\"asIs\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}," + + "{\"columnName\":\"toBeNormalized1\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}," + + "{\"columnName\":\"toBeNormalized2\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]"; + String targetSchemaWithNullableFields = "[{\"columnName\":\"asIs\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}, " + + "{\"columnName\":\"nullable\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}," + + "{\"columnName\":\"normalized\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"map\", \"values\": \"string\"}}]"; JsonArray inputSchema; JsonArray outputSchema; WorkUnitState state; - @BeforeMethod - public void beforeMethod() { + + private void setup(String sourceSchema, String targetSchema, int batchSize) { underTest = new JsonNormalizerConverter(); state = new WorkUnitState(); state.setProp("ms.target.schema", targetSchema); - state.setProp("ms.normalizer.batch.size", 2); + state.setProp("ms.normalizer.batch.size", batchSize); underTest.init(state); inputSchema = new Gson().fromJson(sourceSchema, JsonArray.class); outputSchema = new Gson().fromJson(targetSchema, JsonArray.class); } @Test - public void testConvertSchema() { - Assert.assertTrue(outputSchema.equals(underTest.convertSchema(inputSchema, state))); - } + public void testConvertSchemaAndRecord() { + setup(sourceSchemaOrdinary, targetSchemaOrdinary, 2); + // test convert schema + Assert.assertEquals(underTest.convertSchema(inputSchema, state), outputSchema); - @Test - public void testConvertRecord() { - underTest.convertSchema(inputSchema, state); + // test convert record JsonObject record = new JsonObject(); record.addProperty("asIs", "dummy"); record.addProperty("toBeNormalized1", "dummy"); @@ -52,7 +59,6 @@ public void testConvertRecord() { underTest.convertRecord(outputSchema, record, state); Iterable recordIterable = underTest.convertRecord(outputSchema, record, state); JsonObject jsonObject = recordIterable.iterator().next(); - Assert.assertEquals(jsonObject.getAsJsonArray("normalized").size(), 2); // There's 1 record in the buffer before before passing eof underTest.convertRecord(outputSchema, record,state); JsonObject eof = new JsonObject(); @@ -62,4 +68,38 @@ public void testConvertRecord() { // When there are no records in the buffer calling before eof Assert.assertFalse(underTest.convertRecord(outputSchema, eof, state).iterator().hasNext()); } + + public void testConvertSchemaAndRecordWithNullableFields() { + setup(sourceSchemaWithNullableFields, targetSchemaWithNullableFields, 1); + // test convert schema + Assert.assertEquals(underTest.convertSchema(inputSchema, state), outputSchema); + + // test convert record + JsonObject record = new JsonObject(); + record.addProperty("asIs", "dummy"); + record.addProperty("nullable", "dummy"); + record.addProperty("toBeNormalized1", "dummy"); + record.addProperty("toBeNormalized2", "dummy"); + Iterable recordIterable = underTest.convertRecord(outputSchema, record, state); + JsonObject jsonObject = recordIterable.iterator().next(); + Assert.assertEquals(jsonObject.entrySet().size(), 3); + Assert.assertEquals(jsonObject.getAsJsonObject("normalized").entrySet().size(), 2); + // There's 1 record in the buffer before before passing eof + JsonObject recordWithNullableField = new JsonObject(); + recordWithNullableField.addProperty("asIs", "dummy"); + recordWithNullableField.add("toBeNormalized0", JsonNull.INSTANCE); + // map type should not include this null value + recordWithNullableField.addProperty("toBeNormalized0", (String) null); + recordWithNullableField.addProperty("toBeNormalized1", "dummy"); + recordWithNullableField.addProperty("toBeNormalized2", "dummy"); + recordIterable = underTest.convertRecord(outputSchema, recordWithNullableField, state); + jsonObject = recordIterable.iterator().next(); + Assert.assertEquals(jsonObject.entrySet().size(), 2); + Assert.assertEquals(jsonObject.getAsJsonObject("normalized").entrySet().size(), 2); + JsonObject eof = new JsonObject(); + eof.addProperty("EOF", "EOF"); + // When there are no records in the buffer calling before eof + Assert.assertFalse(underTest.convertRecord(outputSchema, eof, state).iterator().hasNext()); + } + } \ No newline at end of file diff --git a/cdi-core/src/test/java/com/linkedin/cdi/extractor/AvroExtractorTest.java b/cdi-core/src/test/java/com/linkedin/cdi/extractor/AvroExtractorTest.java index 361a9f3..876db07 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/extractor/AvroExtractorTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/extractor/AvroExtractorTest.java @@ -8,11 +8,18 @@ import com.google.common.collect.ImmutableMap; import com.google.gson.JsonArray; import com.google.gson.JsonObject; +import com.linkedin.cdi.connection.MultistageConnection; +import com.linkedin.cdi.exception.RetriableAuthenticationException; +import com.linkedin.cdi.keys.AvroExtractorKeys; +import com.linkedin.cdi.keys.JobKeys; +import com.linkedin.cdi.source.HttpSource; +import com.linkedin.cdi.source.MultistageSource; +import com.linkedin.cdi.util.JsonUtils; +import com.linkedin.cdi.util.WorkUnitStatus; import java.io.InputStream; import java.util.HashMap; import java.util.List; import java.util.Map; -import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; import org.apache.avro.generic.GenericData; @@ -20,15 +27,6 @@ import org.apache.commons.lang.StringUtils; import org.apache.gobblin.configuration.SourceState; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.connection.MultistageConnection; -import com.linkedin.cdi.exception.RetriableAuthenticationException; -import com.linkedin.cdi.keys.AvroExtractorKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.source.HttpSource; -import com.linkedin.cdi.source.MultistageSource; -import com.linkedin.cdi.util.JsonUtils; -import com.linkedin.cdi.util.WorkUnitStatus; import org.apache.gobblin.source.workunit.Extract; import org.apache.gobblin.source.workunit.WorkUnit; import org.apache.gobblin.util.AvroUtils; @@ -39,20 +37,21 @@ import org.joda.time.format.DateTimeFormatter; import org.mockito.Mockito; import org.powermock.reflect.Whitebox; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.BeforeTest; import org.testng.annotations.Test; -import static com.linkedin.cdi.configuration.MultistageProperties.*; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static com.linkedin.cdi.configuration.StaticConstants.*; import static org.mockito.Mockito.*; @Test -@Slf4j public class AvroExtractorTest { - + private static final Logger LOG = LoggerFactory.getLogger(AvroExtractorTest.class); private final static String DATA_SET_URN_KEY = "com.linkedin.somecase.SeriesCollection"; private final static String ACTIVATION_PROP = "{\"name\": \"survey\", \"type\": \"unit\", \"units\": \"id1,id2\"}"; private final static String DATA_FINAL_DIR = "/jobs/testUser/gobblin/useCaseRoot"; @@ -82,7 +81,7 @@ public void setUp() throws RetriableAuthenticationException { List wus = new MultistageSource().getWorkunits(new SourceState()); workUnit = wus.get(0); - workUnit.setProp(MultistageProperties.DATASET_URN_KEY.getConfig(), DATA_SET_URN_KEY); + workUnit.setProp(DATASET_URN.getConfig(), DATA_SET_URN_KEY); jobKeys = mock(JobKeys.class); workUnitStatus = mock(WorkUnitStatus.class); @@ -95,7 +94,7 @@ public void setUp() throws RetriableAuthenticationException { // mock for state when(state.getWorkunit()).thenReturn(workUnit); when(state.getProp(MSTAGE_ACTIVATION_PROPERTY.getConfig(), new JsonObject().toString())).thenReturn(ACTIVATION_PROP); - when(state.getPropAsLong(MSTAGE_WORKUNIT_STARTTIME_KEY.getConfig(), 0L)).thenReturn(WORK_UNIT_START_TIME_KEY); + when(state.getPropAsLong(MSTAGE_WORK_UNIT_SCHEDULING_STARTTIME.getConfig(), 0L)).thenReturn(WORK_UNIT_START_TIME_KEY); when(state.getProp(DATA_PUBLISHER_FINAL_DIR.getConfig(), StringUtils.EMPTY)).thenReturn(DATA_FINAL_DIR); when(state.getProp(MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION.getConfig(), StringUtils.EMPTY)).thenReturn(FILE_PERMISSION); Extract extract = new Extract(Extract.TableType.SNAPSHOT_ONLY, "com.linkedin.test", "test"); @@ -329,4 +328,232 @@ public void testGetSchema() throws Exception { Assert.assertEquals(avroSchema.getName(), "test"); Assert.assertEquals(avroSchema.getNamespace(), "com.linkedin.test"); } + + /** + * When ms.data.field is an array + * data = { + * "results": [ + * { + * "field1": "a", + * "field2": "aa" + * }, + * { + * "field1": "b", + * "field2": "bb" + * }, + * { + * "field1": "c", + * "field2": "cc" + * } + * ] + * } + * ms.data.field = "results" + * @throws Exception exception + */ + @Test + public void testMSDataField1() throws Exception { + InputStream inputStream = getClass().getResourceAsStream("/avro/ArrayFieldTest.avro"); + WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); + + when(sourceState.getProp("ms.output.schema", "" )).thenReturn(""); + + // replace mocked keys with default keys + realHttpSource.getWorkunits(sourceState); + avroExtractor.jobKeys = jobKeys; + avroExtractor.setAvroExtractorKeys(new AvroExtractorKeys()); + when(jobKeys.getSourceParameters()).thenReturn(realHttpSource.getJobKeys().getSourceParameters()); + when(jobKeys.getDataField()).thenReturn("results"); + when(multistageConnection.executeFirst(avroExtractor.workUnitStatus)).thenReturn(status); + + // schema should be of type record + Schema schema = avroExtractor.getSchema(); + Assert.assertEquals(schema.getType(), Schema.Type.RECORD); + + // there should be 1 records processed + GenericRecord rst = avroExtractor.readRecord(null); + /* expected data = { + * "results": [ + * { + * "field1": "a", + * "field2": "aa" + * }, + * { + * "field1": "b", + * "field2": "bb" + * }, + * { + * "field1": "c", + * "field2": "cc" + * } + * ] + * } + */ + Assert.assertEquals(AvroUtils.getFieldValue(rst, "results.0.field1").get().toString(), "a"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "results.0.field2").get().toString(), "aa"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "results.1.field1").get().toString(), "b"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "results.1.field2").get().toString(), "bb"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "results.2.field1").get().toString(), "c"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "results.2.field2").get().toString(), "cc"); + while (avroExtractor.hasNext()) { + avroExtractor.readRecord(null); + } + Assert.assertEquals(1, avroExtractor.getAvroExtractorKeys().getProcessedCount()); + } + + /** + * When ms.data.field is a single record + * data = { + * "results": { + * "field1": "a", + * "field2": "aa" + * } + * } + * ms.data.field = "results" + * @throws Exception exception + */ + @Test + public void testMSDataField2() throws Exception { + InputStream inputStream = getClass().getResourceAsStream("/avro/SingleRecordArrayFieldTest.avro"); + WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); + + when(sourceState.getProp("ms.output.schema", "" )).thenReturn(""); + + // replace mocked keys with default keys + realHttpSource.getWorkunits(sourceState); + avroExtractor.jobKeys = jobKeys; + avroExtractor.setAvroExtractorKeys(new AvroExtractorKeys()); + when(jobKeys.getSourceParameters()).thenReturn(realHttpSource.getJobKeys().getSourceParameters()); + when(jobKeys.getDataField()).thenReturn("results"); + when(multistageConnection.executeFirst(avroExtractor.workUnitStatus)).thenReturn(status); + + // schema should be of type record + Schema schema = avroExtractor.getSchema(); + Assert.assertEquals(schema.getType(), Schema.Type.RECORD); + + // there should be 1 records processed + GenericRecord rst = avroExtractor.readRecord(null); + /* + * expected data = { + * "results": { + * "field1": "a", + * "field2": "aa" + * } + * } + */ + Assert.assertEquals(AvroUtils.getFieldValue(rst, "results.field1").get().toString(), "a"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "results.field2").get().toString(), "aa"); + while (avroExtractor.hasNext()) { + avroExtractor.readRecord(null); + } + Assert.assertEquals(1, avroExtractor.getAvroExtractorKeys().getProcessedCount()); + } + + /** + * When ms.data.field is deep in a nested structure + * data + * Record 1 { + * "results": [ + * { + * "wrapper": { + * "field1": [ + * { + * "field11": "a11", + * "field12": "a12" + * }, + * { + * "field11": "aa11", + * "field12": "aa12" + * } + * ] + * }, + * "field2": "aa" + * } + * ] + * } + * Record 2 { + * "results": [ + * { + * "wrapper": { + * "field1": [ + * { + * "field11": "b11", + * "field12": "b12" + * }, + * { + * "field11": "bb11", + * "field12": "bb12" + * } + * ] + * }, + * "field2": "bb" + * } + * ] + * } + * ms.data.field = "results.0.wrapper.field1" + * @throws Exception exception + */ + @Test + public void testMSDataField3() throws Exception { + InputStream inputStream = getClass().getResourceAsStream("/avro/NestedDataFieldTest.avro"); + WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); + + when(sourceState.getProp("ms.output.schema", "" )).thenReturn(""); + + // replace mocked keys with default keys + realHttpSource.getWorkunits(sourceState); + avroExtractor.jobKeys = jobKeys; + avroExtractor.setAvroExtractorKeys(new AvroExtractorKeys()); + when(jobKeys.getSourceParameters()).thenReturn(realHttpSource.getJobKeys().getSourceParameters()); + when(jobKeys.getDataField()).thenReturn("results.0.wrapper.field1"); + when(multistageConnection.executeFirst(avroExtractor.workUnitStatus)).thenReturn(status); + + // schema should be of type record + Schema schema = avroExtractor.getSchema(); + Assert.assertEquals(schema.getType(), Schema.Type.RECORD); + + // there should be 2 records processed + GenericRecord rst = avroExtractor.readRecord(null); + /* + * expected data = { + * "field1": [ + * { + * "field11": "a11", + * "field12": "a12" + * }, + * { + * "field11": "aa11", + * "field12": "aa12" + * } + * ] + * } + */ + Assert.assertEquals(AvroUtils.getFieldValue(rst, "field1.0.field11").get().toString(), "a11"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "field1.0.field12").get().toString(), "a12"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "field1.1.field11").get().toString(), "aa11"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "field1.1.field12").get().toString(), "aa12"); + + rst = avroExtractor.readRecord(null); + /* + * expected data = { + * "field1": [ + * { + * "field11": "b11", + * "field12": "b12" + * }, + * { + * "field11": "bb11", + * "field12": "bb12" + * } + * ] + * } + */ + Assert.assertEquals(AvroUtils.getFieldValue(rst, "field1.0.field11").get().toString(), "b11"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "field1.0.field12").get().toString(), "b12"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "field1.1.field11").get().toString(), "bb11"); + Assert.assertEquals(AvroUtils.getFieldValue(rst, "field1.1.field12").get().toString(), "bb12"); + while (avroExtractor.hasNext()) { + avroExtractor.readRecord(null); + } + Assert.assertEquals(2, avroExtractor.getAvroExtractorKeys().getProcessedCount()); + } } diff --git a/cdi-core/src/test/java/com/linkedin/cdi/extractor/CsvExtractorTest.java b/cdi-core/src/test/java/com/linkedin/cdi/extractor/CsvExtractorTest.java deleted file mode 100644 index e74ee35..0000000 --- a/cdi-core/src/test/java/com/linkedin/cdi/extractor/CsvExtractorTest.java +++ /dev/null @@ -1,734 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.extractor; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import com.google.gson.JsonParser; -import com.opencsv.CSVParserBuilder; -import com.opencsv.CSVReader; -import com.opencsv.CSVReaderBuilder; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.lang.reflect.Method; -import java.nio.charset.StandardCharsets; -import java.util.Deque; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang.StringUtils; -import org.apache.gobblin.configuration.SourceState; -import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.connection.MultistageConnection; -import com.linkedin.cdi.exception.RetriableAuthenticationException; -import com.linkedin.cdi.filter.JsonSchemaBasedFilter; -import com.linkedin.cdi.keys.CsvExtractorKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.source.HttpSource; -import com.linkedin.cdi.source.MultistageSource; -import com.linkedin.cdi.util.WorkUnitStatus; -import org.apache.gobblin.source.extractor.utils.InputStreamCSVReader; -import org.apache.gobblin.source.workunit.WorkUnit; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Period; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; -import org.mockito.Mockito; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.reflect.Whitebox; -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.BeforeTest; -import org.testng.annotations.Test; - -import static com.linkedin.cdi.configuration.MultistageProperties.*; -import static org.mockito.Mockito.*; - - -@Test -@Slf4j -public class CsvExtractorTest { - - private final static String DATA_SET_URN_KEY = "com.linkedin.somecase.SeriesCollection"; - private final static String ACTIVATION_PROP = "{\"name\": \"survey\", \"type\": \"unit\", \"units\": \"id1,id2\"}"; - private final static String DATA_FINAL_DIR = "/jobs/testUser/gobblin/useCaseRoot"; - private final static String FILE_PERMISSION = "775"; - private final static long ONE_HOUR_IN_MILLS = 3600000L; - private final static long WORK_UNIT_START_TIME_KEY = 1590994800000L; - JsonArray outputJsonSchema; - JsonObject schema; - private WorkUnitState state; - private SourceState sourceState; - private MultistageSource multiStageSource; - private HttpSource httpSource; - private HttpSource realHttpSource; - private WorkUnit workUnit; - private JobKeys jobKeys; - private CsvExtractor csvExtractor; - private WorkUnitStatus workUnitStatus; - private CsvExtractorKeys csvExtractorKeys; - private MultistageConnection multistageConnection; - - @BeforeMethod - public void setUp() throws RetriableAuthenticationException { - state = mock(WorkUnitState.class); - sourceState = mock(SourceState.class); - multiStageSource = mock(MultistageSource.class); - httpSource = mock(HttpSource.class); - realHttpSource = new HttpSource(); - - List wus = new MultistageSource().getWorkunits(new SourceState()); - workUnit = wus.get(0); - workUnit.setProp(MultistageProperties.DATASET_URN_KEY.getConfig(), DATA_SET_URN_KEY); - - jobKeys = mock(JobKeys.class); - workUnitStatus = mock(WorkUnitStatus.class); - - csvExtractorKeys = mock(CsvExtractorKeys.class); - when(csvExtractorKeys.getActivationParameters()).thenReturn(new JsonObject()); - - - outputJsonSchema = new JsonArray(); - schema = new JsonObject(); - - // mock for state - when(state.getWorkunit()).thenReturn(workUnit); - when(state.getProp(MSTAGE_ACTIVATION_PROPERTY.getConfig(), new JsonObject().toString())).thenReturn(ACTIVATION_PROP); - when(state.getPropAsLong(MSTAGE_WORKUNIT_STARTTIME_KEY.getConfig(), 0L)).thenReturn(WORK_UNIT_START_TIME_KEY); - when(state.getProp(DATA_PUBLISHER_FINAL_DIR.getConfig(), StringUtils.EMPTY)).thenReturn(DATA_FINAL_DIR); - when(state.getProp(MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION.getConfig(), StringUtils.EMPTY)).thenReturn(FILE_PERMISSION); - - // mock for source state - when(sourceState.getProp("extract.table.type", "SNAPSHOT_ONLY")).thenReturn("SNAPSHOT_ONLY"); - when(sourceState.contains("source.conn.use.proxy.url")).thenReturn(true); - - - // mock for source - when(multiStageSource.getJobKeys()).thenReturn(jobKeys); - - // mock for source keys - when(jobKeys.getOutputSchema()).thenReturn(outputJsonSchema); - when(jobKeys.getDerivedFields()).thenReturn(new HashMap<>()); - - csvExtractor = new CsvExtractor(state, multiStageSource.getJobKeys()); - csvExtractor.setCsvExtractorKeys(csvExtractorKeys); - csvExtractor.jobKeys = jobKeys; - - multistageConnection = Mockito.mock(MultistageConnection.class); - when(multistageConnection.executeFirst(workUnitStatus)).thenReturn(workUnitStatus); - when(multistageConnection.executeNext(workUnitStatus)).thenReturn(workUnitStatus); - csvExtractor.setConnection(multistageConnection); } - - @BeforeTest - public void setup() { - if (System.getProperty("hadoop.home.dir") == null) { - System.setProperty("hadoop.home.dir", "/tmp"); - } - } - - /** - * testing vanilla CSV Extractor - */ - @Test - void testExtractCSV1() throws RetriableAuthenticationException { - InputStream inputStream = getClass().getResourceAsStream("/csv/common-crawl-files.csv"); - WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); - - when(sourceState.getProp("ms.output.schema", "" )).thenReturn(""); - when(state.getProp("ms.csv.separator", "")).thenReturn("u0009"); - - // replace mocked keys with default keys - realHttpSource.getWorkunits(sourceState); - csvExtractor.jobKeys = realHttpSource.getJobKeys(); - csvExtractor.setCsvExtractorKeys(new CsvExtractorKeys()); - - when(multistageConnection.executeFirst(csvExtractor.workUnitStatus)).thenReturn(status); - - csvExtractor.readRecord(null); - while (csvExtractor.hasNext()) { - String[] rst = csvExtractor.readRecord(null); - } - Assert.assertEquals(10, csvExtractor.getCsvExtractorKeys().getProcessedCount()); - } - - /** - * testing u0004 - */ - @Test - void testExtractCSV2() throws RetriableAuthenticationException { - InputStream inputStream = getClass().getResourceAsStream("/csv/ctl_d_text.dat"); - WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); - - when(sourceState.getProp("ms.output.schema", "" )).thenReturn(""); - when(state.getProp("ms.csv.separator", "")).thenReturn("u0004"); - - realHttpSource.getWorkunits(sourceState); - CsvExtractor extractor = new CsvExtractor(state, realHttpSource.getHttpSourceKeys()); - when(multistageConnection.executeFirst(extractor.workUnitStatus)).thenReturn(status); - extractor.setConnection(multistageConnection); - - extractor.readRecord(null); - while (extractor.hasNext()) { - String[] x = extractor.readRecord(null); - Assert.assertNotNull(x); - Assert.assertEquals(15, x.length); - } - Assert.assertEquals(2, extractor.getCsvExtractorKeys().getProcessedCount()); - } - - @Test - void testExtractCSV3() throws RetriableAuthenticationException { - InputStream inputStream = getClass().getResourceAsStream("/csv/comma-separated.csv"); - WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); - - when(sourceState.getProp("ms.output.schema", "" )).thenReturn(""); - when(MultistageProperties.MSTAGE_CSV_SEPARATOR.getValidNonblankWithDefault(state)).thenReturn("u002c"); - - realHttpSource.getWorkunits(sourceState); - CsvExtractor extractor = new CsvExtractor(state, realHttpSource.getHttpSourceKeys()); - when(multistageConnection.executeFirst(extractor.workUnitStatus)).thenReturn(status); - extractor.setConnection(multistageConnection); - - extractor.readRecord(null); - while (extractor.hasNext()) { - String[] record = extractor.readRecord(null); - Assert.assertNotNull(record); - Assert.assertEquals(record.length, 2); - } - Assert.assertEquals(5, extractor.getCsvExtractorKeys().getProcessedCount()); - } - - /** - * testing CSV extractor with Gunzip processor - */ - @Test - void testExtractGzippedCSV() throws RetriableAuthenticationException { - InputStream inputStream = getClass().getResourceAsStream("/gzip/cc-index.paths.gz"); - WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); - - when(sourceState.getProp("ms.output.schema", "" )).thenReturn(""); - when(state.getProp("ms.extract.preprocessors", "")).thenReturn("com.linkedin.cdi.preprocessor.GunzipProcessor"); - - realHttpSource.getWorkunits(sourceState); - CsvExtractor extractor = new CsvExtractor(state, realHttpSource.getHttpSourceKeys()); - when(multistageConnection.executeFirst(extractor.workUnitStatus)).thenReturn(status); - extractor.setConnection(multistageConnection); - - extractor.readRecord(null); - while (extractor.hasNext()) { - extractor.readRecord(null); - } - Assert.assertEquals(302, extractor.getCsvExtractorKeys().getProcessedCount()); - } - - - /** - * testing CSV Extractor schema inference - * In this case, a column name contains an illegal character. Since ms.enable.cleansing is enabled by default, - * "$" in the header should be converted to "_" but the actual data will not be cleansed. - */ - @Test - void testExtractCSVSchemaInference() throws RetriableAuthenticationException { - InputStream inputStream = getClass().getResourceAsStream("/csv/ids_need_cleansing.csv"); - WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); - - when(sourceState.getProp("ms.derived.fields", new JsonArray().toString())).thenReturn("[{\"name\": \"snapshotDate\", \"formula\": {\"type\": \"epoc\", \"source\": \"currentdate\"}}]"); - // The following line is added intentionally to make sure that column projection is not enabled when user does not specify the output schema - when(state.getProp("ms.csv.column.projection", StringUtils.EMPTY)).thenReturn("0,4,2-3"); - when(state.getProp("ms.csv.column.header", StringUtils.EMPTY)).thenReturn("true"); - when(state.getProp(MSTAGE_ENABLE_CLEANSING.getConfig(), StringUtils.EMPTY)).thenReturn(""); - when(state.getProp("ms.csv.column.header", StringUtils.EMPTY)).thenReturn("true"); - when(state.getPropAsBoolean("ms.csv.column.header")).thenReturn(true); - when(sourceState.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); - - realHttpSource.getWorkunits(sourceState); - CsvExtractor extractor = new CsvExtractor(state, realHttpSource.getHttpSourceKeys()); - when(multistageConnection.executeFirst(extractor.workUnitStatus)).thenReturn(status); - extractor.setConnection(multistageConnection); - - JsonParser parser = new JsonParser(); - JsonArray schema = parser.parse(extractor.getSchema()).getAsJsonArray(); - Assert.assertEquals(schema.get(0).getAsJsonObject().get("columnName").getAsString(), "id_0"); - Assert.assertEquals(schema.size(), 2); - - // check if schema has been added - String[] row; - row = extractor.readRecord(null); - Assert.assertNotNull(row); - Assert.assertEquals(row[0], "497766636$"); - Assert.assertEquals(row.length, 2); - while (extractor.hasNext()) { - row = extractor.readRecord(null); - Assert.assertNotNull(row); - Assert.assertEquals(row.length, 2); - } - Assert.assertEquals(10, extractor.getCsvExtractorKeys().getProcessedCount()); - } - - /** - * Various tests for column projection - */ - @Test - void testColumnProjection() throws RetriableAuthenticationException { - // testing column projection with schema and ms.csv.column.projection specified - testColumnProjectionHelper("/csv/flat.csv", - "[{\"columnName\":\"col1\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col3\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col4\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col5\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]", - true, - false, null, "0,4,2-3"); - - // testing column projection with schema and header, but without ms.csv.column.projection specified - testColumnProjectionHelper("/csv/flat.csv", - "[{\"columnName\":\"col1\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col3\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col4\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col5\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]", - true, - false, null, ""); - - // testing column projection with schema, but without header and ms.csv.column.projection specified - testColumnProjectionHelper("/csv/flat_without_header.csv", - "[{\"columnName\":\"col1\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col2\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col3\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col4\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]", - false, - false, null, ""); - - // testing column projection with schema and header, but schema contains some fields not in the header - testColumnProjectionHelper("/csv/flat.csv", - "[{\"columnName\":\"col11\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col3\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col4\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col5\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]", - true, - true, new String[][]{{"val1", "val2", "val3", "val4"}, {"val6", "val7", "val8", "val9"}}, ""); - - // testing column projection with schema and header, but headers are in upper case - testColumnProjectionHelper("/csv/flat_uppercase_header.csv", - "[{\"columnName\":\"col1\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col3\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col4\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}, {\"columnName\":\"col5\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]", - true, - true, new String[][]{{"val1", "val3", "val4", "val5"}, {"val6", "val8", "val9", "val10"}}, ""); - } - - /** - * Utility function to test column projection - * @param filePath csv file path string - * @param outputSchema output schema - * @param hasHeader flag for having header - * @param shouldValidateContent flag for checking content explicitly - * @param contents array of contents for explicit checking - * @param columnProjection explicit column projection string - */ - private void testColumnProjectionHelper(String filePath, String outputSchema, boolean hasHeader, - boolean shouldValidateContent, String[][] contents, String columnProjection) - throws RetriableAuthenticationException { - InputStream inputStream = getClass().getResourceAsStream(filePath); - WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); - when(sourceState.getProp("ms.output.schema", new JsonArray().toString())).thenReturn(outputSchema); - when(state.getProp("ms.csv.column.projection", StringUtils.EMPTY)).thenReturn(columnProjection); - when(state.getProp("ms.csv.column.header", StringUtils.EMPTY)).thenReturn(String.valueOf(hasHeader)); - when(state.getPropAsBoolean("ms.csv.column.header")).thenReturn(hasHeader); - - realHttpSource.getWorkunits(sourceState); - CsvExtractor extractor = new CsvExtractor(state, realHttpSource.getHttpSourceKeys()); - when(multistageConnection.executeFirst(extractor.workUnitStatus)).thenReturn(status); - extractor.setConnection(multistageConnection); - - // check if schema has been added - JsonParser parser = new JsonParser(); - String schema = extractor.getSchema(); - Assert.assertEquals(parser.parse(schema).getAsJsonArray().size(), 4); - String[] row; - int index = 0; - row = extractor.readRecord(null); - Assert.assertNotNull(row); - Assert.assertEquals(row.length, 4); - if(shouldValidateContent) { - Assert.assertEquals(row, contents[index++]); - } - while (extractor.hasNext()) { - row = extractor.readRecord(null); - Assert.assertNotNull(row); - Assert.assertEquals(row.length, 4); - if(shouldValidateContent) { - Assert.assertEquals(row, contents[index++]); - } - } - Assert.assertEquals(2, extractor.getCsvExtractorKeys().getProcessedCount()); - } - - /** - * testing the interaction between add derived field with column projection - * column projection defined and the column excluded in the middle - */ - @Test - void testAddDerivedFieldWithColumnProjection1() throws RetriableAuthenticationException { - InputStream inputStream = getClass().getResourceAsStream("/csv/ids_flat.csv"); - WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); - - when(sourceState.getProp("ms.output.schema", new JsonArray().toString())).thenReturn("[{\"columnName\":\"id0\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"date\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id2\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id3\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id4\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id5\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id6\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id7\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id8\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]"); - when(sourceState.getProp("ms.derived.fields", new JsonArray().toString())).thenReturn("[{\"name\": \"date\", \"formula\": {\"type\": \"epoc\", \"source\": \"date\", \"format\": \"yyyy-MM-dd Z\"}}]"); - when(state.getProp("ms.csv.column.projection", StringUtils.EMPTY)).thenReturn("0,2-9"); - when(state.getProp("ms.csv.column.header", StringUtils.EMPTY)).thenReturn("true"); - when(state.getPropAsBoolean("ms.csv.column.header")).thenReturn(true); - - realHttpSource.getWorkunits(sourceState); - CsvExtractor extractor = new CsvExtractor(state, realHttpSource.getHttpSourceKeys()); - extractor.setConnection(multistageConnection); - extractor.setJobKeys(realHttpSource.getJobKeys()); - when(multistageConnection.executeFirst(extractor.workUnitStatus)).thenReturn(status); - - // check if schema has been added - JsonParser parser = new JsonParser(); - String schema = extractor.getSchema(); - Assert.assertEquals(parser.parse(schema).getAsJsonArray().size(), 10); - - int index = 0; - long[] dates = new long[]{1586502000000L, 1586588400000L, 1586674800000L, 1586761200000L, 1586847600000L, - 1586934000000L, 1587020400000L, 1587106800000L, 1587193200000L, 1587279600000L}; - String[] row; - row = extractor.readRecord(null); - Assert.assertNotNull(row); - Assert.assertEquals(row.length, 10); - Assert.assertEquals(Long.parseLong(row[9]), dates[index++]); - while (extractor.hasNext()) { - row = extractor.readRecord(null); - Assert.assertNotNull(row); - Assert.assertEquals(row.length, 10); - Assert.assertEquals(Long.parseLong(row[9]), dates[index++]); - } - Assert.assertEquals(10, extractor.getCsvExtractorKeys().getProcessedCount()); - - } - - /** - * testing the interaction between add derived field with column projection - * header exists and the column excluded in the middle - */ - @Test - void testAddDerivedFieldWithColumnProjection2() throws RetriableAuthenticationException { - InputStream inputStream = getClass().getResourceAsStream("/csv/ids_flat.csv"); - WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); - - when(sourceState.getProp("ms.output.schema", new JsonArray().toString())).thenReturn("[{\"columnName\":\"id0\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"date\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id2\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id3\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id4\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id5\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id6\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id7\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"id8\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]"); - when(sourceState.getProp("ms.derived.fields", new JsonArray().toString())).thenReturn("[{\"name\": \"date\", \"formula\": {\"type\": \"epoc\", \"source\": \"date\", \"format\": \"yyyy-MM-dd Z\"}}]"); - when(state.getProp("ms.csv.column.header", StringUtils.EMPTY)).thenReturn("true"); - when(state.getPropAsBoolean("ms.csv.column.header")).thenReturn(true); - - realHttpSource.getWorkunits(sourceState); - CsvExtractor extractor = new CsvExtractor(state, realHttpSource.getHttpSourceKeys()); - extractor.setConnection(multistageConnection); - extractor.setJobKeys(realHttpSource.getJobKeys()); - when(multistageConnection.executeFirst(extractor.workUnitStatus)).thenReturn(status); - - // check if schema has been added - JsonParser parser = new JsonParser(); - String schema = extractor.getSchema(); - Assert.assertEquals(parser.parse(schema).getAsJsonArray().size(), 10); - - int index = 0; - long[] dates = new long[]{1586502000000L, 1586588400000L, 1586674800000L, 1586761200000L, 1586847600000L, - 1586934000000L, 1587020400000L, 1587106800000L, 1587193200000L, 1587279600000L}; - String[] row; - row = extractor.readRecord(null); - Assert.assertNotNull(row); - Assert.assertEquals(row.length, 10); - Assert.assertEquals(Long.parseLong(row[9]), dates[index++]); - while (extractor.hasNext()) { - row = extractor.readRecord(null); - Assert.assertNotNull(row); - Assert.assertEquals(row.length, 10); - Assert.assertEquals(Long.parseLong(row[9]), dates[index++]); - } - Assert.assertEquals(10, extractor.getCsvExtractorKeys().getProcessedCount()); - - } - - @Test - void testCSVParser() { - String input = "S1234\u001AS12345\u001ATrue\u001Atest@gmail.com\u001Atest\u001AAtar-תיווך ושיווק נדל\"ן\u001AONLINE"; - InputStream stream = new ByteArrayInputStream(input.getBytes()); - CSVReader reader = new CSVReaderBuilder(new InputStreamReader(stream, StandardCharsets.UTF_8)) - .withCSVParser(new CSVParserBuilder().withSeparator("\u001A".charAt(0)).withQuoteChar("\u0000".charAt(0)).build()) - .build(); - Assert.assertEquals(7,reader.iterator().next().length); - } - - @Test - void testInputStreamCSVReader () throws IOException { - String input = "S1234\u001AS12345\u001ATrue\u001Atest@gmail.com\u001Atest\u001AAtar-תיווך ושיווק נדל\"ן\u001AONLINE"; - InputStreamCSVReader reader = new InputStreamCSVReader(input,"\u001A".charAt(0),"\u0000".charAt(0)); - Assert.assertEquals(7,reader.splitRecord().size()); - } - - @Test - public void testProcessInputStream() throws RetriableAuthenticationException { - Iterator csvIterator = Mockito.mock(Iterator.class); - when(csvExtractorKeys.getCsvIterator()).thenReturn(csvIterator); - CsvExtractor extractor = new CsvExtractor(state, multiStageSource.getJobKeys()); - extractor.setConnection(multistageConnection); - extractor.setJobKeys(new JobKeys()); - when(multistageConnection.executeNext(extractor.workUnitStatus)).thenReturn(null); - doNothing().when(state).setWorkingState(WorkUnitState.WorkingState.FAILED); - Assert.assertFalse(extractor.processInputStream(10)); - - when(multistageConnection.executeNext(extractor.workUnitStatus)).thenReturn(workUnitStatus); - Map messages = ImmutableMap.of("contentType", "non-text/csv"); - when(workUnitStatus.getMessages()).thenReturn(messages); - Assert.assertFalse(extractor.processInputStream(10)); - - messages = ImmutableMap.of("contentType", "text/csv", "schema", "test_schema"); - when(workUnitStatus.getMessages()).thenReturn(messages); - when(workUnitStatus.getSchema()).thenReturn(new JsonArray()); - when(workUnitStatus.getBuffer()).thenReturn(null); - Assert.assertFalse(extractor.processInputStream(10)); - - when(workUnitStatus.getBuffer()).thenReturn(new ByteArrayInputStream("test_string".getBytes())); - when(csvExtractorKeys.getCsvIterator()).thenReturn(null); - when(multistageConnection.executeFirst(extractor.workUnitStatus)).thenReturn(workUnitStatus); - when(csvExtractorKeys.getSeparator()).thenReturn(","); - when(csvExtractorKeys.getQuoteCharacter()).thenReturn("\""); - when(csvExtractorKeys.getEscapeCharacter()).thenReturn("u005C"); - Assert.assertTrue(extractor.processInputStream(10)); - - schema.addProperty("someKey", "someValue"); - //when(outputJsonSchema.getSchema()).thenReturn(schema); - doThrow(new RuntimeException()).when(csvExtractorKeys).setCsvIterator(any()); - Assert.assertFalse(extractor.processInputStream(10)); - } - - @Test - public void testExpandColumnProjection() throws Exception { - state = new WorkUnitState(); - WorkUnitState workUnitState = PowerMockito.spy(state); - initExtractor(workUnitState); - - csvExtractor = new CsvExtractor(workUnitState, multiStageSource.getJobKeys()); - Method method = CsvExtractor.class.getDeclaredMethod("expandColumnProjection", String.class, int.class); - method.setAccessible(true); - Assert.assertEquals(method.invoke(csvExtractor, "0,4,2-3", 4).toString(), "[0, 2, 3, 4]"); - - Assert.assertEquals(method.invoke(csvExtractor, "0,4,2-3", 3).toString(), "[0, 2, 3, 4]"); - Assert.assertEquals(method.invoke(csvExtractor, null, 4).toString(), "[]"); - Assert.assertEquals(method.invoke(csvExtractor, "", 4).toString(), "[]"); - Assert.assertEquals(method.invoke(csvExtractor, "-1-4", 4).toString(), "[]"); - Assert.assertEquals(method.invoke(csvExtractor, "2--2", 4).toString(), "[]"); - Assert.assertEquals(method.invoke(csvExtractor, "-2--3", 4).toString(), "[]"); - Assert.assertEquals(method.invoke(csvExtractor, "3-3", 4).toString(), "[]"); - verify(workUnitState, atLeast(7)).setWorkingState(WorkUnitState.WorkingState.FAILED); - - when(workUnitState.getProp(MSTAGE_CSV_COLUMN_HEADER.getConfig(), StringUtils.EMPTY)).thenReturn("false"); - csvExtractor = new CsvExtractor(workUnitState, multiStageSource.getJobKeys()); - method = CsvExtractor.class.getDeclaredMethod("expandColumnProjection", String.class, int.class); - method.setAccessible(true); - Assert.assertEquals(method.invoke(csvExtractor, "3-1", 4).toString(), "[]"); - Assert.assertEquals(method.invoke(csvExtractor, "-1", 4).toString(), "[]"); - Assert.assertEquals(method.invoke(csvExtractor, "abc", 4).toString(), "[]"); - verify(workUnitState, atLeast(3)).setWorkingState(WorkUnitState.WorkingState.FAILED); - } - - @Test - public void testProcessGzipInputStream() throws RetriableAuthenticationException { - CsvExtractor extractor = new CsvExtractor(state, multiStageSource.getJobKeys()); - extractor.setConnection(multistageConnection); - extractor.setCsvExtractorKeys(new CsvExtractorKeys()); - extractor.setJobKeys(new JobKeys()); - when(multistageConnection.executeFirst(extractor.workUnitStatus)).thenReturn(workUnitStatus); - when(multistageConnection.executeNext(extractor.workUnitStatus)).thenReturn(workUnitStatus); - - Map messages = ImmutableMap.of("contentType", "application/gzip", "schema", "test_schema"); - when(workUnitStatus.getMessages()).thenReturn(messages); - Assert.assertFalse(extractor.processInputStream(10)); - } - - @Test - public void testAddDerivedFields() throws Exception { - initExtractor(state); - csvExtractor.setTimezone("America/Los_Angeles"); - - // derived field is in unsupported type - Map> derivedFields = ImmutableMap.of("formula", - ImmutableMap.of("type", "non-epoc", "source", "start_time", "format", "yyyy-MM-dd")); - when(jobKeys.getDerivedFields()).thenReturn(derivedFields); - when(csvExtractorKeys.getColumnToIndexMap()).thenReturn(ImmutableMap.of()); - Object[] row = new Object[]{new String[1]}; - String[] res = Whitebox.invokeMethod(csvExtractor, "addDerivedFields", row); - // Since the type is supported, we created a new record with new columns. - // In reality, the work unit will fail when processing the derived field's value. - Assert.assertEquals(res.length, 2); - Assert.assertNull(res[0]); - - // derived field is empty early exit - derivedFields = ImmutableMap.of(); - when(jobKeys.getDerivedFields()).thenReturn(derivedFields); - when(csvExtractorKeys.getColumnToIndexMap()).thenReturn(ImmutableMap.of()); - row = new Object[]{new String[1]}; - res = Whitebox.invokeMethod(csvExtractor, "addDerivedFields", row); - Assert.assertEquals(res.length, 1); - Assert.assertNull(res[0]); - - // derived field is currentdate - derivedFields = ImmutableMap.of("current_date", - ImmutableMap.of("type", "epoc", "source", "currentdate")); - when(jobKeys.getDerivedFields()).thenReturn(derivedFields); - when(csvExtractorKeys.getColumnToIndexMap()).thenReturn(ImmutableMap.of("a", 0)); - row = new Object[]{new String[]{"a"}}; - res = Whitebox.invokeMethod(csvExtractor, "addDerivedFields", row); - Assert.assertEquals(res.length, 2); - Assert.assertEquals(res[0], "a"); - Assert.assertTrue(Math.abs(Long.parseLong(res[1]) - DateTime.now().getMillis()) < ONE_HOUR_IN_MILLS); - - // derived field is P1D - derivedFields = ImmutableMap.of("current_date", - ImmutableMap.of("type", "epoc", "source", "P1D")); - when(jobKeys.getDerivedFields()).thenReturn(derivedFields); - when(csvExtractorKeys.getColumnToIndexMap()).thenReturn(ImmutableMap.of("a", 0)); - row = new Object[]{new String[]{"a"}}; - res = Whitebox.invokeMethod(csvExtractor, "addDerivedFields", row); - Assert.assertEquals(res.length, 2); - Assert.assertEquals(res[0], "a"); - DateTimeZone timeZone = DateTimeZone.forID("America/Los_Angeles"); - Period period = Period.parse("P1D"); - long p1d = DateTime.now().withZone(timeZone).minus(period).dayOfMonth().roundFloorCopy().getMillis(); - Assert.assertTrue(Math.abs(Long.parseLong(res[1]) - p1d) < ONE_HOUR_IN_MILLS); - - // derived field is in the specified format - derivedFields = ImmutableMap.of("current_date", - ImmutableMap.of("type", "epoc", "source", "start_time", "format", "yyyy-MM-dd")); - when(jobKeys.getDerivedFields()).thenReturn(derivedFields); - when(csvExtractorKeys.getColumnToIndexMap()).thenReturn(ImmutableMap.of("start_time", 0)); - row = new Object[]{new String[]{"2020-06-01"}}; - res = Whitebox.invokeMethod(csvExtractor, "addDerivedFields", row); - Assert.assertEquals(res.length, 2); - Assert.assertEquals(res[0], "2020-06-01"); - DateTimeFormatter datetimeFormatter = DateTimeFormat.forPattern("yyyy-MM-dd"); - Assert.assertEquals(Long.parseLong(res[1]), datetimeFormatter.parseDateTime("2020-06-01").getMillis()); - - // derived field is NOT in the specified format - derivedFields = ImmutableMap.of("current_date", - ImmutableMap.of("type", "epoc", "source", "start_time", "format", "yyyy-MM-dd")); - when(jobKeys.getDerivedFields()).thenReturn(derivedFields); - when(csvExtractorKeys.getColumnToIndexMap()).thenReturn(ImmutableMap.of("start_time", 0)); - row = new Object[]{new String[]{"notdatatime"}}; - res = Whitebox.invokeMethod(csvExtractor, "addDerivedFields", row); - // Since the type is supported, we created a new record with new columns. - // In reality, the work unit will fail when processing the derived field's value. - Assert.assertEquals(res.length, 2); - Assert.assertEquals(res[0], "notdatatime"); - Assert.assertEquals(res[1], ""); - - // derived fields are from variables - JsonObject parameters = new JsonObject(); - parameters.addProperty("dateString", "2019-11-01 12:00:00"); - parameters.addProperty("someInteger", 123456); - parameters.addProperty("someNumber", 123.456); - parameters.addProperty("someEpoc", 1601038688000L); - csvExtractor.currentParameters = parameters; - - derivedFields = ImmutableMap.of("dateString", - ImmutableMap.of("type", "string", "source", "{{dateString}}"), - "someInteger", - ImmutableMap.of("type", "integer", "source", "{{someInteger}}"), - "someEpoc", - ImmutableMap.of("type", "epoc", "source", "{{someEpoc}}"), - "someNumber", - ImmutableMap.of("type", "number", "source", "{{someNumber}}")); - when(jobKeys.getDerivedFields()).thenReturn(derivedFields); - when(csvExtractorKeys.getColumnToIndexMap()).thenReturn(ImmutableMap.of("start_time", 0)); - row = new Object[]{new String[]{"2020-06-01"}}; - res = Whitebox.invokeMethod(csvExtractor, "addDerivedFields", row); - Assert.assertEquals(res.length, 5); - Assert.assertEquals(res[0], "2020-06-01"); - Assert.assertEquals(res[1], "2019-11-01 12:00:00"); - Assert.assertEquals(res[2], "123456"); - Assert.assertEquals(res[3], "1601038688000"); - Assert.assertEquals(res[4], "123.456"); - } - - @Test - public void testSkipRowAndSaveHeader() throws Exception { - initExtractor(state); - String[] someData = new String[]{"some_date"}; - String[] moreData = new String[]{"more_data"}; - List rows = ImmutableList.of(someData, moreData); - CsvExtractorKeys csvExtractorKeys = new CsvExtractorKeys(); - CsvExtractorKeys spy = spy(csvExtractorKeys); - csvExtractor.setCsvExtractorKeys(spy); - when(spy.getRowsToSkip()).thenReturn(2); - when(spy.getColumnHeader()).thenReturn(false); - Whitebox.invokeMethod(csvExtractor, "skipRowAndSaveHeader", rows.iterator()); - verify(spy, atMost(0)).setHeaderRow(someData); - verify(spy, atMost(0)).setHeaderRow(moreData); - } - - @Test - public void testInferSchemaWithSample() throws Exception { - initExtractor(state); - - String[] someData = new String[]{"some_date"}; - String[] moreData = new String[]{"more_data"}; - List rows = ImmutableList.of(someData, moreData); - csvExtractorKeys = new CsvExtractorKeys(); - CsvExtractorKeys spy = spy(csvExtractorKeys); - csvExtractor.setCsvExtractorKeys(spy); - when(spy.getRowsToSkip()).thenReturn(0); - Deque deque = new LinkedList(); - when(spy.getSampleRows()).thenReturn(deque); - - when(spy.getHeaderRow()).thenReturn(new String[]{"col1", "col2"}); - Assert.assertEquals( - Whitebox.invokeMethod(csvExtractor, "inferSchemaWithSample", rows.iterator()).toString(), - "[{\"columnName\":\"col0\",\"isNullable\":false,\"dataType\":{\"type\":\"string\"}}]"); - - when(spy.getHeaderRow()).thenReturn(null); - Whitebox.invokeMethod(csvExtractor, "inferSchemaWithSample", rows.iterator()); - Assert.assertEquals( - Whitebox.invokeMethod(csvExtractor, "inferSchemaWithSample", rows.iterator()).toString(), - "[{\"columnName\":\"col0\",\"isNullable\":false,\"dataType\":{\"type\":\"string\"}}]" - ); - } - - @Test - public void testSetRowFilter() { - JsonSchemaBasedFilter filter = Mockito.mock(JsonSchemaBasedFilter.class); - JsonArray schema = new JsonArray(); - csvExtractor.rowFilter = filter; - csvExtractor.setRowFilter(schema); - - csvExtractor.rowFilter = null; - when(state.getProp(MultistageProperties.MSTAGE_ENABLE_SCHEMA_BASED_FILTERING.getConfig(), StringUtils.EMPTY)).thenReturn("false"); - csvExtractor.setRowFilter(new JsonArray()); - Assert.assertNull(csvExtractor.rowFilter); - } - - @Test - public void testAddParsedCSVData() throws Exception { - initExtractor(state); - Method method = CsvExtractor.class.getDeclaredMethod("addParsedCSVData", String.class, String.class, JsonObject.class); - method.setAccessible(true); - method.invoke(csvExtractor, "key1", "true", schema); - Assert.assertEquals(schema.get("key1").getAsBoolean(), true); - - method.invoke(csvExtractor, "key2", "false", schema); - Assert.assertEquals(schema.get("key2").getAsBoolean(), false); - - method.invoke(csvExtractor, "key3", "1.234F", schema); - Assert.assertEquals(schema.get("key3").getAsFloat(), 1.234F); - - method.invoke(csvExtractor, "key4", "something else", schema); - Assert.assertEquals(schema.get("key4").getAsString(), "something else"); - } - - private void initExtractor(WorkUnitState state) { - when(state.getProp(MSTAGE_CSV_COLUMN_HEADER.getConfig(), StringUtils.EMPTY)).thenReturn("true"); - when(state.getPropAsBoolean(MSTAGE_CSV_COLUMN_HEADER.getConfig())).thenReturn(true); - when(state.getPropAsInt(MSTAGE_CSV_SKIP_LINES.getConfig(), 0)).thenReturn(2); - when(state.getProp(MSTAGE_CSV_SEPARATOR.getConfig(), StringUtils.EMPTY)).thenReturn(","); - when(state.getProp(MSTAGE_CSV_QUOTE_CHARACTER.getConfig(), StringUtils.EMPTY)).thenReturn("\""); - when(state.getProp(MSTAGE_CSV_ESCAPE_CHARACTER.getConfig(), StringUtils.EMPTY)).thenReturn("u005C"); - when(state.getProp(MSTAGE_EXTRACT_PREPROCESSORS_PARAMETERS.getConfig(), new JsonObject().toString())).thenReturn(StringUtils.EMPTY); - when(state.getProp(MSTAGE_EXTRACT_PREPROCESSORS.getConfig(), StringUtils.EMPTY)).thenReturn(StringUtils.EMPTY); - } -} diff --git a/cdi-core/src/test/java/com/linkedin/cdi/extractor/FileDumpExtractorTest2.java b/cdi-core/src/test/java/com/linkedin/cdi/extractor/FileDumpExtractorTest2.java deleted file mode 100644 index e541952..0000000 --- a/cdi-core/src/test/java/com/linkedin/cdi/extractor/FileDumpExtractorTest2.java +++ /dev/null @@ -1,199 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.extractor; - -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.UnsupportedEncodingException; -import java.nio.charset.StandardCharsets; -import java.util.HashMap; -import java.util.List; -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.commons.lang3.tuple.MutablePair; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.gobblin.configuration.SourceState; -import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.connection.MultistageConnection; -import com.linkedin.cdi.exception.RetriableAuthenticationException; -import com.linkedin.cdi.keys.FileDumpExtractorKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.source.MultistageSource; -import com.linkedin.cdi.util.VariableUtils; -import com.linkedin.cdi.util.WorkUnitStatus; -import org.apache.gobblin.source.workunit.WorkUnit; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.mockito.Mockito; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.testng.PowerMockTestCase; -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import static com.linkedin.cdi.configuration.MultistageProperties.*; -import static org.mockito.Mockito.*; - - -/** - * Test FileDumpExtractor under following scenarios: - * - * Scenario 1: download a file and save to /tmp - */ -@PrepareForTest({VariableUtils.class, FileSystem.class}) -public class FileDumpExtractorTest2 extends PowerMockTestCase { - - private final static String DATA_SET_URN_KEY = "com.linkedin.SeriesCollection"; - private final static String ACTIVATION_PROP = "{\"name\": \"survey\", \"type\": \"unit\", \"units\": \"id1,id2\"}"; - private final static String DATA_FINAL_DIR = "/jobs/testUser/gobblin/useCaseRoot"; - private final static String FILE_PERMISSION = "775"; - private final static long WORK_UNIT_START_TIME_KEY = 1590994800000L; - - private WorkUnitState state; - private MultistageSource source; - private WorkUnit workUnit; - private FileDumpExtractorKeys fileDumpExtractorKeys; - private WorkUnitStatus workUnitStatus; - private JobKeys jobKeys; - private FileDumpExtractor fileDumpExtractor; - private MultistageConnection multistageConnection; - - @BeforeMethod - public void setUp() { - state = Mockito.mock(WorkUnitState.class); - source = Mockito.mock(MultistageSource.class); - - List wus = new MultistageSource().getWorkunits(new SourceState()); - workUnit = wus.get(0); - workUnit.setProp(MultistageProperties.DATASET_URN_KEY.getConfig(), DATA_SET_URN_KEY); - - fileDumpExtractorKeys = Mockito.mock(FileDumpExtractorKeys.class); - workUnitStatus = Mockito.mock(WorkUnitStatus.class); - jobKeys = Mockito.mock(JobKeys.class); - - when(state.getProp(MSTAGE_ACTIVATION_PROPERTY.getConfig(), new JsonObject().toString())).thenReturn(ACTIVATION_PROP); - when(state.getPropAsLong(MSTAGE_WORKUNIT_STARTTIME_KEY.getConfig(), 0L)).thenReturn(WORK_UNIT_START_TIME_KEY); - when(state.getWorkunit()).thenReturn(workUnit); - when(state.getProp(DATA_PUBLISHER_FINAL_DIR.getConfig(), StringUtils.EMPTY)).thenReturn(DATA_FINAL_DIR); - when(state.getProp(MSTAGE_EXTRACTOR_TARGET_FILE_PERMISSION.getConfig(), StringUtils.EMPTY)).thenReturn(FILE_PERMISSION); - - fileDumpExtractor = new FileDumpExtractor(state, source.getJobKeys()); - fileDumpExtractor.setFileDumpExtractorKeys(fileDumpExtractorKeys); - fileDumpExtractor.jobKeys = jobKeys; - - multistageConnection = Mockito.mock(MultistageConnection.class); - fileDumpExtractor.setConnection(multistageConnection); - } - - /** - * Test FileDumpExtractor Constructor with a happy path - */ - @Test - public void testFileDumpExtractorConstructor() { - FileDumpExtractor fileDumpExtractor = new FileDumpExtractor(state, source.getJobKeys()); - Assert.assertEquals(fileDumpExtractor.getFileDumpExtractorKeys().getFileName(), StringUtils.EMPTY); - Assert.assertEquals(fileDumpExtractor.getFileDumpExtractorKeys().getFileWritePermissions(), FILE_PERMISSION); - Assert.assertEquals(fileDumpExtractor.getFileDumpExtractorKeys().getFileDumpLocation(), DATA_FINAL_DIR); - Assert.assertEquals(fileDumpExtractor.getFileDumpExtractorKeys().getCurrentFileNumber(), 0); - } - - /** - * Test FileDumpExtractor Constructor when a RuntimeException is thrown - */ - @Test(expectedExceptions = RuntimeException.class) - public void testFileDumpExtractorConstructorWithException() { - doThrow(new RuntimeException()).when(state).getProp(DATA_PUBLISHER_FINAL_DIR.getConfig(), StringUtils.EMPTY); - new FileDumpExtractor(state, source.getJobKeys()); - } - - /** - * Test readRecord - */ - @Test(expectedExceptions = RuntimeException.class) - public void testReadRecord() throws IOException, RetriableAuthenticationException { - when(jobKeys.getPaginationFields()).thenReturn(new HashMap<>()); - when(jobKeys.getPaginationInitValues()).thenReturn(new HashMap<>()); - when(jobKeys.isPaginationEnabled()).thenReturn(false); - when(multistageConnection.executeNext(fileDumpExtractor.workUnitStatus)).thenReturn(workUnitStatus); - when(workUnitStatus.getBuffer()).thenReturn(new ByteArrayInputStream("test_string".getBytes())); - when(fileDumpExtractorKeys.getCurrentFileNumber()).thenReturn(Long.valueOf(10)); - when(fileDumpExtractorKeys.getFileName()).thenReturn("file_name"); - when(fileDumpExtractorKeys.getFileDumpLocation()).thenReturn("dir"); - when(fileDumpExtractorKeys.getFileWritePermissions()).thenReturn("775"); - when(fileDumpExtractorKeys.getCurrentFileNumber()).thenReturn(Long.valueOf(1)); - PowerMockito.mockStatic(FileSystem.class); - FSDataOutputStream out = Mockito.mock(FSDataOutputStream.class); - PowerMockito.when(FileSystem.create(any(), any(), any())).thenReturn(out); - PowerMockito.doNothing().when(out).flush(); - PowerMockito.doNothing().when(out).close(); - - Assert.assertNull(fileDumpExtractor.readRecord("")); - - when(jobKeys.isPaginationEnabled()).thenReturn(true); - - doThrow(new RuntimeException()).when(fileDumpExtractorKeys).incrCurrentFileNumber(); - fileDumpExtractor.readRecord(""); - } - - /** - * Test processInputStream with two scenarios - * 1: Happy path - * 2: Invalid file name provided - * - * @throws IOException - */ - @Test - public void testProcessInputStream() throws RetriableAuthenticationException { - // replace mocked source key with default source key - fileDumpExtractor.jobKeys = new JobKeys(); - - when(fileDumpExtractorKeys.getActivationParameters()).thenReturn(new JsonObject()); - when(fileDumpExtractorKeys.getPayloads()).thenReturn(new JsonArray()); - when(multistageConnection.executeNext(fileDumpExtractor.workUnitStatus)).thenReturn(null); - Assert.assertFalse(fileDumpExtractor.processInputStream(10)); - - WorkUnitStatus unitStatus = Mockito.mock(WorkUnitStatus.class); - when(multistageConnection.executeNext(fileDumpExtractor.workUnitStatus)).thenReturn(unitStatus); - fileDumpExtractor.getFileDumpExtractorKeys().setFileName(StringUtils.EMPTY); - Assert.assertFalse(fileDumpExtractor.processInputStream(10)); - - when(unitStatus.getBuffer()).thenReturn(null); - fileDumpExtractor.getFileDumpExtractorKeys().setFileName("test_file_name"); - Assert.assertFalse(fileDumpExtractor.processInputStream(10)); - - JobKeys keys = Mockito.mock(JobKeys.class); - when(source.getJobKeys()).thenReturn(keys); - when(keys.isPaginationEnabled()).thenReturn(true); - InputStream input = new ByteArrayInputStream("test_string".getBytes()); - when(unitStatus.getBuffer()).thenReturn(input); - Assert.assertFalse(fileDumpExtractor.processInputStream(10)); - } - /** - * Test getFileName with two scenarios - * 1: Happy path - * 2: Unresolved placeholder - */ - @Test - public void testGetFileName() throws Exception { - PowerMockito.mockStatic(VariableUtils.class); - String fileNameTemplate = "testFileTemplate"; - when(state.getProp(MSTAGE_EXTRACTOR_TARGET_FILE_NAME.getConfig(), StringUtils.EMPTY)).thenReturn(fileNameTemplate); - String fileName = IOUtils.toString(this.getClass().getResourceAsStream("/other/sample-data-include-long-file-name.txt"), StandardCharsets.UTF_8.name()); - String filePath = String.join("/", "dir", fileName); - Pair pair = new MutablePair<>(filePath, new JsonObject()); - PowerMockito.when(VariableUtils.replaceWithTracking(any(), any())).thenReturn(pair); - FileDumpExtractorKeys extractorKeys = new FileDumpExtractor(state, source.getJobKeys()).getFileDumpExtractorKeys(); - Assert.assertEquals(extractorKeys.getFileName(), String.join("/", "dir", fileName.substring(0, 255 - 1))); - - PowerMockito.doThrow(new UnsupportedEncodingException()).when(VariableUtils.class, "replaceWithTracking", any(), any()); - extractorKeys = new FileDumpExtractor(state, source.getJobKeys()).getFileDumpExtractorKeys(); - Assert.assertEquals(extractorKeys.getFileName(), fileNameTemplate); - } -} \ No newline at end of file diff --git a/cdi-core/src/test/java/com/linkedin/cdi/extractor/JsonExtractorTest.java b/cdi-core/src/test/java/com/linkedin/cdi/extractor/JsonExtractorTest.java index 1f3c980..7d62f74 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/extractor/JsonExtractorTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/extractor/JsonExtractorTest.java @@ -12,36 +12,35 @@ import com.google.gson.JsonNull; import com.google.gson.JsonObject; import com.google.gson.JsonPrimitive; -import java.io.ByteArrayInputStream; -import java.io.InputStream; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import org.apache.commons.lang.StringUtils; -import org.apache.gobblin.configuration.SourceState; -import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; import com.linkedin.cdi.connection.MultistageConnection; import com.linkedin.cdi.exception.RetriableAuthenticationException; import com.linkedin.cdi.keys.JobKeys; import com.linkedin.cdi.keys.JsonExtractorKeys; -import com.linkedin.cdi.source.HttpSource; import com.linkedin.cdi.source.MultistageSource; import com.linkedin.cdi.util.JsonUtils; import com.linkedin.cdi.util.ParameterTypes; import com.linkedin.cdi.util.SchemaBuilder; import com.linkedin.cdi.util.WorkUnitStatus; -import org.apache.gobblin.runtime.JobState; +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.commons.lang.StringUtils; +import org.apache.gobblin.configuration.SourceState; +import org.apache.gobblin.configuration.WorkUnitState; import org.apache.gobblin.source.workunit.WorkUnit; import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; import org.mockito.Mockito; import org.powermock.reflect.Whitebox; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import static com.linkedin.cdi.configuration.MultistageProperties.*; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static org.mockito.Mockito.*; @@ -77,9 +76,9 @@ public void setUp() throws RetriableAuthenticationException { workUnitStatus = Mockito.mock(WorkUnitStatus.class); state = Mockito.mock(WorkUnitState.class); when(state.getProp(MSTAGE_ACTIVATION_PROPERTY.getConfig(), new JsonObject().toString())).thenReturn(ACTIVATION_PROP); - when(state.getPropAsLong(MSTAGE_WORKUNIT_STARTTIME_KEY.getConfig(), 0L)).thenReturn(WORKUNIT_STARTTIME_KEY); + when(state.getPropAsLong(MSTAGE_WORK_UNIT_SCHEDULING_STARTTIME.getConfig(), 0L)).thenReturn(WORKUNIT_STARTTIME_KEY); when(state.getWorkunit()).thenReturn(workUnit); - workUnit.setProp(MultistageProperties.DATASET_URN_KEY.getConfig(), DATA_SET_URN_KEY); + workUnit.setProp(DATASET_URN.getConfig(), DATA_SET_URN_KEY); when(source.getJobKeys()).thenReturn(jobKeys); when(jobKeys.getSourceParameters()).thenReturn(new JsonArray()); when(jobKeys.getPaginationInitValues()).thenReturn(new HashMap<>()); @@ -118,7 +117,7 @@ public void testReadRecord() throws RetriableAuthenticationException { when(jobKeys.getOutputSchema()).thenReturn(outputSchema); when(jsonExtractorKeys.getCurrentPageNumber()).thenReturn(Long.valueOf(0)); when(jsonExtractorKeys.getSessionKeyValue()).thenReturn("session_key"); - workUnit.setProp(MultistageProperties.DATASET_URN_KEY.getConfig(), "com.linkedin.xxxxx.UserGroups"); + workUnit.setProp(DATASET_URN.getConfig(), "com.linkedin.xxxxx.UserGroups"); Iterator jsonElementIterator = ImmutableList.of().iterator(); when(jsonExtractorKeys.getJsonElementIterator()).thenReturn(jsonElementIterator); when(jsonExtractorKeys.getProcessedCount()).thenReturn(Long.valueOf(0)); @@ -225,36 +224,6 @@ public void testGetElementByJsonPathWithEdgeCases() { Assert.assertEquals(JsonUtils.get(row, jsonPath), JsonNull.INSTANCE); } - /** - * Test Extractor shall stop the session when total count of records is met - */ - @Test - void testStopConditionTotalCountMet() throws RetriableAuthenticationException { - InputStream inputStream = getClass().getResourceAsStream("/json/last-page-with-data.json"); - WorkUnitStatus status = WorkUnitStatus.builder().buffer(inputStream).build(); - status.setTotalCount(TOTAL_COUNT); - - SourceState sourceState = mock(SourceState.class); - when(sourceState.getProp("ms.data.field", "")).thenReturn("items"); - when(sourceState.getProp("ms.total.count.field", "")).thenReturn("totalResults"); - when(sourceState.getProp("ms.pagination", "")).thenReturn("{\"fields\": [\"offset\", \"limit\"], \"initialvalues\": [0, 5000]}"); - when(sourceState.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); - MultistageSource source = new HttpSource(); - List wus = source.getWorkunits(sourceState); - WorkUnitState state = new WorkUnitState(wus.get(0), new JobState()); - - JsonExtractor extractor = new JsonExtractor(state, source.getJobKeys()); - extractor.setConnection(multistageConnection); - extractor.getJsonExtractorKeys().setTotalCount(TOTAL_COUNT); - - extractor.workUnitStatus = WorkUnitStatus.builder().build(); - when(multistageConnection.executeFirst(extractor.workUnitStatus)).thenReturn(status); - - Assert.assertFalse(extractor.processInputStream(TOTAL_COUNT)); - // If total count not reached, should not fail - Assert.assertTrue(extractor.processInputStream(TOTAL_COUNT-1)); - } - @Test public void testAddDerivedFields() throws Exception { Map> derivedFields = ImmutableMap.of("formula", @@ -317,6 +286,7 @@ public void testAddDerivedFields() throws Exception { Assert.assertEquals(actual.entrySet().size(), 2); Assert.assertTrue(actual.has("formula")); Assert.assertEquals(actual.get("start_time").toString(), "\"1592809200000\""); + // negative regex case derivedFields = ImmutableMap.of("formula", ImmutableMap.of("type", "regexp", "source", "uri", "format", "/syncs/([0-9]+)$")); @@ -331,6 +301,7 @@ public void testAddDerivedFields() throws Exception { Assert.assertEquals(actual.get("start_time").toString(), "\"1592809200000\""); Assert.assertEquals(actual.get("formula").toString(), "\"no match\""); Assert.assertEquals(actual.get("uri").toString(), "\"invalid_uri\""); + // positive regex case derivedFields = ImmutableMap.of("formula", ImmutableMap.of("type", "regexp", "source", "uri", "format", "/syncs/([0-9]+)$")); @@ -356,16 +327,31 @@ public void testAddDerivedFields() throws Exception { Assert.assertTrue(actual.has("formula")); Assert.assertEquals(actual.get("formula").toString(), "true"); + // Testing derived fields from push downs that are two levels deep + derivedFields = ImmutableMap.of("formula", + ImmutableMap.of("type", "string", "source", "result.key1")); + when(jobKeys.getDerivedFields()).thenReturn(derivedFields); + pushDowns.addProperty("nested", "result.key1"); + row.add("result", gson.fromJson("{\"key1\": \"value1\"}", JsonObject.class)); + when(jsonExtractorKeys.getPushDowns()).thenReturn(pushDowns); + actual = Whitebox.invokeMethod(jsonExtractor, "addDerivedFields", row); + Assert.assertEquals(actual.entrySet().size(), 4); + Assert.assertTrue(actual.has("formula")); + Assert.assertEquals(actual.get("formula").toString(), "\"value1\""); + // Testing derived fields from variable JsonObject parameters = new JsonObject(); - parameters.addProperty("dateString", "2019-11-01 12:00:00"); + parameters.addProperty("dateString", "2019-11-01"); + parameters.addProperty("dateTimeString", "2019-11-01 12:00:00"); parameters.addProperty("someInteger", 123456); parameters.addProperty("someNumber", 123.456); parameters.addProperty("someEpoc", 1601038688000L); jsonExtractor.currentParameters = parameters; derivedFields = ImmutableMap.of("dateString", - ImmutableMap.of("type", "string", "source", "{{dateString}}"), + ImmutableMap.of("type", "epoc", "source", "{{dateString}}", "format", "yyyy-MM-dd"), + "dateTimeString", + ImmutableMap.of("type", "string", "source", "{{dateTimeString}}"), "someInteger", ImmutableMap.of("type", "integer", "source", "{{someInteger}}"), "someEpoc", @@ -375,8 +361,11 @@ public void testAddDerivedFields() throws Exception { when(jobKeys.getDerivedFields()).thenReturn(derivedFields); pushDowns.addProperty("non-formula", "testValue"); actual = Whitebox.invokeMethod(jsonExtractor, "addDerivedFields", row); - Assert.assertEquals(actual.entrySet().size(), 7); - Assert.assertEquals(actual.get("dateString").toString(), "\"2019-11-01 12:00:00\""); + Assert.assertEquals(actual.entrySet().size(), 9); + DateTimeFormatter datetimeFormatter = DateTimeFormat.forPattern("yyyy-MM-dd"); + DateTime dateTime = datetimeFormatter.parseDateTime("2019-11-01"); + Assert.assertEquals(actual.get("dateString").toString(), String.valueOf(dateTime.getMillis())); + Assert.assertEquals(actual.get("dateTimeString").toString(), "\"2019-11-01 12:00:00\""); Assert.assertEquals(actual.get("someInteger").toString(), "123456"); Assert.assertEquals(actual.get("someNumber").toString(), "123.456"); Assert.assertEquals(actual.get("start_time").toString(), "\"1592809200000\""); @@ -440,13 +429,24 @@ public void testRetrieveSessionKeyValue() throws Exception { } /** - * Test getTotalCountValue with non-JsonArray payload + * Test getTotalCountValue with jsonObject data field + */ + @Test + public void testGetTotalCountValueWithJsonObjectDataField() throws Exception { + when(source.getJobKeys().getTotalCountField()).thenReturn(""); + when(source.getJobKeys().getDataField()).thenReturn("items"); + JsonObject data = gson.fromJson("{\"records\":{\"totalRecords\":2},\"items\":{\"callId\":\"001\"}}", JsonObject.class); + Assert.assertEquals(Whitebox.invokeMethod(jsonExtractor, "getTotalCountValue", data), Long.valueOf(1)); + } + + /** + * Test getTotalCountValue with invalid data field * Expect: RuntimeException */ @Test(expectedExceptions = RuntimeException.class) - public void testGetTotalCountValueWithJsonObjectPayload() throws Exception { + public void testGetTotalCountValueWithInvalidDataField() throws Exception { when(source.getJobKeys().getTotalCountField()).thenReturn(""); - when(source.getJobKeys().getDataField()).thenReturn("items"); + when(source.getJobKeys().getDataField()).thenReturn("items.callId"); JsonObject data = gson.fromJson("{\"records\":{\"totalRecords\":2},\"items\":{\"callId\":\"001\"}}", JsonObject.class); Assert.assertEquals(Whitebox.invokeMethod(jsonExtractor, "getTotalCountValue", data), Long.valueOf(0)); } diff --git a/cdi-core/src/test/java/com/linkedin/cdi/extractor/MultistageExtractorTest.java b/cdi-core/src/test/java/com/linkedin/cdi/extractor/MultistageExtractorTest.java deleted file mode 100644 index 7b89b19..0000000 --- a/cdi-core/src/test/java/com/linkedin/cdi/extractor/MultistageExtractorTest.java +++ /dev/null @@ -1,509 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.extractor; - -import com.google.common.collect.ImmutableMap; -import com.google.gson.Gson; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.lang.reflect.Method; -import java.nio.charset.Charset; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang.StringUtils; -import org.apache.gobblin.configuration.ConfigurationKeys; -import org.apache.gobblin.configuration.SourceState; -import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.filter.JsonSchemaBasedFilter; -import com.linkedin.cdi.keys.ExtractorKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.preprocessor.GpgDecryptProcessor; -import com.linkedin.cdi.preprocessor.GunzipProcessor; -import com.linkedin.cdi.preprocessor.InputStreamProcessor; -import com.linkedin.cdi.source.HttpSource; -import com.linkedin.cdi.source.MultistageSource; -import com.linkedin.cdi.util.ParameterTypes; -import com.linkedin.cdi.util.WorkUnitStatus; -import org.apache.gobblin.runtime.JobState; -import org.apache.gobblin.source.workunit.WorkUnit; -import org.joda.time.DateTime; -import org.mockito.Mockito; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.testng.PowerMockTestCase; -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import static org.mockito.Mockito.*; - - -@PrepareForTest({Thread.class, IOUtils.class}) -public class MultistageExtractorTest extends PowerMockTestCase { - private Gson gson; - private ExtractorKeys extractorKeys; - private MultistageExtractor multistageExtractor; - private MultistageSource source; - private WorkUnitState state; - private WorkUnitStatus workUnitStatus; - private JobKeys jobKeys; - private JsonArray jsonSchema; - private JsonArray outputSchema; - - @BeforeMethod - public void setUp() { - gson = new Gson(); - extractorKeys = Mockito.mock(ExtractorKeys.class); - state = mock(WorkUnitState.class); - workUnitStatus = Mockito.mock(WorkUnitStatus.class); - source = mock(MultistageSource.class); - jobKeys = Mockito.mock(JobKeys.class); - jsonSchema = new JsonArray(); - outputSchema = new JsonArray(); - multistageExtractor = new MultistageExtractor(state, source.getJobKeys()); - multistageExtractor.extractorKeys = extractorKeys; - multistageExtractor.jobKeys = jobKeys; - } - - @Test - public void testInitialization() { - WorkUnitState state = mock(WorkUnitState.class); - when(state.getProp("ms.derived.fields", new JsonArray().toString())).thenReturn("[{\"name\": \"activityDate\", \"formula\": {\"type\": \"epoc\", \"source\": \"fromDateTime\", \"format\": \"yyyy-MM-dd'T'HH:mm:ss'Z'\"}}]"); - when(state.getProp("ms.output.schema", new JsonArray().toString())).thenReturn(""); - when(state.getProp("ms.activation.property", new JsonObject().toString())).thenReturn("{\"a\":\"x\"}"); - - SourceState sourceState = mock(SourceState.class); - when(sourceState.contains("source.conn.use.proxy.url")).thenReturn(true); - when(sourceState.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); - MultistageSource source = new HttpSource(); - source.getWorkunits(sourceState); - - MultistageExtractor extractor = new MultistageExtractor(state, source.getJobKeys()); - Assert.assertNotNull(source.getJobKeys().getDerivedFields()); - } - - @Test - public void testJobProperties() { - WorkUnitState state = mock(WorkUnitState.class); - when(state.getProp("ms.derived.fields", new JsonArray().toString())).thenReturn("[{\"name\": \"activityDate\", \"formula\": {\"type\": \"epoc\", \"source\": \"fromDateTime\", \"format\": \"yyyy-MM-dd'T'HH:mm:ss'Z'\"}}]"); - when(state.getProp("ms.output.schema", new JsonArray().toString())).thenReturn(""); - - SourceState sourceState = mock(SourceState.class); - - when(state.getProp("ms.activation.property", new JsonObject().toString())).thenReturn("{\"a\":\"x\"}"); - Assert.assertNotNull(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.getProp(state)); - Assert.assertNotNull(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.getValidNonblankWithDefault(state)); - Assert.assertTrue(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.validate(state)); - Assert.assertTrue(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.validateNonblank(state)); - - when(state.getProp("ms.activation.property", new JsonObject().toString())).thenReturn("{\"a\"}"); - Assert.assertFalse(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.validate(state)); - Assert.assertFalse(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.validateNonblank(state)); - Assert.assertNotNull(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.getValidNonblankWithDefault(state)); - - when(state.getProp("ms.activation.property", new JsonObject().toString())).thenReturn("{}"); - Assert.assertTrue(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.validate(state)); - Assert.assertFalse(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.validateNonblank(state)); - Assert.assertNotNull(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.getValidNonblankWithDefault(state)); - - when(state.getProp("ms.activation.property", new JsonObject().toString())).thenReturn(""); - Assert.assertTrue(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.validate(state)); - Assert.assertFalse(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.validateNonblank(state)); - Assert.assertNotNull(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.getValidNonblankWithDefault(state)); - } - - - @Test - public void testWorkUnitWatermark(){ - SourceState state = mock(SourceState.class); - when(state.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); - MultistageSource source = new MultistageSource(); - List workUnits = source.getWorkunits(state); - WorkUnitState workUnitState = new WorkUnitState(workUnits.get(0)); - JsonExtractor extractor = new JsonExtractor(workUnitState, source.getJobKeys()); - - // low watermark by default is 2017-01-01 - Assert.assertEquals("1546329600000", extractor.getWorkUnitWaterMarks().get("low").getAsString()); - } - - @Test - public void testGetOnePreprocessor() { - WorkUnitState state = mock(WorkUnitState.class); - when(state.getProp("ms.derived.fields", new JsonArray().toString())).thenReturn( - "[]"); - when(state.getProp("ms.output.schema", new JsonArray().toString())).thenReturn(""); - when(state.getProp("ms.activation.property", new JsonObject().toString())).thenReturn( - "{\"a\":\"x\"}"); - when(state.getProp("ms.extract.preprocessor.parameters", new JsonObject().toString())).thenReturn( - "{\"com.linkedin.cdi.preprocessor.GpgProcessor\":" + - "{\"keystore_path\" :\"some path\", \"keystore_password\" : \"some password\"}}"); - when(state.getProp("ms.extract.preprocessors", new String())).thenReturn( - "com.linkedin.cdi.preprocessor.GpgProcessor"); - - SourceState sourceState = mock(SourceState.class); - when(sourceState.contains("source.conn.use.proxy.url")).thenReturn(true); - when(sourceState.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); - MultistageSource source = new HttpSource(); - source.getWorkunits(sourceState); - - MultistageExtractor extractor = new MultistageExtractor(state, source.getJobKeys()); - - List res = extractor.getPreprocessors(state); - Assert.assertEquals(res.size(), 1); - Assert.assertTrue(res.get(0) instanceof GpgDecryptProcessor); - } - - @Test - public void testGetTwoPreprocessors() { - WorkUnitState state = mock(WorkUnitState.class); - when(state.getProp("ms.derived.fields", new JsonArray().toString())).thenReturn( - "[]"); - when(state.getProp("ms.output.schema", new JsonArray().toString())).thenReturn(""); - when(state.getProp("ms.activation.property", new JsonObject().toString())).thenReturn( - "{\"a\":\"x\"}"); - when(state.getProp("ms.extract.preprocessor.parameters", new JsonObject().toString())).thenReturn( - "{\"com.linkedin.cdi.preprocessor.GpgProcessor\":" + - "{\"keystore_path\" :\"some path\", \"keystore_password\" : \"some password\"}}"); - when(state.getProp("ms.extract.preprocessors", new String())).thenReturn( - "com.linkedin.cdi.preprocessor.GpgProcessor,"+ - "com.linkedin.cdi.preprocessor.GunzipProcessor"); - - SourceState sourceState = mock(SourceState.class); - when(sourceState.contains("source.conn.use.proxy.url")).thenReturn(true); - when(sourceState.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); - MultistageSource source = new HttpSource(); - source.getWorkunits(sourceState); - - MultistageExtractor extractor = new MultistageExtractor(state, source.getJobKeys()); - - List res = extractor.getPreprocessors(state); - Assert.assertEquals(res.size(), 2); - Assert.assertTrue(res.get(0) instanceof GpgDecryptProcessor); - Assert.assertTrue(res.get(1) instanceof GunzipProcessor); - } - - @Test - public void testGetSchema() { - Assert.assertNull(multistageExtractor.getSchema()); - } - - @Test - public void testGetExpectedRecordCount() { - Assert.assertEquals(multistageExtractor.getExpectedRecordCount(), 0); - } - - @Test - public void testGetHighWatermark() { - Assert.assertEquals(multistageExtractor.getHighWatermark(), 0); - } - - @Test - public void testReadRecord() { - Assert.assertNull(multistageExtractor.readRecord(null)); - } - - @Test - public void testClose() { - when(state.getWorkingState()).thenReturn(WorkUnitState.WorkingState.CANCELLED); - multistageExtractor.close(); - } - - @Test - public void testProcessInputStream() { - MultistageSource source = new MultistageSource(); - List wus = source.getWorkunits(new SourceState()); - WorkUnitState state = new WorkUnitState(wus.get(0), new JobState()); - multistageExtractor = new MultistageExtractor(state, source.getJobKeys()); - multistageExtractor.initialize(new ExtractorKeys()); - Assert.assertFalse(multistageExtractor.processInputStream(100L)); - } - - @Test - public void testSetRowFilter() { - JsonSchemaBasedFilter filter = Mockito.mock(JsonSchemaBasedFilter.class); - JsonArray schema = new JsonArray(); - multistageExtractor.rowFilter = filter; - multistageExtractor.setRowFilter(schema); - - multistageExtractor.rowFilter = null; - when(state.getProp(MultistageProperties.MSTAGE_ENABLE_SCHEMA_BASED_FILTERING.getConfig(), StringUtils.EMPTY)).thenReturn("false"); - multistageExtractor.setRowFilter(new JsonArray()); - Assert.assertNull(multistageExtractor.rowFilter); - } - - @Test - public void testGetOrInferSchema() { - MultistageSource source = new MultistageSource(); - List wus = source.getWorkunits(new SourceState()); - WorkUnitState state = new WorkUnitState(wus.get(0), new JobState()); - multistageExtractor = new MultistageExtractor(state, source.getJobKeys()); - multistageExtractor.initialize(new ExtractorKeys()); - - JsonObject schema = new JsonObject(); - schema.addProperty("testAttribute", "something"); - - JsonArray schemaArray = new JsonArray(); - Map defaultFieldTypes = new HashMap<>(); - - Assert.assertEquals(multistageExtractor.getOrInferSchema(), schemaArray); - - ExtractorKeys extractorKeys = Mockito.mock(ExtractorKeys.class); - JsonArray inferredSchema = new JsonArray(); - JsonObject schemaObj = new JsonObject(); - schemaObj.addProperty("type", "null"); - multistageExtractor.extractorKeys = extractorKeys; - when(extractorKeys.getInferredSchema()).thenReturn(inferredSchema); - when(extractorKeys.getActivationParameters()).thenReturn(schemaObj); - when(extractorKeys.getPayloads()).thenReturn(new JsonArray()); - when(jobKeys.hasSourceSchema()).thenReturn(false); - Assert.assertEquals(multistageExtractor.getOrInferSchema(), schemaArray); - - when(jobKeys.hasSourceSchema()).thenReturn(true); - Assert.assertEquals(multistageExtractor.getOrInferSchema(), schemaArray); - } - - @Test - public void testHoldExecutionUnitPresetStartTime() throws Exception { - multistageExtractor.extractorKeys = extractorKeys; - //current time + 3 s - Long currentSeconds = DateTime.now().plusSeconds(3).getMillis(); - when(extractorKeys.getDelayStartTime()).thenReturn(currentSeconds); - - PowerMockito.mockStatic(Thread.class); - PowerMockito.doNothing().when(Thread.class); - Thread.sleep(100L); - multistageExtractor.holdExecutionUnitPresetStartTime(); - - when(extractorKeys.getDelayStartTime()).thenReturn(DateTime.now().plusSeconds(3).getMillis()); - PowerMockito.doThrow(new InterruptedException()).when(Thread.class); - Thread.sleep(100L); - multistageExtractor.holdExecutionUnitPresetStartTime(); - } - - @Test - public void testsFailWorkUnit() { - state = new WorkUnitState(); - WorkUnitState stateSpy = spy(state); - multistageExtractor.state = stateSpy; - multistageExtractor.failWorkUnit(StringUtils.EMPTY); - verify(stateSpy).setWorkingState(WorkUnitState.WorkingState.FAILED); - multistageExtractor.failWorkUnit("NON_EMPTY_ERROR_STRING"); - } - - @Test - public void testDeriveEpoc() { - String format = "yyyy-MM-dd"; - String strValue = "2020-06-20"; - Assert.assertNotEquals(multistageExtractor.deriveEpoc(format, strValue), StringUtils.EMPTY); - - strValue = "2018-07-14Txsdfs"; - Assert.assertNotEquals(multistageExtractor.deriveEpoc(format, strValue), StringUtils.EMPTY); - - format = "yyyy-MM-dd'T'HH:mm:ssZ"; - strValue = "2018/07/14T14:31:30+0530"; - Assert.assertEquals(multistageExtractor.deriveEpoc(format, strValue), StringUtils.EMPTY); - } - - @Test - public void testsAddDerivedFieldsToAltSchema() { - Map items = ImmutableMap.of("type", "some_type", "source", "token.full_token"); - Map> derivedFields = ImmutableMap.of("formula", items); - JsonArray outputSchema = gson.fromJson("[{\"token.full_token\": {\"type\":\"string\"}}]", JsonArray.class); - when(source.getJobKeys()).thenReturn(jobKeys); - when(jobKeys.getOutputSchema()).thenReturn(outputSchema); - when(jobKeys.getDerivedFields()).thenReturn(derivedFields); - Assert.assertEquals(multistageExtractor.addDerivedFieldsToAltSchema().toString(), - "[{\"columnName\":\"formula\",\"dataType\":{\"type\":\"string\"}}]"); - } - - @Test - public void testExtractText() throws Exception { - Assert.assertEquals(multistageExtractor.extractText(null), StringUtils.EMPTY); - - String expected = "test_string"; - InputStream input = new ByteArrayInputStream(expected.getBytes()); - when(state.getProp(MultistageProperties.MSTAGE_SOURCE_DATA_CHARACTER_SET.getConfig(), StringUtils.EMPTY)).thenReturn("UTF-8"); - Assert.assertEquals(multistageExtractor.extractText(input), expected); - - PowerMockito.mockStatic(IOUtils.class); - PowerMockito.doThrow(new IOException()).when(IOUtils.class, "toString", input, Charset.forName("UTF-8")); - multistageExtractor.extractText(input); - Assert.assertEquals(multistageExtractor.extractText(null), StringUtils.EMPTY); - } - - @Test - public void testCheckContentType() { - String expectedContentType = "application/json"; - Map messages = new HashMap<>(); - when(workUnitStatus.getMessages()).thenReturn(messages); - Assert.assertTrue(multistageExtractor.checkContentType(workUnitStatus, expectedContentType)); - - messages.put("contentType", expectedContentType); - when(workUnitStatus.getMessages()).thenReturn(messages); - Assert.assertTrue(multistageExtractor.checkContentType(workUnitStatus, expectedContentType)); - - messages.put("contentType", "non-expected-contentType"); - when(workUnitStatus.getMessages()).thenReturn(messages); - Assert.assertFalse(multistageExtractor.checkContentType(workUnitStatus, expectedContentType)); - - when(workUnitStatus.getMessages()).thenReturn(null); - Assert.assertTrue(multistageExtractor.checkContentType(workUnitStatus, expectedContentType)); - HashSet expectedContentTypeSet = new LinkedHashSet<>( - Arrays.asList("text/csv", "application/gzip", "application/json") - ); - messages.clear(); - when(workUnitStatus.getMessages()).thenReturn(messages); - Assert.assertTrue(multistageExtractor.checkContentType(workUnitStatus, expectedContentTypeSet)); - - messages.put("contentType", expectedContentType); - when(workUnitStatus.getMessages()).thenReturn(messages); - Assert.assertTrue(multistageExtractor.checkContentType(workUnitStatus, expectedContentTypeSet)); - - messages.put("contentType", "non-expected-contentType"); - when(workUnitStatus.getMessages()).thenReturn(messages); - Assert.assertFalse(multistageExtractor.checkContentType(workUnitStatus, expectedContentTypeSet)); - - when(workUnitStatus.getMessages()).thenReturn(null); - Assert.assertTrue(multistageExtractor.checkContentType(workUnitStatus, expectedContentTypeSet)); - } - - /** - * test getting session key value when the value is in the headers - */ - @Test - public void testGetSessionKeyValue() { - String headers = "{\"cursor\": \"123\"}"; - Map messages = new HashMap<>(); - messages.put("headers", headers); - when(workUnitStatus.getMessages()).thenReturn(messages); - - JsonObject sessionKeyField = gson.fromJson("{\"name\": \"cursor\"}", JsonObject.class); - when(source.getJobKeys()).thenReturn(jobKeys); - when(jobKeys.getSessionKeyField()).thenReturn(sessionKeyField); - - Assert.assertEquals(multistageExtractor.getSessionKey(workUnitStatus), "123"); - } - - @Test - public void testMinimumSchema() { - WorkUnitState state = new WorkUnitState(); - state.setProp(ConfigurationKeys.EXTRACT_PRIMARY_KEY_FIELDS_KEY, "id"); - state.setProp(ConfigurationKeys.EXTRACT_DELTA_FIELDS_KEY, "date"); - MultistageSource source = new MultistageSource<>(); - MultistageExtractor extractor = new MultistageExtractor<>(state, source.getJobKeys()); - JsonArray schema = extractor.createMinimumSchema(); - String expected = "[{\"columnName\":\"id\",\"isNullable\":true,\"dataType\":{\"type\":\"string\"}},{\"columnName\":\"date\",\"isNullable\":true,\"dataType\":{\"type\":\"timestamp\"}}]"; - Assert.assertEquals(schema.toString(), expected); - } - - @Test - public void testMinimumSchemaEmpty() { - WorkUnitState state = new WorkUnitState(); - state.setProp(ConfigurationKeys.EXTRACT_PRIMARY_KEY_FIELDS_KEY, ""); - state.setProp(ConfigurationKeys.EXTRACT_DELTA_FIELDS_KEY, "date"); - MultistageSource source = new MultistageSource<>(); - MultistageExtractor extractor = new MultistageExtractor<>(state, source.getJobKeys()); - JsonArray schema = extractor.createMinimumSchema(); - String expected = "[{\"columnName\":\"date\",\"isNullable\":true,\"dataType\":{\"type\":\"timestamp\"}}]"; - Assert.assertEquals(schema.toString(), expected); - } - /** - * ReplaceVariablesInParameters() replace placeholders with their real values. This process - * is called substitution. - * - * When the substituted parameter starts with tmp, the parameter is removed from the final. - * - * @throws Exception - */ - @Test - public void testReplaceVariablesInParameters() throws Exception { - WorkUnitState state = new WorkUnitState(); - MultistageSource source = new MultistageSource<>(); - MultistageExtractor extractor = new MultistageExtractor<>(state, source.getJobKeys()); - - JsonObject parameters = gson.fromJson("{\"param1\":\"value1\"}", JsonObject.class); - JsonObject replaced = extractor.replaceVariablesInParameters(parameters); - Assert.assertEquals(replaced, parameters); - - parameters = gson.fromJson("{\"param1\":\"value1\",\"param2\":\"{{param1}}\"}", JsonObject.class); - JsonObject parameters2Expected = gson.fromJson("{\"param1\":\"value1\",\"param2\":\"value1\"}", JsonObject.class); - replaced = extractor.replaceVariablesInParameters(parameters); - Assert.assertEquals(replaced, parameters2Expected); - - parameters = gson.fromJson("{\"tmpParam1\":\"value1\",\"param2\":\"{{tmpParam1}}\"}", JsonObject.class); - parameters2Expected = gson.fromJson("{\"param2\":\"value1\"}", JsonObject.class); - replaced = extractor.replaceVariablesInParameters(parameters); - Assert.assertEquals(replaced, parameters2Expected); - } - - @Test - public void testAppendActivationParameter() throws Exception { - MultistageExtractor extractor = Mockito.mock(MultistageExtractor.class); - ExtractorKeys extractorKeys = Mockito.mock(ExtractorKeys.class); - extractor.extractorKeys = extractorKeys; - - JsonObject obj = gson.fromJson("{\"survey\": \"id1\"}", JsonObject.class); - when(extractorKeys.getActivationParameters()).thenReturn(obj); - - Method method = MultistageExtractor.class.getDeclaredMethod("appendActivationParameter", JsonObject.class); - method.setAccessible(true); - - Assert.assertEquals(method.invoke(extractor, obj), obj); - } - - @Test - public void testGetUpdatedWorkUnitVariableValues() throws Exception { - MultistageExtractor extractor = Mockito.mock(MultistageExtractor.class); - WorkUnitStatus wuStatus = Mockito.mock(WorkUnitStatus.class); - - when(extractor.getWorkUnitStatus()).thenReturn(wuStatus); - when(wuStatus.getPageSize()).thenReturn(100L); - when(wuStatus.getPageNumber()).thenReturn(5L); - when(wuStatus.getPageStart()).thenReturn(1L); - when(wuStatus.getSessionKey()).thenReturn("test_session_key"); - - JsonObject jsonObject = new JsonObject(); - jsonObject.addProperty(ParameterTypes.SESSION.toString(), "{\"name\": \"status\"}"); - jsonObject.addProperty(ParameterTypes.PAGESTART.toString(), 1); - jsonObject.addProperty(ParameterTypes.PAGESIZE.toString(), 100); - jsonObject.addProperty(ParameterTypes.PAGENO.toString(), 5); - - Method method = MultistageExtractor.class.getDeclaredMethod("getUpdatedWorkUnitVariableValues", JsonObject.class); - method.setAccessible(true); - - Assert.assertEquals(method.invoke(extractor, jsonObject).toString(), - "{\"session\":\"test_session_key\",\"pagestart\":1,\"pagesize\":100,\"pageno\":5}"); - - when(wuStatus.getPageSize()).thenReturn(-1L); - Assert.assertEquals(method.invoke(extractor, jsonObject).toString(), - "{\"pagesize\":100,\"session\":\"test_session_key\",\"pagestart\":1,\"pageno\":5}"); - } - - @Test - public void testGetInitialWorkUnitVariableValues() throws Exception { - MultistageExtractor extractor = Mockito.mock(MultistageExtractor.class); - Method method = MultistageExtractor.class.getDeclaredMethod("getInitialWorkUnitVariableValues"); - method.setAccessible(true); - - JobKeys jobKeys = Mockito.mock(JobKeys.class); - extractor.jobKeys = jobKeys; - JsonObject waterMarkObj = gson.fromJson("{\"watermark\":{\"low\":-100,\"high\":1564642800}}", JsonObject.class); - when(extractor.getWorkUnitWaterMarks()).thenReturn(waterMarkObj); - when(jobKeys.getPaginationInitValues()).thenReturn(ImmutableMap.of(ParameterTypes.PAGESIZE, 10L)); - Assert.assertEquals(method.invoke(extractor).toString(), - "{\"watermark\":{\"watermark\":{\"low\":-100,\"high\":1564642800}},\"pagesize\":10}"); - } - - -} \ No newline at end of file diff --git a/cdi-core/src/test/java/com/linkedin/cdi/factory/ApacheHttpClientFactoryTest.java b/cdi-core/src/test/java/com/linkedin/cdi/factory/DefaultConnectionClientFactoryTest.java similarity index 82% rename from cdi-core/src/test/java/com/linkedin/cdi/factory/ApacheHttpClientFactoryTest.java rename to cdi-core/src/test/java/com/linkedin/cdi/factory/DefaultConnectionClientFactoryTest.java index 7d6ce34..dc50379 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/factory/ApacheHttpClientFactoryTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/factory/DefaultConnectionClientFactoryTest.java @@ -18,7 +18,7 @@ @PrepareForTest({HttpClientBuilder.class}) -public class ApacheHttpClientFactoryTest extends PowerMockTestCase { +public class DefaultConnectionClientFactoryTest extends PowerMockTestCase { @Mock private HttpClientBuilder httpClientBuilder; @@ -30,10 +30,10 @@ public class ApacheHttpClientFactoryTest extends PowerMockTestCase { */ @Test public void testGet() { - ApacheHttpClientFactory factory = new ApacheHttpClientFactory(); + DefaultConnectionClientFactory factory = new DefaultConnectionClientFactory(); PowerMockito.mockStatic(HttpClientBuilder.class); PowerMockito.when(HttpClientBuilder.create()).thenReturn(httpClientBuilder); when(httpClientBuilder.build()).thenReturn(closeableHttpClient); - Assert.assertEquals(factory.get(new State()), closeableHttpClient); + Assert.assertEquals(factory.getHttpClient(new State()), closeableHttpClient); } } \ No newline at end of file diff --git a/cdi-core/src/test/java/com/linkedin/cdi/factory/SecureConnectionClientFactoryTest.java b/cdi-core/src/test/java/com/linkedin/cdi/factory/SecureConnectionClientFactoryTest.java new file mode 100644 index 0000000..1ba6821 --- /dev/null +++ b/cdi-core/src/test/java/com/linkedin/cdi/factory/SecureConnectionClientFactoryTest.java @@ -0,0 +1,44 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.factory; + +import com.linkedin.cdi.configuration.PropertyCollection; +import com.linkedin.cdi.factory.sftp.SftpClient; +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import org.apache.gobblin.configuration.SourceState; +import org.apache.http.client.HttpClient; +import org.testng.annotations.Test; + +public class SecureConnectionClientFactoryTest { + + @Test (enabled = false) + public void testHttpConnection() throws IOException { + SourceState state = new SourceState(); + state.setProp(PropertyCollection.MSTAGE_SSL.getConfig(), "REPLACEME"); + HttpClient httpClient = new SecureConnectionClientFactory().getHttpClient(state); + ((Closeable) httpClient).close(); + } + + @Test (enabled = false) + public void testSftpConnection() { + SourceState state = new SourceState(); + state.setProp(PropertyCollection.SOURCE_CONN_HOST.getConfig(), "REPLACEME"); + state.setProp(PropertyCollection.SOURCE_CONN_USERNAME.getConfig(), "REPLACEME"); + state.setProp(PropertyCollection.SOURCE_CONN_PORT.getConfig(), "22"); + state.setProp(PropertyCollection.SOURCE_CONN_PRIVATE_KEY.getConfig(), "REPLACEME"); + state.setProp(PropertyCollection.MSTAGE_SFTP_CONN_TIMEOUT_MILLIS.getConfig(), "360000"); + ConnectionClientFactory factory = new SecureConnectionClientFactory(); + SftpClient client = factory.getSftpChannelClient(state); + try { + client.getSftpChannel(); + } catch (Exception e) { + System.out.println(e.getMessage()); + } + List files = client.ls("REPLACEME"); + client.close(); + } +} diff --git a/cdi-core/src/test/java/com/linkedin/cdi/filter/AvroSchemaBasedFilterTest.java b/cdi-core/src/test/java/com/linkedin/cdi/filter/AvroSchemaBasedFilterTest.java index 18bd747..c25af45 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/filter/AvroSchemaBasedFilterTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/filter/AvroSchemaBasedFilterTest.java @@ -16,6 +16,7 @@ import com.linkedin.cdi.keys.AvroExtractorKeys; import com.linkedin.cdi.util.AvroSchemaUtils; import com.linkedin.cdi.util.JsonIntermediateSchema; +import org.apache.gobblin.converter.avro.UnsupportedDateTypeException; import org.apache.gobblin.source.workunit.Extract; import org.testng.Assert; import org.testng.annotations.BeforeMethod; @@ -52,7 +53,7 @@ public void setUp() throws RetriableAuthenticationException { } @Test - public void testFilter() { + public void testFilter() throws UnsupportedDateTypeException { // The case where one column is filtered out JsonArray rawSchemaArray = GSON.fromJson( "[{\"columnName\":\"id0\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]", JsonArray.class); diff --git a/cdi-core/src/test/java/com/linkedin/cdi/filter/MultistageSchemaBasedFilterTest.java b/cdi-core/src/test/java/com/linkedin/cdi/filter/MultistageSchemaBasedFilterTest.java index c0f29fe..b35e4c0 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/filter/MultistageSchemaBasedFilterTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/filter/MultistageSchemaBasedFilterTest.java @@ -5,6 +5,7 @@ package com.linkedin.cdi.filter; import com.linkedin.cdi.util.JsonIntermediateSchema; +import org.apache.gobblin.converter.avro.UnsupportedDateTypeException; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; @@ -13,7 +14,7 @@ public class MultistageSchemaBasedFilterTest { @Test - public void testFilter() { + public void testFilter() throws UnsupportedDateTypeException { JsonIntermediateSchema schema = Mockito.mock(JsonIntermediateSchema.class); MultistageSchemaBasedFilter filter = new MultistageSchemaBasedFilter(schema); Assert.assertEquals(filter.filter("input"), null); diff --git a/cdi-core/src/test/java/com/linkedin/cdi/keys/JobKeysTest.java b/cdi-core/src/test/java/com/linkedin/cdi/keys/JobKeysTest.java index f2119b9..9b5484b 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/keys/JobKeysTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/keys/JobKeysTest.java @@ -7,17 +7,16 @@ import com.google.gson.Gson; import com.google.gson.JsonArray; import com.google.gson.JsonObject; +import com.linkedin.cdi.util.JsonUtils; +import com.linkedin.cdi.util.ParameterTypes; +import com.linkedin.cdi.util.SchemaBuilder; +import com.linkedin.cdi.util.WorkUnitPartitionTypes; import gobblin.configuration.SourceState; import java.lang.reflect.Method; import java.util.HashMap; import java.util.Map; import org.apache.commons.lang3.StringUtils; import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.util.JsonUtils; -import com.linkedin.cdi.util.ParameterTypes; -import com.linkedin.cdi.util.SchemaBuilder; -import com.linkedin.cdi.util.WorkUnitPartitionTypes; import org.mockito.Mockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.testng.PowerMockTestCase; @@ -25,6 +24,7 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static org.mockito.Mockito.*; @@ -115,29 +115,29 @@ public void testValidation() { Assert.assertTrue(jobKeys.validate(state)); // test output schema validation with a wrong type - state.setProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "{}"); + state.setProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), "{}"); Assert.assertFalse(jobKeys.validate(state)); // test output schema validation with an empty array - state.setProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "[{}]"); + state.setProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), "[{}]"); Assert.assertFalse(jobKeys.validate(state)); // test output schema validation with an incorrect structure String schema = "[{\"columnName\":\"test\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}]"; - state.setProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), schema); + state.setProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), schema); jobKeys.initialize(state); Assert.assertFalse(jobKeys.validate(state)); schema = "[{\"columnName\":\"test\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]"; - state.setProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), schema); + state.setProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), schema); jobKeys.setOutputSchema(jobKeys.parseOutputSchema(state)); Assert.assertTrue(jobKeys.validate(state)); - state.setProp(MultistageProperties.MSTAGE_WORK_UNIT_PARTITION.getConfig(), "lovely"); + state.setProp(MSTAGE_WORK_UNIT_PARTITION.getConfig(), "lovely"); jobKeys.setWorkUnitPartitionType(null); Assert.assertFalse(jobKeys.validate(state)); - state.setProp(MultistageProperties.MSTAGE_WORK_UNIT_PARTITION.getConfig(), "{\"weekly\": [\"2020-01-01\", \"2020-02-1\"]}"); + state.setProp(MSTAGE_WORK_UNIT_PARTITION.getConfig(), "{\"weekly\": [\"2020-01-01\", \"2020-02-1\"]}"); jobKeys.setWorkUnitPartitionType(WorkUnitPartitionTypes.COMPOSITE); Assert.assertFalse(jobKeys.validate(state)); } @@ -148,8 +148,8 @@ public void testGetDefaultFieldTypes() throws Exception { Method method = JobKeys.class.getDeclaredMethod("parseDefaultFieldTypes", State.class); method.setAccessible(true); - State state = Mockito.mock(State.class); - when(state.getProp(MultistageProperties.MSTAGE_DATA_DEFAULT_TYPE.getConfig(), new JsonObject().toString())).thenReturn("{\"testField\":100}"); + State state = new SourceState(); + state.setProp(MSTAGE_DATA_DEFAULT_TYPE.getConfig(), "{\"testField\":100}"); Assert.assertEquals(method.invoke(jobkeys, state).toString(), "{testField=100}"); } @@ -175,9 +175,8 @@ public void testGetPaginationInitialValues() throws Exception { Method method = JobKeys.class.getDeclaredMethod("parsePaginationInitialValues", State.class); method.setAccessible(true); - State state = Mockito.mock(State.class); - when(state.getProp(MultistageProperties.MSTAGE_PAGINATION.getConfig(), new JsonObject().toString())) - .thenReturn("{\"fields\": [\"offset\", \"limit\"], \"initialvalues\": [0, 5000]}"); + State state = new SourceState(); + state.setProp(MSTAGE_PAGINATION.getConfig(), "{\"fields\": [\"offset\", \"limit\"], \"initialvalues\": [0, 5000]}"); method.invoke(jobkeys, state); Map paginationInitValues = jobkeys.getPaginationInitValues(); Assert.assertEquals((long) paginationInitValues.get(ParameterTypes.PAGESTART), 0L); @@ -188,14 +187,14 @@ public void testGetPaginationInitialValues() throws Exception { public void testGetPaginationFields() throws Exception { JobKeys jobkeys = new JobKeys(); State state = Mockito.mock(State.class); - when(state.getProp(MultistageProperties.MSTAGE_PAGINATION.getConfig(), new JsonObject().toString())) + when(state.getProp(MSTAGE_PAGINATION.getConfig(), new JsonObject().toString())) .thenReturn("{\"fields\": [\"\", \"\"], \"initialvalues\": [0, 5000]}"); Method method = JobKeys.class.getDeclaredMethod("parsePaginationFields", State.class); method.setAccessible(true); method.invoke(jobkeys, state); Assert.assertEquals(jobkeys.getPaginationInitValues().size(), 0); - when(state.getProp(MultistageProperties.MSTAGE_PAGINATION.getConfig(), new JsonObject().toString())) + when(state.getProp(MSTAGE_PAGINATION.getConfig(), new JsonObject().toString())) .thenReturn("{\"initialvalues\": [0, 5000]}"); method.invoke(jobkeys, state); Assert.assertEquals(jobkeys.getPaginationInitValues().size(), 0); diff --git a/cdi-core/src/test/java/com/linkedin/cdi/keys/JobKeysTestNoMock.java b/cdi-core/src/test/java/com/linkedin/cdi/keys/JobKeysTestNoMock.java new file mode 100644 index 0000000..9c54f93 --- /dev/null +++ b/cdi-core/src/test/java/com/linkedin/cdi/keys/JobKeysTestNoMock.java @@ -0,0 +1,20 @@ +// Copyright 2021 LinkedIn Corporation. All rights reserved. +// Licensed under the BSD-2 Clause license. +// See LICENSE in the project root for license information. + +package com.linkedin.cdi.keys; + +import gobblin.configuration.SourceState; +import org.apache.gobblin.configuration.State; +import org.junit.Assert; +import org.testng.annotations.Test; + + +public class JobKeysTestNoMock { + @Test + public void testValidate() { + State state = new SourceState(); + state.setProp("csv.max.failures", "10"); + Assert.assertFalse(new JobKeys().validate(state)); + } +} diff --git a/cdi-core/src/test/java/com/linkedin/cdi/preprocessor/GpgEncryptProcessorTest.java b/cdi-core/src/test/java/com/linkedin/cdi/preprocessor/GpgEncryptProcessorTest.java index b45ed47..de63d88 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/preprocessor/GpgEncryptProcessorTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/preprocessor/GpgEncryptProcessorTest.java @@ -5,9 +5,6 @@ package com.linkedin.cdi.preprocessor; import com.google.gson.JsonObject; -import java.io.IOException; -import java.io.PipedInputStream; -import java.io.PipedOutputStream; import org.testng.Assert; import org.testng.annotations.Test; @@ -25,19 +22,4 @@ public void testConvertFileName() { OutputStreamProcessor processor = new GpgEncryptProcessor(parameters); Assert.assertEquals(processor.convertFileName(fileName), "abc.zip.gpg"); } - - @Test - public void testEncryption() throws IOException { - JsonObject parameters = new JsonObject(); - parameters.addProperty("cipher", "AES_256"); - parameters.addProperty("keystore_path",this.getClass().getResource("/key/public.key").toString()); - parameters.addProperty("key_name","48A84F2FA6E38870"); - - PipedInputStream is = new PipedInputStream(); - PipedOutputStream os = new PipedOutputStream(is); - - OutputStreamProcessor processor = new GpgEncryptProcessor(parameters); - Assert.assertNotNull(processor.process(os)); - - } } diff --git a/cdi-core/src/test/java/com/linkedin/cdi/source/HttpSourceTest.java b/cdi-core/src/test/java/com/linkedin/cdi/source/HttpSourceTest.java deleted file mode 100644 index fe4d106..0000000 --- a/cdi-core/src/test/java/com/linkedin/cdi/source/HttpSourceTest.java +++ /dev/null @@ -1,399 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.source; - -import com.google.gson.Gson; -import com.google.gson.JsonArray; -import com.google.gson.JsonElement; -import com.google.gson.JsonObject; -import java.io.InputStreamReader; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang.StringUtils; -import org.apache.gobblin.configuration.ConfigurationKeys; -import org.apache.gobblin.configuration.SourceState; -import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.extractor.JsonExtractor; -import com.linkedin.cdi.helpers.GobblinMultiStageTestHelpers; -import com.linkedin.cdi.keys.HttpKeys; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.util.EncryptionUtils; -import com.linkedin.cdi.util.ParameterTypes; -import org.apache.gobblin.runtime.embedded.EmbeddedGobblin; -import org.apache.gobblin.source.workunit.WorkUnit; -import org.mockito.Mockito; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.testng.PowerMockTestCase; -import org.powermock.reflect.Whitebox; -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import static com.linkedin.cdi.configuration.MultistageProperties.*; -import static com.linkedin.cdi.source.HttpSource.*; -import static org.mockito.Mockito.*; - - -@Slf4j -@PrepareForTest({EncryptionUtils.class}) -public class HttpSourceTest extends PowerMockTestCase { - - private Gson gson; - private WorkUnitState state; - private HttpSource source; - private JobKeys jobKeys; - private SourceState sourceState; - private String token; - private JsonObject pagination; - private JsonObject sessionKeyField; - private String totalCountField; - private JsonArray parameters; - private JsonArray encryptionFields; - private String dataField; - private Long callInterval; - private Long waitTimeoutSeconds; - private Boolean enableCleansing; - private Boolean workUnitPartialPartition; - private JsonArray watermark; - private JsonArray secondaryInput; - private String httpClientFactory; - private JsonObject httpRequestHeaders; - private String sourceUri; - private String httpRequestMethod; - private String extractorClass; - private JsonObject authentication; - private JsonObject httpStatus; - private JsonObject httpStatusReasons; - - @BeforeMethod - public void setUp() { - gson = new Gson(); - state = Mockito.mock(WorkUnitState.class); - jobKeys = Mockito.mock(JobKeys.class); - sourceState = Mockito.mock(SourceState.class); - source = new HttpSource(); - } - - @Test(enabled = false) - public void testAuthentication() { - HttpSource source = new HttpSource(); - - SourceState state = mock(SourceState.class); - when(state.getProp("ms.watermark", "")).thenReturn("[{\"name\": \"system\",\"type\": \"datetime\", \"range\": {\"from\": \"2017-01-01\", \"to\": \"-\"}}]"); - when(state.getProp("extract.table.type", "SNAPSHOT_ONLY")).thenReturn("SNAPSHOT_ONLY"); - when(state.getProp("extract.namespace", "")).thenReturn("test"); - when(state.getProp("extract.table.name", "")).thenReturn("table1"); - when(state.getProp("source.conn.username", "")).thenReturn("X7CWBD5V4T6DR77WY23YSHACH55K2OXA"); - when(state.getProp("source.conn.password", "")).thenReturn(""); - when(state.getProp("ms.source.uri", "")).thenReturn("https://host/v2/users"); - when(state.getProp("ms.authentication", new JsonObject().toString())).thenReturn("{\"method\":\"basic\",\"encryption\":\"base64\", \"header\": \"Authorization\"}"); - when(state.getProp("ms.http.request.headers", new JsonObject().toString())).thenReturn("{\"Content-Type\": \"application/json\"}"); - when(state.getProp("ms.http.request.method", "")).thenReturn("GET"); - when(state.getProp("ms.session.key.field", new JsonObject().toString())).thenReturn("{\"name\": \"records.cursor\"}"); - when(state.getProp("ms.parameters", new JsonArray().toString())).thenReturn("[{\"name\":\"cursor\",\"type\":\"session\"}]"); - when(state.getProp("ms.data.field", "")).thenReturn("users"); - when(state.getProp("ms.total.count.field", "")).thenReturn("records.totalRecords"); - when(state.getProp("ms.work.unit.partition", "")).thenReturn(""); - when(state.getProp("ms.pagination", new JsonObject().toString())).thenReturn("{}"); - - List workUnits = source.getWorkunits(state); - - Assert.assertFalse(source.getJobKeys().isPaginationEnabled()); - Assert.assertNotNull(source.getJobKeys()); - Assert.assertNotNull(source.getHttpSourceKeys()); - Assert.assertNotNull(source.getJobKeys().getSourceParameters()); - Assert.assertTrue(workUnits.size() == 1); - Assert.assertEquals(source.getHttpSourceKeys().getHttpRequestHeaders().toString(), "{\"Content-Type\":\"application/json\"}"); - - WorkUnitState unitState = new WorkUnitState(workUnits.get(0)); - - JsonExtractor extractor = new JsonExtractor(unitState, source.getHttpSourceKeys()); - - JsonObject record = extractor.readRecord(new JsonObject()); - - // should return 14 columns - Assert.assertEquals(14, record.entrySet().size()); - Assert.assertTrue(extractor.getWorkUnitStatus().getTotalCount() > 0); - Assert.assertTrue(extractor.getWorkUnitStatus().getSessionKey().length() > 0); - } - - /* - * basic test with no watermark created. - */ - @Test(enabled=false) - public void getWorkUnitsTestEmpty() { - HttpSource source = new HttpSource(); - List workUnits = source.getWorkunits(GobblinMultiStageTestHelpers.prepareSourceStateWithoutWaterMark()); - Assert.assertTrue(workUnits.size() == 1); - Assert.assertEquals(workUnits.get(0).getLowWatermark().getAsJsonObject().get("value").toString(), "-1"); - Assert.assertEquals(workUnits.get(0).getExpectedHighWatermark().getAsJsonObject().get("value").toString(), "-1"); - } - - /* - * basic test with watermark. - */ - @Test(enabled=false) - public void getWorkUnitsTest() { - HttpSource source = new HttpSource(); - List workUnits = source.getWorkunits(GobblinMultiStageTestHelpers.prepareSourceStateWithWaterMark()); - Assert.assertTrue(workUnits.size() == 1); - - //time stamps below corresponds to the date given in watermark fields in test data. - Assert.assertEquals(GobblinMultiStageTestHelpers - .getDateFromTimeStamp( - Long.parseLong(workUnits.get(0).getLowWatermark().getAsJsonObject().get("value").toString())), - "2019-08-01"); - Assert.assertEquals(GobblinMultiStageTestHelpers - .getDateFromTimeStamp( - Long.parseLong(workUnits.get(0).getExpectedHighWatermark().getAsJsonObject().get("value").toString())), - "2019-08-02"); - } - - /* - * precondition check failure test. - */ - @Test(enabled=false) - public void preConditionCheckFail() { - boolean isIllegalState = false; - try { - HttpSource source = new HttpSource(); - SourceState state = GobblinMultiStageTestHelpers.prepareSourceStateWithWaterMark(); - when(state.getProp(ConfigurationKeys.EXTRACT_NAMESPACE_NAME_KEY)).thenReturn(null); - List workUnits = source.getWorkunits(state); - } catch (Exception e) { - isIllegalState = e.getClass().getCanonicalName() - .contains("IllegalStateException"); - } - Assert.assertTrue(isIllegalState); - } - - @Test - public void testGetAuthenticationHeader() { - SourceState state = new SourceState(); - HttpSource httpSource = new HttpSource(); - state.setProp("source.conn.username", "1"); - state.setProp("source.conn.password", "2"); - - state.setProp("ms.authentication", "{\"method\":\"basic\",\"encryption\":\"base64\", \"header\": \"Authorization\"}"); - httpSource.initialize(state); - Assert.assertEquals(httpSource.getHttpSourceKeys().getHttpRequestHeadersWithAuthentication().toString(), "{Authorization=Basic MToy}"); - - state.setProp("ms.authentication", "{\"method\":\"bearer\",\"encryption\":\"base64\", \"header\": \"Authorization\"}"); - httpSource.initialize(state); - Assert.assertEquals(httpSource.getHttpSourceKeys().getHttpRequestHeadersWithAuthentication().toString(), "{Authorization=Bearer MToy}"); - - state.setProp("ms.authentication", "{\"method\":\"bearer\",\"encryption\":\"base64\", \"header\": \"Authorization\", \"token\": \"xyz\"}"); - httpSource.initialize(state); - Assert.assertEquals(httpSource.getHttpSourceKeys().getHttpRequestHeadersWithAuthentication().toString(), "{Authorization=Bearer eHl6}"); - } - - /** - * Test getAuthenticationHeader - */ - @Test - public void testGetAuthenticationHeader2() { - PowerMockito.mockStatic(EncryptionUtils.class); - - HttpKeys httpSourceKeys = mock(HttpKeys.class); - source.setHttpSourceKeys(httpSourceKeys); - - JsonObject authObj = gson.fromJson("{\"method\":\"some-method\",\"encryption\":\"base32\",\"header\":\"Authorization\"}", JsonObject.class); - when(httpSourceKeys.getAuthentication()).thenReturn(authObj); - Assert.assertEquals(source.getAuthenticationHeader(state), new HashMap<>()); - - authObj = gson.fromJson("{\"method\":\"oauth\",\"encryption\":\"base32\",\"header\":\"Authorization\",\"token\":\"sdf23someresfsdwrw24234\"}", JsonObject.class); - when(httpSourceKeys.getAuthentication()).thenReturn(authObj); - String token = "someDecryptedToken"; - when(EncryptionUtils.decryptGobblin(any(), any())).thenReturn(token); - Assert.assertEquals(source.getAuthenticationHeader(state).get("Authorization"), OAUTH_TOKEN_PREFIX + TOKEN_PREFIX_SEPARATOR + token); - - authObj = gson.fromJson("{\"method\":\"custom\",\"encryption\":\"base32\",\"header\":\"Authorization\",\"token\":\"sdf23someresfsdwrw24234\"}", JsonObject.class); - when(httpSourceKeys.getAuthentication()).thenReturn(authObj); - Assert.assertEquals(source.getAuthenticationHeader(state).get("Authorization"), token); - } - - /** - * This test, by simulation, verifies that when the http error message is contained in a normal response, - * we will be able to retrieve that if the content type is different from the expected - * - * The test queries non-existent S3 endpoint, which returns 404 error as expected, but we are simulating 404 - * as success by overriding status codes. - * - * @throws Exception - */ - @Test (enabled=false) - void testHttpErrorInNormalResponse() throws Exception { - EmbeddedGobblin job = new EmbeddedGobblin("test"); - Assert.assertTrue(job.jobFile(getClass().getResource("/pull/http-error.pull").getPath()).run().isSuccessful()); - } - - /** - * Test getExtractor - */ - @Test - public void testGetExtractor() { - initializeHelper(); - PowerMockito.mockStatic(EncryptionUtils.class); - when(EncryptionUtils.decryptGobblin(token, state)).thenReturn(token); - source.getExtractor(state); - jobKeys = source.getJobKeys(); - Map paginationFields = new HashMap<>(); - Map paginationInitValues = new HashMap<>(); - JsonArray fields = pagination.get("fields").getAsJsonArray(); - for (int i = 0; i < fields.size(); i++) { - switch (fields.get(i).getAsString()) { - case "page_start": - paginationFields.put(ParameterTypes.PAGESTART, "page_start"); - break; - case "page_size": - paginationFields.put(ParameterTypes.PAGESIZE, "page_size"); - break; - case "page_number": - paginationFields.put(ParameterTypes.PAGENO, "page_number"); - break; - } - } - - JsonArray initialvalues = pagination.get("initialvalues").getAsJsonArray(); - for (int i = 0; i < initialvalues.size(); i++) { - switch (i) { - case 0: - paginationInitValues.put(ParameterTypes.PAGESTART, initialvalues.get(0).getAsLong()); - break; - case 1: - paginationInitValues.put(ParameterTypes.PAGESIZE, initialvalues.get(1).getAsLong()); - break; - case 2: - paginationInitValues.put(ParameterTypes.PAGENO, initialvalues.get(2).getAsLong()); - break; - } - } - - Assert.assertEquals(jobKeys.getPaginationFields(), paginationFields); - Assert.assertEquals(jobKeys.getPaginationInitValues(), paginationInitValues); - Assert.assertEquals(jobKeys.getSessionKeyField(), sessionKeyField); - Assert.assertEquals(jobKeys.getTotalCountField(), totalCountField); - Assert.assertEquals(jobKeys.getSourceParameters(), parameters); - Assert.assertEquals(jobKeys.getEncryptionField(), encryptionFields); - Assert.assertEquals(jobKeys.getDataField(), dataField); - Assert.assertEquals(jobKeys.getCallInterval(), callInterval.longValue()); - Assert.assertEquals(jobKeys.getSessionTimeout(), waitTimeoutSeconds.longValue() * 1000); - Assert.assertEquals(jobKeys.getWatermarkDefinition(), watermark); - Assert.assertEquals(jobKeys.getSecondaryInputs(), secondaryInput); - Assert.assertEquals(source.getHttpSourceKeys().getAuthentication(), authentication); - Assert.assertEquals(source.getHttpSourceKeys().getSourceUri(), sourceUri); - Assert.assertEquals(source.getHttpSourceKeys().getHttpRequestMethod(), httpRequestMethod); - - Map> httpStatuses = new HashMap<>(); - for (Map.Entry entry : httpStatus.entrySet()) { - String key = entry.getKey(); - List codes = new ArrayList<>(); - for (int i = 0; i < entry.getValue().getAsJsonArray().size(); i++) { - codes.add(entry.getValue().getAsJsonArray().get(i).getAsInt()); - } - httpStatuses.put(key, codes); - } - Assert.assertEquals(source.getHttpSourceKeys().getHttpStatuses(), httpStatuses); - - Map> StatusesReasons = new HashMap<>(); - for (Map.Entry entry : httpStatusReasons.entrySet()) { - String key = entry.getKey(); - List reasons = new ArrayList<>(); - for (int i = 0; i < entry.getValue().getAsJsonArray().size(); i++) { - reasons.add(entry.getValue().getAsJsonArray().get(i).getAsString()); - } - StatusesReasons.put(key, reasons); - } - Assert.assertEquals(source.getHttpSourceKeys().getHttpStatusReasons(), StatusesReasons); - } - - /** - * Test getHttpStatuses - */ - @Test - public void testGetHttpStatuses() throws Exception { - String statuses = "{\"success\":{\"someKey\":\"someValue\"},\"warning\":null}"; - when(state.getProp(MSTAGE_HTTP_STATUSES.getConfig(), new JsonObject().toString())).thenReturn(statuses); - Assert.assertEquals(Whitebox.invokeMethod(source, "getHttpStatuses", state), new HashMap<>()); - } - - /** - * Test getHttpStatusReasons - */ - @Test - public void testGetHttpStatusReasons() throws Exception { - String reasons = "{\"success\":{\"someReason\":\"someValue\"},\"warning\":null}"; - when(state.getProp(MSTAGE_HTTP_STATUS_REASONS.getConfig(), new JsonObject().toString())).thenReturn(reasons); - Assert.assertEquals(Whitebox.invokeMethod(source, "getHttpStatusReasons", state), new HashMap<>()); - } - - private void initializeHelper() { - JsonObject allKeys = gson.fromJson(new InputStreamReader(this.getClass().getResourceAsStream("/json/sample-data-for-source.json")), JsonObject.class); - pagination = allKeys.get(MSTAGE_PAGINATION.getConfig()).getAsJsonObject(); - when(state.getProp(MSTAGE_PAGINATION.getConfig(), new JsonObject().toString())).thenReturn(pagination.toString()); - - sessionKeyField = allKeys.get(MSTAGE_SESSION_KEY_FIELD.getConfig()).getAsJsonObject(); - when(state.getProp(MSTAGE_SESSION_KEY_FIELD.getConfig(), new JsonObject().toString())).thenReturn(sessionKeyField.toString()); - - totalCountField = allKeys.get(MSTAGE_TOTAL_COUNT_FIELD.getConfig()).getAsString(); - when(state.getProp(MSTAGE_TOTAL_COUNT_FIELD.getConfig(), StringUtils.EMPTY)).thenReturn(totalCountField); - - parameters = allKeys.get(MSTAGE_PARAMETERS.getConfig()).getAsJsonArray(); - when(state.getProp(MSTAGE_PARAMETERS.getConfig(), new JsonArray().toString())).thenReturn(parameters.toString()); - - encryptionFields = allKeys.get(MSTAGE_ENCRYPTION_FIELDS.getConfig()).getAsJsonArray(); - when(state.getProp(MSTAGE_ENCRYPTION_FIELDS.getConfig(), new JsonArray().toString())).thenReturn(encryptionFields.toString()); - - dataField = allKeys.get(MSTAGE_DATA_FIELD.getConfig()).getAsString(); - when(state.getProp(MSTAGE_DATA_FIELD.getConfig(), StringUtils.EMPTY)).thenReturn(dataField); - - callInterval = allKeys.get(MSTAGE_CALL_INTERVAL.getConfig()).getAsLong(); - when(state.getPropAsLong(MSTAGE_CALL_INTERVAL.getConfig(), 0L)).thenReturn(callInterval); - - waitTimeoutSeconds = allKeys.get(MSTAGE_WAIT_TIMEOUT_SECONDS.getConfig()).getAsLong(); - when(state.getPropAsLong(MSTAGE_WAIT_TIMEOUT_SECONDS.getConfig(), 0L)).thenReturn(waitTimeoutSeconds); - - enableCleansing = allKeys.get(MSTAGE_ENABLE_CLEANSING.getConfig()).getAsBoolean(); - when(state.getPropAsBoolean(MSTAGE_ENABLE_CLEANSING.getConfig())).thenReturn(enableCleansing); - - workUnitPartialPartition = allKeys.get(MSTAGE_WORK_UNIT_PARTIAL_PARTITION.getConfig()).getAsBoolean(); - when(state.getPropAsBoolean(MSTAGE_WORK_UNIT_PARTIAL_PARTITION.getConfig())).thenReturn(workUnitPartialPartition); - - watermark = allKeys.get(MSTAGE_WATERMARK.getConfig()).getAsJsonArray(); - when(state.getProp(MSTAGE_WATERMARK.getConfig(), new JsonArray().toString())).thenReturn(watermark.toString()); - - secondaryInput = allKeys.get(MSTAGE_SECONDARY_INPUT.getConfig()).getAsJsonArray(); - when(state.getProp(MSTAGE_SECONDARY_INPUT.getConfig(), new JsonArray().toString())).thenReturn(secondaryInput.toString()); - - httpClientFactory = allKeys.get(MSTAGE_HTTP_CLIENT_FACTORY.getConfig()).getAsString(); - when(state.getProp(MSTAGE_HTTP_CLIENT_FACTORY.getConfig(), StringUtils.EMPTY)).thenReturn(httpClientFactory); - - httpRequestHeaders = allKeys.get(MSTAGE_HTTP_REQUEST_HEADERS.getConfig()).getAsJsonObject(); - when(state.getProp(MSTAGE_HTTP_REQUEST_HEADERS.getConfig(), new JsonObject().toString())).thenReturn(httpRequestHeaders.toString()); - - sourceUri = allKeys.get(MSTAGE_SOURCE_URI.getConfig()).getAsString(); - when(state.getProp(MSTAGE_SOURCE_URI.getConfig(), StringUtils.EMPTY)).thenReturn(sourceUri); - - httpRequestMethod = allKeys.get(MSTAGE_HTTP_REQUEST_METHOD.getConfig()).getAsString(); - when(state.getProp(MSTAGE_HTTP_REQUEST_METHOD.getConfig(), StringUtils.EMPTY)).thenReturn(httpRequestMethod); - - extractorClass = allKeys.get(MSTAGE_EXTRACTOR_CLASS.getConfig()).getAsString(); - when(state.getProp(MSTAGE_EXTRACTOR_CLASS.getConfig(), StringUtils.EMPTY)).thenReturn(extractorClass); - - authentication = allKeys.get(MSTAGE_AUTHENTICATION.getConfig()).getAsJsonObject(); - token = authentication.get("token").getAsString(); - when(state.getProp(MSTAGE_AUTHENTICATION.getConfig(), new JsonObject().toString())).thenReturn(authentication.toString()); - - httpStatus = allKeys.get(MSTAGE_HTTP_STATUSES.getConfig()).getAsJsonObject(); - when(state.getProp(MSTAGE_HTTP_STATUSES.getConfig(), new JsonObject().toString())).thenReturn(httpStatus.toString()); - - httpStatusReasons = allKeys.get(MSTAGE_HTTP_STATUS_REASONS.getConfig()).getAsJsonObject(); - when(state.getProp(MSTAGE_HTTP_STATUS_REASONS.getConfig(), new JsonObject().toString())).thenReturn(httpStatusReasons.toString()); - } -} \ No newline at end of file diff --git a/cdi-core/src/test/java/com/linkedin/cdi/source/MultistageSource2Test.java b/cdi-core/src/test/java/com/linkedin/cdi/source/MultistageSource2Test.java deleted file mode 100644 index e1c44f7..0000000 --- a/cdi-core/src/test/java/com/linkedin/cdi/source/MultistageSource2Test.java +++ /dev/null @@ -1,99 +0,0 @@ -// Copyright 2021 LinkedIn Corporation. All rights reserved. -// Licensed under the BSD-2 Clause license. -// See LICENSE in the project root for license information. - -package com.linkedin.cdi.source; - -import com.google.gson.Gson; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import java.io.InputStreamReader; -import java.util.List; -import org.apache.commons.lang.StringUtils; -import org.apache.gobblin.configuration.SourceState; -import com.linkedin.cdi.util.VariableUtils; -import org.apache.gobblin.source.workunit.WorkUnit; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.testng.PowerMockTestCase; -import org.testng.Assert; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -import static com.linkedin.cdi.configuration.MultistageProperties.*; -import static org.mockito.Mockito.*; - - -@PrepareForTest({VariableUtils.class, MultistageSource.class}) -public class MultistageSource2Test extends PowerMockTestCase { - - private MultistageSource source; - private SourceState state; - private Gson gson; - @BeforeClass - public void setUp() { - source = new MultistageSource(); - state = mock(SourceState.class); - gson = new Gson(); - } - - @Test - public void testGetWorkunits() { - initializeHelper(state); - - List wuList = source.getWorkunits(state); - Assert.assertEquals(wuList.size(), 1); - WorkUnit workUnit = wuList.get(0); - Assert.assertEquals(workUnit.getSpecProperties().getProperty(MSTAGE_WATERMARK_GROUPS.getConfig()), "[\"watermark.system\",\"watermark.unit\"]"); - } - - @Test - public void testInitialize() { - initializeHelper(state); - - when(state.getProp(MSTAGE_ENABLE_CLEANSING.getConfig(), StringUtils.EMPTY)).thenReturn("true"); - when(state.getProp(MSTAGE_SECONDARY_INPUT.getConfig(), new JsonArray().toString())) - .thenReturn("[{\"fields\":[\"uuid\"],\"category\":\"authentication\",\"authentication\":{}}]"); - source.initialize(state); - - when(state.getProp(MSTAGE_ENABLE_CLEANSING.getConfig(), StringUtils.EMPTY)).thenReturn(""); - when(state.getProp(MSTAGE_SECONDARY_INPUT.getConfig(), new JsonArray().toString())) - .thenReturn("[{\"path\":\"${job.dir}/${extract.namespace}/getResults\",\"fields\":[\"access_token\"],\"category\":\"authentication\",\"retry\":{}}]"); - source.initialize(state); - - when(state.getProp(MSTAGE_ENABLE_CLEANSING.getConfig(), StringUtils.EMPTY)).thenReturn("false"); - source.initialize(state); - } - - private void initializeHelper(SourceState state) { - JsonObject allKeys = gson.fromJson(new InputStreamReader(this.getClass().getResourceAsStream("/json/sample-data-for-source.json")), JsonObject.class); - - when(state.getProp(MSTAGE_PAGINATION.getConfig(), new JsonObject().toString())).thenReturn(allKeys.get(MSTAGE_PAGINATION.getConfig()).getAsJsonObject().toString()); - when(state.getProp(MSTAGE_SESSION_KEY_FIELD.getConfig(), new JsonObject().toString())).thenReturn(allKeys.get(MSTAGE_SESSION_KEY_FIELD.getConfig()).getAsJsonObject().toString()); - when(state.getProp(MSTAGE_TOTAL_COUNT_FIELD.getConfig(), StringUtils.EMPTY)).thenReturn(allKeys.get(MSTAGE_TOTAL_COUNT_FIELD.getConfig()).getAsString()); - when(state.getProp(MSTAGE_PARAMETERS.getConfig(), new JsonArray().toString())).thenReturn(allKeys.get(MSTAGE_PARAMETERS.getConfig()).getAsJsonArray().toString()); - when(state.getProp(MSTAGE_ENCRYPTION_FIELDS.getConfig(), new JsonArray().toString())).thenReturn(allKeys.get(MSTAGE_ENCRYPTION_FIELDS.getConfig()).getAsJsonArray().toString()); - when(state.getProp(MSTAGE_DATA_FIELD.getConfig(), StringUtils.EMPTY)).thenReturn(allKeys.get(MSTAGE_DATA_FIELD.getConfig()).getAsString()); - when(state.getPropAsLong(MSTAGE_CALL_INTERVAL.getConfig(), 0L)).thenReturn(allKeys.get(MSTAGE_CALL_INTERVAL.getConfig()).getAsLong()); - when(state.getPropAsLong(MSTAGE_WAIT_TIMEOUT_SECONDS.getConfig(), 0L)).thenReturn(allKeys.get(MSTAGE_WAIT_TIMEOUT_SECONDS.getConfig()).getAsLong()); - when(state.getPropAsBoolean(MSTAGE_ENABLE_CLEANSING.getConfig())).thenReturn(allKeys.get(MSTAGE_ENABLE_CLEANSING.getConfig()).getAsBoolean()); - when(state.getPropAsBoolean(MSTAGE_WORK_UNIT_PARTIAL_PARTITION.getConfig())).thenReturn(allKeys.get(MSTAGE_WORK_UNIT_PARTIAL_PARTITION.getConfig()).getAsBoolean()); - when(state.getProp(MSTAGE_WATERMARK.getConfig(), new JsonArray().toString())).thenReturn(allKeys.get(MSTAGE_WATERMARK.getConfig()).getAsJsonArray().toString()); - when(state.getProp(MSTAGE_SECONDARY_INPUT.getConfig(), new JsonArray().toString())).thenReturn(allKeys.get(MSTAGE_SECONDARY_INPUT.getConfig()).getAsJsonArray().toString()); - when(state.getProp(MSTAGE_HTTP_CLIENT_FACTORY.getConfig(), StringUtils.EMPTY)).thenReturn(allKeys.get(MSTAGE_HTTP_CLIENT_FACTORY.getConfig()).getAsString()); - when(state.getProp(MSTAGE_HTTP_REQUEST_HEADERS.getConfig(), new JsonObject().toString())).thenReturn(allKeys.get(MSTAGE_HTTP_REQUEST_HEADERS.getConfig()).getAsJsonObject().toString()); - when(state.getProp(MSTAGE_SOURCE_URI.getConfig(), StringUtils.EMPTY)).thenReturn(allKeys.get(MSTAGE_SOURCE_URI.getConfig()).getAsString()); - when(state.getProp(MSTAGE_HTTP_REQUEST_METHOD.getConfig(), StringUtils.EMPTY)).thenReturn(allKeys.get(MSTAGE_HTTP_REQUEST_METHOD.getConfig()).getAsString()); - when(state.getProp(MSTAGE_EXTRACTOR_CLASS.getConfig(), StringUtils.EMPTY)).thenReturn(allKeys.get(MSTAGE_EXTRACTOR_CLASS.getConfig()).getAsString()); - when(state.getProp(MSTAGE_AUTHENTICATION.getConfig(), new JsonObject().toString())).thenReturn(allKeys.get(MSTAGE_AUTHENTICATION.getConfig()).getAsJsonObject().toString()); - when(state.getProp(MSTAGE_HTTP_STATUSES.getConfig(), new JsonObject().toString())).thenReturn(allKeys.get(MSTAGE_HTTP_STATUSES.getConfig()).getAsJsonObject().toString()); - when(state.getProp(MSTAGE_HTTP_STATUS_REASONS.getConfig(), new JsonObject().toString())).thenReturn(allKeys.get(MSTAGE_HTTP_STATUS_REASONS.getConfig()).getAsJsonObject().toString()); - - when(state.getProp(MSTAGE_SOURCE_S3_PARAMETERS.getConfig(), new JsonObject().toString())).thenReturn("{\"region\" : \"us-east-1\", \"connection_timeout\" : 10}"); - when(state.getProp(MSTAGE_SOURCE_FILES_PATTERN.getConfig(), StringUtils.EMPTY)).thenReturn(StringUtils.EMPTY); - when(state.getPropAsInt(MSTAGE_S3_LIST_MAX_KEYS.getConfig())).thenReturn(100); - when(state.getProp(SOURCE_CONN_USERNAME.getConfig(), StringUtils.EMPTY)).thenReturn(StringUtils.EMPTY); - when(state.getProp(SOURCE_CONN_PASSWORD.getConfig(), StringUtils.EMPTY)).thenReturn(StringUtils.EMPTY); - when(state.getProp(MSTAGE_EXTRACTOR_TARGET_FILE_NAME.getConfig(), StringUtils.EMPTY)).thenReturn(StringUtils.EMPTY); - when(state.getProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), StringUtils.EMPTY)).thenReturn(""); - } -} \ No newline at end of file diff --git a/cdi-core/src/test/java/com/linkedin/cdi/source/MultistageSourceTest.java b/cdi-core/src/test/java/com/linkedin/cdi/source/MultistageSourceTest.java index 30f7d03..24ceb7b 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/source/MultistageSourceTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/source/MultistageSourceTest.java @@ -8,23 +8,23 @@ import com.google.gson.Gson; import com.google.gson.JsonArray; import com.google.gson.JsonObject; +import com.linkedin.cdi.factory.reader.SchemaReader; +import com.linkedin.cdi.keys.JobKeys; +import com.linkedin.cdi.util.EndecoUtils; +import com.linkedin.cdi.util.WatermarkDefinition; +import com.linkedin.cdi.util.WorkUnitPartitionTypes; import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.TimeZone; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.gobblin.configuration.SourceState; import org.apache.gobblin.configuration.State; import org.apache.gobblin.configuration.WorkUnitState; -import com.linkedin.cdi.configuration.MultistageProperties; -import com.linkedin.cdi.factory.reader.SchemaReader; -import com.linkedin.cdi.keys.JobKeys; -import com.linkedin.cdi.util.EndecoUtils; -import com.linkedin.cdi.util.WatermarkDefinition; -import com.linkedin.cdi.util.WorkUnitPartitionTypes; import org.apache.gobblin.source.extractor.WatermarkInterval; import org.apache.gobblin.source.extractor.extract.LongWatermark; import org.apache.gobblin.source.workunit.Extract; @@ -39,11 +39,14 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static org.mockito.Mockito.*; public class MultistageSourceTest { private final static DateTimeFormatter JODA_DATE_TIME_FORMATTER = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ssZ"); + private final static DateTimeZone PST_TIMEZONE = DateTimeZone.forTimeZone(TimeZone.getTimeZone("PST")); + private final static DateTimeFormatter DTF_PST_TIMEZONE = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss").withZone(PST_TIMEZONE); private Gson gson; private MultistageSource source; @@ -55,38 +58,33 @@ public void setUp() { @Test public void testWorkUnitPartitionDef(){ - SourceState state = mock(SourceState.class); - when(state.getProp("ms.work.unit.partition", "")).thenReturn("daily"); - when(state.getProp("ms.pagination", new JsonObject().toString())).thenReturn("{}"); - when(state.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); + SourceState state = new SourceState(); + state.setProp("ms.work.unit.partition", "daily"); + state.setProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), ""); MultistageSource source = new MultistageSource(); source.getWorkunits(state); String expected = "daily"; - Assert.assertEquals(expected, MultistageProperties.MSTAGE_WORK_UNIT_PARTITION.getProp(state)); + Assert.assertEquals(expected, MSTAGE_WORK_UNIT_PARTITION.get(state)); } @Test public void testWorkUnitPacingDef(){ - SourceState state = mock(SourceState.class); - when(state.getPropAsInt("ms.work.unit.pacing.seconds", 0)).thenReturn(10); - when(state.getProp("ms.pagination", new JsonObject().toString())).thenReturn("{}"); - when(state.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); + SourceState state = new SourceState(); + state.setProp("ms.work.unit.pacing.seconds", "10"); MultistageSource source = new MultistageSource(); source.getWorkunits(state); - Assert.assertEquals(((Integer) MultistageProperties.MSTAGE_WORK_UNIT_PACING_SECONDS.getProp(state)).intValue(), 10); + Assert.assertEquals(((Integer) MSTAGE_WORK_UNIT_PACING_SECONDS.get(state)).intValue(), 10); } @Test public void testWorkUnitPacingConversion(){ - SourceState state = mock(SourceState.class); - when(state.getPropAsInt("ms.work.unit.pacing.seconds", 0)).thenReturn(10); - when(state.getProp("ms.pagination", new JsonObject().toString())).thenReturn("{\"fields\": [\"start\"]}"); - when(state.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); + SourceState state = new SourceState(); + state.setProp("ms.work.unit.pacing.seconds", "10"); MultistageSource source = new MultistageSource(); source.getWorkunits(state); - Assert.assertEquals(MultistageProperties.MSTAGE_WORK_UNIT_PACING_SECONDS.getMillis(state).longValue(), 10000L); + Assert.assertEquals(MSTAGE_WORK_UNIT_PACING_SECONDS.getMillis(state).longValue(), 10000L); } @Test @@ -105,6 +103,36 @@ public void testGetWorkUnitsTooManyPartitions() { Assert.assertEquals(wuList.size(), 3 * 30 * 24); } + @Test (expectedExceptions = RuntimeException.class) + public void testGetWorkUnitsMinimumUnits() { + SourceState state = new SourceState(); + state.setProp("ms.watermark", + "[{\"name\": \"system\",\"type\": \"datetime\", \"range\": {\"from\": \"2000-01-01\", \"to\": \"-\"}}]"); + state.setProp("extract.table.type", "SNAPSHOT_ONLY"); + state.setProp("extract.namespace", "test"); + state.setProp("extract.table.name", "table1"); + state.setProp("ms.work.unit.partition", "hourly"); + state.setProp("ms.pagination", "{}"); + state.setProp("ms.work.unit.min.units", "1"); + MultistageSource source = new MultistageSource(); + source.getWorkunits(state); + } + + @Test (expectedExceptions = RuntimeException.class) + public void testGetWorkUnitsMinimumUnits2() { + SourceState state = new SourceState(); + state.setProp("ms.watermark", + "[{\"name\": \"system\",\"type\": \"datetime\", \"range\": {\"from\": \"2000-01-01\", \"to\": \"-\"}}, " + + "{\"name\": \"unitWatermark\",\"type\": \"unit\", \"units\": \"unit1\"}]"); + state.setProp("extract.table.type", "SNAPSHOT_ONLY"); + state.setProp("extract.namespace", "test"); + state.setProp("extract.table.name", "table1"); + state.setProp("ms.pagination", "{}"); + state.setProp("ms.work.unit.min.units", "2"); + MultistageSource source = new MultistageSource(); + source.getWorkunits(state); + } + @Test public void testGetWorkUnitsDefault(){ SourceState state = new SourceState(); @@ -117,7 +145,7 @@ public void testGetWorkUnitsDefault(){ MultistageSource source = new MultistageSource(); source.getWorkunits(state); - //Assert.assertEquals(source.getMyProperty(MultistageProperties.WORK_UNIT_PARTITION), "weekly"); + //Assert.assertEquals(source.getMyProperty(WORK_UNIT_PARTITION), "weekly"); Extract extract = source.createExtractObject(true); WorkUnit workUnit = WorkUnit.create(extract, new WatermarkInterval(new LongWatermark(1483257600000L), new LongWatermark(1572660000000L))); @@ -126,53 +154,34 @@ public void testGetWorkUnitsDefault(){ workUnit.setProp("watermark.unit", "NONE"); WorkUnit workUnit1 = (WorkUnit) source.getWorkunits(state).get(0); Assert.assertEquals(workUnit1.getLowWatermark().toString(), workUnit.getLowWatermark().toString()); - Assert.assertEquals(workUnit1.getProp(MultistageProperties.DATASET_URN_KEY.toString()), "[watermark.system.1483257600000, watermark.unit.{}]"); - Assert.assertEquals(workUnit1.getProp(MultistageProperties.MSTAGE_WATERMARK_GROUPS.toString()), "[\"watermark.system\",\"watermark.unit\"]"); - } - - @Test - public void testParallismMaxSetting() { - SourceState state = mock(SourceState.class); - when(state.getPropAsInt("ms.work.unit.parallelism.max",0)).thenReturn(0); - when(state.getProp("ms.pagination", new JsonObject().toString())).thenReturn(""); - - Assert.assertFalse(MultistageProperties.MSTAGE_WORK_UNIT_PARALLELISM_MAX.validateNonblank(state)); - - when(state.getPropAsInt("ms.work.unit.parallelism.max",0)).thenReturn(10); - Assert.assertTrue(MultistageProperties.MSTAGE_WORK_UNIT_PARALLELISM_MAX.validateNonblank(state)); + Assert.assertEquals(workUnit1.getProp(DATASET_URN.toString()), "[watermark.system.1483257600000, watermark.unit.{}]"); + Assert.assertEquals(workUnit1.getProp(MSTAGE_WATERMARK_GROUPS.toString()), "[\"watermark.system\",\"watermark.unit\"]"); } @Test public void testDerivedFields() { - SourceState sourceState = mock(SourceState.class); - when(sourceState.getProp("extract.table.type", "SNAPSHOT_ONLY")).thenReturn("SNAPSHOT_ONLY"); - when(sourceState.getProp("extract.namespace", "")).thenReturn("test"); - when(sourceState.getProp("extract.table.name", "")).thenReturn("table1"); - when(sourceState.getProp("ms.derived.fields", new JsonArray().toString())).thenReturn("[{\"name\": \"activityDate\", \"formula\": {\"type\": \"epoc\", \"source\": \"fromDateTime\", \"format\": \"yyyy-MM-dd'T'HH:mm:ss'Z'\"}}]"); - when(sourceState.getProp("ms.output.schema", new JsonArray().toString())).thenReturn(""); - when(sourceState.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); + SourceState sourceState = new SourceState(); + sourceState.setProp("ms.derived.fields", "[{\"name\": \"activityDate\", \"formula\": {\"type\": \"epoc\", \"source\": \"fromDateTime\", \"format\": \"yyyy-MM-dd'T'HH:mm:ss'Z'\"}}]"); MultistageSource source = new MultistageSource(); source.getWorkunits(sourceState); - Assert.assertEquals(source.getJobKeys().getDerivedFields().keySet().toString(), "[activityDate]"); } @Test public void testOutputSchema(){ - SourceState state = mock(SourceState.class); - when(state.getProp("ms.output.schema", new JsonArray().toString())).thenReturn(""); - when(state.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); + SourceState state = new SourceState(); + state.setProp("ms.output.schema", ""); MultistageSource source = new MultistageSource(); source.getWorkunits(state); Assert.assertEquals(0, source.getJobKeys().getOutputSchema().size()); // wrong format should be ignored - when(state.getProp("ms.output.schema", new JsonArray().toString())).thenReturn("{\"name\": \"responseTime\"}"); + state.setProp("ms.output.schema", "{\"name\": \"responseTime\"}"); source.getWorkunits(state); Assert.assertEquals(0, source.getJobKeys().getOutputSchema().size()); // wrong format should be ignored - when(state.getProp("ms.output.schema", new JsonArray().toString())).thenReturn("[{\"name\": \"responseTime\"}]"); + state.setProp("ms.output.schema", "[{\"name\": \"responseTime\"}]"); source.getWorkunits(state); Assert.assertEquals(1, source.getJobKeys().getOutputSchema().size()); Assert.assertEquals(1, source.getJobKeys().getOutputSchema().size()); @@ -181,7 +190,7 @@ public void testOutputSchema(){ @Test public void testSourceParameters(){ SourceState sourceState = mock(SourceState.class); - when(sourceState.getProp(MultistageProperties.MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); + when(sourceState.getProp(MSTAGE_OUTPUT_SCHEMA.getConfig(), "")).thenReturn(""); MultistageSource source = new MultistageSource(); source.getWorkunits(sourceState); Assert.assertNotNull(source.getJobKeys().getSourceParameters()); @@ -230,7 +239,7 @@ public void testIsSecondaryAuthenticationEnabledWithInvalidSecondaryInput() { } @Test - public void testReadSecondaryAuthentication() { + public void testReadSecondaryAuthentication() throws InterruptedException { JsonArray secondaryInput = gson.fromJson("[{\"fields\": [\"access_token\"], \"category\": \"authentication\"}]", JsonArray.class); JobKeys jobKeys = Mockito.mock(JobKeys.class); State state = Mockito.mock(State.class); @@ -243,7 +252,7 @@ public void testReadSecondaryAuthentication() { public void testGetUpdatedWorkUnitActivation() { WorkUnit workUnit = Mockito.mock(WorkUnit.class); JsonObject authentication = gson.fromJson("{\"method\": \"basic\", \"encryption\": \"base64\", \"header\": \"Authorization\"}", JsonObject.class); - when(workUnit.getProp(MultistageProperties.MSTAGE_ACTIVATION_PROPERTY.toString(), StringUtils.EMPTY)).thenReturn(StringUtils.EMPTY); + when(workUnit.getProp(MSTAGE_ACTIVATION_PROPERTY.toString(), StringUtils.EMPTY)).thenReturn(StringUtils.EMPTY); Assert.assertEquals(source.getUpdatedWorkUnitActivation(workUnit, authentication), authentication.toString()); } @@ -290,18 +299,18 @@ private void testGenerateWorkUnitsHelper(WatermarkDefinition.WatermarkTypes wate @Test public void testCheckFullExtractState() throws Exception { - State state = Mockito.mock(State.class); + State state = new SourceState(); Map map = Mockito.mock(Map.class); Method method = MultistageSource.class.getDeclaredMethod("checkFullExtractState", State.class, Map.class); method.setAccessible(true); - when(state.getProp("extract.table.type", StringUtils.EMPTY)).thenReturn("APPEND_ONLY"); + state.setProp("extract.table.type", "APPEND_ONLY"); when(map.isEmpty()).thenReturn(true); Assert.assertTrue((Boolean) method.invoke(source, state, map)); when(map.isEmpty()).thenReturn(false); Assert.assertFalse((Boolean) method.invoke(source, state, map)); - when(state.getProp("ms.enable.dynamic.full.load", StringUtils.EMPTY)).thenReturn("true"); + state.setProp("ms.enable.dynamic.full.load", "true"); Assert.assertFalse((Boolean) method.invoke(source, state, map)); } @@ -672,7 +681,7 @@ public void testPassingSchema2WorkUnits() { "[{\"columnName\":\"column1\",\"isNullable\":true,\"dataType\":{\"type\":\"timestamp\"}}]", JsonArray.class); - state.setProp(MultistageProperties.MSTAGE_SOURCE_SCHEMA_URN.toString(), urn); + state.setProp(MSTAGE_SOURCE_SCHEMA_URN.toString(), urn); MultistageSource source = new MultistageSource<>(); SchemaReader mockFactory = mock(SchemaReader.class); @@ -685,4 +694,38 @@ public void testPassingSchema2WorkUnits() { Assert.assertNotNull(source.generateWorkUnits(new ArrayList<>(), new HashMap<>())); } + @Test + public void testAvoidWatermarkGoingBeyondLeftBoundary() { + // state and source: define 2 day grace period + SourceState state = new SourceState(); + state.setProp("ms.grace.period.days", "2"); + MultistageSource source = new MultistageSource<>(); + source.setSourceState(state); + source.jobKeys.initialize(state); + + // watermark definition: define from and to date watermark + String jsonDef = "[{\"name\": \"system\",\"type\": \"datetime\", \"range\": {\"from\": \"2021-06-18\", \"to\": \"2021-06-19\"}}]"; + Gson gson = new Gson(); + JsonArray defArray = gson.fromJson(jsonDef, JsonArray.class); + WatermarkDefinition watermarkDefinition = new WatermarkDefinition(defArray.get(0).getAsJsonObject(), + false, WorkUnitPartitionTypes.DAILY); + List definitions = ImmutableList.of(watermarkDefinition); + + // previous highwatermarks: simulate state-store entry + Map previousHighWatermarks = Mockito.mock(HashMap.class); + when(previousHighWatermarks.containsKey(any())).thenReturn(true); + when(previousHighWatermarks.get(any())).thenReturn( + DTF_PST_TIMEZONE.parseDateTime("2021-06-19T00:00:00").getMillis()); + + // expected workunits + WorkUnit expectedWorkUnit = WorkUnit.create(null, + new WatermarkInterval( + new LongWatermark(DTF_PST_TIMEZONE.parseDateTime("2021-06-18T00:00:00").getMillis()), + new LongWatermark(DTF_PST_TIMEZONE.parseDateTime("2021-06-19T00:00:00").getMillis()))); + + List actualWorkUnits = source.generateWorkUnits(definitions, previousHighWatermarks); + Assert.assertEquals(actualWorkUnits.size(), 1); + Assert.assertEquals(actualWorkUnits.get(0).getLowWatermark(), expectedWorkUnit.getLowWatermark()); + Assert.assertEquals(actualWorkUnits.get(0).getExpectedHighWatermark(), expectedWorkUnit.getExpectedHighWatermark()); + } } diff --git a/cdi-core/src/test/java/com/linkedin/cdi/util/AvroSchemaUtilsTest.java b/cdi-core/src/test/java/com/linkedin/cdi/util/AvroSchemaUtilsTest.java index 13261ea..9204523 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/util/AvroSchemaUtilsTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/util/AvroSchemaUtilsTest.java @@ -7,7 +7,6 @@ import com.google.gson.Gson; import com.google.gson.JsonArray; import java.util.List; -import lombok.SneakyThrows; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.gobblin.configuration.WorkUnitState; @@ -27,9 +26,8 @@ public class AvroSchemaUtilsTest { JsonArray schemaArray = new Gson().fromJson(schemaString, JsonArray.class); Schema schema; - @SneakyThrows @BeforeMethod - public void beforeMethod() { + public void beforeMethod() throws UnsupportedDateTypeException { state = mock(WorkUnitState.class); Extract extract = new Extract(Extract.TableType.SNAPSHOT_ONLY, "com.linkedin.test", "test"); when(state.getExtract()).thenReturn(extract); @@ -65,4 +63,11 @@ public void testCreateEOF() { Assert.assertEquals(row.getSchema().getFields().size(), 1); Assert.assertEquals(row.get("EOF"), "EOF"); } + + @Test + public void testDeepCopy() { + GenericRecord row = AvroSchemaUtils.createEOF(state); + GenericRecord copiedRow = AvroSchemaUtils.deepCopy(row.getSchema(), row); + Assert.assertEquals(row, copiedRow); + } } \ No newline at end of file diff --git a/cdi-core/src/test/java/com/linkedin/cdi/util/EncryptionUtilsTest.java b/cdi-core/src/test/java/com/linkedin/cdi/util/EncryptionUtilsTest.java index bfcb81a..6949477 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/util/EncryptionUtilsTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/util/EncryptionUtilsTest.java @@ -7,7 +7,6 @@ import com.google.gson.JsonObject; import gobblin.configuration.SourceState; import org.apache.gobblin.codec.StreamCodec; -import com.linkedin.cdi.configuration.MultistageProperties; import org.apache.gobblin.password.PasswordManager; import org.mockito.Mock; import org.powermock.api.mockito.PowerMockito; @@ -17,6 +16,7 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import static com.linkedin.cdi.configuration.PropertyCollection.*; import static org.mockito.Mockito.*; @@ -32,7 +32,7 @@ public class EncryptionUtilsTest extends PowerMockTestCase { public void setUp() { String masterKeyLoc = this.getClass().getResource("/key/master.key").toString(); state = new SourceState(); - state.setProp(MultistageProperties.ENCRYPT_KEY_LOC.toString(), masterKeyLoc); + state.setProp(ENCRYPT_KEY_LOC.toString(), masterKeyLoc); PowerMockito.mockStatic(PasswordManager.class); PowerMockito.when(PasswordManager.getInstance(state)).thenReturn(passwordManager); } diff --git a/cdi-core/src/test/java/com/linkedin/cdi/util/HttpRequestMethodTest.java b/cdi-core/src/test/java/com/linkedin/cdi/util/HttpRequestMethodTest.java index 8f15490..85e3e72 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/util/HttpRequestMethodTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/util/HttpRequestMethodTest.java @@ -8,13 +8,13 @@ import com.google.common.collect.ImmutableMap; import com.google.gson.Gson; import com.google.gson.JsonObject; +import com.linkedin.cdi.factory.http.HttpRequestMethod; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URLEncoder; import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.Map; -import okhttp3.HttpUrl; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.http.client.methods.HttpDelete; @@ -38,7 +38,7 @@ */ @Test -@PrepareForTest({HttpUrl.class, VariableUtils.class}) +@PrepareForTest({VariableUtils.class}) public class HttpRequestMethodTest extends PowerMockTestCase { final static String FROM_DATETIME = "2017-01-02T00:00:00-0800"; @@ -156,17 +156,6 @@ public void testGetHttpRequest() throws IOException { Assert.assertEquals(HttpRequestMethod.POST.getHttpRequest(String.format(BASE_URI, VERSION_2), parameters, headers).toString(), expected); } - /** - * Test appendParameters with null uri - * Expected: null - */ - @Test - public void testAppendParametersWithNullUri() { - PowerMockito.mockStatic(HttpUrl.class); - when(HttpUrl.parse(null)).thenReturn(null); - Assert.assertEquals(HttpRequestMethod.PUT.appendParameters(null, null), null); - } - private void addContentType() { headers.clear(); headers.put(CONTENT_TYPE, CONTENT_TYPE_VALUE); diff --git a/cdi-core/src/test/java/com/linkedin/cdi/util/JsonParameterTest.java b/cdi-core/src/test/java/com/linkedin/cdi/util/JsonParameterTest.java index 2755b99..8585a6e 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/util/JsonParameterTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/util/JsonParameterTest.java @@ -11,11 +11,12 @@ import java.io.InputStreamReader; import java.lang.reflect.Method; import org.apache.gobblin.configuration.State; -import com.linkedin.cdi.configuration.MultistageProperties; import org.testng.Assert; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import static com.linkedin.cdi.configuration.PropertyCollection.*; + /** * Unit test for {@link JsonParameter} @@ -48,7 +49,7 @@ public void testParameterEncryption() { String encrypted = "ENC(M6nV+j0lhqZ36RgvuF5TQMyNvBtXmkPl)"; String masterKeyLoc = this.getClass().getResource("/key/master.key").toString(); SourceState state = new SourceState(); - state.setProp(MultistageProperties.ENCRYPT_KEY_LOC.toString(), masterKeyLoc); + state.setProp(ENCRYPT_KEY_LOC.toString(), masterKeyLoc); JsonArray jsonArray = gson.fromJson(new InputStreamReader(this.getClass().getResourceAsStream("/json/parameter-encryption.json")), JsonArray.class); Assert.assertEquals(expected, JsonParameter.getParametersAsJsonString(jsonArray.toString(), new JsonObject(), state)); } @@ -162,23 +163,23 @@ public void testListParameterByExtractMode() { SourceState state = new SourceState(); String expected = "{\"column\":\"createdDate\"}"; - state.setProp(MultistageProperties.EXTRACT_IS_FULL.toString(), true); + state.setProp(EXTRACT_IS_FULL.toString(), true); Assert.assertEquals(expected, JsonParameter.getParametersAsJsonString(msParameters.toString(), new JsonObject(), state)); expected = "{\"column\":\"updatedDate\"}"; - state.setProp(MultistageProperties.EXTRACT_IS_FULL.toString(), false); + state.setProp(EXTRACT_IS_FULL.toString(), false); Assert.assertEquals(expected, JsonParameter.getParametersAsJsonString(msParameters.toString(), new JsonObject(), state)); expected = "{\"column\":\"createdDate\"}"; - state.setProp(MultistageProperties.EXTRACT_IS_FULL.toString(), false); + state.setProp(EXTRACT_IS_FULL.toString(), false); Assert.assertEquals(expected, JsonParameter.getParametersAsJsonString(msParameters2.toString(), new JsonObject(), state)); expected = "{\"column\":\"createdDate\"}"; - state.setProp(MultistageProperties.EXTRACT_IS_FULL.toString(), false); + state.setProp(EXTRACT_IS_FULL.toString(), false); Assert.assertEquals(expected, JsonParameter.getParametersAsJsonString(msParameters3.toString(), new JsonObject(), state)); expected = "{\"column\":\"\"}"; - state.setProp(MultistageProperties.EXTRACT_IS_FULL.toString(), false); + state.setProp(EXTRACT_IS_FULL.toString(), false); Assert.assertEquals(expected, JsonParameter.getParametersAsJsonString(msParameters4.toString(), new JsonObject(), state)); } } \ No newline at end of file diff --git a/cdi-core/src/test/java/com/linkedin/cdi/util/JsonUtilsTest.java b/cdi-core/src/test/java/com/linkedin/cdi/util/JsonUtilsTest.java index cb688ba..921a1ec 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/util/JsonUtilsTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/util/JsonUtilsTest.java @@ -4,6 +4,8 @@ package com.linkedin.cdi.util; +import com.google.gson.Gson; +import com.google.gson.JsonArray; import com.google.gson.JsonObject; import org.testng.Assert; import org.testng.annotations.Test; @@ -59,4 +61,22 @@ public void testReplace() { a.addProperty("name2", "value1"); Assert.assertEquals(JsonUtils.replace(a, b).toString(), "{\"name1\":\"newValue1\",\"name2\":\"value1\"}"); } + + @Test + public void testHas() { + Gson gson = new Gson(); + JsonObject a = gson.fromJson("{\"name1\": \"value1\", \"result\": {\"name2\": \"value2\"}}", JsonObject.class); + Assert.assertTrue(JsonUtils.has(a, "name1")); + Assert.assertTrue(JsonUtils.has(a, "result.name2")); + Assert.assertFalse(JsonUtils.has(a, "name2")); + Assert.assertFalse(JsonUtils.has(a, "name3")); + } + + @Test + public void testFilter() { + Gson gson = new Gson(); + JsonArray sample = gson.fromJson("[{\"name1\": \"value1\", \"result\": {\"name2\": \"value2\"}}]", JsonArray.class); + Assert.assertTrue(JsonUtils.filter("name1", "value1", sample).size() == 1); + Assert.assertTrue(JsonUtils.filter("name1", "noneexist", sample).size() == 0); + } } diff --git a/cdi-core/src/test/java/com/linkedin/cdi/util/SchemaUtilsTest.java b/cdi-core/src/test/java/com/linkedin/cdi/util/SchemaUtilsTest.java index 8b23a8c..a252a93 100644 --- a/cdi-core/src/test/java/com/linkedin/cdi/util/SchemaUtilsTest.java +++ b/cdi-core/src/test/java/com/linkedin/cdi/util/SchemaUtilsTest.java @@ -4,11 +4,15 @@ package com.linkedin.cdi.util; +import com.google.common.collect.Lists; +import com.google.gson.JsonArray; import java.util.Arrays; import java.util.List; import org.testng.Assert; import org.testng.annotations.Test; +import static com.linkedin.cdi.util.JsonUtils.*; + public class SchemaUtilsTest { @@ -34,4 +38,51 @@ public void testIsValidOutputSchema() { sourceColumns = Arrays.asList("a", "B", "C"); Assert.assertFalse(SchemaUtils.isValidOutputSchema(schemaColumns, sourceColumns)); } + + @Test + public void testIsNullable() { + String schema = "[{\"columnName\":\"token\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}]"; + JsonArray schemaArray = GSON.fromJson(schema, JsonArray.class); + Assert.assertFalse(SchemaUtils.isNullable(Lists.newArrayList("token"), schemaArray)); + Assert.assertTrue(SchemaUtils.isNullable(Lists.newArrayList("token", "url"), schemaArray)); + + schema = "[{\"columnName\":\"token\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"record\",\"name\":\"token\",\"values\":[{\"columnName\":\"url\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}]}}]"; + schemaArray = GSON.fromJson(schema, JsonArray.class); + Assert.assertFalse(SchemaUtils.isNullable(Lists.newArrayList("token"), schemaArray)); + Assert.assertFalse(SchemaUtils.isNullable(Lists.newArrayList("token", "url"), schemaArray)); + Assert.assertFalse(SchemaUtils.isNullable("token.url", schemaArray)); + + schema = "[{\"columnName\":\"token\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"record\",\"name\":\"token\",\"values\":[{\"columnName\":\"url\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]}}]"; + schemaArray = GSON.fromJson(schema, JsonArray.class); + Assert.assertFalse(SchemaUtils.isNullable(Lists.newArrayList("token"), schemaArray)); + Assert.assertTrue(SchemaUtils.isNullable(Lists.newArrayList("token", "url"), schemaArray)); + + schema = "[{\"columnName\":\"token\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"record\",\"name\":\"token\",\"values\":[{\"columnName\":\"url\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}]}}]"; + schemaArray = GSON.fromJson(schema, JsonArray.class); + Assert.assertTrue(SchemaUtils.isNullable(Lists.newArrayList("token"), schemaArray)); + Assert.assertTrue(SchemaUtils.isNullable(Lists.newArrayList("token", "url"), schemaArray)); + + schema = "[{\"columnName\":\"token\", \"dataType\":{\"type\":\"record\",\"name\":\"token\",\"values\":[{\"columnName\":\"url\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}]}}]"; + schemaArray = GSON.fromJson(schema, JsonArray.class); + Assert.assertTrue(SchemaUtils.isNullable(Lists.newArrayList("token"), schemaArray)); + Assert.assertTrue(SchemaUtils.isNullable(Lists.newArrayList("token", "url"), schemaArray)); + + schema = "[{\"columnName\":\"token\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"record\",\"name\":\"token\",\"values\":[{\"columnName\":\"url\",\"dataType\":{\"type\":\"string\"}}]}}]"; + schemaArray = GSON.fromJson(schema, JsonArray.class); + Assert.assertFalse(SchemaUtils.isNullable(Lists.newArrayList("token"), schemaArray)); + Assert.assertTrue(SchemaUtils.isNullable(Lists.newArrayList("token", "url"), schemaArray)); + + schema = "[{\"columnName\":\"token\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"array\",\"name\":\"token\",\"items\":{\"name\":\"contextItem\",\"dataType\":{\"name\":\"contextItem\",\"type\":\"record\",\"values\":[{\"columnName\":\"system\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"string\"}}]}}}}]"; + schemaArray = GSON.fromJson(schema, JsonArray.class); + Assert.assertFalse(SchemaUtils.isNullable(Lists.newArrayList("token"), schemaArray)); + + // token.url not exist in schema + Assert.assertTrue(SchemaUtils.isNullable(Lists.newArrayList("token", "url"), schemaArray)); + Assert.assertFalse(SchemaUtils.isNullable(Lists.newArrayList("token", "system"), schemaArray)); + + schema = "[{\"columnName\":\"token\",\"isNullable\":\"false\",\"dataType\":{\"type\":\"array\",\"name\":\"token\",\"items\":{\"name\":\"contextItem\",\"dataType\":{\"name\":\"contextItem\",\"type\":\"record\",\"values\":[{\"columnName\":\"system\",\"isNullable\":\"true\",\"dataType\":{\"type\":\"string\"}}]}}}}]"; + schemaArray = GSON.fromJson(schema, JsonArray.class); + Assert.assertTrue(SchemaUtils.isNullable(Lists.newArrayList("token", "system"), schemaArray)); + + } } \ No newline at end of file diff --git a/cdi-core/src/test/resources/avro/ArrayFieldTest.avro b/cdi-core/src/test/resources/avro/ArrayFieldTest.avro new file mode 100644 index 0000000..8474945 Binary files /dev/null and b/cdi-core/src/test/resources/avro/ArrayFieldTest.avro differ diff --git a/cdi-core/src/test/resources/avro/NestedDataFieldTest.avro b/cdi-core/src/test/resources/avro/NestedDataFieldTest.avro new file mode 100644 index 0000000..2588071 Binary files /dev/null and b/cdi-core/src/test/resources/avro/NestedDataFieldTest.avro differ diff --git a/cdi-core/src/test/resources/avro/SingleRecordArrayFieldTest.avro b/cdi-core/src/test/resources/avro/SingleRecordArrayFieldTest.avro new file mode 100644 index 0000000..9d5a68c Binary files /dev/null and b/cdi-core/src/test/resources/avro/SingleRecordArrayFieldTest.avro differ diff --git a/cdi-core/src/test/resources/csv/ids_multiple_header_lines.csv b/cdi-core/src/test/resources/csv/ids_multiple_header_lines.csv new file mode 100644 index 0000000..6a52fd0 --- /dev/null +++ b/cdi-core/src/test/resources/csv/ids_multiple_header_lines.csv @@ -0,0 +1,13 @@ +id_0,date +id_0,date +id_0,date +497766636,2020-04-10 -07:00 +583903591,2020-04-11 -07:00 +666997959,2020-04-12 -07:00 +131360915,2020-04-13 -07:00 +704160186,2020-04-14 -07:00 +121446123,2020-04-15 -07:00 +106921314,2020-04-16 -07:00 +147490822,2020-04-17 -07:00 +553282504,2020-04-18 -07:00 +216647067,2020-04-19 -07:00 \ No newline at end of file diff --git a/cdi-core/src/test/resources/json/sample-data-for-source.json b/cdi-core/src/test/resources/json/sample-data-for-source.json index aac61cc..50e68f5 100644 --- a/cdi-core/src/test/resources/json/sample-data-for-source.json +++ b/cdi-core/src/test/resources/json/sample-data-for-source.json @@ -80,7 +80,7 @@ } } ], - "ms.http.client.factory": "com.linkedin.cdi.factory.ApacheHttpClientFactory", + "ms.connection.client.factory": "com.linkedin.cdi.factory.DefaultConnectionClientFactory", "ms.http.request.headers": { "Content-Type": "application/json", "key": "adwpsdfsftcc9cj749fnb8xxsdfsn" diff --git a/cdi-core/src/test/resources/pull/s3-csv.pull b/cdi-core/src/test/resources/pull/s3-csv.pull index 75865a6..1188481 100644 --- a/cdi-core/src/test/resources/pull/s3-csv.pull +++ b/cdi-core/src/test/resources/pull/s3-csv.pull @@ -2,7 +2,7 @@ source.class=com.linkedin.cdi.source.S3SourceV2 ms.extract.preprocessors=com.linkedin.cdi.preprocessor.GunzipProcessor ms.extractor.class=com.linkedin.cdi.extractor.CsvExtractor converter.classes=org.apache.gobblin.converter.csv.CsvToJsonConverterV2,org.apache.gobblin.converter.avro.JsonIntermediateToAvroConverter -ms.http.client.factory=com.linkedin.cdi.factory.ApacheHttpClientFactory +ms.http.client.factory=com.linkedin.cdi.factory.DefaultConnectionClientFactory extract.namespace=com.linkedin.test extract.table.name=test diff --git a/cdi-core/src/test/resources/pull/s3-filedump.pull b/cdi-core/src/test/resources/pull/s3-filedump.pull index f8d9cce..903e558 100644 --- a/cdi-core/src/test/resources/pull/s3-filedump.pull +++ b/cdi-core/src/test/resources/pull/s3-filedump.pull @@ -2,7 +2,7 @@ source.class=com.linkedin.cdi.source.S3SourceV2 #ms.extract.preprocessors=com.linkedin.cdi.preprocessor.GunzipProcessor ms.extractor.class=com.linkedin.cdi.extractor.FileDumpExtractor converter.classes=org.apache.gobblin.converter.csv.CsvToJsonConverterV2,org.apache.gobblin.converter.avro.JsonIntermediateToAvroConverter -ms.http.client.factory=com.linkedin.cdi.factory.ApacheHttpClientFactory +ms.http.client.factory=com.linkedin.cdi.factory.DefaultConnectionClientFactory extract.namespace=com.linkedin.test extract.table.name=test diff --git a/docs/components/AvroExtractor.md b/docs/components/AvroExtractor.md new file mode 100644 index 0000000..c1a3465 --- /dev/null +++ b/docs/components/AvroExtractor.md @@ -0,0 +1,3 @@ + + +[Back to Summary](summary.md) \ No newline at end of file diff --git a/docs/components/CsvExtractor.md b/docs/components/CsvExtractor.md new file mode 100644 index 0000000..b6cd2e9 --- /dev/null +++ b/docs/components/CsvExtractor.md @@ -0,0 +1,2 @@ + +[Back to Summary](summary.md) \ No newline at end of file diff --git a/docs/components/FileDumpExtractor.md b/docs/components/FileDumpExtractor.md new file mode 100644 index 0000000..5db9018 --- /dev/null +++ b/docs/components/FileDumpExtractor.md @@ -0,0 +1,4 @@ +ms.extractor.target.file.permission + + +[Back to Summary](summary.md) diff --git a/docs/components/GunzipProcessor.md b/docs/components/GunzipProcessor.md new file mode 100644 index 0000000..b6cd2e9 --- /dev/null +++ b/docs/components/GunzipProcessor.md @@ -0,0 +1,2 @@ + +[Back to Summary](summary.md) \ No newline at end of file diff --git a/docs/components/InFlowValidationConverter.md b/docs/components/InFlowValidationConverter.md new file mode 100644 index 0000000..b6cd2e9 --- /dev/null +++ b/docs/components/InFlowValidationConverter.md @@ -0,0 +1,2 @@ + +[Back to Summary](summary.md) \ No newline at end of file diff --git a/docs/components/JsonExtractor.md b/docs/components/JsonExtractor.md new file mode 100644 index 0000000..b6cd2e9 --- /dev/null +++ b/docs/components/JsonExtractor.md @@ -0,0 +1,2 @@ + +[Back to Summary](summary.md) \ No newline at end of file diff --git a/docs/components/S3SourceV2.md b/docs/components/S3SourceV2.md new file mode 100644 index 0000000..b6cd2e9 --- /dev/null +++ b/docs/components/S3SourceV2.md @@ -0,0 +1,2 @@ + +[Back to Summary](summary.md) \ No newline at end of file diff --git a/docs/components/normalizer-converter.md b/docs/components/normalizer-converter.md new file mode 100644 index 0000000..b6cd2e9 --- /dev/null +++ b/docs/components/normalizer-converter.md @@ -0,0 +1,2 @@ + +[Back to Summary](summary.md) \ No newline at end of file diff --git a/docs/components/summary.md b/docs/components/summary.md new file mode 100644 index 0000000..2004e72 --- /dev/null +++ b/docs/components/summary.md @@ -0,0 +1,9 @@ +# Key Component +## [AvroExtractor.md](AvroExtractor.md) +## [CsvExtractor.md](CsvExtractor.md) +## [FileDumpExtractor.md](FileDumpExtractor.md) +## [GunzipProcessor.md](GunzipProcessor.md) +## [InFlowValidationConverter.md](InFlowValidationConverter.md) +## [JsonExtractor.md](JsonExtractor.md) +## [Normalizer-converter.md](normalizer-converter.md) +## [S3SourceV2.md](S3SourceV2.md) diff --git a/docs/concepts/authentication-method.md b/docs/concepts/authentication-method.md new file mode 100644 index 0000000..5ea2b9a --- /dev/null +++ b/docs/concepts/authentication-method.md @@ -0,0 +1 @@ +basic|bearer|oauth|custom diff --git a/docs/concepts/encryption-method.md b/docs/concepts/encryption-method.md new file mode 100644 index 0000000..87ee1cc --- /dev/null +++ b/docs/concepts/encryption-method.md @@ -0,0 +1,2 @@ +base64 +none diff --git a/docs/concepts/iso-date-interval.md b/docs/concepts/iso-date-interval.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/concepts/job-type.md b/docs/concepts/job-type.md new file mode 100644 index 0000000..5cbfc36 --- /dev/null +++ b/docs/concepts/job-type.md @@ -0,0 +1,78 @@ +# Job Patterns + +DIL has a multistage architecture, +and a data integration job may be divided into multiple stages, +and each stage is configured as an Azkaban job, and Azkaban +jobs are orchestrated to make one or more workflows. + +Typically, we have the following logical types of Azkaban jobs in +data integration workflows. These job types are not differentiated by +any particular parameter; **they are configured the same way, and +they look alike**; only they serve different **purposes**. +They are differentiated here because each of +them follow certain **patterns**. Understanding these logical types +of jobs can help configure them quicker. + +## Authentication Job + +This is for tasks like getting a new access token +or temporary password. The output of authentication job is normally +stored in a restricted storage area, like an HDFS folder accessible only +to the flow proxy user. A dedicated authentication job is needed only +when credentials need to be refreshed in each flow execution. This +is typically for OAuth2 authentication with limited TTL on access tokens. +For example, Google API and Salesforce API both use OAuth2 +with 1 hour TTL on access token. + +## Initiation Job + +This is typically for asynchronous data integration. +The initiation request sends a set of parameters to the third party system +to request it start an action, like data extraction. The initiation job +normally returns an id/key that can be used in subsequent jobs. For +example, in Eloqua ingestion, an initiation job sends a request to Eloqua +to start an export process, and the return is an export id. + +## Status Checking Job + +A status checking job ensures the data is +ready for consumption on the third party system. +This can be used in asynchronous data ingestion and file downloads. +In the asynchronous ingestion scenario, this job will keep checking status with the 3P system +every once a while (e.g. every 15 minutes) until the status turns to +ready or timeout. In the file downloading scenario, the status checking +job can keep checking the availability of source data until they are +present or timeout. + +## Data Extraction Job + +A data extraction job dumps data from the third +party system. The extraction job differs from other jobs in that it normally +has one or more converters, and its target is normally the data lake, while +`initiation` and `authentication` jobs might just write data to a staging +area. + +## Data Preparation Job + +A data preparation job transforms data in ways +so that it can be processed by subsequent jobs; this is specifically design for +the egression. DIL as a data integration tool doesn't provide strong +data transformation features for the ingestion scenarios. Ingested data can go +through transformation using other tools more conveniently. The data preparation +job only provide very limited transformation that is absolutely needed for +the egression. Tasks like group data by batches and wrapping data with some top +tier indications are typical when sending data out. + +## Data Egression Job + +A data egression job sends data to third party systems +and retrieves the response from the third party system. An `egression` job would typically +take a payload from the secondary input. + +## Validation Job + +A validation job can compare data from the primary input and +from the secondary input. The comparison normally happens in a converter. A `validation` +job fails when validation fails. + +[Back to Summary](summary.md#job-pattern) \ No newline at end of file diff --git a/docs/concepts/json-path.md b/docs/concepts/json-path.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/concepts/pagination.md b/docs/concepts/pagination.md new file mode 100644 index 0000000..462e903 --- /dev/null +++ b/docs/concepts/pagination.md @@ -0,0 +1,153 @@ +# Pagination + +Pagination is typically used to fetch a large dataset from cloud over HTTP, where +one fetch can only optimally get a limited chunk of data. In such case, +the data is fetched through a series of pages. + +## Related + +- [ms.pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.pagination.md), +- [ms.session.key.field](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.session.key.field.md) +- [ms.total.count.field](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.total.count.field.md) +- [asynchronous pattern](https://github.com/linkedin/data-integration-library/blob/master/docs/patterns/asynchronous-ingestion-pattern.md) + +## Pagination Methods + +There are different ways to control pagination. DIL supports the following +mechanisms. + +### Method 1: Session Key + +Pagination can be established through a session key or cursor. The data provider will +maintain the session key on the server side. The session key is exchanged between the +server and the client in each request and each response. + +- APIs returning Json payload can embed the session key +in the payload as a Json element or in the response header. + +- APIs returning Csv payload can embed the session key +in response header. + +Session keys are designed by the data provider, and it comes in different forms. + +#### Request Id + Indicator + +The session key can be as simple as a "hasMore" indicator, but that usually require +a client request id accompany each request. For example: + +The following will define a session key getting +value from "hasMore" field of the response, and the pagination +will stop once the condition is met. + +- `ms.session.key.field={"name": "hasMore", "condition": {"regexp": "false|False"}}` + +#### Cursor + +The session key can be dynamic and change in each response. +The following will define a session key getting value from the "records.cursor" + +- `ms.session.key.field={"name": "records.cursor"}` + +The cursor is provided to the client in each response, and the client (DIL) +will provide the same cursor in the subsequent request. + +### Method 2: Record Position + +Page Start is the starting row index, normally zero-based. APIs may +call Page Start in different ways, commonly "offset", "page_start", or "start_record". + +Page Size is the page size! It is normally a natural number. APIs +may call Page Size in different ways, commonly "limit", or "page_size". +Page Size is normally fixed, and it won't change across pages, except +that the last page may have a smaller size. A page size smaller than +the expected size is a signal of pagination ending. + +Page Number is normally a one-based page index. Again, APIs may call it +in different ways. Page number generally increment by 1 in each page. + +#### Offset-limit Pair + +The following pagination works through a offset-limit pair. + +`ms.pagination={"fields": ["offset", "limit"], "initialvalues": [0, 5000]}` + +This will define a `pagestart` variable that is initialized as 0, and +gets value from the "offset" field of the response. So every response +will come back with different offset for next page, on and on, +until an empty page is returned. + +This will also define a `pagesize` variable that is initialized +as 5000, and gets value from the "limit" field of the response. +However, in most cases, the pagesize stays the same in almost all +use cases. + +When source of "offset" is not provided, DIL will calculate the +offset for next page assuming each page will be a full page. In the +following example, only initial values are configured. + +- `ms.pagination={"initialvalues": [0, 5000]}` + +This will define a continuous 5000-record page stream. The limit +is 5000, and the offset will be accumulated, 0, 5000, 10000.... +This will continue until other conditions are met to stop. + +#### Page Number + +This can be an explicitly defined page number, if ms.pagination is defined. + +This can also be an implicitly defined page number, if ms.pagination +is not defined. + +Page number can be sequentially increased, or can be retrieved +from a field in the response + +### Method 3: Total Record Count + +This works when the response contains only a field indicating +the total number of records. This field should be defined in +`ms.total.count.field`. + +For example: `ms.total.count.field=totalResults` + +The total record count is not a variable in parameters. +It is simply a value to control when pagination should stop. + +## Pagination Stop Conditions + +The pagination stops when one of following condition is met: + +- An empty response +- A blank session key +- The session key value met the stop condition +- The number of rows processed reached total row count +- Session timeout + +## Pagination Availability by Protocol + +- HTTP source supports all above methods +- JDBC source supports paging by record position. +- S3 source doesn't support pagination +- SFTP source doesn't support pagination +- HDFS source doesn't support pagination + +## References + +Session Key Pagination +- Gong.io API +- Zendesk API +- Eloqua API +- Salesforce bulk API + +Record Positioning +- Qualaroo API +- RightNow JDBC +- Eloqua API +- Zoom API + +Total Record Count +- Gong.io API +- Zoom API + +[Back to Summary](summary.md#pagination) + + \ No newline at end of file diff --git a/docs/concepts/schema.md b/docs/concepts/schema.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/concepts/secondary-input.md b/docs/concepts/secondary-input.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/concepts/secret-encryption.md b/docs/concepts/secret-encryption.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/concepts/session-control.md b/docs/concepts/session-control.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/concepts/single-flow.md b/docs/concepts/single-flow.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/concepts/summary.md b/docs/concepts/summary.md new file mode 100644 index 0000000..b6951c9 --- /dev/null +++ b/docs/concepts/summary.md @@ -0,0 +1,59 @@ +# Key Concepts + +## [Authentication Methods](authentication-method.md) + +## [Encryption Method](encryption-method.md) + +## [ISO Duration](iso-date-interval.md) + +## [Job Pattern](job-type.md) + +There are several logical types of Azkaban jobs in +data integration workflows. These job types are not differentiated by +any particular parameter; **they are configured the same way, and +they look alike**; only they serve different **purposes**. +They are differentiated here because each of +them follow certain **patterns**. Understanding these logical types +of jobs can help configure them quicker. + +## [Json Path](json-path.md) + +## [Pagination](pagination.md) + +Pagination is typically used to fetch a large dataset from cloud over HTTP, where +one fetch can only optimally get a limited chunk of data. In such case, +the data is fetched through a series of pages. + +## [Schema](schema.md) + +## [Secondary Input](secondary-input.md) + +## [Secret Encryption](secret-encryption.md) + +## [Session Control](session-control.md) + +## [Single Flow](single-flow.md) + +## [Variable](variables.md) + +A variable provides dynamic input through substitution to parameters. + +## [Watermark](watermark.md) + +There are two types of watermarks: + +- time-based LongWatermark +- unit-based UnitWaterMark. + +Time watermark defines a time range, with a `from` datetime and a +`to` datetime. DIL internally handles time watermark values in milliseconds. + +A "Unit" watermark holds individual values, like ids. It is a list of string values. + +## [Work Unit](work-unit.md) + +"Time" watermarks can generate partitions, and "unit" watermarks have units. + +Time watermark and unit watermark together creates work units, and DIL +maintains execution state including watermarks for each work unit. + diff --git a/docs/concepts/variables.md b/docs/concepts/variables.md new file mode 100644 index 0000000..0e73762 --- /dev/null +++ b/docs/concepts/variables.md @@ -0,0 +1,137 @@ +# Variables + +A variable provides dynamic input through substitution to parameters. + +## Variable Definition + +Variables are named with only the following letters: +- a-z +- A-Z +- 0-9 +- "_" (Underscore) +- "-" (Hyphen) +- "." (dot) +- "$" (dollar sign) + +Variables are defined and generated through following means: + +### Option 1: [ms.parameters](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.parameters.md) + +Each parameter defined in `ms.parameters` is also a variable. Parameters defined this +way are called "parameters" because they will be used in final requests to +integration points (IPs). + +Each parameter is also a variable, but other variables are not used in final requests. +Other variables only participate in value substitution. + +Note this recursive nature of variables and parameters, so you will +never define a parameter with a substitution variable of itself, that +will be a loop. + +### Option 2: [ms.watermark](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.watermark.md) + +Each `time watermark` defined 2 variables, low and high, but currently +we support only 1 time watermark. +So there are 2 time watermark variables. + +Each unit watermark define a variable, +but currently we support only 1 unit watermark. So there +is 1 unit watermark variables from properties. The unit watermark +can have multiple units (values), each of them will be +distributed into one of the work units, and therefore, +each work unit has 1 unique value under the same variable name. + +### Option 3: [ms.pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.pagination.md) + +Each pagination attribute define a variable, these include +"pagestart", "pagesize", and "pageno". They get values from fields +defined in ms.pagination, or derived from response data + +Example: `ms.pagination={"fields": ["offset", "limit"], "initialvalues": [0, 5000]}` + +### Option 4: [ms.session.key.field](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.session.key.field.md) + +A session variable gets value from the field defined in +`ms.session.key.field`. + +### Option 5: [ms.secondary.input](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.secondary.input.md) + +An `activation` secondary input can have multiple records and multiple fields. +Hence, every `activation` type secondary input can generate one or more variables. +One variable is generate from each of the `fields` selected. + +However, `activation` records are distributed to multiple work units, each work +unit has only 1 row from the activation. So variables generated this way +are work unit specific, i.e., each work unit has the same variable +but different value. For example, for Qualaroo, the first job, job1, +generates a list of surveys, and the list of surveys is used as secondary +input in the second job, job2. In job2, we select the surveyId +field from the secondary input, therefore, we have a variable called "surveyId", +but we have many surveys, thus, we have many work units, +each work unit has a parameter called "surveyId", and it has a +value assigned to that work unit. + +An `authentication` secondary input can have 1 and only 1 records, +but allow multiple fields. That means it can generate multiple variables, +with each variable having only 1 value. +`authentication` variable will be shared across all work units. + +## Parameters + +Parameters are used to execute requests (Http request, JDBC request, and Sftp request, etc). +Parameters are also variables, but other variables are just kept internally. +Other variables are not used to execute requests. + +All variables are used for parameter substitution. Therefore, parameters +can be used in variable substitution. + +Some data source doesn't allow extra parameters in requests. +For example, SFDC fails if unknown parameters are in the Http request. +Therefore, parameters need to be selective, and only +parameters that will work with the source should be defined in `ms.parameters`. + +## Usage of Variables + +Variables can be used for substitution in parameters (defined through ms.parameters), +using the syntax of double brackets `{{variableName}}`. + +If the variable is a parameter that starts with "**tmp**", then once it is used, +the parameter is removed from the pool. + +A parameter can be used for substitution in `ms.extractor.target.file.name`, +but it will stay in the pool, because the substitution +happens in Extractor. + +### HTTP Usage + +For HTTP protocol, variables are used for substitution in the following properties: + +- [ms.source.uri](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.source.uri.md) +- [ms.http.request.headers](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.http.request.headers.md) +- [ms.authentication](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.authentication.md) +- [ms.derived.fields](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.derived.fields.md) +- [ms.extractor.target.file.name](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.extractor.target.file.name.md) + +The rest parameters will be appended as URL parameters to `ms.source.uri` +if it is a GET Http request. + +The rest parameters will be added to the Request Entity if it +is a **POST**, **PUT**, **DELETE** request. + +### JDBC Usage + +For JDBC protocol, variables are used for substitution in the following properties: + +- [ms.source.uri](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.source.uri.md) +- [ms.derived.fields](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.derived.fields.md) +- [ms.jdbc.statement](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.jdbc.statement.md) +- [ms.extractor.target.file.name](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.extractor.target.file.name.md) + +### S3 Usage + +For S3 protocol, variables are used for substitution in the following properties: +- [ms.source.uri](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.source.uri.md) +- [ms.derived.fields](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.derived.fields.md) +- [ms.extractor.target.file.name](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.extractor.target.file.name.md) + +[Back to Summary](summary.md#variables) \ No newline at end of file diff --git a/docs/concepts/watermark.md b/docs/concepts/watermark.md new file mode 100644 index 0000000..96c83fd --- /dev/null +++ b/docs/concepts/watermark.md @@ -0,0 +1,221 @@ +# Watermark + +There are two types of watermarks: + +- time-based LongWatermark +- unit-based UnitWaterMark. + +## Related +- [ms.watermark](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.watermark.md) +- [ms.grace.period.days](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.grace.period.days.md) +- [ms.abstinent.period.days](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.abstinent.period.days.md) + +## Time Watermark + +Time watermark defines a time range, with a `from` datetime and a +`to` datetime. DIL internally handles time watermark values in milliseconds. +For example: +- `ms.watermark = [{"name": "system","type": "datetime", "range": {"from": "2017-01-02", "to": "-"}}]` + +Both the `from` and `to` can take the following configuration format: +- **ISO Datetime**: `yyyy-MM-dd HH:mm:ss.SSSSSSz` or `yyyy-MM-ddTHH:mm:ss.SSSSSSz` +- **ISO 8601 duration**: `PnDTmH`, which is a pattern of "P\\d+D(T\\d+H){0,1}" + +When using the ISO 8601 duration format, `n` can be any natural numbers, +it defines n days before current date time. For example P1D defined 1 day +before current day. `m` can be any natural number between 0 and 23, +it defines m hours before the current time. The hour part is optional. +Examples: P0D, P1D, P0DT7H. + +"**-**" (Hyphen) can be used in `to` as a shortcut for "P0D". + +The `from` datetime is usually static, and `to` datetime is usually dynamic. +see [ms.watermark](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.watermark.md). + +A time watermark has an effective range which is defined by an +`effective cut off date` and watermark `to` date. The `effective +cut off date` is decided by the last highest watermark, grace period, +and abstinent period. The formula is: + +- `effective cut off date = last highest watermark + abstinent period - grace period` +- `effective cut off date` is the same as `from` if it is the first job execution and last +highest watermark is not established yet + +#### Rounding of `to` Datetime + +If time watermark is partitioned using multi-day partition (i.e. monthly and weekly), +the "to date" will be rounded to day level if `to` is `PnD`, and the "to date" will be +rounded to hour level if `to` is `PnDTmH`. + +If time watermark is not partitioned or is daily/hourly partitioned, +the "to date" will include hours and minutes, etc. + +### Grace Period + +Grace Period addresses the late arrival problem, which is +very common if the ingestion source is a data warehouse. +See [ms.grace.period.days](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.grace.period.days.md) + +For example, if we define a time watermark as from **2020-01-01** to **P0D**. Then +on the first run on 2020-01-15. The effective range is 2020-01-01 to 2020-01-15. +Then on the next run on 2020-01-16, the effective range will be 2020-01-12 to +2020-01-16, if we give 3 grace period days and 0 abstinent day. + +See following illustration for a general case over 2 executions: +![Time Watermark Illustration](../images/effective-range.png) + +Therefore, _Grace Period moves effective cut off date back by "GracePeriod" days_, +increasing the effective range, and thus allow more incremental updates +be included in the extract. + +### Abstinent Period + +Abstinent Period has the opposite effect as Grace Period. It addresses the +repeated download problem. +See [ms.abstinent.period.days](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.abstinent.period.days.md) + +The repeated download problem, typically in a file download situation, requires +us avoid downloading the downloaded files by X number of days. + +_Abstinent Period moves effective cut off date forward by "AbstinentPeriod" days_, +reducing or even invalidating effective range, and thus allow few or +no incremental updates be included in the extract. + +In a file downloading situation, each file is a work unit. Each file has its watermark. +A file will download if the `current date (system)` falls between the effective range. + +With the same watermark definition as above, if there is no grace period (for simplicity), +and there is 1 day abstinent day: +- on 2020-01-16, the effective range is 0, therefore the file is not downloaded +- on 2020-01-17, the file starts downloading again + +![Abstinent Period Illustration](../images/abstinent-period.png) + +If put in the context that the server provides daily files for the past 7 days, +we can expect the following behavior if we set Abstinent days to **7 or more**: + +- on 2020-01-15, file20200115 is generated, and file20200115 is downloaded + +- on 2020-01-16, file20200116 is generated, and file20200116 is downloaded, +and file20200115 is ignored + +- on 2020-01-17, file20200117 is generated, and file20200117 is downloaded, +and file20200116 and file20200115 are ignored + +so on, hence no repeated download although the server keeps 7-day's files. + +If Abstinent days is 0, then on 2020-01-16, the effective range would be +[2020-01-15, 2020-01-16), and it will repeat downloading file file20200115. + +### Cut Off Date + +`cut off date` is calculated during runtime. It is based on job configuration +and state store records. `cut off date` is calculated at milliseconds level. +The `effective cut off date` is the `cut off date` after considering execution history, which +is tracked in state store. + +- `effective cut off date = last highest watermark + abstinent period - grace period` +- If it is the first job execution and last highest watermark is not present, + then `effective cut off date` is the same as `from` + +Each job has an `overall cut off date` and each partition has a `partition cut off date`. + +In runtime, the `overall effective cut off date` is calculated by considering all partitions. +The `overall effective cut off date` is used to decide whether to execute a partition. A partition +executes if its `partition cut off date` is larger than the `overall effective cut off date`. +A partition without execution history, including when the partition failed in its first execution, +has practically no `partition cut off date`; therefore, it always executes. + +### Partitioning + +Time watermark can be partitioned into different periods: monthly, weekly, +daily, and hourly. Parameter [ms.work.unit.partition](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.work.unit.partition.md) +defines partition method. + +When time watermark is partitioned, it partitions per `from` date and `to` date. +The `effective cut off date` has no role in defining the partitions. +This keeps partitions consistent. + +#### Multi-day Partitions and Partial Partition + +Monthly and weekly partitions are considered multi-day partitions. + +Monthly and weekly partitioning can result in a partial partition in the end. + +For example: + +If from date is 2019-01-01, and the to date is "-", and it is 2020-02-21, +and it is **monthly** partitioned, then the first flow execution will +have 13 full monthly partitions and 1 partition monthly partition +from **[2020-02-01, 2020-02-21)**. In the next flow execution, the first +13 monthly partitions will stay the same, the last partial partition +becomes **[2020-02-01, 2020-02-22)**. + +If from date is 2020-01-06 (Monday), and the to date is "-", and +it is 2020-02-21 (Friday), and it is **weekly** partitioned, +then the last partition will be 2020-02-17 to 2020-02-21. + +Partial partition can be disabled by [ms.work.unit.partial.partition](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.work.unit.partial.partition.md). + +Partitions maintain their own states (high watermark). So, in above +example, after the first successful flow execution the partition +[2019-01-01, 2019-01-31) has high watermark as 2019-01-31. + +A job's effective cut off date is **the largest high watermark of all partitions**. +Taking above example, after the first successful execution the job's effective cut off date +will be 2020-02-18 (3 day grace period), because the largest high +watermark of all partitions is 2020-02-21. + +**_A partition is executed if its end date is larger than the effective cut off date._** + +The job's effective cut off date ensures old partitions will not be +re-extracted after the first flow execution because older +partitions' end date is smaller than the job's effective cut off date. + +However, if a partition fails in first execution, its last high watermark will be +the same as partition start (in milliseconds), and DIL will ignore that in next +execution. The failed partition will be treated like a new partition. + +When above process is applied to daily partitions, however, the situation +will appear differently. + +For example, if `from` = "2020-01-01", and `to` = "P0D", +and it is "daily" partitioned,and it is 2020-02-21, then after the first +successful execution, each date partition will have a high watermark +of the midnight of next date (in milliseconds). So the last partition +2020-02-20 will have high watermark as 2020-02-21 12:00:00 AM. +Assuming GracePeriod = 3, in next flow execution, +the job's effective cut off date would be 2020-02-19 12:00:00 AM, +therefore, partition 02-19, 02-20, and 02-21 will be re-executed +because their last high watermarks are larger than the job's +effective cut off date. + +Note: watermark `from` date is allowed to have hours, +for example 2020-02-21 10:00:00, in such case the partitions +will also end by that hour, not by 12:00:00 AM. + +- Daily partitions are derived by adding 1 day to `from` date, and so on. +- Weekly partitions are derived by adding 7 days to `from` date, and so on. +- Monthly partitions are derived by adding 1 month to `from` date, and so on. + +## Unit Watermark + +A "Unit" watermark holds individual values, like ids. It is a list of string values. + +There are two ways a unit watermark is defined. + +- Explicit definition using [ms.watermark](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.watermark.md) +- Implicit definition using [ms.secondary.input](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.secondary.input.md) + +A "Unit" watermark can also be defined via ms.watermark explicitly. For example: +- `ms.watermark=[{"name": "surveyId", "type": "unit", "units": "id1,id2,id3"]}]` + +Typically, a secondary input provides a list of ids. Those ids will +be made into unit watermarks internally. For example, in Qualaroo +configuration, we have: +- `ms.secondary.input=[{"path": "${job.dir}/${preceding.table.name}", "fields": ["id"]}]` + +As the input file contains a list of survey ids, that will +effectively define a unit watermark for each survey id. + +[Back to Summary](summary.md#watermark) \ No newline at end of file diff --git a/docs/concepts/work-unit.md b/docs/concepts/work-unit.md new file mode 100644 index 0000000..004931a --- /dev/null +++ b/docs/concepts/work-unit.md @@ -0,0 +1,36 @@ +# Work Unit + +"Time" watermarks can generate partitions, and "unit" watermarks have units. + +Time watermark and unit watermark together creates work units, and DIL +maintains execution state including watermarks for each work unit. + +Partitions and Units make a matrix. Assuming we have m periods and n units, +the matrix will be n x m. **Each combination of partitions and units makes +a work unit**. That means there will be n x m work units. + +The following scenarios are possible: + +- Only time watermark is defined, no secondary input nor explicit unit +watermark, and time watermark is **not partitioned**, +there will be only 1 work unit +- Only time watermark is defined, no secondary input nor explicit unit +watermark, and time watermark is partitioned into **m partitions**, +there will be m work units +- Only unit watermark is defined, either through secondary input or +through explicit definition, default none-partition time watermark, +there will be n work units +- Both time watermark and unit watermark are defined, there will +be n x m work units + +In any of above cases, each work unit tracks its own high watermark. + +**Note**: Work units are executed as tasks in runtime. Therefore, tasks and +work units are exchangeable concepts. + +## Related +- [ms.watermark](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.watermark.md) +- [ms.grace.period.days](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.grace.period.days.md) +- [ms.abstinent.period.days](https://github.com/linkedin/data-integration-library/blob/master/docs/parameters/ms.abstinent.period.days.md) + +[Back to Summary](summary.md#work-unit) \ No newline at end of file diff --git a/docs/get-started/create-a-http-ingestion-flow.md b/docs/get-started/create-a-http-ingestion-flow.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/get-started/create-a-jdbc-ingestion-flow.md b/docs/get-started/create-a-jdbc-ingestion-flow.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/get-started/create-a-s3-ingestion-flow.md b/docs/get-started/create-a-s3-ingestion-flow.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/get-started/create-a-sftp-ingestion-flow.md b/docs/get-started/create-a-sftp-ingestion-flow.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/get-started/where-how-what.md b/docs/get-started/where-how-what.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/how-to/abstinent-period.md b/docs/how-to/abstinent-period.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/how-to/authentication-job.md b/docs/how-to/authentication-job.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/how-to/grace-period.md b/docs/how-to/grace-period.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/how-to/status-check-job.md b/docs/how-to/status-check-job.md new file mode 100644 index 0000000..efa09ff --- /dev/null +++ b/docs/how-to/status-check-job.md @@ -0,0 +1,81 @@ +# Config a Status Checking Job + +A status checking job ensures the data is +ready for consumption on the third party system. +This can be used in asynchronous data ingestion and file downloads. + +In the asynchronous ingestion scenario, this job will keep checking status with the 3P system +every once a while (e.g. every 15 minutes) until the status turns to +ready or timeout. + +In the file downloading scenario, the status checking +job can keep checking the availability of source data until they are +present or timeout. + +A status checking job tries to read a small piece of information +from the third party system, and it then verifies per the given criteria +for success or failure. + +## Base Configuration + +Just like other jobs, the base configurations include source, +authentication, and others in order to retrieve the needed +status information. + +Typically, the job would have: + +1. `extract.table.type=snapshot_only` +2. `data.publisher.final.dir=<>` +3. `state.store.enabled=false` + +## Session Control + +Status checking can be achieved through session control. +See [ms.session.key.field](../parameters/summary.md#mssessionkeyfield). +This way of configuration is most commonly used in [asynchronous ingestion](../patterns/summary.md#asynchronous-ingestion-pattern). + +For example, the following uses the "status" field in the response +return from the third party system as a session key. It expects the +status to have a value "success" or "ready" when data is ready on the system +for consumption. + +- `ms.session.key.field={"name": "status", "condition": {"regexp": "success|ready"}}` + +When the status is not "success" nor "ready", the session +will keep going until [time out](../parameters/summary.md#mswaittimeoutseconds). +When the session is live, the job will `paginate` by sending requests to the third +party system on [intervals](../parameters/summary.md#mscallintervalmillis). + +Another example, the following uses the "status" field in the "result" field (nested) +in the response from the third party system as the session key. +The job expects a value "complete" for success, and "failed" for failure. +When the success criteria is met, the job will complete successfully. When the +failure criteria is met, the job will fail. Otherwise, the job will keep paginating +until time out. + +- `ms.session.key.field={"name": "result.status", "condition": {"regexp": "^complete$"}, "failCondition": {"regexp": "^failed$"}}` + +## Minimum Record Validation + +Status checking can be achieved through minimum record count validation. +See [ms.work.unit.min.records](../parameters/summary.md#msworkunitminrecords). +This way of configuration is most commonly used in [2 step file download](../patterns/summary.md#2-step-file-download-pattern). + +For example, the following job pulls the today's file from an SFTP server, +and it will fail the work unit if today's file is not present because the +minimum required record is 1. By the Gobblin retry mechanism, +when the work unit fails, the job will retry 10 times with an exponentially +growing interval (in between the retries). + +The configuration used the "currentDate" +dynamic variable in the path so that only today's files are listed, and +"currentDate" is defined as a variable that gets value from the high +watermark, which is "P0D", i.e. the current date. + +- `ms.work.unit.min.records=1` +- `ms.watermark=[{"name": "system","type": "datetime", "range": {"from": "P1D", "to": "P0D"}}]` +- `ms.parameters=[{"name":"currentDate","type":"watermark","watermark":"dateRange","value":"high","format":"datetime","pattern":"yyyyMMdd"}]` +- `ms.source.uri=/home/user/download/file_*{{currentDate}}*.txt.gpg` +- `task.maxretries=10` +- `job.commit.policy=full` + diff --git a/docs/how-to/summary.md b/docs/how-to/summary.md new file mode 100644 index 0000000..8bb8252 --- /dev/null +++ b/docs/how-to/summary.md @@ -0,0 +1,6 @@ +# [Config a Status Checking Job](status-check-job.md) + +A [status checking job](../concepts/job-type.md#status-checking-job) ensures the data is +ready for consumption on the third party system. + +# [Config a Authentication Job](authentication-job.md) \ No newline at end of file diff --git a/docs/images/abstinent-period.png b/docs/images/abstinent-period.png new file mode 100644 index 0000000..25be508 Binary files /dev/null and b/docs/images/abstinent-period.png differ diff --git a/docs/images/effective-range.png b/docs/images/effective-range.png new file mode 100644 index 0000000..8fa126f Binary files /dev/null and b/docs/images/effective-range.png differ diff --git a/docs/images/sfdc-bulk-ingestion.png b/docs/images/sfdc-bulk-ingestion.png new file mode 100644 index 0000000..98782cc Binary files /dev/null and b/docs/images/sfdc-bulk-ingestion.png differ diff --git a/docs/parameters/categories.md b/docs/parameters/categories.md new file mode 100644 index 0000000..4139b84 --- /dev/null +++ b/docs/parameters/categories.md @@ -0,0 +1,129 @@ +# Auditing Properties + +The following job properties are essential for auditing function: + +- [ms.audit.enabled](ms.audit.enabled.md) +- [ms.kafka.brokers](ms.kafka.brokers.md) +- [ms.kafka.clientId](ms.kafka.clientId.md) +- [ms.kafka.schema.registry.url](ms.kafka.schema.registry.url.md) +- [ms.kafka.audit.topic.name](ms.kafka.audit.topic.name.md) + +# Authentication Properties + +The following are related to authentication: + +- [ms.authentication](ms.authentication.md) +- [ms.http.request.headers](ms.http.request.headers.md) +- [ms.secondary.input](ms.secondary.input.md) +- [ms.parameters](ms.parameters.md) +- source.conn.username +- source.conn.password + +# Connection Properties + +The following are related to connections: + +- [ms.connection.client.factory](ms.connection.client.factory.md) +- [ms.source.uri](ms.source.uri.md) +- [ms.source.s3.parameters](ms.source.s3.parameters.md) +- [ms.sftp.conn.timeout.millis](ms.sftp.conn.timeout.millis.md) +- [ms.ssl](ms.ssl.md) +- source.conn.username +- source.conn.password + +The following could also be related to connections as they define variables: + +- [ms.secondary.input](ms.secondary.input.md) +- [ms.parameters](ms.parameters.md) + +# Conversion Properties + +The following are related to conversion (converters): + +- [ms.data.explicit.eof](ms.data.explicit.eof.md) +- [ms.derived.fields](ms.derived.fields.md) +- [ms.enable.schema.based.filtering](ms.enable.schema.based.filtering.md) +- [ms.encryption.fields](ms.encryption.fields.md) +- [ms.extract.preprocessors](ms.extract.preprocessors.md) +- [ms.extract.preprocessor.parameters](ms.extract.preprocessor.parameters.md) +- [ms.extractor.class](ms.extractor.class.md) +- [ms.extractor.target.file.name](ms.extractor.target.file.name.md) +- [ms.extractor.target.file.permission](ms.extractor.target.file.permission.md) +- [ms.normalizer.batch.size](ms.normalizer.batch.size.md) +- [ms.output.schema](ms.output.schema.md) +- [ms.source.schema.urn](ms.source.schema.urn.md) +- [ms.target.schema](ms.target.schema.md) +- [ms.target.schema.urn](ms.target.schema.urn.md) +- [ms.validation.attributes](ms.validation.attributes.md) + +# Extractor Properties +- [ms.csv](ms.csv.md) + +# Execution Properties +- [ms.enable.dynamic.full.load](ms.enable.dynamic.full.load.md) + +# HTTP Properties + +The following are related to HTTP sources: + +- [ms.authentication](ms.authentication.md) +- [ms.http.request.headers](ms.http.request.headers.md) +- [ms.http.request.method](ms.http.request.method.md) +- [ms.http.response.type](ms.http.response.type.md) +- [ms.http.statuses](ms.http.statuses.md) +- [ms.http.conn.max](ms.http.conn.max.md) +- [ms.http.conn.per.route.max](ms.http.conn.per.route.max.md) +- [ms.http.conn.ttl.seconds](ms.http.conn.ttl.seconds.md) + +# Pagination Properties +- [ms.call.interval.millis](ms.call.interval.millis.md) +- [ms.pagination](ms.pagination.md) +- [ms.session.key.field](ms.session.key.field.md) +- [ms.wait.timeout.seconds](ms.wait.timeout.seconds.md) + +# Schema Properties + +The following are related to schema: + +- [ms.data.default.type](ms.data.default.type.md) +- [ms.output.schema](ms.output.schema.md) +- [ms.source.schema.urn](ms.source.schema.urn.md) +- [ms.target.schema](ms.target.schema.md) +- [ms.target.schema.urn](ms.target.schema.urn.md) +- [ms.schema.cleansing](ms.schema.cleansing.md) +- [ms.enable.cleansing](ms.enable.cleansing.md) +- [ms.enable.schema.based.filtering](ms.enable.schema.based.filtering.md) +- [ms.jdbc.schema.refactor](ms.jdbc.schema.refactor.md) +- [ms.kafka.schema.registry.url](ms.kafka.schema.registry.url.md) +- ms.converter.keep.null.strings + +# Source Properties + +- [ms.data.field](ms.data.field.md) +- [ms.jdbc.statement](ms.jdbc.statement.md) +- [ms.parameters](ms.parameters.md) +- [ms.s3.list.max.keys](ms.s3.list.max.keys.md) +- [ms.session.key.field](ms.session.key.field.md) +- [ms.source.data.character.set](ms.source.data.character.set.md) +- [ms.source.files.pattern](ms.source.files.pattern.md) +- [ms.source.s3.parameters](ms.source.s3.parameters.md) +- [ms.source.schema.urn](ms.source.schema.urn.md) +- [ms.source.uri](ms.source.uri.md) +- [ms.total.count.field](ms.total.count.field.md) +- [ms.wait.timeout.seconds](ms.wait.timeout.seconds.md) + +# Watermark Work Unit Properties + +The following are related to watermarks and work units: + +- [ms.abstinent.period.days](ms.abstinent.period.days.md) +- [ms.grace.period.days](ms.grace.period.days.md) +- [ms.secondary.input](ms.secondary.input.md) +- [ms.watermark](ms.watermark.md) +- [ms.work.unit.min.records](ms.work.unit.min.records.md) +- [ms.work.unit.min.units](ms.work.unit.min.units.md) +- [ms.work.unit.pacing.seconds](ms.work.unit.pacing.seconds.md) +- [ms.work.unit.parallelism.max](ms.work.unit.parallelism.max.md) +- [ms.work.unit.partial.partition](ms.work.unit.partial.partition.md) +- [ms.work.unit.partition](ms.work.unit.partition.md) + diff --git a/docs/parameters/csv.max.failures.md b/docs/parameters/csv.max.failures.md new file mode 100644 index 0000000..7a4563c --- /dev/null +++ b/docs/parameters/csv.max.failures.md @@ -0,0 +1,3 @@ +## csv.max.failures + +**Deprecated, use [ms.csv](ms.csv.md)** \ No newline at end of file diff --git a/docs/parameters/deprecated.md b/docs/parameters/deprecated.md new file mode 100644 index 0000000..01f6406 --- /dev/null +++ b/docs/parameters/deprecated.md @@ -0,0 +1,25 @@ +# Deprecated Job Properties + +The following job properties are deprecated, some are no longer being effective, +and some are replaced by new parameters. + +## ms.watermark.type + +`ms.watermark.type` was designed as an DIL internal property. It is no longer being used. + +**Alert**: `ms.watermark.type` can cause parsing error if it is used in GaaS flow specs because +its prefix matches another effective parameter ms.watermark. Azkaban projects that still +have this job property will have no harm. + +## ms.encoding + +`ms.encoding` is replaced by [ms.source.data.character.set](ms.source.data.character.set.md). + +## Factories + +- ms.http.client.factory +- ms.source.schema.reader.factory +- ms.target.schema.reader.factory + +Above factories are replaced by [ms.connection.client.factory](ms.connection.client.factory.md) + diff --git a/docs/parameters/gobblin.converter.class.md b/docs/parameters/gobblin.converter.class.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/parameters/gobblin.source.class.md b/docs/parameters/gobblin.source.class.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/parameters/ms.abstinent.period.days.md b/docs/parameters/ms.abstinent.period.days.md new file mode 100644 index 0000000..8876e5f --- /dev/null +++ b/docs/parameters/ms.abstinent.period.days.md @@ -0,0 +1,42 @@ +# ms.abstinent.period.days + +**Tags**: +[watermark & work unit](categories.md#watermark-work-unit-properties) + +**Type**: integer + +**Default value**: 0 + +## Related + +- [key concept: watermark](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/watermark.md) +- [key concept: work-unit](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/work-unit.md) +- [Property: ms.grace.period.days](ms.grace.period.days.md) +- [How to: abstinent period](https://github.com/linkedin/data-integration-library/blob/master/docs/how-to/abstinent-period.md) +- [How to: grace period](https://github.com/linkedin/data-integration-library/blob/master/docs/how-to/grace-period.md) + +## Description + +Abstinent Period is designed to avoid re-extracting a dataset repeatedly. This is particularly useful +for situations like downloading files in large quantity.

    + +Abstinent Period works the opposite way of Grace Period. In an incremental ETL execution, +Abstinent Period is added to a work unit's last execution time for form a new low watermark for +that work unit, while Grace Period is subtracted from the work unit's last execution time. + +Assuming we will control all data extraction through a time range, including file downloads, and +assuming files were all uploaded to source on 6/30, and further assuming we can only download 100 files +per day, and there are 1000 files, therefore, the plan should be downloading 100 files per day, and run the job for +10 consecutive days. + +Assuming we start downloading on 7/1, files downloaded on 7/1 will be downloaded again on 7/2 because +their cut off time is 7/1, which is the last actual high watermark, and which is before the new extraction time (7/2). +See how cut off time is calculated [here](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/watermark.md). + +An abstinent period 30 is thus added to the last actual high watermark, allowing us move the cutoff time forward. +Therefore, if there is an abstinent period of 30 days, the downloaded files will not be downloaded +again in 30 days. + +Abstinent period can be set to a large number so that the same file will never be downloaded again. + +[back to summary](summary.md#msabstinentperioddays) diff --git a/docs/parameters/ms.audit.enabled.md b/docs/parameters/ms.audit.enabled.md new file mode 100644 index 0000000..83869ae --- /dev/null +++ b/docs/parameters/ms.audit.enabled.md @@ -0,0 +1,26 @@ +# ms.audit.enabled + +**Tags**: +[auditing](categories.md#auditing-properties) + +**Type**: boolean + +**Format**: true/false + +**Default value**: false + +## Related +- [ms.kafka.brokers](ms.kafka.brokers.md) +- [ms.kafka.clientId](ms.kafka.clientId.md) +- [ms.kafka.schema.registry.url](ms.kafka.schema.registry.url.md) +- [ms.kafka.audit.topic.name](ms.kafka.audit.topic.name.md) + +## Description + +Setting ms.audit.enabled to true will enable outgoing data auditing. Auditing will trace all outgoing data +including parameters and payloads to data lake through Kafka. + +Auditing is an important part of egression, but ingestion requests can also be audited. + +[back to summary](summary.md#msauditenabled) + \ No newline at end of file diff --git a/docs/parameters/ms.authentication.md b/docs/parameters/ms.authentication.md new file mode 100644 index 0000000..0b528ca --- /dev/null +++ b/docs/parameters/ms.authentication.md @@ -0,0 +1,74 @@ +# ms.authentication + +**Tags**: +[authentication](categories.md#authentication-properties) +[http](categories.md#http-properties) + +**Type**: string + +**Format**: JsonObject + +**Default value**: blank + +**Accepts Variable Substitution**: Yes + +## Related +- [ms.http.request.headers](ms.http.request.headers.md) +- [ms.secondary.input](ms.secondary.input.md) +- [ms.parameters](ms.parameters.md) +- source.conn.username +- source.conn.password + +## Description + +ms.authentication job property defines the authentication of a request. It works with HTTP protocol only +for now, but could be used in other protocols. + +ms.authentication is designed to be an JsonObject with the following fields: + +- **method**: `method` field specifies the authentication scheme, it is a string allowing these +values: basic|bearer|oauth|custom. see [authentication method](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/authentication-method.md). + +- **encryption**:`encryption` field specifies how to encrypt the credentials, it is a string allowing +`base64` or `none`. see [encryption method](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/encryption-method.md). + +- **header**: `header` field specifies what HTTP request header tag to associate the credential withs. In most +cases, this is `Authorization`. + +- **token**: `token` field specifies the values like access token, bearer token, or refresh token etc. If +not specified, a token is made from the concatenation of the values of source.conn.username +and source.conn.password. Tokens are considered secrets, and job configuration should have its encrypted +value. see [secret encryption](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/secret-encryption.md). +Encrypted tokens are decrypted, and encrypted with the `encryption` algorithm, before being used in requests. + +## Example 1: Base Authentication + +With base authentication, source.conn.username and source.conn.password specifies user name and password, or +user key and secret key. Then user name and password are concatenated with `:` as separator. The combined +string is then encrypted using base64 algorithm. + +A typical configuration would be like this: + +`ms.authentication = {"method": "bearer", "encryption": "base64", "header": "Authorization"}` + +The combined user name and password string can also be supplied through the `token` field for cleaner +configuration. This avoids using source.conn.username and source.conn.password job properties. + +`ms.authentication={"method": "bearer", "encryption": "base64", "header": "Authorization", "token": "xxxx"}` + +## Example 2: Bearer Token Authentication + +Bearer token can be specified directly: + +`ms.authentication={"method": "bearer", "encryption": "none", "header": "Authorization", "token": "xxxx"}` + +or as a variable: + +`ms.authentication={"method": "bearer", "encryption": "none", "header": "Authorization", "token": "{{access_token}}"}` + +The typical application of the second way is OAuth2 where the access token need to be refreshed ahead of +every job execution, and then the updated access token can be read in through secondary input. When the +access token is read in through secondary input, it is stored in a variable. And that variable can then +be referenced in the `token` field. + +[back to summary](summary.md#msauthentication) \ No newline at end of file diff --git a/docs/parameters/ms.call.interval.millis.md b/docs/parameters/ms.call.interval.millis.md new file mode 100644 index 0000000..478e59c --- /dev/null +++ b/docs/parameters/ms.call.interval.millis.md @@ -0,0 +1,25 @@ +# ms.call.interval.millis + +**Tags**: +[pagination](categories.md#pagination-properties) + +**Type**: long + +**Default value**: 0 + +## Related + +## Description + +ms.call.interval.millis specifies the minimum time elapsed between requests in the pagination process. +When a page is retrieved sooner than the interval, to avoid QPS violation, the thread will wait until +the interval has passed. + +ms.call.interval.millis works within an executor thread. In cases of parallel execution, where the +number of executor threads is more than one, ms.call.interval.millis should be multiple of the interval +allowed by the QPS to avoid QPS violations cross threads. + +APIs might have quota by second and quota by the hour or day. + +[back to summary](summary.md#mscallintervalmillis) + diff --git a/docs/parameters/ms.connection.client.factory.md b/docs/parameters/ms.connection.client.factory.md new file mode 100644 index 0000000..a8a717d --- /dev/null +++ b/docs/parameters/ms.connection.client.factory.md @@ -0,0 +1,17 @@ +# ms.connection.client.factory + +**Tags**: [connection](categories.md#connection-properties) + +**Type**: string + +**Default value**: `com.linkedin.cdi.DefaultConnectionClientFactory` + +## Description + +ms.connection.client.factory allows vendors specify customized connections with proxy or enhanced security. + +The default factory works with general cloud data sources. It can be used when integrating data within +a company's network. + +[back to summary](summary.md#msconnectionclientfactory) + \ No newline at end of file diff --git a/docs/parameters/ms.converter.csv.max.failures.md b/docs/parameters/ms.converter.csv.max.failures.md new file mode 100644 index 0000000..4cd5a5e --- /dev/null +++ b/docs/parameters/ms.converter.csv.max.failures.md @@ -0,0 +1,3 @@ +## ms.converter.csv.max.failures + +**Deprecated, use [ms.csv](ms.csv.md)** \ No newline at end of file diff --git a/docs/parameters/ms.csv.column.header.index.md b/docs/parameters/ms.csv.column.header.index.md new file mode 100644 index 0000000..a02ebb2 --- /dev/null +++ b/docs/parameters/ms.csv.column.header.index.md @@ -0,0 +1,5 @@ +# ms.csv.column.header.index + +**Deprecated, use [ms.csv](ms.csv.md)** + +[back to summary](summary.md#mscsvcolumnheaderindex) diff --git a/docs/parameters/ms.csv.column.header.md b/docs/parameters/ms.csv.column.header.md new file mode 100644 index 0000000..50f0375 --- /dev/null +++ b/docs/parameters/ms.csv.column.header.md @@ -0,0 +1,5 @@ +# ms.csv.column.header + +**Deprecated, use [ms.csv](ms.csv.md)** + +[back to summary](summary.md#mscsvcolumnheader) diff --git a/docs/parameters/ms.csv.column.projection.md b/docs/parameters/ms.csv.column.projection.md new file mode 100644 index 0000000..e04ac60 --- /dev/null +++ b/docs/parameters/ms.csv.column.projection.md @@ -0,0 +1,6 @@ +# ms.csv.column.projection + +**Deprecated, use [ms.csv](ms.csv.md)** + +[back to summary](summary.md#mscsvcolumnprojection) + diff --git a/docs/parameters/ms.csv.default.field.type.md b/docs/parameters/ms.csv.default.field.type.md new file mode 100644 index 0000000..c67c415 --- /dev/null +++ b/docs/parameters/ms.csv.default.field.type.md @@ -0,0 +1,5 @@ +# ms.csv.default.field.type + +**Deprecated, use [ms.csv](ms.csv.md)** + +[back to summary](summary.md#mscsvdefaultfieldtype) diff --git a/docs/parameters/ms.csv.escape.character.md b/docs/parameters/ms.csv.escape.character.md new file mode 100644 index 0000000..d493a34 --- /dev/null +++ b/docs/parameters/ms.csv.escape.character.md @@ -0,0 +1,5 @@ +# ms.csv.escape.character + +**Deprecated, use [ms.csv](ms.csv.md)** + +[back to summary](summary.md#mscsvescapecharacter) diff --git a/docs/parameters/ms.csv.md b/docs/parameters/ms.csv.md new file mode 100644 index 0000000..23691ab --- /dev/null +++ b/docs/parameters/ms.csv.md @@ -0,0 +1,63 @@ +# ms.csv + +**Tags**: +[extractor](categories.md#extractor-properties), + +**Type**: string + +**Format**: JsonObject + +**Default value**: {} (blank JsonObject) + +## Related + +## Description + +`ms.csv` defines csv extraction and conversion parameters. + +`ms.csv` comes as a JsonObject, and it can have any of the following +attributes: + +- **linesToSkip**, specifies how many lines of data to skip in the CSV payload. +The linesToSkip need to be more than the columnHeaderIndex. +For example, if columnHeaderIndex = 0, the number of lines to skip need to be at least 1. +When the linesToSkip is not set explicitly, and the columnHeaderIndex is set, linesToSkip = columnHeaderIndex + 1. +When neither linesToSkip and columnHeaderIndex are set, linesToSkip = 0. +If more lines need to be skipped after the header, then set this parameter explicitly. +- **columnHeaderIndex**, specifies the 0-based row index of the header columns if they are available. +The valid value range is [-1, Integer.MAX_VALUE). The default value is -1, which means no header row. +CSV files may have 1 or more descriptive lines before the actual data. These descriptive lines, +including the column header line, should be skipped. +Note the column header line can be in any place of the skipped lines. +- **escapeCharacter**, specifies how characters can be escaped. Default is "u005C" (backslash \). +This can be specified as a variation of unicode without a backslash (\) before 'u'. +For example: \ can be specified as "u005c". +- **quoteCharacter**, specifies how source data are enclosed by columns. Default is double-quote ("). +This can be specified as a variation of unicode without a backslash (\) before 'u'. +For example: | can be specified as "u007C". +- **fieldSeparator**, specifies the field delimiter in the source csv data. The default is comma. +This can be specified as a variation of unicode without a backslash (\) before 'u'. +For example: tab (\t) can be specified as "u0009". +- **recordSeparator**, also called line separator, specifies the line or record +delimiter. The default is system line separator. +This can be specified as a variation of unicode without a backslash (\) before 'u'. +- **columnProjection**, defines how CSV columns should be arranged and filtered after parse, +before being sent to converter and writer to persist. +This feature is primarily used to extract selected columns from csv source without a header. +Column projection definition is a comma-separated string, where each value is either an +integer or a range, with each number representing the 0 based index of the field. +Column projection definition is inclusive, i.e., only the selected fields are included +in the final dataset, if a column projection is defined. +For example, to include the 0th, 2nd, 3rd, and 4th column from a source that has 6 columns, +set the value to: `"columnProjection": "0,2-4"` +- **defaultFieldType**, specifies a default type to supersede field type inference. +By default, CsvExtractor tries to infer the true type of fields when inferring schema +However, in some cases, the inference is not accurate, and users may prefer to keep all fields as strings. +In this case `"defaultFieldType": "string"`. +Supported types: string | int | long | double | boolean | float. +- **maxFailures**, this is for the future CSV converter. +- **keepNullString**, this is for the future CSV converter. + +See [CsvExtractor](https://github.com/linkedin/data-integration-library/blob/master/docs/components/CsvExtractor.md) + +[back to summary](summary.md#mscsv) diff --git a/docs/parameters/ms.csv.quote.character.md b/docs/parameters/ms.csv.quote.character.md new file mode 100644 index 0000000..9ffe89c --- /dev/null +++ b/docs/parameters/ms.csv.quote.character.md @@ -0,0 +1,5 @@ +# ms.csv.quote.character + +**Deprecated, use [ms.csv](ms.csv.md)** + +[back to summary](summary.md#mscsvquotecharacter) diff --git a/docs/parameters/ms.csv.separator.md b/docs/parameters/ms.csv.separator.md new file mode 100644 index 0000000..3508681 --- /dev/null +++ b/docs/parameters/ms.csv.separator.md @@ -0,0 +1,5 @@ +# ms.csv.separator + +**Deprecated, use [ms.csv](ms.csv.md)** + +[back to summary](summary.md#mscsvseparator) diff --git a/docs/parameters/ms.csv.skip.lines.md b/docs/parameters/ms.csv.skip.lines.md new file mode 100644 index 0000000..07bd5ad --- /dev/null +++ b/docs/parameters/ms.csv.skip.lines.md @@ -0,0 +1,5 @@ +# ms.csv.skip.lines + +**Deprecated, use [ms.csv](ms.csv.md)** + +[back to summary](summary.md#mscsvskiplines) diff --git a/docs/parameters/ms.data.default.type.md b/docs/parameters/ms.data.default.type.md new file mode 100644 index 0000000..5011b9f --- /dev/null +++ b/docs/parameters/ms.data.default.type.md @@ -0,0 +1,28 @@ +# ms.data.default.type + +**Tags**: +[schema](categories.md#schema-properties) + +**Type**: string + +**Format**: A JsonObject + +**Default value**: blank + +**Related**: + +## Description + +`ms.data.default.type` provides a way to explicitly specifies data +types for certain fields. This is necessary when the source data has +empty fields, like placeholders, and DIL cannot infer its type properly. + +### Example + +In the following example, 2 fields are placeholders, and they don't have any +value. We can explicitly put a type for them. + +- `ms.data.default.type={"extension": "string", "personalMeetingUrls": "string"}` + +[back to summary](summary.md#msdatadefaulttype) + diff --git a/docs/parameters/ms.data.explicit.eof.md b/docs/parameters/ms.data.explicit.eof.md new file mode 100644 index 0000000..e0ab5d2 --- /dev/null +++ b/docs/parameters/ms.data.explicit.eof.md @@ -0,0 +1,34 @@ +# ms.data.explicit.eof + +**Tags**: +[conversion](categories.md#conversion-properties) + +**Type**: boolean + +**Format**: true/false + +**Default value**: false + +**Related**: + +## Description + +`ms.data.explicit.eof` specifies whether an explicit EOF record should +be sent to converter after processing all records. + +If enabled, DIL will send a record `{"EOF":"EOF"}` to converter signaling +no more records. + +By default, Gobblin use a null record to signaling the end of processing, +but that signal is gobbled by the execution context, and not passed to +converter at all. Therefore, converter doesn't get the chance to wrap up +things on hand. + +Knowing the end of processing is important for Normalizer Converter, so +that it can flush out the last batch of records. + +### Example + + +[back to summary](summary.md#msdataexpliciteof) + diff --git a/docs/parameters/ms.data.field.md b/docs/parameters/ms.data.field.md new file mode 100644 index 0000000..da50906 --- /dev/null +++ b/docs/parameters/ms.data.field.md @@ -0,0 +1,49 @@ +# ms.data.field + +**Tags**: +[source](categories.md#source-properties) + +**Type**: string + +**Format**: a Json path string separated by "." (dot) + +**Default value**: blank + +**Related**: + +## Description + +In a nested response, like JSON or Avro, `ms.data.field` specifies +where the core data (payload) is. + +If data.field is not specified, or is blank, then the whole +response will be treated as the payload. + +If DIL cannot find the element using the path, as specified by +`ms.data.field`, it will generate an error. + +For nested JsonObjects or Avro Records, the syntax for specifying +the path is concatenating the path segments with ".". + +If at certain level, the element is a JsonArray or Avro Array, this +function allows pick only **one** element from the array. The syntax of +picking the element is "A.n". Here element A has an array sub-element, +and we will pick the N(th) element from the array. If the array item is +an object (JsonObject or Avro Record), the nested path can go on. + +### Example + +The following picks the core data, or payload, from the `results` element. +- `ms.data.field=results` + +If the core data is in the "elements" element of +the "result" element, it can be specified as: +- `ms.data.field=result.elements` + +The following picks `partialFailureError` from the response, and +then pick `details` from it, then pick the first row in the array, +and then pick the `errors` element. +- `ms.data.field=partialFailureError.details.0.errors` + +[back to summary](summary.md#msdatafield) + diff --git a/docs/parameters/ms.derived.fields.md b/docs/parameters/ms.derived.fields.md new file mode 100644 index 0000000..b13635b --- /dev/null +++ b/docs/parameters/ms.derived.fields.md @@ -0,0 +1,93 @@ +# ms.derived.fields + +**Tags**: +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Format**: A JsonArray of JsonObjects + +**Default value**:blank + +## Related + +## Description + +Derived Fields are calculated fields that serve critical roles in data ingestion process, such as compaction. This includes, but is not +limited by, the following: + +- Convert a date string to a EPOC date value so that downstream compaction/partitioning can use the long EPOC value +- Extract a part of a field to form a primary key or delta key +- Provide a calculated value based on flow execution state or watermark, such as the often used extraction date derived field +- Lift up a nested element in the response to the toplevel and make it a toplevel field because only toplevel fields can be primary keys or delta keys +- Persist a job execution variable, such as the work unit identifier, into the final dataset + +ms.derived.fields can have an array of derived field definitions with each derived field definition being a JsonObject. + +Each derived field definition will have a "name" and a "formula". The "name" specifies the eventual field name of the derived field +in the output dataset, and the "formula" specifies how to make the derived field. + +The formula can be specified by "type", "source", and "format". + +DIL supports 6 types of derived fields: + +- `epoc`: it provides a time value field in the form of epoc (millisecond level). +- `regexp`: it provides a string value field by extracting the value from another field using a Java REGEXP pattern +- `string`: it provides a string value field by taking from another field, which can be a nested field, without transformation +- `integer`: it provides a integer value field by taking from another integer field, which can be a nested field, without transformation +- `number`: it provides a number value field by taking from another number field, which can be a nested field, without transformation +- `boolean`: it provides a boolean value field by taking from another boolean field, which can be a nested field, without transformation + +DIL supports 3 sources of derivation: +- from another field in the payload, in this case, the source can be the field name, or a JsonPath that leads to the field +- from a dynamic DIL [variable](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/variables.md) +- from predefined DIL symbols, which can be CURRENTDATE, or PxD (see [date interval](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/iso-date-interval.md)) + +When the type is `epoc`, a format is required to specify how to convert the source to the desired EPOC value. + +### Example 1: the following defines a derived field using regular expression to subtract part of a source field

    +`[{ + "name": "surveyid", + "formula": { + "type": "regexp", + "source": "survey_url", + "format": "https.*\\/surveys\\/([0-9]+)$" + } +}]` + +### Example 2: the following defines an epoc timestamp field to meet compaction requirement

    +`[{ + "name": "callDate", + "formula": { + "type": "epoc", + "source": "started", + "format": "yyyy-MM-dd" + } +}]` + +### Example 3: the following defines derived fields from variables that are generated from secondary input. + +Assuming the secondary input provides an id value that we would like to make it into the final dataset as a +permanent field. +`ms.secondary.input=[{"path": "/path/to/hdfs/inputFileDir/2019/08/07/19/", "fields": ["id", "tempId"]}]`

    + +Then the derived field can be configured as: + +`[{"name": "id", "formula": {"type": "string", "source": "{{id}}"}}]` + +### Example 4: the following defines a derived field from pagination control variables. + +Assuming the pagination is controlled by page number and page size, and we would like to persist the page size in the +final dataset. + +`ms.parameters=[{"name": "pageNo", "type": "pageno"},{"name": "size", "type": "pagesize"}]` + +Then the derived field can be defined as: + +`[{"name": "pagesize", "formula": {"type": "integer", "source": "{{pagesize}}"}}]` + +### Example 5: the following defines an epoc timestamp field based on flow execution time

    +`[{"name": "extractedDate", "formula": {"type": "epoc", "source": "CURRENTDATE"}}]` + + +[back to summary](summary.md#msderivedfields) diff --git a/docs/parameters/ms.enable.cleansing.md b/docs/parameters/ms.enable.cleansing.md new file mode 100644 index 0000000..650f57e --- /dev/null +++ b/docs/parameters/ms.enable.cleansing.md @@ -0,0 +1,41 @@ +# ms.enable.cleansing + +**Tags**: +[schema](categories.md#schema-properties) + +**Type**: boolean + +**Format**: true/false + +**Default value**: true + +**Related**: + +## Description + +Schema cleansing replaces special characters in the schema element names based +on a pattern. By default, it will replace all blank spaces, $, and @ to underscores. + +**Alert**: This feature should be used only on need basis, +for example, where source data element names are un-conforming, such as +containing spaces, and needed standardization. In large datasets cleansing +can be expensive. + +This configuration has no impact on schemas from metadata stores. + +If defined, [ms.schema.cleansing](ms.schema.cleansing.md) +supersedes `ms.enable.cleansing` +If `ms.schema.cleansing` is not defined, DIL will check `ms.enable.cleansing`. +If `ms.enable.cleansing` is true, DIL will do the [default cleansing](ms.schema.cleansing.md). + +### Statement of Direction + +`ms.enable.cleansing` will be deprecated. + +### Examples + +The following disables schema cleansing if `ms.schema.cleansing` is not defined. + +- `ms.enable.cleansing=false` + +[back to summary](summary.md#msenablecleansing) \ No newline at end of file diff --git a/docs/parameters/ms.enable.dynamic.full.load.md b/docs/parameters/ms.enable.dynamic.full.load.md new file mode 100644 index 0000000..5c4f0af --- /dev/null +++ b/docs/parameters/ms.enable.dynamic.full.load.md @@ -0,0 +1,35 @@ +# ms.enable.dynamic.full.load + +**Tags**: +[execution](categories.md#execution-properties) + +**Type**: boolean + +**Default value**: true + +**Related**: + +## Description + +`ms.enable.dynamic.full.load` enables or disables dynamic full load. +When enabled (default) and `extract.is.full = false`, DIL will dynamically +perform a full load if it is a SNAPSHOT_ONLY extract or +if there is no pre-existing watermarks of the job. + +Dynamic full load is a DIL [Single Flow](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/single-flow.md) +feature that aims to alleviate users from coding 2 separate flows, +one for the full load and one for the incremental load. + +When `extract.is.full` is true, `ms.enable.dynamic.full.load` has no use. + +To use DIL single flow architecture, set `extract.is.full` to **false** explicitly, +or not setting it and let it use the default value (false). You don't need to +set `ms.enable.dynamic.full.load`, as its default is true. + + +### Example + +`ms.enable.dynamic.full.load=true` + +[back to summary](summary.md#msenabledynamicfullload) + diff --git a/docs/parameters/ms.enable.schema.based.filtering.md b/docs/parameters/ms.enable.schema.based.filtering.md new file mode 100644 index 0000000..efec5f4 --- /dev/null +++ b/docs/parameters/ms.enable.schema.based.filtering.md @@ -0,0 +1,29 @@ +# ms.enable.schema.based.filtering + +**Tags**: +[conversion](categories.md#conversion-properties) + +**Type**: boolean + +**Default value**: true + +**Related**: + +## Description + +`ms.enable.schema.based.filtering` enables or disables schema-based filtering, +or column projection. When enabled, only fields specified schema +are projected to final dataset. + +Each Extractor will enforce a compliance filter based on given schema, +currently this is soft enforced. Use case can turn the filtering off by +setting this parameter to false. + +Whe a flow uses a normalizer converter, this generally should be disabled. + +### Example + +`ms.enable.schema.based.filtering=false` + +[back to summary](summary.md#msenableschemabasedfiltering) + diff --git a/docs/parameters/ms.encryption.fields.md b/docs/parameters/ms.encryption.fields.md new file mode 100644 index 0000000..9d72f7a --- /dev/null +++ b/docs/parameters/ms.encryption.fields.md @@ -0,0 +1,37 @@ +# ms.encryption.fields + +**Tags**: +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Format**: JsonArray + +**Default value**: blank + +**Related**: + +## Description + +`ms.encryption.fields` specifies a list of fields to be encrypted before +they are passed to converters. + +In this job property you can specify the fields (array of fields) +that needs to be encrypted by the Gobblin Encryption utility. + +These fields can be of JsonPrimitive type (string/int/boolean/etc.) or +JsonObject type (with nested structure). For the later, the field +name can be Json Path. + +Fields to be encrypted cannot have null values, and the data type of +the fields cannot be nullable (need to have isNullable: false). When +[ms.output.schema](ms.output.schema.md) is defined, this will be +checked against the definition in it. + +### Example + +`ms.encryption.fields=["access_token", "client_secret", "refresh_token"]` +`ms.encryption.fields=["emailAddress", "settings.publicKey"]` + +[back to summary](summary.md#msencryptionfields) + diff --git a/docs/parameters/ms.extract.preprocessor.parameters.md b/docs/parameters/ms.extract.preprocessor.parameters.md new file mode 100644 index 0000000..8e48d96 --- /dev/null +++ b/docs/parameters/ms.extract.preprocessor.parameters.md @@ -0,0 +1,31 @@ +# ms.extract.preprocessor.parameters + +**Tags**: +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Format**: A JsonObject + +**Related**: +- [job property: ms.extract.preprocessors](ms.extract.preprocessors.md) + +## Description + +When a source file is encrypted, it requires credentials to decrypt. +`ms.extract.preprocessor.parameters` defines parameters to pass into the +preprocessor along with the input. + +For GPG based decryption/encryption, parameters needed are: +- "**action**" : string, decrypt/encrypt +- "**keystore_password**" : string, some password, +- "**keystore_path**" : string, path to the secret keyring, +- "**cipher**" : string, optional, cipher algorithm to use, default to CAST5 (128 bit key, as per RFC 2144) +- "**key_name**" : string, optional, the key id, a long value, of the public Gpg key as a Hex string + +### Example + +The following provides key and password to GPG decryption: +- `ms.extract.preprocessor.parameters={"com.linkedin.cdi.preprocessor.GpgDecryptProcessor": {"keystore_path" :"/some path/secret.gpg", "keystore_password" : "ENC(some password)"}}` + +[back to summary](summary.md#msextractpreprocessorparameters) diff --git a/docs/parameters/ms.extract.preprocessors.md b/docs/parameters/ms.extract.preprocessors.md new file mode 100644 index 0000000..bb618b0 --- /dev/null +++ b/docs/parameters/ms.extract.preprocessors.md @@ -0,0 +1,37 @@ +# ms.extract.preprocessors + +**Tags**: +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Default value**: blank + +**Related**: +- [job property: ms.extract.preprocessor.parameters](ms.extract.preprocessor.parameters.md) + +## Description + +`ms.extract.preprocessors` define one or more preprocessor classes that +handles the incoming data before they can be processed by the extractor. + +When input data is compressed or encrypted, the input stream needs to +be preprocessed before it can be passed to an DIL extractor to parse. + +`ms.extract.preprocessors` is a comma delimited string if there are +more than 1 preprocessors. + +Currently, DIL can consume GZIP'ed and/or GPG encrypted data. + +### Example + +The following define preprocessors for GPG encrypted, GZIP compressed, and +compressed and then encrypted data. + +- `ms.extract.preprocessors=com.linkedin.cdi.preprocessor.GpgDecryptProcessor` +- `ms.extract.preprocessors=com.linkedin.cdi.preprocessor.GunzipProcessor` +- `ms.extract.preprocessors=com.linkedin.cdi.preprocessor.GpgProcessor,com.linkedin.cdi.preprocessor.GunzipProcessor` + + + +[back to summary](summary.md#msextractpreprocessors) diff --git a/docs/parameters/ms.extractor.class.md b/docs/parameters/ms.extractor.class.md new file mode 100644 index 0000000..0ac2b1e --- /dev/null +++ b/docs/parameters/ms.extractor.class.md @@ -0,0 +1,32 @@ +# ms.extractor.class + +**Tags**: +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Default value**: blank + +**Related**: +- [AvroExtractor](https://github.com/linkedin/data-integration-library/blob/master/docs/components/AvroExtractor.md) +- [JsonExtractor](https://github.com/linkedin/data-integration-library/blob/master/docs/components/JsonExtractor.md) +- [CsvExtractor](https://github.com/linkedin/data-integration-library/blob/master/docs/components/CsvExtractor.md) +- [FileDumpExtractor](https://github.com/linkedin/data-integration-library/blob/master/docs/components/FileDumpExtractor.md) + +## Description + +`ms.extractor.class` specifies the extractor class to use for data parsing. +The choice of extractor is based data format. Currently, DIL designed 4 +classes of extractors. + +- **AvroExtractor**: if the incoming data is Avro format +- **CsvExtractor**: if the incoming data is Csv format +- **JsonExtractor**: if the incoming data is Json format +- **FileDumpExtractor**: if the incoming data is to be dumped to file system without +going through converters and writers. In this case, the incoming data can be any format. + +### Example + +`ms.extractor.class=com.linkedin.cdi.extractor.AvroExtractor` + +[back to summary](summary.md#msextractorclass) diff --git a/docs/parameters/ms.extractor.target.file.name.md b/docs/parameters/ms.extractor.target.file.name.md new file mode 100644 index 0000000..1236239 --- /dev/null +++ b/docs/parameters/ms.extractor.target.file.name.md @@ -0,0 +1,29 @@ +# ms.extractor.target.file.name + +**Tags**: +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Default value**: blank + +**Accepts DIL Variables**: yes + +**Related**: +- [ms.extractor.target.file.permission](ms.extractor.target.file.permission.md) + +## Description + +`ms.extractor.target.file.name` specify the file name when +FileDumpExtractor is used. The file name can be specified as a +string container DIL variables. + +[FileDumpExtractor](https://github.com/linkedin/data-integration-library/blob/master/docs/components/FileDumpExtractor.md) +extractor will dump the InputStream directly to HDFS as a file +without going through converters or writers. + +### Example + +`ms.extractor.target.file.name={{s3key}}` + +[back to summary](summary.md#msextractortargetfilename) diff --git a/docs/parameters/ms.extractor.target.file.permission.md b/docs/parameters/ms.extractor.target.file.permission.md new file mode 100644 index 0000000..a2dadb9 --- /dev/null +++ b/docs/parameters/ms.extractor.target.file.permission.md @@ -0,0 +1,26 @@ +# ms.extractor.target.file.permission + +**Tags**: +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Default value**: "755" + +**Related**: +- [ms.extractor.target.file.name](ms.extractor.target.file.name.md) + +## Description + +`ms.extractor.target.file.permission` set file permission when +FileDumpExtractor is used. + +[FileDumpExtractor](https://github.com/linkedin/data-integration-library/blob/master/docs/components/FileDumpExtractor.md) +extractor will dump the InputStream directly to HDFS as a file +without going through converters or writers. + +### Example + +`ms.extractor.target.file.permission=750` + +[back to summary](summary.md#msextractortargetfilepermission) diff --git a/docs/parameters/ms.grace.period.days.md b/docs/parameters/ms.grace.period.days.md new file mode 100644 index 0000000..564c2fa --- /dev/null +++ b/docs/parameters/ms.grace.period.days.md @@ -0,0 +1,57 @@ +# ms.grace.period.days + +**Tags**: +[watermark & work unit](categories.md#watermark-work-unit-properties) + +**Type**: integer + +**Default value**: 0 + +**Related**: +- [job property: ms.abstinent.period.days](ms.abstinent.period.days.md) + +## Description + +`ms.grace.period.days` addresses the late arrival problem, which is +very common if the ingestion source is a data warehouse. + +`ms.grace.period.days` defines a Grace Period for incremental extraction, +and it adds extra buffer to cutoff timestamp during the +incremental load so that more data can be included. + +Grace Period is for overlapped data extraction, it assumes that the source can +have late arrivals, which are older data that showed up in source +after last extract. For example, a record was modified 2 days ago, +but did not show up in source until today. In such case, if we extract based on +the record's `last update date`, the last extraction would have missed that record, +and today's extraction will again miss it if we cut off by last +extraction time (yesterday). + +A grace period is thus subtracted from the cutoff timestamp, allowing us +move the cutoff time backward by the Grace Period, allowing late arrivals +be captured in incremental extraction. + +`ms.http.grace.period.days` itself alone doesn't change extraction logic, +it only changes the work unit watermark. Job configuration need to +use the watermark derived variables in requests properly in order achieve the +goal of reliable incremental extraction. + +### Example + +If we set `ms.http.grace.period.days=2`, and we run the data ingestion job +daily, after last extraction yesterday, the cutoff time would be last +ETL time without grace period. With grace period, the cutoff time becomes +`last ETL time - 2 days`. + +That means the work unit watermark becomes [`last ETL time - 2 days`, `current time`). + +If a variable "fromTime" is defined on the watermark (see [ms.parameters](ms.parameters.md)), +it would have the value `last ETL time - 2 days`. + +If the variable is used in request like `http://domain/path?updateDate>={{fromTime}}`, +then the extraction will include data that was updated since `last ETL time - 2 days`. + +With grace period, the extraction would only include data that was updated +since `last ETL time`, and it would potentially miss late arrivals. + +[back to summary](summary.md#msgraceperioddays) diff --git a/docs/parameters/ms.http.conn.max.md b/docs/parameters/ms.http.conn.max.md new file mode 100644 index 0000000..0b220f7 --- /dev/null +++ b/docs/parameters/ms.http.conn.max.md @@ -0,0 +1,31 @@ +# ms.http.conn.max + +**Tags**: +[http](categories.md#http-properties) + +**Type**: Integer + +**Default value**: 50, 0 will be translated to the default value + +**Maximum value**: 500 + +**Related**: +- [ms.http.conn.per.route.max](ms.http.conn.per.route.max.md) + + +## Description + +`ms.http.conn.max` defines maximum number of connections to keep +in a connection pool. It limits the total connections to an HTTP +server. + +In a MAPREDUCE mode, because each mapper runs on a separate container, +this parameter can only limit the total connections from each container; +therefore, the total connections to HTTP server, from all mappers, +can be more than this value. + +In actual execution, DIL closes the HTTP connection once a data stream +is processed; hence we should not expect very high number of connections +to the server. + +[back to summary](summary.md#mshttpconnmax) diff --git a/docs/parameters/ms.http.conn.per.route.max.md b/docs/parameters/ms.http.conn.per.route.max.md new file mode 100644 index 0000000..b2382a2 --- /dev/null +++ b/docs/parameters/ms.http.conn.per.route.max.md @@ -0,0 +1,31 @@ +# ms.http.conn.per.route.max + +**Tags**: +[http](categories.md#http-properties) + +**Type**: Integer + +**Default value**: 20, 0 will be translated to the default value + +**Maximum value**: 200 + +**Related**: +- [ms.http.conn.max](ms.http.conn.max.md) + + +## Description + +`ms.http.conn.per.route.max` defines maximum number of connections to keep +in a connection pool. It limits the total connections to a particular +path, or endpoint, on the HTTP server. + +In a MAPREDUCE mode, because each mapper runs on a separate container, +this parameter can only limit the total connections from each container; +therefore, the total connections to HTTP server, from all mappers, +can be more than this value. + +In actual execution, DIL closes the HTTP connection once a data stream +is processed; hence we should not expect very high number of connections +to the server. + +[back to summary](summary.md#mshttpconnperroutemax) diff --git a/docs/parameters/ms.http.conn.ttl.seconds.md b/docs/parameters/ms.http.conn.ttl.seconds.md new file mode 100644 index 0000000..a8d58c5 --- /dev/null +++ b/docs/parameters/ms.http.conn.ttl.seconds.md @@ -0,0 +1,21 @@ +# ms.http.conn.ttl.seconds + +**Tags**: +[http](categories.md#http-properties) + +**Type**: Integer + +**Default value**: 10 + +**Related**: +- [ms.http.conn.max](ms.http.conn.max.md) +- [ms.http.conn.per.route.max](ms.http.conn.per.route.max.md) + + +## Description + +`ms.http.conn.ttl.seconds` defines maximum idle time allowed when there +is no activity on an HTTP connection. When there is no activity after +TTL passed, the connection is disconnected. + +[back to summary](summary.md#mshttpconnmax) diff --git a/docs/parameters/ms.http.request.headers.md b/docs/parameters/ms.http.request.headers.md new file mode 100644 index 0000000..f57488f --- /dev/null +++ b/docs/parameters/ms.http.request.headers.md @@ -0,0 +1,36 @@ +# ms.http.request.headers + +**Tags**: +[http](categories.md#http-properties), +[authentication](categories.md#authentication-properties) + +**Type**: string + +**Format**: A JsonObject + +**Default value**: "{}" (a blank JsonObject) + +**Related**: +- [ms.authentication](ms.authentication.md) +- [ms.secondary.input](ms.secondary.input.md) +- [ms.parameters](ms.parameters.md) +- source.conn.username +- source.conn.password + +## Description + +`ms.http.request.headers` specifies custom headers including Content-Type that are to be +included in HTTP requests. + +### Examples + +The following define request content type: + +- `ms.http.request.headers={"Content-Type": "application/json"}` +- `ms.http.request.headers={"Content-Type": "application/x-www-form-urlencoded"}` + +The following defines request content type and other headers: + +- `ms.http.request.headers={"Content-Type": "application/json", "developer-token": "...", "login-customer-id":"..."}` + +[back to summary](summary.md#mshttprequestheaders) diff --git a/docs/parameters/ms.http.request.method.md b/docs/parameters/ms.http.request.method.md new file mode 100644 index 0000000..259c7c2 --- /dev/null +++ b/docs/parameters/ms.http.request.method.md @@ -0,0 +1,26 @@ +# ms.http.request.method + +**Tags**: +[http](categories.md#http-properties) + +**Type**: string + +**Format**: one of the values of get, put, post, and delete + +**Default value**: blank (equivalent to GET) + +## Related + +## Description + +The expected HTTP method to send the requests, decided by the API. + +- GET: all parameters are specified in the URL as URL parameters +- POST: parameters are sent to API in request body +- PUT: parameters are sent to API in request body +- DELETE: parameters are sent to API in request body + +**Note**: URL parameters are URL encoded. + +[back to summary](summary.md#mshttprequestmethod) + diff --git a/docs/parameters/ms.http.response.type.md b/docs/parameters/ms.http.response.type.md new file mode 100644 index 0000000..a96b2a1 --- /dev/null +++ b/docs/parameters/ms.http.response.type.md @@ -0,0 +1,31 @@ +# ms.http.response.type + +**Tags**: +[http](categories.md#http-properties) + +**Type**: string + +**Format**: A JsonObject + +**Default value**: "{}" (a blank JsonObject) + +**Related**: + +## Description + +`ms.http.response.type` specifies less common response types in addition to + the default ones "application/json" or "text/csv". + +If you have a custom response type other than "application/json" or "text/csv", +you can configure expected response using this parameter. + +When `ms.http.response.type` is not configured, the default +for CsvExtractor is "text/csv", and for JsonExtractor, it is "application/json". + +### Example + +The following accepts "application/x-gzip" in addition to "text/csv" +in a CsvExtractor +- `ms.http.response.type={"Content-Type":"application/x-gzip"}` + +[back to summary](summary.md#mshttpresponsetype) diff --git a/docs/parameters/ms.http.statuses.md b/docs/parameters/ms.http.statuses.md new file mode 100644 index 0000000..8c446fb --- /dev/null +++ b/docs/parameters/ms.http.statuses.md @@ -0,0 +1,44 @@ +# ms.http.statuses + +**Tags**: +[http](categories.md#http-properties) + +**Type**: string + +**Format**: A JsonObject + +**Default value**: `{"success":[200,201,202], "pagination_error":[401]}"` + +**Related**: + +## Description + +`ms.http.statuses` defines status codes that should be treated as success, +warning, or error. + +It is a JsonObject with the following attributes: + +- **success**: a list of status codes that should be deemed as successful +- **warning**: a list of status codes that should be deemed as successful with warning +- **error**: a list of status codes that should be deemed as errors +- **pagination_error**: a list of status codes that should be deemed as transient errors, +and requires the DIL to refresh authentication token in next page + +By default, if this parameter is not set, 200 (OK), 201 (CREATED), and 202 (ACCEPTED) +will be treated as success; anything else below 400 will be treated as warning; and +anything 400 and above will be treated as error. Warnings will be logged but will not +cause job failure. Errors will cause job failure. + +In cases where 4xx codes, like 404 (NOT FOUND), happened frequently, and a failure is +not desirable, exceptions can be reclassified as `warning`. +For example, if 404 is to be treated as `warning` instead of an `error`, it +can be configured as +- `ms.http.statuses={"success": [200, 201, 202], "warning": [404]}` + +### Example + +In following configuration, we make 404 an warning, and make 206 a failure indicating +that partial content is not acceptable: +- `ms.http.statuses={"success": [200], "warning": [404], "error": [206]}` + +[back to summary](summary.md#mshttpstatuses) diff --git a/docs/parameters/ms.jdbc.schema.refactor.md b/docs/parameters/ms.jdbc.schema.refactor.md new file mode 100644 index 0000000..e549167 --- /dev/null +++ b/docs/parameters/ms.jdbc.schema.refactor.md @@ -0,0 +1,25 @@ +# ms.jdbc.schema.refactor + +**Tags**: +[schema](categories.md#schema-properties) + +**Type**: string + +**Format**: one of the values: toupper, tolower, or none + +**Default value**: none + +**Related**: + +## Description + +`ms.jdbc.schema.refactor` specifies the function to apply to JDBC schema. +The choices are `toupper`, `tolower`, or `none` + +### Examples + +The following is a sample of MySQL SELECT statement. + +- `ms.jdbc.statement=select * from ${source.entity} limit {{limit}} offset {{offset}}` + +[back to summary](summary.md#msjdbcschemarefactor) \ No newline at end of file diff --git a/docs/parameters/ms.jdbc.statement.md b/docs/parameters/ms.jdbc.statement.md new file mode 100644 index 0000000..2cd1002 --- /dev/null +++ b/docs/parameters/ms.jdbc.statement.md @@ -0,0 +1,26 @@ +# ms.jdbc.statement + +**Tags**: +[source](categories.md#source-properties) + +**Type**: string + +**Default value**: blank + +**Related**: + +## Description + +`ms.jdbc.statement` specifies the SQL statement for data retrieval. The value +can be any validate statement on any JDBC source. + +DIL doesn't explicitly restrict or support syntax of the statement. +The source database decides whether to accept or fail the statement. + +### Examples + +The following is a sample of MySQL SELECT statement. + +- `ms.jdbc.statement=select * from ${source.entity} limit {{limit}} offset {{offset}}` + +[back to summary](summary.md#msjdbcstatement) \ No newline at end of file diff --git a/docs/parameters/ms.kafka.audit.topic.name.md b/docs/parameters/ms.kafka.audit.topic.name.md new file mode 100644 index 0000000..f1a9056 --- /dev/null +++ b/docs/parameters/ms.kafka.audit.topic.name.md @@ -0,0 +1,20 @@ +# ms.kafka.audit.topic.name + +**Tags**: +[auditing](categories.md#auditing-properties) + +**Type**: string + +**Default value**: blank + +**Related**: +- [ms.kafka.brokers](ms.kafka.brokers.md) +- [ms.kafka.clientId](ms.kafka.clientId.md) +- [ms.kafka.schema.registry.url](ms.kafka.schema.registry.url.md) + +## Description + +`ms.kafka.audit.topic.name` specifies the auditing topic name, where +DIL wil send auditing events to if auditing is enabled. + +[back to summary](summary.md#mskafkaaudittopicname) \ No newline at end of file diff --git a/docs/parameters/ms.kafka.brokers.md b/docs/parameters/ms.kafka.brokers.md new file mode 100644 index 0000000..b982f4b --- /dev/null +++ b/docs/parameters/ms.kafka.brokers.md @@ -0,0 +1,24 @@ +# ms.kafka.brokers + +**Tags**: +[auditing](categories.md#auditing-properties) + +**Type**: string + +**Default value**: blank + +**Related**: +- [ms.kafka.clientId](ms.kafka.clientId.md) +- [ms.kafka.schema.registry.url](ms.kafka.schema.registry.url.md) +- [ms.kafka.audit.topic.name](ms.kafka.audit.topic.name.md) + +## Description + +`ms.kafka.brokers` specifies the Kafka broker host, such as `kafka.corp.com:1234`. + +**Note**: This is not fully ready for open source use. + + +[back to summary](summary.md#mskafkabroker) + + \ No newline at end of file diff --git a/docs/parameters/ms.kafka.clientId.md b/docs/parameters/ms.kafka.clientId.md new file mode 100644 index 0000000..aa3f5f6 --- /dev/null +++ b/docs/parameters/ms.kafka.clientId.md @@ -0,0 +1,20 @@ +# ms.kafka.clientId + +**Tags**: [auditing](categories.md#auditing-properties) + +**Type**: string + +**Default value**: blank + +**Related**: +- [ms.kafka.brokers](ms.kafka.brokers.md) +- [ms.kafka.schema.registry.url](ms.kafka.schema.registry.url.md) +- [ms.kafka.audit.topic.name](ms.kafka.audit.topic.name.md) + +## Description + +`ms.kafka.clientId` specifies the Kafka client id, such as `dil-audit`. + +**Note**: This is not fully ready for open source use. + +[back to summary](summary.md#mskafkaclientid) \ No newline at end of file diff --git a/docs/parameters/ms.kafka.schema.registry.url.md b/docs/parameters/ms.kafka.schema.registry.url.md new file mode 100644 index 0000000..3829bc4 --- /dev/null +++ b/docs/parameters/ms.kafka.schema.registry.url.md @@ -0,0 +1,18 @@ +# ms.kafka.schema.registry.url + +**Tags**: [auditing](categories.md#auditing-properties) + +**Type**: string + +**Default value**: blank + +**Related**: +- [ms.kafka.brokers](ms.kafka.brokers.md) +- [ms.kafka.clientId](ms.kafka.clientId.md) +- [ms.kafka.audit.topic.name](ms.kafka.audit.topic.name.md) + +## Description + +`ms.kafka.schema.registry.url` specifies the auditing schema registry URL. + +[back to summary](summary.md#mskafkaschemaregistryurl) \ No newline at end of file diff --git a/docs/parameters/ms.normalizer.batch.size.md b/docs/parameters/ms.normalizer.batch.size.md new file mode 100644 index 0000000..72c644d --- /dev/null +++ b/docs/parameters/ms.normalizer.batch.size.md @@ -0,0 +1,64 @@ +# ms.normalizer.batch.size + +**Tags**: +[schema](categories.md#schema-properties), +[conversion](categories.md#conversion-properties) + +**Type**: integer + +**Default value**: 500 + +**Minimum value**: 1 + +**Related**: + +- [job property: ms.target.schema](ms.target.schema.md) +- [job property: ms.target.schema.urn](ms.target.schema.urn.md) +- [job property: ms.source.schema.urn](ms.source.schema.urn.md) + +## Description + +`ms.normalizer.batch.size` specifies the batch size for the normalizer converter +to group rows. + +[NormalizerConverter](https://github.com/linkedin/data-integration-library/blob/master/docs/components/normalizer-converter.md), +including JsonNormalizerConverter and AvroNormalizerConvert, can group multiple columns into a single column, and group +multiple rows to a single record. It compare the source schema with the target schema, +and put all columns in the source but not in the target into the first column that +is in the target but not in the source. + +Assuming the source schema has A,**B,C,D** 4 columns, and the target has A, **X** 2 columns, +and `ms.normalizer.batch.size=100`, then **B,C,D** are in the source but not in the target, +and **X** is in the target but not in the source. Therefore, for each record, B, C, D will be grouped +into a sub-record of a "map" type, and for each 100 records, the sub-records will +be formed into a new column "X" of "**array**" type. + +Assuming the source schema has A,**B,C,D** 4 columns, and the target has A, **X** 2 columns, +and **`ms.normalizer.batch.size=1`**, then **B,C,D** are in the source but not in the target, +and **X** is in the target but not in the source. Therefore, for each record, B, C, D will be grouped +into the new column "X" of "**map**" type. + +Per above rule, setting `ms.normalizer.batch.size` to 1 has special effects of condensing a sparse +table. Assuming we have sparse table of 1 fixed column A, and 100s of sparse +columns (c1, c2 .... c300) +So some records may values in column A a few columns (c1, c2, ...), and other records may +have values in column A and a few other columns (c20, c21, ...). In such case, +the normalizer converter will put all sparse columns into a map field of key value +pairs, and only columns with values will be included in the map field. + +### Example: surveys + +For example, surveys may have a few fixed columns, and many question columns. But each +survey may have a different set of questions. Therefore in order to store the surveys +in one table, one option would be putting all question-answser pairs into a +response column. So the input data could be like: +- {"survey": "A", "q1":"...", "q2":"...", "q3":"...", ...} +- {"survey": "B", "qa":"...", "qb":"...", "qc":"...", ...} + +With `ms.normalizer.batch.size=1`, and target schema like: {survey, responses} + +The output data could be like +- {"survey": "A", "responses": {"q1":"...", "q2":"...", "q3":"...", ...}} +- {"survey": "B", "responses": {"qa":"...", "qb":"...", "qc":"...", ...}} + +[back to summary](summary.md#msnormalizerbatchsize) \ No newline at end of file diff --git a/docs/parameters/ms.output.schema.md b/docs/parameters/ms.output.schema.md new file mode 100644 index 0000000..0146685 --- /dev/null +++ b/docs/parameters/ms.output.schema.md @@ -0,0 +1,29 @@ +# ms.output.schema + +**Tags**: +[schema](categories.md#schema-properties), +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Format**: A JsonArray + +**Default value**: blank + +**Related**: +- [key concept: schema](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/schema.md) + +## Description + +`ms.output.schema` defines the output schema of extractors. Therefore, +it is also the input schema of the first converter. + +You can leave this as blank, run the job, and DIL will automatically +infer the schema, and output in the log. +Then, you can copy and paste the schema(Avro-flavor schema) as the value here. + +### Examples + +`ms.output.schema=[{"columnName":"path","isNullable":"false","dataType":{"type":"string"}}]` + +[back to summary](summary.md#msoutputschema) \ No newline at end of file diff --git a/docs/parameters/ms.pagination.md b/docs/parameters/ms.pagination.md new file mode 100644 index 0000000..b819965 --- /dev/null +++ b/docs/parameters/ms.pagination.md @@ -0,0 +1,87 @@ +# ms.pagination + +**Tags**: +[pagination](categories.md#pagination-properties) + +**Type**: string + +**Format**: A JsonObject + +**Default value**: "{}" (a blank JsonObject) + +**Related**: +- [key concept: pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/pagination.md) +- [ms.session.key.field](ms.session.key.field.md) +- [ms.total.count.field](ms.total.count.field.md) + +## Description + +`ms.pagination` defines key pagination attributes. +`ms.pagination` is a JsonObject with 2 members: + +- **fields**: is an array of up to 3 string elements, each denote a source key column for: + 1. page start, or offset + 2. page size, or limit of each page + 3. page no, if page no is used to control instead of using page start and page size +- **initialvalues**: is an array of up to 3 integers, each denote an initial value for: + 1. page start, or offset + 2. pagesize, or limit of each page + 3. page no, if page no is used to control instead of using page start and page size + +The field names defined by `fields` are element names in the response +from data source. + +`fields` are optional. If not defined, but initial offset and page size are given, DIL will automatically +increase page number by 1 and increase offset by page size. DIL will auto-increase +page number by 1 for each page whenever pagination is enabled. + +Pagination is enabled whenever `fields` or `initialvalues` are provided. +If none of them are provided, pagination is disabled. + +Variables can be defined using the special pagination types. +See [ms.parameters](ms.parameters.md). + +### Example 1 + +Say your source doesn't provide offset, but provides the size of +each page under key "page_size", and the number of total pages under +key "page_number". You can set page start to 0 since offset is not +being used. And you can set page_size to be the value you want, and +set page_number to 1. The page_number will auto-increment by 1 with +each subsequent request sent to the data source. + +The job configuration is: + +- Specify that we will read "page_size" and "page_number" from the response +`ms.pagination={"fields": ["", "page_size", "page_number"], "initialvalues": [0, 100, 1]}` +- Define variables `size` and `pageNo` +`ms.parameter=[{"name": "pageNo", "type": "pageno"},{"name": "size", "type": "pagesize"}]` +- Specify that we will use variable `size` and `pageNo` in URL +`ms.source.uri=https://api.abc.com/q?page_size={{size}}&page_number={{pageNo}}` + +The first request URL becomes: +`https://api.abc.com/q?page_size=100&page_number=1` + +The second request URL becomes: +`https://api.abc.com/q?page_size=100&page_number=2` + +### Example 2 + +Say your source doesn't provide page numbers, but accepts pagination +by updating the offset. + +The job configuration is: +- Specify that we will read "offset" and "limit" from the response +`ms.pagination={"fields": ["offset", "limit"], "initialvalues": [0, 5000]}` +- Define variables `offset` and `limit` +`ms.parameter=[{"name": "offset", "type": "pagestart"}, {"name": "limit", "type": "pagesize"}]` +- Specify that we will use variables `offset` and `limit` in URL +`ms.source.uri=https://api.abc.com/q?offset={{offset}}&limit={{limit}}` + +And the first request URL becomes: +`https://api.abc.com/q?offset=0&limit=5000` + +And the second request will auto increment the offset, and the URL becomes: +`https://api.abc.com/q?offset=5000&limit=5000` + +[back to summary](summary.md#mspagination) \ No newline at end of file diff --git a/docs/parameters/ms.parameters.md b/docs/parameters/ms.parameters.md new file mode 100644 index 0000000..3174b57 --- /dev/null +++ b/docs/parameters/ms.parameters.md @@ -0,0 +1,171 @@ +# ms.parameters + +**Tags**: +[source](categories.md#source-properties) +[authentication](categories.md#authentication-properties) + +**Type**: string + +**Format**: A JsonArray of JsonObjects + +**Default value**: "[]" (a blank JsonArray) + +## Related +- [key concept: variables](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/variables.md) +- [key concept: watermark](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/watermark.md) +- [key concept: pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/pagination.md) +- [job property: ms.session.key.field](ms.session.key.field.md) +- [job property: ms.watermark](ms.watermark.md) +- [job property: ms.pagination](ms.pagination.md) + +## Description + +ms.parameter defines a list of **named parameters**, +which are also [**variables**](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/variables.md) +that can be referenced in other configuration properties +using the syntax of double brackets **{{variableName}}**. + +Parameters are special variables that are used to execute requests (Http request, JDBC request, and Sftp request, etc). +Other variables are just kept internally, they are not used to execute requests. + +Parameters can have either static values or dynamically derived +values using a formula. + +Parameters can recursively use other defined variables in definition, +but the recursion level shall not be more than 1. + +Parameters will have values in string format. Presently other formats, +like integer, are not considered, instead, integer values +will be carried as strings. + +### Types of Parameters + +The following types of parameters are designed: + +- **list**: list is the default type, which means a primitive string. + +- **object**: a `object` parameter contains recursively defined parameters. + +- **watermark**, a `watermark` parameter derives its value from [watermarks](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/watermark.md). +Because watermarks are work unit specific, so parameters defined as `watermark` type +would have different values for each work unit. When define a parameter off a +time watermark, the value can be "low" or "high", which means using +low watermark or high watermark values, and the format can be "**datetime**", +"**epoc-second**", or **unspecified**. If format is not specified, it will be +**epoch millisecond**. That's the default format. When the format is "**datetime**", +a pattern can specified for the output. The pattern is normally ISO +Java date time string. Timezone is supported. Milli-seconds are supported. +Micro-seconds are not supported, but you can format the milli-second +in 6 digits, i.e., the last 3 digits will be just 0s. + +- **session**, a `session` parameter derives its value from the +[session control](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/session-control.md) variable. + +- **pagestart**, a `pagestart` parameter derives its value from the starting row +number of next page during the pagination process. +The page-start parameter can be named differently in each application. +Examples are "offset", "start", "from" etc. + +- **pagesize**, a `pagesize` parameter derives its value from the given +pagination size during the pagination process. +The page-size parameter can be named differently in each application. +Examples are "limit", "size" etc. + +- **pageno**, a `pageno` parameter derives its value from the page number of +next page during the pagination process. +The page-no variable can be named differently in each application. Examples are "next", "page_number" etc. + +- **jsonarray**, a `jsonarray` parameter defines are raw JsonArray. A JsonArray +parameter can take a very complex JsonArray object, and it doesn't require +each element be recursively defined, like what `object` type variables do. +For example: `ms.parameters=[{'name':'groupBys','type':'jsonarray','value':[{'heading':'CASE_ID','dimensionName':'CASE_ID','groupType':'FIELD','details':{}}, {'heading':'CASE','dimensionName':'CASE','groupType':'FIELD','details':{}}]` + +- **jsonobject**, a `jsonobject` parameter defines are raw JsonObject. +A JsonObject parameter can take a very complex JsonObject object, +and it doesn't require each element be recursively defined, +like what `object` type parameter does. For example: +`ms.parameters=[{"name":"token","type":"JSONOBJECT", "value": {"client_id": "999999", "scopes": ["read", "write"]}}]` +**Note** the value of "value" element is not quoted, otherwise, it will be +treated as a primitive string. + +### Sending Dynamic Requests + +**Terminology**: in following description, we call parameters used in URI as URI Parameters. + +_The following is explained using HTTP requests as example, the variable substitution mechanism works +for JDBC and S3._ + +For HTTP requests, variables will be used to form the final URI. In such case, the variable can be +used in the URI path segments or as URI parameter. + +- When used in URI path, the variable name need to be specified in [URI template](ms.source.uri.md) +as a variable contained in {{}}. + +- When used as URI parameters, the variable name can be specified in URI template. If not used in the URI template, +the variable name and its derived value will be coded as KV pairs, then these KV pair will be sent to the +data source as request parameters; therefore, the variable name need to be acceptable to source. Keep in mind, some data sources will +reject unknown request parameters. This is particularly important for HTTP POST and HTTP PUT requests, +the variable name will be used as-is in the form of "variable name": "variable value" in the request body; +therefore, the variable name have to match URI source expectation. + +For example, if a source accepts URI like http://domainname/endpoint?cursor=xxxx, and the "cursor" parameter is optional, +then the variable should be named as "cursor", and the URI template should be set as http://domainname/endpoint in pull file. +In such case, a "?cursor=xxx" will be appended to the final URI when cursor is present, otherwise, no URI parameter is added +to the URI. This is important because some data sources reject any session control request parameters in the first request. + +However, if the cursor URI parameter is not optional, the URI template could be coded as http://domain/endpoint?cursor={{p1}}, +then the variable can be named as "p1", and the variable value will replace {{p1}} before the request is sent to the URI source. + +## Examples 1: setting pagination in URI + +In this example, the URI needs 3 mandatory variables, and they can be named as p1, p2, and p3. +We can configure the parameters as following: + +`ms.source.uri=https://domain.com/api/bulk/2.0/syncs/{{p1}}/data?offset={{p2}}&limit={{p3}}`

    + +`ms.parameter=[ + {"name": "p1", "value": "3837498"}, + {"name": "p2", "type": "pagestart"}, + {"name": "p3", "type": "pagesize"}]`

    + +## Example 2: setting session control + +In this example, the URI needs 1 optional variable, and the parameter has to be named as +required by the source. Here is the configuration: + +`ms.source.uri=https://domain.com/users`

    + +`ms.parameter=[{"name":"cursor","type":"session"}]` + +The first request will be sent to URL `https://domain.com/users`. Assume the response from the +first request is `{"cursor" : "582939"}`, then the second request will be sent +to URL `https://domain.com/users?cursor=582939` + + +## Example 3: setting date range in request + +In this example, we want a date range to be passed to the request as query parameters, and we want +each work unit to work on a different portion of the data range. + +First, we partition the time watermark to daily work units. + +`ms.work.unit.partition=daily` + +Then, we define variables in ms.parameter like this: + +`ms.parameter= +[{"name":"fromDate","type":"watermark","watermark":"system","value":"low","format":"datetime","pattern":"yyyy-MM-dd"}, +{"name":"toDate","type":"watermark","watermark":"system","value":"high","format":"datetime","pattern":"yyyy-MM-dd"}] +` +That will make sure each work unit has a "fromDate" and a "toDate" variable, and their values are assigned +based on the date range that the work unit processes. + +Then we define source endpoint looks like this: + +`ms.source.uri= +https://api.zoom.us/v2/metrics/webinars?type=past&from={{fromDate}}&to={{toDate}} +` + +In execution time, each day range will be processed by a work unit. + +[back to summary](summary.md#msparameters) diff --git a/docs/parameters/ms.s3.list.max.keys.md b/docs/parameters/ms.s3.list.max.keys.md new file mode 100644 index 0000000..4a4d981 --- /dev/null +++ b/docs/parameters/ms.s3.list.max.keys.md @@ -0,0 +1,35 @@ +# ms.s3.list.max.keys + +**Tags**: +[source](categories.md#source-properties) + +**Type**: integer + +**Default value**: 1000 + +**Minimum value**: 1 + +**Related**: + +## Description + +`ms.s3.list.max.keys` limit the number of keys when doing a "list" operation +on a S3 bucket. + +In retrieve files from S3, DIL will first try listing the keys using the path +from [ms.source.uri](ms.source.uri.md), +DIL will only perform a "download" if there is only 1 key. + +If there are multiple keys from the given location, DIL will just write the +list of keys out, and no download performed. + +In order to download multiple files, a "list" job is required to list all +keys and save them in a list file, then a "download" job to get them one by one, +with each key being processed by 1 work unit. +See [Two Step File Download](https://github.com/linkedin/data-integration-library/blob/master/docs/patterns/two-step-file-download-pattern.md). + +### Statement of Direction + +`ms.s3.list.max.keys` will be merged into `ms.source.s3.parameters`. + +[back to summary](summary.md#mss3listmaxkeys) \ No newline at end of file diff --git a/docs/parameters/ms.schema.cleansing.md b/docs/parameters/ms.schema.cleansing.md new file mode 100644 index 0000000..20473f4 --- /dev/null +++ b/docs/parameters/ms.schema.cleansing.md @@ -0,0 +1,55 @@ +# ms.schema.cleansing + +**Tags**: +[schema](categories.md#schema-properties) + +**Type**: string + +**Format**: A JsonObjects + +**Default value**: "{}" (a blank JsonOjbect) + +**Related**: + +## Description + +Schema cleansing replaces special characters in the schema element names based +on a pattern. By default, it will replace all blank spaces, $, and @ to underscores. + +`ms.schema.cleansing` is a JsonObject, and it supports the following elements: + +- **enabled** : true|false +- **pattern**: if enabled, it has default value "(\\s|\\$|@)" +- **replacement**: if enabled, it has default value "_" +- **nullable**: whether to force nullability. + +`nullable` has default value "false". +If true, all fields will be forced to be nullable. +If false, the schema inference will try to detect nullability from samples. + +This configuration has no impact on schemas from metadata stores. + +If defined, `ms.schema.cleansing` supersedes [ms.enable.cleansing](ms.enable.cleansing.md) +If `ms.schema.cleansing` is not defined, DIL will check `ms.enable.cleansing`. +If `ms.enable.cleansing` is true, DIL will do the default cleansing. + +**Alert**: This feature should be used only on need basis, +for example, where source data element names are un-conforming, such as +containing spaces, and needed standardization. In large datasets cleansing +can be expensive. + +### Statement of Direction + +`ms.enable.cleansing` will be deprecated. + +### Examples + +The following makes all inferred schema fields nullable. + +- `ms.schema.cleansing={"enabled": "true", "nullable": "true"}` + +The following additionally replaces "-" (hyphen) with "_"(underscore). + +- `ms.schema.cleansing={"enabled": "true", "pattern": "(\\s|\\$|@|-)"}` + +[back to summary](summary.md#msschemacleansing) \ No newline at end of file diff --git a/docs/parameters/ms.secondary.input.md b/docs/parameters/ms.secondary.input.md new file mode 100644 index 0000000..923e2d7 --- /dev/null +++ b/docs/parameters/ms.secondary.input.md @@ -0,0 +1,110 @@ +# ms.secondary.input + +**Tags**: +[watermark & work unit](categories.md#watermark-work-unit-properties), +[authentication](categories.md#authentication-properties) + +**Type**: string + +**Format**: A JsonArray of JsonObjects, with each JsonObject defines +one secondary input. + +**Default value**: "[]" (a blank JsonArray) + +**Related**: + +## Description + +Secondary inputs provides additional directives to job execution, in addition to +the primary inputs of job execution, which is its metadata, i.e, job configurations. + +`ms.secondary.input` property has attributes to support the following functions: + +- **location**: from where the HDFS data will be loaded as secondary input +- **fields**: fields that needs to be extracted and added into the work unit state +- **filters**: rules to include/exclude records from secondary input +- **Tags**: specify the type of secondary input + +The following are filtering rules: + + - if multiple fields are filtered, the relationship is AND, that means + all condition must be met + - if a filter is defined on a field, and field value is NULL, the record is rejected + - if a filter is defined on a field, and the field value is not NULL, the + record will be rejected if its value doesn't match the pattern + - if no filter is defined on a field, the default filter ".*" is applied to + the field, and NULL values are accepted + +DIL designed 3 categories of secondary inputs: + +- **activation**: `activation` secondary input creates work units +- **authentication**: `authentication` secondary input provides authentication information, +like access tokens +- **payload**: `payload` secondary input specifies the location to pick additional data that +will only be interpreted by connection. + +### Activation Secondary Input + +`activation` secondary inputs are used to "activate" the job. That means the job will +generate work units based on the given values. + +For example, if we have a file of a list of Ids to extract from a source, then +we can define an `activation` category of secondary input based on the file. + +- `ms.secondary.input=[{"path":"/path","fields":["id"],"category":"activation"}]` + +At the beginning of job execution, DIL will read the list file and retrieve the `id`s. +Then each `id` leads to the generation of one work unit, which has an attribute `id:value`. + +### Authentication Secondary Input + +`authentication` secondary inputs are used to read dynamic authentication credentials or tokens +from a storage location, so that they don't need to be coded in metadata, which is static. + +Typical usage is that we use a separate job to get a refreshed access token, and save +the access token on media, then the primary integration job can read the access token +for subsequent requests. + +`authentication` secondary input doesn't generate work units, and it is passed to all +work units, i.e, all work units get the same authentication credentials/tokens. + +### Payload Secondary Input + +`payload` secondary inputs are used to specify raw payload locations. + +Payloads are simply passed to connections without processing. The connection will decide +what to do about it. +For example, HTTP connection will read the records from the payload, and attach 1 +row to 1 HTTP request. If there are multiple rows, HTTP connection will page (see [pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/pagination.md)) +through them. Therefore, each row of the payload is processed by 1 HTTP request. + +`payload` secondary input is typically used in the egression flows. If there are many +rows to send out, they can be "batched" so that the payload file has fewer number of +records. + +### Examples + +In the following, we have a file with a list of ids and their statuses. We +expect the job will generate 1 work unit for each id of status "OK" or "Success". +The gobblin job will read records from that location and extract the two +fields and inject them into the work units. Each work unit then has 2 variables, +"id" and "tempId", that can be used as request parameters. + +- ` ms.secondary.input=[{ + "path": "/path/to/hdfs/inputFileDir", + "fields": ["id", "tempId"], + "filters": {"status": "(OK|Success)"}, + "category" "activation" + }] +` + +In the following, we only process certain ids from a list. This is typically used +in back fill. + +- `ms.secondary.input=[{"path": "/path/ids", "fields": ["id"], "filters": {"id": "(19|28|89)"}}]` + +In the following, we egress the normalized records from a prior job (`${preprocess.table.name}`) + +- `ms.secondary.input=[{"path": "${job.dir}/${extract.namespace}/${preprocess.table.name}", "fields": ["conversions"], "category": "payload"}]` + +[back to summary](summary.md#mssecondaryinput) \ No newline at end of file diff --git a/docs/parameters/ms.session.key.field.md b/docs/parameters/ms.session.key.field.md new file mode 100644 index 0000000..7c8cf65 --- /dev/null +++ b/docs/parameters/ms.session.key.field.md @@ -0,0 +1,121 @@ +# ms.session.key.field + +**Tags**: +[source](categories.md#source-properties), +[pagination](categories.md#pagination-properties) + +**Type**: string + +**Format**: A JsonObject + +**Default value**: "{}" (a blank JsonObject) + +**Related**: +- [key concept: pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/pagination.md) +- [ms.pagination](ms.pagination.md) +- [ms.total.count.field](ms.total.count.field.md) + +## Description + +Session is a state management mechanism over stateless connections. +For example, although Restful API is stateless, data sources can maintain +a session in backend by a status field, a session cursor, or through +[pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/pagination.md). + +`ms.session.key.field` specifies the key field in response in order to retrieve the +status for session control and the condition for termination. + +This property takes the form a JsonObject with a **name**, a **condition**, +and a **failCondition**. + +it takes the form a Json object with a "name", "condition", and "failCondition". +- **name** specifies the field in response that gives session info, it is required +- **condition** specifies when the session should stop. +- **failCondition** specifies when the session should fail. +- "condition" and "failCondition" are optional +- "condition" takes precedence over "failCondition" + +The "name" element of ms.session.key.field is one of following: + +- For API returning Json response, "name" should be the Json element that +decides next action or next fetching start position. +- For API returning CSV response, "name" should be the HTTP header tag +that decides next action or next fetching start position + +A condition can be a regular expression or a formula (TODO). Currently, +only regular expression is supported. + +When only session control is enabled but no pagination, the extractor +will keep consuming data from the source until: + +- the stop condition is met, or +- the session timed out + +When both session and pagination are enabled, the extractor will +keep consuming data from the source until + +- the stop condition is met, or +- the session timed out, or +- the total number of records are fetched, or +- a blank response is received from source, or +- an error/warning is received that indicates the response +should not be treated as valid data + +**Alert**: In that regard, when the source gives conflicting signal in turns of +total expected rows and status, the data can have duplicate, and actual +extracted rows in log file should show more rows extracted than expected. + +A session can timeout before the stop condition is met. The timeout +is controlled by property [ms.wait.timeout.seconds](ms.wait.timeout.seconds.md), +which has a default value of 600 seconds. + +This is useful for [asynchronous ingestion](https://github.com/linkedin/data-integration-library/blob/master/docs/patterns/asynchronous-ingestion-pattern.md), +for example, we can have the stop condition set +as the status value of a successful request, such as "completed" or "ready". +When the stop condition is not met during the timeout period, the session +will time out (error), and the session will abort (stop and fail). + +Session key value can come from the Json payload for Rest API, or from an +HTTP header for SOAP API. Rest API returns JSON response. SOAP API +returns XML or CSV response. When SOAP API is instructed to return CSV +response, the session key value mostly is included in an HTTP header. + +Session key can be used in the following 2 scenarios: + +- as a session control mechanism over a series of independent requests +until the stop condition is met (above) +- as a session control mechanism over a sequence of dependent requests, +with the session key value of first request fed into the next request as +input. That means the fetched value of session key, from the +`ms.session.key.field`, can be fed into variables of type [session](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/session-control.md). + +Variables are defined in [ms.parameters](ms.parameters.md). +When a `session` type variable is defined, it will implicitly add to +URL parameters or request bodies if HttpSource is used. +However, for the first HTTP request, because the +session variable has a NULL value, it will not be used in the request. + +### Examples + +In the following, the session key provides a stop condition: + +- `ms.session.key.field={"name": "hasMore", "condition": {"regexp": "false|False"}}` + +In the following, the session key is used as a cursor. A `session` type variable +`cursor` is defined. This variable will get the session key value +in each page after the first page. When the `cursor` variable +is used in requests, it establishes a "session" on the source. + +- `ms.session.key.field={"name": "records.cursor"}` +- `ms.parameters=[{"name":"fromDateTime","type":"watermark","watermark":"system","value":"low","format":"datetime","pattern":"yyyy-MM-dd'T'HH:mm:ss'Z'"},{"name":"toDateTime","type":"watermark","watermark":"system","value":"high","format":"datetime","pattern":"yyyy-MM-dd'T'HH:mm:ss'Z'"},{"name":"cursor","type":"session"}]` + +In the following, say the source doesn't supply a total records or total pages +to indicate when to finish ingesting data from it. Instead, in the +source's response, it contains a boolean field called "end_of_stream". +This field indicates whether the data source is exhausted or not. +Only when this field has the value "true", the source has finished +streaming data. + +- `ms.session.key.field={"name": "end_of_stream", "condition": {"regexp": "^true$"}}` + +[back to summary](summary.md#mssessionkeyfield) \ No newline at end of file diff --git a/docs/parameters/ms.sftp.conn.timeout.millis.md b/docs/parameters/ms.sftp.conn.timeout.millis.md new file mode 100644 index 0000000..6cfeaee --- /dev/null +++ b/docs/parameters/ms.sftp.conn.timeout.millis.md @@ -0,0 +1,19 @@ +# ms.sftp.conn.timeout.millis + +**Tags**: +[connection](categories.md#connection-properties) + +**Type**: Integer + +**Default value**: 60 seconds + +**Related**: + + +## Description + +`ms.sftp.conn.timeout.millis` defines maximum allowed inactive time. The default is 60 seconds. + +SFTP uses SSH channel, so this is practically the SSH timeout time. + +[back to summary](summary.md#mssftpconntimeoutmillis) diff --git a/docs/parameters/ms.source.data.character.set.md b/docs/parameters/ms.source.data.character.set.md new file mode 100644 index 0000000..786dc91 --- /dev/null +++ b/docs/parameters/ms.source.data.character.set.md @@ -0,0 +1,25 @@ +# ms.source.data.character.set + +**Tags**: + [source](categories.md#source-properties) + +**Type**: string + +**Format**: one of the standard codes in Java:StandardCharsets + +**Default value**: "UTF-8" + +**Related**: + +## Description + +`ms.source.data.character.set` specifies a character set to parse JSON or CSV payload. +The default source data character set is UTF-8, which should be good for most use cases. + +See `Java:StandardCharsets` for other common names, such as UTF-16. + +### Example + +`ms.source.data.character.set=UTF-8` + +[back to summary](summary.md#mssourcedatacharacterset) \ No newline at end of file diff --git a/docs/parameters/ms.source.files.pattern.md b/docs/parameters/ms.source.files.pattern.md new file mode 100644 index 0000000..a789e7b --- /dev/null +++ b/docs/parameters/ms.source.files.pattern.md @@ -0,0 +1,40 @@ +# ms.source.files.pattern + +**Tags**: +[source](categories.md#source-properties) + +**Type**: string + +**Format**: A Java regular expression + +**Support DIL Variables**: No + +**Default value**: ".*" + +**Related**: +- [job property: ms.source.uri](ms.source.uri.md) + +## Description + +`ms.source.files.pattern` specifies a pattern to filter files from S3 and SFTP sources. + +`ms.source.files.pattern` meant to provide advanced filtering +that SFTP or S3 "list" command cannot do by using its own patterns. + +### Statement of Direction + +Source file patterns will be moved to [ms.source.uri](ms.source.uri.md), +which supports DIL variables, for S3 and SFTP. HDFS source has +already been using ms.source.uri to specify file patterns. + +### Examples + +To pick only PGP files from the source. + +`ms.source.files.pattern=.+\.pgp($|\n)` + +To pick only source files that start with a particular prefix: + +`ms.source.files.pattern=^2019-01-01.+` + +[back to summary](summary.md#mssourcefilespattern) \ No newline at end of file diff --git a/docs/parameters/ms.source.s3.parameters.md b/docs/parameters/ms.source.s3.parameters.md new file mode 100644 index 0000000..0ca874b --- /dev/null +++ b/docs/parameters/ms.source.s3.parameters.md @@ -0,0 +1,31 @@ +# ms.source.s3.parameters + +**Tags**: +[connection](categories.md#connection-properties), +[source](categories.md#source-properties) + +**Type**: string + +**Format**: A JsonObject + +**Default value**: empty JsonObject (considered as blank) + +**Related**: + + +## Description + +`ms.source.s3.parameters` specifies parameters for S3 connection. +It can have the following attributes: + +- **region**: string, one of [aws region codes](https://docs.aws.amazon.com/general/latest/gr/rande.html) +- **read_timeout_seconds**: integer, read time out in seconds +- **write_timeout_seconds**: integer, write time out in seconds +- **connection_timeout_seconds**: Sets the socket to timeout after failing to establish a connection with the server after milliseconds. +- **connection_max_idle_millis**: Sets the socket to timeout after timeout milliseconds of inactivity on the socket. + +### Example + +`ms.source.s3.parameters={"region" : "us-east-1"}` + +[back to summary](summary.md#mssources3parameters) \ No newline at end of file diff --git a/docs/parameters/ms.source.schema.urn.md b/docs/parameters/ms.source.schema.urn.md new file mode 100644 index 0000000..cb43557 --- /dev/null +++ b/docs/parameters/ms.source.schema.urn.md @@ -0,0 +1,45 @@ +# ms.target.schema.urn + +**Tags**: +[schema](categories.md#schema-properties), +[conversion](categories.md#conversion-properties), +[source](categories.md#source-properties) + +**Type**: string + +**Format**: A DataHub URN pointing to a dataset or registered schema definition + +**Default value**: blank + +**Related**: +- [job property: ms.target.schema.urn](ms.target.schema.urn.md) +- [job property: ms.output.schema](ms.output.schema.md) + +## Description + +Source schema represents the source data structure. Generally, in a data +ingestion scenario, the source data will be read in, projected, filtered, and +converted. Source schema can be read from the source, like for JDBC data sources, or parsed +from actual data, like JSON data, or defined as a string, or defined in a metadata +store. `ms.target.schema.urn` address the option that defines source schema in metadata store. + +We generally don't define source schema in schema string format. Instead, we directly +define the [output schema](ms.output.schema.md) +if necessary. + +`ms.target.schema.urn` is a URN string of the following forms: + +- a **dataset** URN, if the source schema can be represented by a dataset, +the latest schema of the dataset will be read from metadata store, +and then parsed to retrieve fields and types, etc. +- a **registered schema** URN, if the source schema is registered with metadata store in +the form of either a pegasus (PDL) or GraphQL schema. + +### Example + +The following use a pre-defined "registered schema" to represent the +response structure when calling Google batch upload API. + +`ms.source.schema.urn=urn:li:registeredSchema:(PEGASUS,dil-draft-schema/com.linkedin.google.BatchinsertResponse)` + +[back to summary](summary.md#mssourceschemaurn) \ No newline at end of file diff --git a/docs/parameters/ms.source.uri.md b/docs/parameters/ms.source.uri.md new file mode 100644 index 0000000..0596b8b --- /dev/null +++ b/docs/parameters/ms.source.uri.md @@ -0,0 +1,42 @@ +# ms.source.uri + +**Tags**: +[connection](categories.md#connection-properties), +[source](categories.md#source-properties) + +**Type**: string + +**Format**: URI with path segments and parameters + +**Support DIL Variables**: Yes + +## Required +Yes + +**Default value**: +blank + +## Related + +## Description + +ms.source.uri defines the integration point, which is called data source for data ingestion or target for data egression. +It follows the URI format [here](https://en.wikipedia.org/wiki/Uniform_Resource_Identifier). +The only exception is that authority is not supported, because all authority cannot be fit in the URI. + +ms.source.uri supports [variables](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/variables.md) +that allow substitution in runtime, for example: +`ms.source.uri = https://api.zendesk.com/api/v2/incremental/tickets.json?start_time={{startTime}}` + +ms.source.uri requires protocol (scheme) and domain (host). Path segments and URI parameters are optional. + +## Examples + +` +ms.source.uri=https://api.zendesk.com/api/v2/incremental/tickets.json?start_time={{startTime}} +ms.source.uri=jdbc://data.rightnow.com/marketing?useSSL=true +ms.source.uri=https://oauth2.googleapis.com/token +ms.source.uri=https://bucket-name.s3.amazonaws.com/{{s3key}} +` + +[back to summary](summary.md#mssourceuri) diff --git a/docs/parameters/ms.ssl.md b/docs/parameters/ms.ssl.md new file mode 100644 index 0000000..9c702bf --- /dev/null +++ b/docs/parameters/ms.ssl.md @@ -0,0 +1,30 @@ +# ms.ssl + +**Tags**: +[connection](categories.md#connection-properties), + +**Type**: string + +**Format**: JsonObject + +**Default value**: {} (blank JsonObject) + +## Related + +## Description + +`ms.ssl` defines SSL parameters. + +`ms.ssl` comes as a JsonObject, and it can have any of the following +attributes: +- keyStoreType, the key store type, default is "pkcs12" +- keyStorePath, the file path to key store file +- keyStorePassword, the key to decrypt the key store file +- keyPassword, the password to decrypt the key +- trustStorePath, the file with trust certificate +- trustStorePassword, the password to decrypt the trust store file +- connectionTimeoutSeconds, the wait time to establish a connection, the default is 60 seconds +- socketTimeoutSeconds, the wait time for the next packet, the default is 60 seconds +- version, the SSL version, default is "TLSv1.2" + +[back to summary](summary.md#msssl) diff --git a/docs/parameters/ms.target.schema.md b/docs/parameters/ms.target.schema.md new file mode 100644 index 0000000..e6b1ba3 --- /dev/null +++ b/docs/parameters/ms.target.schema.md @@ -0,0 +1,30 @@ +# ms.target.schema + +**Tags**: +[schema](categories.md#schema-properties), +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Format**: A schema definition in the form of a JsonArray of JsonObjects +with each JsonObject being a column definition. + +**Default value**: blank + +**Related**: +- [job property: ms.target.schema.urn](ms.target.schema.urn.md) + +## Description + +`ms.target.schema` defines the target schema in a JsonArray string. +Target schema denotes the schema to be passed to writer, this applies +to situation where the source data are transformed through a converter +or other processes. + +The syntax of the schema string is same as [ms.output.schema](ms.output.schema.md). + +### Example + +`ms.target.schema=[{"columnName":"record","isNullable":"false","dataType":{"type": "string"}}, {"columnName":"uuid","isNullable":"false","dataType":{"type": "string"}}, {"columnName":"date","isNullable":"false","dataType":{"type": "timestamp"}}, {"columnName":"survey_path","isNullable":"false","dataType":{"type": "string"}}, {"columnName":"dilExtractedDate","isNullable":"false","dataType":{"type": "long"}}, {"columnName":"start_date","isNullable":"true","dataType":{"type": "timestamp"}}, {"columnName":"additionalinfo","isNullable":"false","dataType":{"type":"map", "values": "string"}}]` + +[back to summary](summary.md#mstargetschema) \ No newline at end of file diff --git a/docs/parameters/ms.target.schema.urn.md b/docs/parameters/ms.target.schema.urn.md new file mode 100644 index 0000000..4b527d6 --- /dev/null +++ b/docs/parameters/ms.target.schema.urn.md @@ -0,0 +1,31 @@ +# ms.target.schema.urn + +**Tags**: +[schema](categories.md#schema-properties), +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Format**: A DataHub URN pointing to a registered schema definition + +**Default value**: blank + +**Related**: +- [job property: ms.target.schema](ms.target.schema.md) + +## Description + +Target schema denotes the schema to be passed to writer, this applies +to situation where the source data are transformed through a converter +or other processes. + +Generally, target schema should be specified through target schema URN. +to avoid coding long schema strings. +An URN can point to the schema storage location on DataHub, which is +the only supported schema storage for now. + +### Example + +`ms.target.schema.urn=urn:li:registeredSchema:(PEGASUS,draft-schema/com.linkedin.google.UploadClickConversionsRequest)` + +[back to summary](summary.md#mstargetschemaurn) \ No newline at end of file diff --git a/docs/parameters/ms.total.count.field.md b/docs/parameters/ms.total.count.field.md new file mode 100644 index 0000000..ecd9148 --- /dev/null +++ b/docs/parameters/ms.total.count.field.md @@ -0,0 +1,38 @@ +# ms.total.count.field + +**Tags**: +[pagination](categories.md#pagination-properties), +[source](categories.md#source-properties) + +**Type**: string + +**Format**: A [JsonPath](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/json-path.md) separated by ".", like "level1.level2" + +**Default value**: blank + +## Related + +- [key concept: pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/pagination.md) + +Total Count field directs DIL how to retrieve the expected total row counts. This is important when there are large +volume of data and [pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/pagination.md) +is used to retrieve data page by page. In cases of pagination, the expected total row count is one way to indicate +the end of pagination when the actually retrieved total number of rows matches or surpasses the expected total rows. + +Total count field is normally the element name representing the expected total count. It is, therefore, mostly in +simple string format. + +If the response is in Json format, the total count field can be in a nested element, and the configuration will +be a JsonPath to indicate how to find the nested element. + +Example 1: if the response from the source is like this: `{..."total_records" : 48201...}`, then the configuration can be +`ms.total.count.field = total_records`. + +Example 2: if the response from the source is like this `{..."records": {..."totalRecords": 9999...}...}`, then +the configuration can be `ms.total.count.field = records.totalRecords`. + +[back to summary](summary.md#mstotalcountfield) + + + + diff --git a/docs/parameters/ms.validation.attributes.md b/docs/parameters/ms.validation.attributes.md new file mode 100644 index 0000000..431423f --- /dev/null +++ b/docs/parameters/ms.validation.attributes.md @@ -0,0 +1,58 @@ +# ms.validation.attributes + +**Tags**: +[conversion](categories.md#conversion-properties) + +**Type**: string + +**Format**: A JsonObject + +**Default value**: blank + +**Related**: + +## Description + +`ms.validation.attributes` defines a threshold to mark a job as successful or failed. +The threshold can be specified as "success" or "failure" thresholds. The former is +called a "success" rule, and the later is called a "failure" rule. + +This property is required for **InFlowValidationConverter**, which +is validation converter based on simple count comparison. + +In either configuration, job will only succeed if the threshold is met. That means, if the +rule is defined as "success", the success rate has to be above the threshold; +and the failure rate has to be below the threshold if the rule is defined as "failure" type. + +"**success**" rule is used when the data available for validation are successful records. +"success" rule is a lower bound rule, it works this way: +- Job succeeds when the row count in validation set / row count in base set >= threshold +- Job fails when the row count in validation set / row count in base set < threshold + +"**failure**" rule is used when the data available for validation are error records. +"failure" rule is a upper bound rule, it works this way: +- Job succeeds when the row count in validation set / row count in base set < threshold +- Job fails when the row count in validation set / row count in base set >= threshold + +A rule is accepted as a JsonObject with following Keys +- **threshold**: represents the percentage of acceptable failure or required success +- **criteria**: this value can be "fail" or "success" +- **errorColumn**: this value is optional, and it is required in order to filter +the failure records based on a column, and the records having none-blank +value in "errorColumn" are considered failed. + +### Examples + +Failed records cannot be more than 10% of total records: + +`ms.validation.attributes={"threshold": "10", "criteria" : "fail"}` + +There have to be at least 1 successful record: + +`ms.validation.attributes={"threshold": "0", "criteria" : "success"}` + +Failed records cannot be more than 30% of total records: + +`ms.validation.attributes={"threshold": "30", "criteria" : "fail"}` + +[back to summary](summary.md#msvalidationattributes) \ No newline at end of file diff --git a/docs/parameters/ms.wait.timeout.seconds.md b/docs/parameters/ms.wait.timeout.seconds.md new file mode 100644 index 0000000..ee43560 --- /dev/null +++ b/docs/parameters/ms.wait.timeout.seconds.md @@ -0,0 +1,65 @@ +# ms.wait.timeout.seconds + +**Tags**: +[pagination](categories.md#pagination-properties), +[back to summary](summary.md) + +**Type**: integer (in seconds) + +**Default value**: 600 seconds (or 10 minutes) + +**Maximum value**: 24 hours (24 * 3600 seconds) + +**Related**: + +- [job property: ms.call.interval.millis](ms.call.interval.millis.md). +- [job property: ms.session.key.field](ms.session.key.field.md) + +## Description + +`ms.wait.timeout.seconds` is one option to control pagination, it specifies +how long the job will wait before the session ending (success or failure) status is met. + +When there is no total expected-row-count ([ms.total.count.field](ms.total.count.field.md) is blank), +the pagination will keep looping and waiting until either the session +ending condition is met or time out. + +### Use Case + +In [asynchronous](https://github.com/linkedin/data-integration-library/blob/master/docs/patterns/asynchronous-ingestion-pattern.md) +data extraction, a request is submitted to data source, and the +data source will provide updated status when the request is completed, and data +are ready for downloading. +Therefore, the extraction job will keep checking the status after submitting the +request by intervals defined in [ms.call.interval.millis](ms.call.interval.millis.md). + +At the same time, +[ms.session.key.field](ms.session.key.field.md) +can specify the status code when the request is completed or failed. +In each check, DIL will compare the retrieved status with the expected status, and +ends the looping when the status is completed or failed. + +However, DIL will not loop forever, it will timeout if after timeout period none of +completed or failed status is returned. + +### Example 1 + +The following SalesForce status check job will wait for the "JobComplete" signal, which indicating +the request is ready, and it will timeout after 4 hours. + +`ms.session.key.field={"name": "Sforce-Locator", "condition": {"regexp": "JobComplete"}}` + +`ms.wait.timeout.seconds=14400` + +### Example 2 + +The following asynchronous data extraction job will wait for the `complete` or `failed` status +for 600 seconds by checking every 1 second. + +`ms.session.key.field={"name": "result.status", "condition": {"regexp": "^complete$"}, "failCondition": {"regexp": "^failed$"}}` + +`ms.call.interval.millis=1000` + +`ms.wait.timeout.seconds=600` + +[back to summary](summary.md#mswaittimeoutseconds) \ No newline at end of file diff --git a/docs/parameters/ms.watermark.md b/docs/parameters/ms.watermark.md new file mode 100644 index 0000000..8aae7d7 --- /dev/null +++ b/docs/parameters/ms.watermark.md @@ -0,0 +1,147 @@ +# ms.watermark + +**Tags**: +[watermark & work unit](categories.md#watermark-work-unit-properties) + +**Type**: string + +**Format**: a JsonArray of JsonObjects + +**Default value**: 0 + +**Related**: + +- [job property: ms.parameters](ms.parameters.md). +- [job property: ms.work.unit.partition](ms.work.unit.partition.md) +- [key concept: variable](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/variable.md) +- [key concept: work unit](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/work-unit.md) +- [key concept: watermark](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/watermark.md) + +## Description + +`ms.watermark` define named watermarks for work unit generation, execution control, +and incremental processing. DIL supports 2 types of watermarks, `datetime` and `unit`. + +- **datetime watermark**: a datetime watermark defines a datetime range. +- **unit watermark**: a unit watermark defines an array of processing units. + +There should be 1 and only datetime watermark. If a datetime watermark is not defined, +DIL will implicitly generate a datetime watermark with a range from 2019-01-01 to current date. + +There should be no more than 1 unit watermark. + +This document focuses on the syntax of `ms.watermark` property. +To understand how watermark controls execution, please read: [key concept: watermark](../concepts/watermark.md). +To understand how work unit works, please read: [key concept: work unit](../concepts/work-unit.md). + +### More about Datetime Watermark + +A datetime watermark is a reference. It doesn't directly effect or control +job execution. The watermark name and boundaries, low watermark +and high watermark, can be referenced in [variables](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/variable.md), +which can control execution. +See [ms.parameters](ms.parameters.md). + +A datetime watermark is a range, defined by its `from` and `to` field. The range +can be further partitioned per other configurations. +See [ms.work.unit.partition](ms.work.unit.partition.md) + +Therefore, a datetime watermark could generate 1 or more mini-watermarks when +partitioned, and each mini-watermark is mapped to a work unit. Therefore, +each work unit has its own unique watermark. + +The range of a datetime watermark is controlled by a `from` and `to` field, and +each of them can be defined in the following ways: + +- **a datetime string**: The string has to be in the format of `yyyy-MM-dd HH:mm:ss.SSSSSSz` +or `yyyy-MM-ddTHH:mm:ss.SSSSSSz`. For example: "2020-01-01". Hour and below grain +are optional. Timezone is optional, and the default is PST. +- **-(hyphen)**: Hyphen represents the current date time. It will be converted to +system date during the work unit generation phase. +- **PxDTyH([ISO 8601 duration format](https://en.wikipedia.org/wiki/ISO_8601#Durations))**: +A ISO duration is interpreted as a datetime +that is `PxDTyH` **preceding** current date time. For example, if the definition +is P1D, then it means a date time value (milliseconds) of 1 day before current +date time (milliseconds). Apparently, hypen (-) is just a shorthand for P0D. +This format of presentation is interpreted per system date. + +_The `from` value of a datetime watermark is usually static_. +The importance of keeping `from` static is that partitions are generated +based on it. `from` is part of the signature of all work units if no partitioning; + `from` will decide the start datetime values of all partitions if the watermark +is partitioned, and those values will be signatures of their corresponding +work units. + +For example, a monthly-partitioned watermark from **2020-01-01** will generate +partitions, and thus work units, like [2020-01-01, 2020-02-01), [2020-02-01, 2020-03-01), +[2020-03-01, 2020-04-01), and so on. If the from value changed to **2020-01-05**, +partitions will be generated like [2020-01-05, 2020-02-05), [2020-02-05, 2020-03-05), +[2020-03-05, 2020-04-05). Because the start time of partitions is the signature +of the work unit, and it is used to identify the state of the work unit in +state store, the change of `from`, therefore, totally invalidated all prior +work unit states. + +The `from` can be dynamic through the ISO duration format under +the following situations: + +- You are using daily work unit partitioning, and therefore changing the `from` value +by one or more days will not invalidate work unit state stores. +- Prior execution states are not important, and you want to keep the watermark +reference to a small range. In such case, you could define the `from` as something +like `P30D`, which will make the reference timeframe starting from **30 days ago**. + +**Alert** whenever `from` is dynamic, there could be excessive state store +records being generated, because partition signatures are floating. This can +cause small-file problem when state store is on HDFS. + +_On the contrary, `to` value of a datetime watermark is usually dynamic_. Most +commonly, it is "-". The `to` value can be PxD if the reference timeframe has to +end by certain number of days ago. + +For multi-day partitioning, i.e. monthly and weekly partitioning, the `to` value +is rounded to day level to avoid generating empty partitions under certain situations. +For example, +without rounding, a weekly partition from Monday (milliseconds of midnight) to +Monday (milliseconds of current time) could be generated on Mondays. That kind of +partitions can generate an empty range if the watermark is formatted as +yyyy-MM-dd strings. In order to avoid rounding of `to` value to day level, +`to` can be defined as "P0DT0H", indicating DIL to round only to hour level. + +#### datetime watermark examples + +`ms.watermark=[{"name": "system","type": "datetime","range": {"from": "2019-01-01", "to": "-"}}]` + +`ms.watermark=[{"name": "system","type": "datetime","range": {"from": "2021-06-01", "to": "P0D"}}]` + +`ms.watermark=[{"name": "system","type": "datetime","range": {"from": "2021-06-01", "to": "P1D"}}]` + +`ms.watermark=[{"name": "system","type": "datetime","range": {"from": "P7D", "to": "-"}}]` + +### More about Unit Watermark + +A `unit` watermark defines a list of values that will be used by DIL to +generate work units. + +A `unit` watermark can be defined as a JsonArray. +For example, `["a", "b", "c"]`. + +As a shortcut, a `unit` watermark can also be defined as +a comma separated string, like "a,b,c", which then will be converted +to a JsonArray internally. + +A `unit` watermark name can be referenced as a [variable](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/variable.md) +directly. + +#### unit watermark examples + +`ms.watermark = [ +{"name": "system","type": "datetime", "range": {"from": "2021-08-21", "to": "-"}}, +{"name": "bucketId", "type": "unit", "units": "null,0,1,2,3,4,5,6,7,8,9"}] +` + +`ms.watermark = [ +{"name": "dateRange","type": "datetime", "range": {"from": "2020-01-01", "to": "P0D"}}, +{"name": "siteName", "type": "unit", "units": "https://siteA/,https://SiteB/...siteZ"}] +` + +[back to summary](summary.md#mswatermark) \ No newline at end of file diff --git a/docs/parameters/ms.work.unit.min.records.md b/docs/parameters/ms.work.unit.min.records.md new file mode 100644 index 0000000..b81a037 --- /dev/null +++ b/docs/parameters/ms.work.unit.min.records.md @@ -0,0 +1,21 @@ +# ms.work.unit.min.records + +**Tags**: +[watermark & work unit](categories.md#watermark-work-unit-properties) + +**Type**: integer + +**Default value**: 0 + +**Related**: +- [job property: ms.work.unit.min.units](ms.work.unit.min.units.md) + +## Description + +`ms.work.unit.min.records` specifies a minimum number of records that are expected. If the total +processed rows is less than `ms.work.unit.min.records`, the job will fail, generating an alert. + +This can be used when a data ingestion job expects a certain number of records every time. +Setting `ms.work.unit.min.records=1` can detect empty ingestion. + +[back to summary](summary.md) \ No newline at end of file diff --git a/docs/parameters/ms.work.unit.min.units.md b/docs/parameters/ms.work.unit.min.units.md new file mode 100644 index 0000000..035a5be --- /dev/null +++ b/docs/parameters/ms.work.unit.min.units.md @@ -0,0 +1,23 @@ +# ms.work.unit.min.units + +**Tags**: +[watermark & work unit](categories.md#watermark-work-unit-properties) + +**Type**: integer + +**Default value**: 0 + +**Related**: + - [job property: ms.work.unit.min.records](ms.work.unit.min.records.md) + +## Description + +`ms.work.unit.min.units` specify a minimum number of work units required for the job to be successful. +if the number of work units is smaller than `ms.work.unit.min.units`, the job will fail, sending an +alert to operations team. + +This is particularly useful when a data ingestion job expects daily files, for example, but there +is no file on one day, then the job will fail, generating a failure email, alerting there is no +file. + +[back to summary](summary.md) \ No newline at end of file diff --git a/docs/parameters/ms.work.unit.pacing.seconds.md b/docs/parameters/ms.work.unit.pacing.seconds.md new file mode 100644 index 0000000..9abf656 --- /dev/null +++ b/docs/parameters/ms.work.unit.pacing.seconds.md @@ -0,0 +1,31 @@ +# ms.work.unit.pacing.seconds + +**Tags**: +[watermark & work unit](categories.md#watermark-work-unit-properties) + +**Type**: integer + +**Default value**: 0 + +**Related**: + +- [job property: ms.work.unit.partition](ms.work.unit.partition.md) + +## Description + +ms.work.unit.pacing.seconds can spread out work unit execution by adding a waiting time +in the front of each work unit's execution. The amount of wait time is based on the order of +the work units. It is calculated as `i * ms.work.unit.pacing.seconds`, where `i` is the sequence number +of the work unit. + +**Note**: this property can be easily used inappropriately. When there are 3600 work units, and +`ms.work.unit.pacing.seconds=1`, the last work unit will not start processing until 1 hour later, +no matter how fast other work units are processed. + +## Example + +Assuming there are 100 work units, and we set `ms.work.unit.pacing.seconds=10`, then the second +work unit will not start processing until 10th second. Therefore, work units are spread out by +10 second gaps. + +[back to summary](summary.md) \ No newline at end of file diff --git a/docs/parameters/ms.work.unit.parallelism.max.md b/docs/parameters/ms.work.unit.parallelism.max.md new file mode 100644 index 0000000..2419801 --- /dev/null +++ b/docs/parameters/ms.work.unit.parallelism.max.md @@ -0,0 +1,76 @@ +# ms.work.unit.parallelism.max + +**Tags**: [watermark & work unit](categories.md#watermark-work-unit-properties) + +**Type**: integer + +**Default value**: 100 (value 0 will also be interpreted as 100) + +**Max value**: 1000 + +**Related**: + +- [job property: ms.work.unit.partition](ms.work.unit.partition.md) + +## Description + +ms.work.unit.parallelism.max defines maximum possible parallel work +units that can be processed in one job execution. + +Gobblin allows multiple work units be executed in one job. The concurrent +task executor number is controlled by `taskexecutor.threadpool.size`. The thread pool +size decide the number of work units that will be executed currently. +By default, the thread pool is 10. + +The parameter ms.work.unit.parallelism.max put a limit on the total number of +work units for a single job execution. In that sense, the thread pool size +cannot be larger than the total number of work units, because if thread pool +is bigger than the total number of work units, some threads will be idle. +Therefore, ms.work.unit.parallelism.max also sets the maximum parallelism for the job. + +Typically, for jobs processing large number of work units, we will +use these two parameters to control job execution. `ms.work.unit.parallelism.max` controls +how many work units will be processed by the job, and `taskexecutor.threadpool.size` controls +how many task executor threads will be running concurrently. +If there are more work units than the number of task executors, a task executor +will take another work unit once it finished processing its assigned work unit. + +For example, a Common Crawl ingestion has about 56,000 files, each about 1 GB. +Since processing 56K files takes about 2 weeks, we don't want 1 job keep running for +2 weeks. That for sure will fail. Therefore, we set `ms.work.unit.parallelism.max = 300`, +that means each job execution will process 300 files. +Further we don't want to ingest those 300 files all in once, we set `taskexecutor.threadpool.size = 10`, +that means only 10 files will be ingested concurrently. +After finished processing the first 10 files, task executors will move to processing +next 10 files, until all 300 files are processed. Then the job will complete. + +When the job is run next time, the first 300 files will be bypassed +based on their state store records. The next 300 files will be picked up +and each will generate one work unit, total 300 work units. Again, task executors +will process those 300 work units in groups of 10 like in the first job execution. + +That keeps going until all 56K files are processed + +ms.work.unit.parallelism.max is optional. If there are only a few work units +for a job, it is not necessary to set the limit, and all work units +will be processed in 1 job execution. In such case, the task executors will +recursively process work units the same way based on thread pool size. + +Therefore, unless there are 100s or more work units for a job, it is not necessary +to set `ms.work.unit.parallelism.max`, or it can be set to 0 (default), which means no limit. + +If ms.work.unit.parallelism.max is set to any value greater than 0, and there are +more work units than ms.work.unit.parallelism.max, then the Gobblin job need to +be executed repeatedly until all work units are processed. + +## Example +Total work units is 56,000 + +`ms.work.unit.parallelism.max = 300` + +`taskexecutor.threadpool.size = 10` + +Each job execution takes about 1 hour, repeating the job hourly for about 10 days, +until all work units processed. + +[back to summary](summary.md) \ No newline at end of file diff --git a/docs/parameters/ms.work.unit.partial.partition.md b/docs/parameters/ms.work.unit.partial.partition.md new file mode 100644 index 0000000..e10f544 --- /dev/null +++ b/docs/parameters/ms.work.unit.partial.partition.md @@ -0,0 +1,24 @@ +# ms.work.unit.partial.partition + +**Tags**: [watermark & work unit](categories.md#watermark-work-unit-properties) + +**Type**: boolean + +**Format**: true/false + +**Default value**: true + +**Related**: +- [job property: ms.watermark](ms.watermark.md) +- [job property: ms.work.unit.partition](ms.work.unit.partition.md) + +## Description + +ms.work.unit.partial.partition specifies whether the last partition of a multi-day partition scheme can be partial. +If set to true, it allows the last multi-day partition to be partial (partial month or partial week) + +For example, if you watermark date range 01/01-01/16, and set ms.work.unit.partition=weekly, +then the third partition of the data will be partial from 01/15-01/16. If you want this +partition of the data to be dropped, set this value to `false`. + +[back to summary](summary.md) \ No newline at end of file diff --git a/docs/parameters/ms.work.unit.partition.md b/docs/parameters/ms.work.unit.partition.md new file mode 100644 index 0000000..d629fb8 --- /dev/null +++ b/docs/parameters/ms.work.unit.partition.md @@ -0,0 +1,83 @@ +# ms.work.unit.partition + +**Tags**: [watermark & work unit](categories.md#watermark-work-unit-properties) + +**Type**: string + +**Format**: A string of one of the values: monthly, weekly, daily, hourly, and none, or a JsonObject. + +**Default value**: blank (equivalent to `none`) + +**Related**: + +- [job property: ms.watermark](ms.watermark.md) + +## Description + +ms.work.unit.partition defines how the watermark will be partitioned to form +work units. When a watermark is partitioned, each partition will be processed as +a work unit. Partitioning, therefore, allows parallel processing. + +This property supports **plain partitioning** or **structured partitioning**. + +### Plain Partitioning + +In **plain partitioning**, the whole watermark is evenly partitioned. In this case, +the partitioning scheme can be one of the values from `monthly`, `weekly`, `daily`, `hourly`, +and `none`. For example, if you have a job with data range 01/01-03/15, and you want each work +unit to take a week load of data, set this value to `weekly`. + +More information about the various partition schemes: + +- **monthly**: the [watermark](ms.watermark.md) +is partitioned by months from the starting date of the watermark. +For example, if watermark starts from Jan-5, then the first partition is +Jan-5 (milliseconds of midnight on Jan-5 inclusive) +to Feb-5 (milliseconds of midnight Feb-5 exclusive), and so on. +It is recommended to pick a start date at the month beginning. If a month end +date is picked as watermark start date, then the partitioning results can be wacky. + +- **weekly**: the [watermark](ms.watermark.md) +is partitioned by weeks from the starting day of the watermark. +For example, if watermark starts from a Monday, then the first partition is +Monday (milliseconds of midnight on Monday inclusive) +to the next Monday (milliseconds of midnight on next Monday exclusive), and so on. +The watermark start date can be a date that falls on any day. + +- **daily**: the [watermark](ms.watermark.md) +is partitioned by days from the starting date of the watermark, with +each partition starting from a date (milliseconds of midnight of the date inclusive) +to the next date (milliseconds of midnight of next date exclusive), and so on. + +- **hourly**: the [watermark](ms.watermark.md) +is partitioned by hours from the starting date-time of the watermark, with +each partition starting from a date-time (milliseconds of the date-time inclusive) +to the next date-time an hour away (milliseconds of the next date-time exclusive), and so on. + +- **none**: the watermark is not partitioned. + +### Structured Partitioning + +In **structured partitioning**, ms.work.unit.partition is a JsonObject, +and there can be multiple ways to partition the watermark. + +For example, the following will break 2010-2019 by monthly partitions, +and daily partitions afterwards. +`ms.work.unit.partition={"monthly": ["2010-01-01", "2020-01-01"], "daily": ["2020-01-01": "-"]}` + +In such case, the partitions are called composite. For composite partitions to work, +the ranges should be continuous with no gaps or overlaps. In order to avoid gaps and overlaps, +one range should end where the next range starts. + +**Note** the end of partition accepts "-" as current date, but it doesn't access PxD syntax, +the reason being a partition range can be broader than watermark range. + +For a composite partition, if the range definition is unspecified or invalid, +then the there is no partitioning, equivalent to ms.work.unit.partition='' + +For a composite partition, a range is matched against watermark to define partitions, +if a range is smaller than full partition range, for example `{"monthly": ["2020-01-01", "2020-01-18"]}`, +it will still generate a full partition. So to avoid confusion, the range should be, at minimum, 1 partition size. +That means, a range should at least 1 month for monthly, or at least 1 week for weekly etc. + +[back to summary](summary.md) diff --git a/docs/parameters/summary.md b/docs/parameters/summary.md new file mode 100644 index 0000000..3a4661d --- /dev/null +++ b/docs/parameters/summary.md @@ -0,0 +1,465 @@ +# DIL Job Properties +[Browse Properties by Category](categories.md) + +The following are a full list of DIL job properties. Generally DIL properties complement the job properties defined +in Gobblin core. When there is an exception, i.e. when a property replaces one or more Gobblin properties, +the property document will explain. + +## [ms.abstinent.period.days](ms.abstinent.period.days.md) +Abstinent Period is designed to avoid re-extracting a dataset repeatedly. This is particularly useful +for situations like downloading files in large quantity. + +## ms.activation.property + +This is an DIL internal property, and it should not be used explicitly in job configurations.

    + +DIL reads activation entries from secondary input and generates work units for each activation entry, and +each of those work units is given a unique activation parameter using ms.activation.property, therefore +the work unit can be uniquely identified.

    + +Value in ms.activation.property is part of the work unit signature. + +## [ms.audit.enabled](ms.audit.enabled.md) + +Setting ms.audit.enabled to true will enable outgoing data auditing. Auditing will trace all outgoing data +including parameters and payloads to data lake through Kafka. + +## [ms.authentication](ms.authentication.md) + +ms.authentication job property defines the authentication of a request. It works with HTTP protocol only +for now, but could be used in other protocols. + +## ms.backfill + +This is for future back fill automation. It has no use currently. + +## [ms.call.interval.millis](ms.call.interval.millis.md) + +ms.call.interval.millis specifies the minimum time elapsed between requests in the pagination process. +When a page is retrieved sooner than the interval, to avoid QPS violation, the thread will wait until +the interval has passed. + +ms.call.interval.millis works within an executor thread. In cases of parallel execution, where the +number of executor threads is more than one, ms.call.interval.millis should be multiple of the interval +allowed by the QPS to avoid QPS violations cross threads. + +## [ms.connection.client.factory](ms.connection.client.factory.md) + +ms.connection.client.factory allows vendors specify customized connections with proxy or enhanced security. +The default factory is com.linkedin.cdi.DefaultConnectionClientFactory. + +## [ms.csv](ms.csv.md) + +[ms.csv](ms.csv.md) defines CSV extraction and conversion parameters. +It can have the following parameters: + +- **linesToSkip**, specifies how many lines of data to skip in the CSV payload. +The linesToSkip need to be more than the columnHeaderIndex. +For example, if columnHeaderIndex = 0, the number of lines to skip need to be at least 1. +When the linesToSkip is not set explicitly, and the columnHeaderIndex is set, linesToSkip = columnHeaderIndex + 1. +When neither linesToSkip and columnHeaderIndex are set, linesToSkip = 0. +If more lines need to be skipped after the header, then set this parameter explicitly. +- **columnHeaderIndex**, specifies the 0-based row index of the header columns if they are available. +CSV files may have 1 or more descriptive lines before the actual data. These descriptive lines, +including the column header line, should be skipped. +Note the column header line can be in any place of the skipped lines. +- **escapeCharacter**, specifies how characters can be escaped. Default is "u005C" (backslash \). +This can be specified as a variation of unicode without a backslash (\) before 'u'. +For example: \ can be specified as "u005c". +- **quoteCharacter**, specifies how source data are enclosed by columns. Default is double-quote ("). +This can be specified as a variation of unicode without a backslash (\) before 'u'. +For example: | can be specified as "u007C". +- **fieldSeparator**, specifies the field delimiter in the source csv data. The default is comma. +This can be specified as a variation of unicode without a backslash (\) before 'u'. +For example: tab (\t) can be specified as "u0009". +- **recordSeparator**, also called line separator, specifies the line or record +delimiter. The default is system line separator. +This can be specified as a variation of unicode without a backslash (\) before 'u'. +- **columnProjection**, defines how CSV columns should be arranged and filtered after parse, +before being sent to converter and writer to persist. +This feature is primarily used to extract selected columns from csv source without a header. +Column projection definition is a comma-separated string, where each value is either an +integer or a range, with each number representing the 0 based index of the field. +Column projection definition is inclusive, i.e., only the selected fields are included +in the final dataset, if a column projection is defined. +For example, to include the 0th, 2nd, 3rd, and 4th column from a source that has 6 columns, +set the value to: `"columnProjection": "0,2-4"` +- **defaultFieldType**, specifies a default type to supersede field type inference. +By default, CsvExtractor tries to infer the true type of fields when inferring schema +However, in some cases, the inference is not accurate, and users may prefer to keep all fields as strings. +In this case `"defaultFieldType": "string"`. +Supported types: string | int | long | double | boolean | float. + +## [ms.data.default.type](ms.data.default.type.md) + +`ms.data.default.type` provides a way to explicitly specifies data +types for certain fields. This is necessary when the source data has +empty fields, like placeholders, and DIL cannot infer its type properly. + +## [ms.data.explicit.eof](ms.data.explicit.eof.md) + +`ms.data.explicit.eof` specifies whether an explicit EOF record should +be sent to converter after processing all records. + +## [ms.data.field](ms.data.field.md) + +In a nested response, like JSON or Avro, `ms.data.field` specifies +where the core data (payload) is. + +## [ms.derived.fields](ms.derived.fields.md) + +Derived Fields are calculated fields that serve critical roles in data ingestion process, such as compaction. This includes, but is not +limited by, the following: + +- Convert a date string to a EPOC date value so that downstream compaction/partitioning can use the long EPOC value +- Extract a part of a field to form a primary key or delta key +- Provide a calculated value based on flow execution state or watermark, such as the often used extraction date derived field +- Lift up a nested element in the response to the toplevel and make it a toplevel field because only toplevel fields can be primary keys or delta keys +- Persist a job execution variable, such as the work unit identifier, into the final dataset + +## [ms.enable.cleansing](ms.enable.cleansing.md) + +Schema cleansing replaces special characters in the schema element names based +on a pattern. By default, it will replace all blank spaces, $, and @ to underscores. + +## [ms.enable.dynamic.full.load](ms.enable.dynamic.full.load.md) + +`ms.enable.dynamic.full.load` enables or disables dynamic full load. +When enabled (default) and `extract.is.full = false`, DIL will dynamically +perform a full load if it is a SNAPSHOT_ONLY extract or +if there is no pre-existing watermarks of the job. + +Dynamic full load is a DIL [Single Flow](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/single-flow.md) +feature that aims to alleviate users from coding 2 separate flows, +one for the full load and one for the incremental load. + +## [ms.enable.schema.based.filtering](ms.enable.schema.based.filtering.md) + +`ms.enable.schema.based.filtering` enables or disables schema-based filtering, +or column projection. When enabled, only fields specified schema +are projected to final dataset. + +## [ms.encryption.fields](ms.encryption.fields.md) + +`ms.encryption.fields` specifies a list of fields to be encrypted before +they are passed to converters. + +## [ms.extractor.class](ms.extractor.class.md) + +`ms.extractor.class` specifies the extractor class to use for data parsing. +The choice of extractor is based data format. Currently, DIL designed 4 +classes of extractors. + +## [ms.extractor.target.file.name](ms.extractor.target.file.name.md) + +`ms.extractor.target.file.name` specify the file name when +FileDumpExtractor is used. The file name can be specified as a +string container DIL variables. + +## [ms.extractor.target.file.permission](ms.extractor.target.file.permission.md) + +`ms.extractor.target.file.permission` set file permission when +FileDumpExtractor is used. + +## [ms.extract.preprocessors](ms.extract.preprocessors.md) + +`ms.extract.preprocessors` define one or more preprocessor classes that +handles the incoming data before they can be processed by the extractor. +When input data is compressed or encrypted, the input stream needs to +be preprocessed before it can be passed to an DIL extractor to parse. +`ms.extract.preprocessors` is a comma delimited string if there are +more than 1 preprocessors. + +## [ms.extract.preprocessor.parameters](ms.extract.preprocessor.parameters.md) + +When a source file is encrypted, it requires credentials to decrypt. +`ms.extract.preprocessor.parameters` defines parameters to pass into the +preprocessor along with the input. + +## [ms.grace.period.days](ms.grace.period.days.md) + +`ms.grace.period.days` addresses the late arrival problem, which is +very common if the ingestion source is a data warehouse. +`ms.grace.period.days` defines a Grace Period for incremental extraction, +and it adds extra buffer to cutoff timestamp during the +incremental load so that more data can be included. + +## [ms.http.conn.max](ms.http.conn.max.md) + +`ms.http.conn.max` defines maximum number of connections to keep +in a connection pool. It limits the total connections to an HTTP +server. The default value is 50. + +## [ms.http.conn.per.route.max](ms.http.conn.per.route.max.md) + +`ms.http.conn.per.route.max` defines maximum number of connections to keep +in a connection pool. It limits the total connections to a particular +path, or endpoint, on the HTTP server. The default value is 20. + +## [ms.http.conn.ttl.seconds](ms.http.conn.ttl.seconds.md) + +`ms.http.conn.ttl.seconds` defines maximum idle time allowed when there +is no activity on an HTTP connection. When there is no activity after +TTL passed, the connection is disconnected. The default is 10 seconds. + +## [ms.http.request.headers](ms.http.request.headers.md) + +`ms.http.request.headers` specifies custom headers including Content-Type that are to be +included in HTTP requests. + +## [ms.http.request.method](ms.http.request.method.md) + +The expected HTTP method to send the requests, decided by the data source. + +## [ms.http.response.type](ms.http.response.type.md) + +`ms.http.response.type` specifies less common response types in addition to +the default ones "application/json" or "text/csv". + +## [ms.http.statuses](ms.http.statuses.md) + +`ms.http.statuses` defines status codes that should be treated as success, +warning, or error. + +## ms.http.status.reasons + +`ms.http.status.reasons` is for future use. + +`http.status.reasons` define reason codes of special meaning in determining +whether a request was a success or failure. For example, when status is 200, but there is a +reason to indicate the request was not successful, then the status.reason can be set:`{"error": ["not found"]}`. +An HTTP response is considered success if and only if status code is in +http.statuses.success and reason code is not in http.status.reasons.error. + +Currently, we don't allow exceptions being made to revert errors by using reason code. + +## [ms.jdbc.schema.refactor](ms.jdbc.schema.refactor.md) + +`ms.jdbc.schema.refactor` specifies the function to apply to JDBC schema. +The choices are `toupper`, `tolower`, or `none` + +## [ms.jdbc.statement](ms.jdbc.statement.md) + +`ms.jdbc.statement` specifies the SQL statement for data retrieval. The value +can be any validate statement on any JDBC source. + +## [ms.kafka.brokers](ms.kafka.brokers.md) + +This specifies the Kafka broker host, such as `kafka.corp.com:1234` + +## [ms.kafka.clientId](ms.kafka.clientId.md) + +This specifies the Kafka client id, such as `dil-audit` + +## [ms.kafka.schema.registry.url](ms.kafka.schema.registry.url.md) + +`ms.kafka.schema.registry.url` specifies the auditing schema registry URL. + +## [ms.kafka.audit.topic.name](ms.kafka.audit.topic.name.md) + +`ms.kafka.audit.topic.name` specifies the auditing topic name, where +DIL wil send auditing events to if auditing is enabled. + +## [ms.normalizer.batch.size](ms.normalizer.batch.size.md) + +`ms.normalizer.batch.size` specifies the batch size for the normalizer converter +to group rows. Setting `ms.normalizer.batch.size` to 1 has special +effects of condensing a sparse table. + +## [ms.output.schema](ms.output.schema.md) + +`ms.output.schema` defines the output schema of extractors. Therefore, +it is also the input schema of the first converter. + +## [ms.pagination](ms.pagination.md) + +`ms.pagination` defines key pagination attributes. + +## [ms.parameters](ms.parameters.md) + +ms.parameter defines a list of named [variables](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/variables.md) +that can be referenced in other configuration properties using the syntax of double brackets {{variableName}}. + +## ms.payload.property + +`ms.payload.property` is an internal property that DIL uses to pass payloads to work units, +and it should not be used explicitly in job configurations. + +## ms.retention + +`ms.retention` is designed for future use. + +## [ms.s3.list.max.keys](ms.s3.list.max.keys.md) + +`ms.s3.list.max.keys` limit the number of keys when doing a "list" operation +on a S3 bucket. + +## [ms.schema.cleansing](ms.schema.cleansing.md) + +Schema cleansing replaces special characters in the schema element names based +on a pattern. By default, it will replace all blank spaces, $, and @ to underscores. + +## [ms.secondary.input](ms.secondary.input.md) + +Secondary inputs provides additional directives to job execution, in addition to +the primary inputs of job execution, which is its metadata, i.e, job configurations. + +## [ms.session.key.field](ms.session.key.field.md) + +Session is a state management mechanism over stateless connections. +For example, although Restful API is stateless, data sources can maintain +a session in backend by a status field, a session cursor, or through +[pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/pagination.md). + +`ms.session.key.field` specifies the key field in response in order to retrieve the +status for session control and the condition for termination. + +## [ms.sftp.conn.timeout.millis](ms.sftp.conn.timeout.millis.md) + +`ms.sftp.conn.timeout.millis` defines maximum allowed inactive time. The default is 60 seconds. + +## [ms.source.data.character.set](ms.source.data.character.set.md) + +`ms.source.data.character.set` specifies a character set to parse JSON or CSV payload. +The default source data character set is UTF-8, which should be good for most use cases. + +## [ms.source.files.pattern](ms.source.files.pattern.md) + +`ms.source.files.pattern` specifies a pattern to filter files from S3 and SFTP sources. + +## [ms.source.s3.parameters](ms.source.s3.parameters.md) + +`ms.source.s3.parameters` specifies parameters for S3 connection. + +## [ms.source.schema.urn](ms.source.schema.urn.md) + +Source schema represents the source data structure. Generally, in a data +ingestion scenario, the source data will be read in, projected, filtered, and +converted. Source schema can be read from the source, like for JDBC data sources, or parsed +from actual data, like JSON data, or defined as a string, or defined in a metadata +store. `ms.target.schema.urn` address the option that defines source schema in metadata store. + +## [ms.source.uri](ms.source.uri.md) + +[`ms.source.uri`](ms.source.uri.md) +defines the integration point, which is called data source for data ingestion or target for data egression. +It follows the [URI format](https://en.wikipedia.org/wiki/Uniform_Resource_Identifier). +The only exception is that authority is not supported, because all authority cannot be fit in the URI. + +## [ms.ssl](ms.ssl.md) + +[`ms.ssl`](ms.ssl.md) defines SSL parameters. + +## [ms.target.schema](ms.target.schema.md) + +`ms.target.schema` defines the target schema in a JsonArray string. +Target schema denotes the schema to be passed to writer, this applies +to situation where the source data are transformed through a converter +or other processes. + +## [ms.target.schema.urn](ms.target.schema.urn.md) + +Generally, target schema should be specified through target schema URN. +to avoid coding long schema strings. +An URN can point to the schema storage location on DataHub, which is +the only supported schema storage for now. + +## [ms.total.count.field](ms.total.count.field.md) + +Total Count field directs DIL how to retrieve the expected total row counts. This is important when there are large +volume of data and [pagination](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/pagination.md) +is used to retrieve data page by page. In cases of pagination, the expected total row count is one way to indicate +the end of pagination when the actually retrieved total number of rows matches or surpasses the expected total rows. + +## [ms.validation.attributes](ms.validation.attributes.md) + +`ms.validation.attributes` defines a threshold to mark a job as successful or failed. +The threshold can be specified as "success" or "failure" thresholds. The former is +called a "success" rule, and the later is called a "failure" rule. + +This property is required for [InFlowValidationConverter](https://github.com/linkedin/data-integration-library/blob/master/docs/components/InFlowValidationConverter.md), +which is a validation converter based on simple count comparison. + +## [ms.wait.timeout.seconds](ms.wait.timeout.seconds.md) + +`ms.wait.timeout.seconds` is one option to control pagination, it specifies +how long the job will wait before the session ending (success or failure) status is met. + +## [ms.watermark](ms.watermark.md) + +`ms.watermark` define watermarks for work unit generation, execution control, +and incremental processing. DIL supports 2 types of watermarks, `datetime` and `unit`. + +A datetime watermark is a reference. It doesn't directly effect or control +job execution. The watermark name and boundaries, low watermark +and high watermark, can be referenced in [variables](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/variable.md), which can +control execution. +See [ms.parameters](ms.parameters.md). + +A datetime watermark is a range, defined by its `from` and `to` field. The range +can be further partitioned per other configurations. +See [ms.work.unit.partition](ms.work.unit.partition.md) + +Therefore, a datetime watermark could generate 1 or more mini-watermarks when +partitioned, and each mini-watermark is mapped to a work unit. Therefore, +each work unit has its own unique watermark. + +A `unit` watermark defines a list of values that will be used by the DIL to +generate work units. The `unit` watermark name can be referenced as a [variable](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/variable.md) +directly. + +## ms.watermark.groups + +`ms.watermark.groups` is an DIL internal property, and it should not be used explicitly in job configurations. + +DIL uses this property to pass work unit signature to work units. + +**Alert**: setting this in GaaS flowSpecs may cause parsing error. + +## ms.work.unit.scheduling.starttime + +`ms.work.unit.scheduling.starttime` is an DIL internal property, and it should not be used explicitly in job configurations. + +When [work unit pacing](ms.work.unit.pacing.seconds.md) is enabled, the job planner will pass the scheduled start time to work units +so that work unit can wait for their moment to start. + +## [ms.work.unit.min.records](ms.work.unit.min.records.md) + +`ms.work.unit.min.records` specifies a minimum number of records that are expected. If the total +processed rows is less than `ms.work.unit.min.records`, the job will fail, generating an alert. + +## [ms.work.unit.min.units](ms.work.unit.min.units.md) + +`ms.work.unit.min.units` specify a minimum number of work units required for the job to be successful. +if the number of work units is smaller than `ms.work.unit.min.units`, the job will fail, sending an +alert to operations team. + +## [ms.work.unit.pacing.seconds](ms.work.unit.pacing.seconds.md) + +`ms.work.unit.pacing.seconds` can spread out work unit execution by adding a waiting time +in the front of each work unit's execution. The amount of wait time is based on the order of +the work units. It is calculated as `i * ms.work.unit.pacing.seconds`, where `i` is the sequence number +of the work unit. + +## [ms.work.unit.parallelism.max](ms.work.unit.parallelism.max.md) + +`ms.work.unit.parallelism.max` defines maximum possible parallel work +units that can be processed in one job execution. + +## [ms.work.unit.partial.partition](ms.work.unit.partial.partition.md) + +`ms.work.unit.partial.partition` specifies whether the last partition of a multi-day partition scheme can be partial. +If set to true, it allows the last multi-day partition to be partial (partial month or partial week). + +## [ms.work.unit.partition](ms.work.unit.partition.md) + +`ms.work.unit.partition` defines how the watermark will be partitioned to form +work units. When a watermark is partitioned, each partition will be processed as +a work unit. Partitioning, therefore, allows parallel processing. + +# Essential Gobblin Core Properties +The following are Gobblin core properties that are essential to job configuration. This is only a short list, +for a complete list of Gobblin core properties, please refer to Gobblin documentation. + +# [source.class](source.class.md) +# [converter.class](converter.class.md) diff --git a/docs/patterns/asynchronous-ingestion-pattern.md b/docs/patterns/asynchronous-ingestion-pattern.md new file mode 100644 index 0000000..6c2374c --- /dev/null +++ b/docs/patterns/asynchronous-ingestion-pattern.md @@ -0,0 +1,208 @@ +# Asynchronous Ingestion Pattern + +Asynchronous ingestion is common on cloud. In an asynchronous ingestion, a request +for data preparation is sent to the provider, and the data requester +can then check back with the provider on status. When the status changes +to ready, the requester can start the data extraction. + +An asynchronous ingestion can have 3, 4, or 5 steps. Different vendors implement +it slightly differently. But at typical implementation would have: + +- Start a request for data and retrieve a tracking number like "request id" +- Check status using the request id until it is ready +- Extract the data + +The requester can execute the above process asynchronously because it doesn't +need to wait for the data once the request is submitted. It can go do other +things, and check status every one a while. As some data providers use +buffers to stage data for extraction, the data preparation process can be +significant in many cases. So waiting synchronously doesn't make sense +for data extractors. + +## Step 1: Submit a Request + +Step 1 is a separate job. So it can also be called Job 1. + +Step 1 has a simple mission which to submit a request and get the Request ID. The Request +ID can come in different forms. It may be just a code, or a URL. Whichever it is, it should +allow the status checking in step 2. + +### Base Configuration + +Just like other jobs, the base configurations include source, +authentication, and others in order to connect to the data provider. + +Following are typical settings and explanation: + +- `data.publisher.final.dir=<>` +- `data.publisher.replace.final.dir=true` +- `writer.file.path.type=namespace_table` + +**Explanation**: This job is auxiliary, hence the results are normally not +persisted into final storage, but you can also make it into +final storage if needed. In above, `data.publisher.replace.final.dir` +and `writer.file.path.type` settings ensure the staging folder +is cleaned up everytime, so that prior responses are not accumulated. + +- `extract.table.type=snapshot_only` +- `state.store.enabled=false` +- `extract.is.full=true` + +**Explanation**: For the same reason of being auxiliary, we typically don't want to track incremental +values each time the list is pulled. + +### Step-1 Configuration + +- `ms.http.request.method=POST` + +**Explanation**: Because this step actual "creates" something one the server, so it is normally a POST +HTTP request. + +- `ms.output.schema=xxxx` + +**Explanation**: The output schema should match the response structure. At high level, Job 1 is +just a data ingestion job. Only it ingests a small piece of data. Therefore, it should have +a schema to describe the data. If the response is JSON format, it could be left out, as schema +can be inferred from JSON data. + +- `ms.derived.fields=xxxx` + +**Explanation**: When the Request ID is a sub-part of a response field, you would need to use derived fields +feature to extract out the Request ID. For example, the following extract the numbers after "/syncs/" in the "uri" +field of the response. + - `ms.derived.fields=[{"name": "syncid", "formula": {"type": "regexp", "source": "uri", "format": "/syncs/([0-9]+)$"}}]` + +- `-extract.table.name=xxxx` + +**Explanation**: This is the extracted table name for the request response. + +## Step 2: Check Request Status + +See [Status Checking Job](../how-to/status-check-job.md) for additional information. + +Step 2 is a separate job. So it can also be called Job 2. + +Step 2 needs to use session control techniques normally to periodically check status. Too +frequent checking could overwhelm the server and bust QPS. So it is important to +manage the pace. + +### Base Configuration + +Just like other jobs, the base configurations include source, +authentication, and others in order to connect to the data provider. + +Following are typical settings and explanation: + +- `data.publisher.final.dir=<>` +- `data.publisher.replace.final.dir=true` +- `writer.file.path.type=namespace_table` + +**Explanation**: This job is auxiliary, hence the results are normally not +persisted into final storage, but you can also make it into +final storage if needed. In above, `data.publisher.replace.final.dir` +and `writer.file.path.type` settings ensure the staging folder +is cleaned up everytime, so that prior responses are not accumulated. + +- `extract.table.type=snapshot_only` +- `state.store.enabled=false` +- `extract.is.full=true` + +**Explanation**: For the same reason of being auxiliary, we typically don't want to track incremental +values each time the list is pulled. + +### Step-2 Configuration + +- `ms.secondary.input=xxxx` + +**Explanation**: this is how you specify the location of job 1 output so that job 2 can read the request +ID from it. For example, the following directs the DIL to read the request ID from the "syncid" field +of the data file under "${job.dir}/${job1.table.name}". The secondary input file from job 1 needs +to be an Avro file. +-`ms.secondary.input=[{"path": "${job.dir}/${job1.table.name}", "fields": ["syncid"]}]` +See [ms.secondary.input](../parameters/ms.secondary.input.md). + +- `ms.call.interval.millis=xxxx` + +**Explanation**: This specifies how frequently the DIL should check the status. This should be a proper value that is +not too small, which could bust QPS, or too big, which could slow down the process. + +- `extract.table.name=xxxx` + +**Explanation**: This is the extracted table name for the status checking responses. This table might get multiple records +as each check will write a record to it. + +- `ms.session.key.field=xxxx` + +**Explanation**: This is the place to specify when the status is ready, or fail. For example, +the following looks for a "success" or "ready" value in the "status" field of the response. +- `ms.session.key.field={"name": "status", "condition": {"regexp": "success|ready"}}` +See [ms.session.key.field](../parameters/ms.session.key.field.md) and [Status Checking Job](../how-to/status-check-job.md). + +- `ms.wait.timeout.seconds=xxxx` + +**Explanation**: This limit the waiting time. The job wait for "timeout" time and fail if the status doesn't +turn to ready after the "timeout" time. This parameter should have a reasonably estimated value based on server +performance. + +## Step 3: Extract Data + +Step 3 is a separate job. So it can also be called Job 3. + +### Base Configuration + +Just like other jobs, the base configurations include source, +authentication, and others in order to connect to the data provider. + +### Step-3 Configuration + +- `data.publisher.final.dir=<>` + +**Explanation**: This is the step to download the data, hence you would save the data in a +permanent location for subsequent processing. + +- `extract.table.type=snapshot_append` + +**Explanation**: Large datasets are normally extracted incrementally so that we don't need to extract +everything every day. But there are also cases where datasets are extracted in full every time. + +- `state.store.enabled=true` + +**Explanation**: State store would have to be enabled if data is extracted incrementally. Repeated full +extract would not need state store. + +- `extract.is.full=false` + +**Explanation**: This has to be false if data is extracted incrementally. + +- `ms.secondary.input=xxxx` + +**Explanation**: Job 3 normally would also need the request ID from Job 1 to download data. +This is how you specify the location of job 1 output so that job 3 can read the request +ID from it. For example, the following directs the DIL to read the request ID from the "syncid" field +of the data file under "${job.dir}/${job1.table.name}". The secondary input file from job 1 needs +to be an Avro file. +-`ms.secondary.input=[{"path": "${job.dir}/${job1.table.name}", "fields": ["syncid"]}]` +See [ms.secondary.input](../parameters/ms.secondary.input.md). + +- `extract.table.name=xxxx` + +**Explanation**: This is the extracted table name for the final dataset. + +## Other Variations + +OAuth2 authentication can be used in asynchronous ingestion. In such case, we will have an [authentication +job](../how-to/authentication-job.md) up front. + +Then the [secondary input](../parameters/ms.secondary.input.md) would have to include +both the request ID and access token. + +See the SFDC bulk API ingestion example: +![SFDC bulk API ingestion](../images/sfdc-bulk-ingestion.png) + +## Sample Applications + +Sample applications of this pattern include: +- [Salesforce (SFDC) bulk API ingestion](../sample-configs/sfdc-bulk-api-ingestion.md) +- [Eloqua API ingestion](../sample-configs/eloqua-api-ingestion.md) + +[Back to Pattern Summary](summary.md) \ No newline at end of file diff --git a/docs/patterns/egression-validation-pattern b/docs/patterns/egression-validation-pattern new file mode 100644 index 0000000..4ef7ac9 --- /dev/null +++ b/docs/patterns/egression-validation-pattern @@ -0,0 +1,4 @@ + + + +[Back to Pattern Summary](summary.md) \ No newline at end of file diff --git a/docs/patterns/summary.md b/docs/patterns/summary.md new file mode 100644 index 0000000..458b09d --- /dev/null +++ b/docs/patterns/summary.md @@ -0,0 +1,81 @@ +# FLow Design Patterns + +## [Asynchronous Ingestion Pattern](asynchronous-ingestion-pattern.md) + +Asynchronous ingestion is common on cloud. In an asynchronous ingestion, a request +for data preparation is sent to the provider, and the data requester +can then check back with the provider on status. When the status changes +to ready, the requester can start the data extraction. + +An asynchronous ingestion can have 3, 4, or 5 steps. Different vendors implement +it slightly differently. But at typical implementation would have: + +- Start a request for data and retrieve a tracking number like "request id" +- Check status using the request id until it is ready +- Extract the data + +The requester can execute the above process asynchronously because it doesn't +need to wait for the data once the request is submitted. It can go do other +things, and check status every one a while. As some data providers use +buffers to stage data for extraction, the data preparation process can be +significant in many cases. So waiting synchronously doesn't make sense +for data extractors. + +Sample applications of this pattern include: +- [Salesforce (SFDC) bulk API ingestion](../sample-configs/sfdc-bulk-api-ingestion.md) +- [Eloqua API ingestion](../sample-configs/eloqua-api-ingestion.md) + +## [2-step File Download Pattern](two-step-file-download-pattern.md) + +In downloading files from cloud (S3, GCS, Azsure) or FTP (SFTP), the preferred +practice is to do it in 2 steps if the number of objects to be downloaded can be +more than 1. The first step is to list all the files, and save into a staging file; +the second step is to download the files one by one. + +## [2-step Ingestion Pattern](two-step-ingestion-pattern.md) + +In ingesting large amount of data from a provider, we often extract an attribute +that has a short list of values, and use that list of values to divide the +ingestion of the larger dataset. + +DIL support diving large ingestion by [work units](../concepts/work-unit.md). +In a 2-step ingestion pattern, a list of attributes can be used to create work +units. And each work unit can be tracked individually in state store. + +In this pattern, the first step is to extract a list of values by one or more +attributes that can fairly evenly divide the whole dataset. The second step is +extract the whole dataset based on the values obtained in step 1. + +This pattern has a lot of similarity with the [2-step file download pattern](two-step-file-download-pattern.md) +It is a more general pattern of ingestion from any sources. One typical application is +survey response ingestion. In the survey application, survey responses are often +organized by surveys. Therefore, we can extract a list of survey IDs first, and +then extract the responses for each survey ID. + +The overall design of this pattern is: +- a job that gets a small set of attributes (mostly 1 attribute) with a finite list of +values that can very well evenly divide the whole dataset. The best candidates are +partition keys. Others like index columns are also desirable. If data is stored on the +source in separate tables of the same structure, the table names. +- a second job that retrieves the full dataset using job 1 output to create work units. Each +work unit ingests a filtered subset of data. The work units don't have to be executed +in one execution. They can be executed in a series of recurring runs if the number of work +units is high. + +The advantages of using the 2-step method are: + +- A large dataset can be ingested in a very controlled fashion +- Failure in one unit of work doesn't impact the overall success, the retrying process +can rerun the work unit alone if it fails. +- The work units can be ingested in parallel + +Sample applications of this pattern include: + +- [Qualaroo Survey API Ingestion](../sample-configs/qualaroo-survey-api-ingestion.md) +- [Google Search Console API Ingestion](../sample-configs/google-search-console-api-ingestion.md) +- [Zoom Webinar API Ingestion](../sample-configs/zoom-webinar-api-ingestion.md) +- [Decipher Survey API Ingestion](../sample-configs/decipher-survey-api-ingestion.md) +- [Thematic Survey API Ingestion](../sample-configs/thematic-survey-api-ingestion.md) + + +## [Egression with Validation Pattern](egression-validation-pattern.md) diff --git a/docs/patterns/two-step-file-download-pattern.md b/docs/patterns/two-step-file-download-pattern.md new file mode 100644 index 0000000..ad646be --- /dev/null +++ b/docs/patterns/two-step-file-download-pattern.md @@ -0,0 +1,137 @@ +# 2-step File Download Pattern + +In downloading files from cloud (S3, GCS, Azsure) or FTP (SFTP), the preferred +practice is to do it in 2 steps if the number of objects to be downloaded can be +more than 1. The first step is to list all the files, and save into a staging file; +the second step is to download the files one by one. + +The advantages of using the 2-step method are: + +- The files can be downloaded individually through parallel processes. +- The files can be tracked individually, and one file's failure will not impact others. +- The staging file can be useful information for monitoring and troubleshooting. + +Here is how to do so. + +## Step 1: Retrieve File Names + +Step 1 is to configure a job that reads from the source all file names, and +save the list of file names into a file. + +### Base Configuration + +Just like other jobs, the base configurations include source, +authentication, and others in order to retrieve the needed +status information. + +Following are typical settings and explanation: + +- `data.publisher.final.dir=<>` +- `data.publisher.replace.final.dir=true` +- `writer.file.path.type=namespace_table` + +**Explanation**: This job is auxiliary, hence the results are normally not +persisted into final storage, but you can also make it into +final storage if needed. In above, `data.publisher.replace.final.dir` +and `writer.file.path.type` settings ensure the staging folder +is cleaned up everytime, so that prior files are not accumulated. + +- `extract.table.type=snapshot_only` +- `state.store.enabled=false` +- `extract.is.full=true` + +**Explanation**: For the same reason of being auxiliary, we typically don't want to track incremental +values each time the list is pulled. + +### Step-1 Configuration + +- `ms.extractor.target.file.name=` + +**Explanation**: By setting `ms.extractor.target.file.name` +to blank, DIL will output the results of the "list" command +to files. The whole logic works as the following: + +- retrieve a list of files based on `ms.source.uri` configuration +- is `ms.extractor.target.file.name` blank? + - if is blank: + - List the files and output the results as CSV + - if is not blank: + - the number of files == 1 + - dump the file content + - the number of files > 1 + - dump the first file which matches the pattern + +**Note**: Per above logic, it is also sufficient by +setting `ms.source.files.pattern` to blank. + +**Note**: `ms.source.uri` can have patterns acceptable to the "list" +command, e.g. `ms.source.uri=/home/user/download/*.gpg`. + +- `ms.output.schema=[{"columnName":"filePath","isNullable":"true","dataType":{"type":"string"}}]` + +**Explanation**: give the output an explict schema because the data retrieved +from the "list" command has not schema. In this case, we will label the +file names as "filePath". + +- `converter.classes=com.linkedin.cdi.converter.CsvToJsonConverterV3,org.apache.gobblin.converter.avro.JsonIntermediateToAvroConverter` + +**Explanation**: because the output of the "list" command is CSV, we will +convert it to Json and then to Avro. + +## Step 2: Download File Individually + +Step 2 is to configure a job that reads the file names from step 1, and +initiate a downloading work unit for each of them. + +### Base Configuration + +Just like other jobs, the base configurations include source, +authentication, and others in order to retrieve the needed +status information. + +Following are typical settings and explanation: + +- `state.store.enabled=true` + +**Explanation**: Enable state store so that files can be tracked +individually. + +### Step-2 Configuration + +#### Required Configurations + +- `ms.secondary.input=[{"path": "<>", "fields": ["filePath"]}]` + +**Explanation**: configure the location to read the list file, and specify the field +that contain file names (paths). The field name "filePath" will lead to +a dynamic variable being generated internally. + +- `ms.source.uri={{filePath}}` + +**Explanation**: specify that the source is the path name. If +there are 10 files (10 paths), the secondary input will get 10 records, and each +will be assigned to a work unit. Then each work unit will have a path in the +dynamic variable {{filePath}}. + +- `ms.extractor.target.file.name={{filePath}}` + +**Explanation**: by setting `ms.extractor.target.file.name`, DIL will +dump the file. + +#### Optional Configurations + +- `ms.work.unit.parallelism.max=100` + +**Explanation**: optionally can limit the number of files to download when +there are large number of files. In such case, the step 2 job can be repeated +to download all files in batches of 100. + +## Adding a Status Check Step + +The 2-step process can be modified with a status check upfront, +the status check job can ensure files are ready before starting +the downloading process. + +See [how-to: status check](../how-to/status-check-job.md) + +[Back to Pattern Summary](summary.md) \ No newline at end of file diff --git a/docs/patterns/two-step-ingestion-pattern.md b/docs/patterns/two-step-ingestion-pattern.md new file mode 100644 index 0000000..ef6143d --- /dev/null +++ b/docs/patterns/two-step-ingestion-pattern.md @@ -0,0 +1,141 @@ +# 2-step Ingestion Pattern + +In ingesting large amount of data from a provider, we often extract an attribute +that has a short list of values, and use that list of values to divide the +ingestion of the larger dataset. + +DIL support diving large ingestion by [work units](../concepts/work-unit.md). +In a 2-step ingestion pattern, a list of attributes can be used to create work +units. And each work unit can be tracked individually in state store. + +In this pattern, the first step is to extract a list of values by one or more +attributes that can fairly evenly divide the whole dataset. The second step is +extract the whole dataset based on the values obtained in step 1. + +This pattern has a lot of similarity with the [2-step file download pattern](two-step-file-download-pattern.md) +It is a more general pattern of ingestion from any sources. One typical application is +survey response ingestion. In the survey application, survey responses are often +organized by surveys. Therefore, we can extract a list of survey IDs first, and +then extract the responses for each survey ID. + +The overall design of this pattern is: +- a job that gets a small set of attributes (mostly 1 attribute) with a finite list of +values that can very well evenly divide the whole dataset. The best candidates are +partition keys. Others like index columns are also desirable. If data is stored on the +source in separate tables of the same structure, the table names. +- a second job that retrieves the full dataset using job 1 output to create work units. Each +work unit ingests a filtered subset of data. The work units don't have to be executed +in one execution. They can be executed in a series of recurring runs if the number of work +units is high. + +The advantages of using the 2-step method are: + +- A large dataset can be ingested in a very controlled fashion +- Failure in one unit of work doesn't impact the overall success, the retrying process +can rerun the work unit alone if it fails. +- The work units can be ingested in parallel + +Here is how to do so. + +## Step 1: Retrieve Partitioning Attributes + +Step 1 is to configure a job that reads only the partitioning attributes +from the source, and save the list of values into a file, in avro format. + +### Base Configuration + +Just like other jobs, the base configurations include source, +authentication, and others in order to retrieve the needed +status information. + +Following are typical settings and explanation: + +- `data.publisher.final.dir=<>` +- `data.publisher.replace.final.dir=true` +- `writer.file.path.type=namespace_table` + +**Explanation**: This job is auxiliary, hence the results are normally not +persisted into final storage, but you can also make it into +final storage if needed. In above, `data.publisher.replace.final.dir` +and `writer.file.path.type` settings ensure the staging folder +is cleaned up everytime, so that prior files are not accumulated. + +- `extract.table.type=snapshot_only` +- `state.store.enabled=false` +- `extract.is.full=true` + +**Explanation**: For the same reason of being auxiliary, we typically don't want to track incremental +values each time the list is pulled. + +### Step-1 Configuration + +- `ms.output.schema=xxx` + +**Explanation**: give the output an explict schema if needed. This is optional. +If the output is Json format, the schema +inference usually works well because it is normally very simple. If the source is JDBC, the +schema can be retrieved from the dataset. + +- `extract.table.name=xxx` + +**Explanation**: the `extract.table.name` will used in step 2 to create work units. + + +## Step 2: Ingest the Full Dataset + +Step 2 is to configure a job that reads the values from step 1, and +initiate a downloading work unit for each of them. If there are 2 or more +attributes, the combinations of values as listed +in the step 1 output table are used to create work units. + +### Base Configuration + +Just like other jobs, the base configurations include source, +authentication, and others in order to retrieve the needed +status information. + +Following are typical settings and explanation: + +- `state.store.enabled=true` + +**Explanation**: Enable state store so that work units can be tracked +individually. + +### Step-2 Configuration + +#### Required Configurations + +- `ms.secondary.input=[{"path": "<>", "fields": ["attribute1", "attribute2"...]}]` + +**Explanation**: configure the location to read the list file. The list file can contain +extra fields, but the job can take only the needed ones. This job property also allows +filtering, to include or exclude certain records. After filtering, each record makes a work unit. +The fields chosen will make the signature of each work unit. See [ms.secondary.input](../parameters/ms.secondary.input.md) + +- `ms.watermark=xxxx` + +**Explanation**: each work unit can have watermarks of its own, but the definition of watermark +is the same across all units created by the secondary input. If the definition of watermark +creates time partitions, each unit from secondary input will have the same number of time partitions. +The combinations (matrix) of time partitions and units from secondary input makes the final work units. +See [concept: work unit](../concepts/work-unit.md) + +#### Optional Configurations + +- `ms.work.unit.parallelism.max=100` + +**Explanation**: optionally can limit the number of work units to be processed in one execution. +And the step 2 can be repeated to ingest the full dataset in batches of 100 work units each time. + +## Sample Applications + +Sample applications of this pattern include: + +- [Qualaroo Survey API Ingestion](../sample-configs/qualaroo-survey-api-ingestion.md) +- [Google Search Console API Ingestion](../sample-configs/google-search-console-api-ingestion.md) +- [Zoom Webinar API Ingestion](../sample-configs/zoom-webinar-api-ingestion.md) +- [Decipher Survey API Ingestion](../sample-configs/decipher-survey-api-ingestion.md) +- [Thematic Survey API Ingestion](../sample-configs/thematic-survey-api-ingestion.md) + + +[Back to Pattern Summary](summary.md#2-step-ingestion-pattern) \ No newline at end of file diff --git a/docs/sample-configs/decipher-survey-api-ingestion.md b/docs/sample-configs/decipher-survey-api-ingestion.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/sample-configs/eloqua-api-ingestion.md b/docs/sample-configs/eloqua-api-ingestion.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/sample-configs/google-search-console-api-ingestion.md b/docs/sample-configs/google-search-console-api-ingestion.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/sample-configs/ingest-csv-from-s3.md b/docs/sample-configs/ingest-csv-from-s3.md new file mode 100644 index 0000000..101065d --- /dev/null +++ b/docs/sample-configs/ingest-csv-from-s3.md @@ -0,0 +1,80 @@ +# Sample 1: S3/CSV Ingestion + +This sample Gobblin job configuration demonstrates how to ingest data from AWS S3 and dump to a local HDFS data lake. + +The job can be executed through Gobblin Standalone if parameters are made into a .pull file. It can also be executed on Azkaban when parameters are made into a .job file. + +## Define the ingestion protocol (required) + +Source.class is a Gobblin Core parameter. It defines the protocol to be used. In this case, we are using S3. The class for S3 is [S3SourceV2](https://github.com/linkedin/data-integration-library/blob/master/docs/components/S3SourceV2.md). Here V2 means that it uses S3 Version 2 SDK. + +> source.class=com.linkedin.cdi.source.S3SourceV2 + +## Define data source parameters (required) + +Next, we will give more specific information about the data source. + +ms.source.uri is DIL parameter. It defines how the source is identified. In this case, `commoncrawl.s3.amazonaws.com` specifies the bucket `commoncrawl` on S3. The rest of the URI specifies the S3 key of the resource (data file). + +> ms.source.uri=https://commoncrawl.s3.amazonaws.com/crawl-data/CC-MAIN-2019-43/cc-index.paths.gz + +## Define the format of data being ingested (required) + +Ms.extractor.class is a DIL parameter. It defines the format of input data stream, and thus specifies which data parser to use extract the metadata and payload from input stream. + +The extractor class for CSV is [CsvExtractor](https://github.com/linkedin/data-integration-library/blob/master/docs/components/CsvExtractor.md). This extractor parses the CSV structure of the payload together with other tasks. + +> ms.extractor.class=com.linkedin.cdi.extractor.CsvExtractor + +## Define the data conversion (optional) + +Converter.class is a Gobblin Core parameter. It defines a series conversions before data are stored on data lake. In this case, we convert CSV to JSON, and convert to JSON to AVRO. + +> converter.classes=org.apache.gobblin.converter.csv.CsvToJsonConverterV2,org.apache.gobblin.converter.avro.JsonIntermediateToAvroConverter + +## Define preprocessing (optional) + +Because of compression and encryption, payload data, downloaded from data sources, may not be directly parsable. They may have to be unzipped or unencrypted before they can be parsed by extractors. + +ms.extract.preprocessors is DIL parameter. It can define one or more preprocessors. In this case, we have only one [GunzipProcessor](https://github.com/linkedin/data-integration-library/blob/master/docs/components/GunzipProcessor.md). + +> ms.extract.preprocessors=com.linkedin.cdi.preprocessor.GunzipProcessor + +## Define source data schema (optional) +ms.output.schema is a DIL parameter. It defines the [schema](https://github.com/linkedin/data-integration-library/blob/master/docs/concepts/schema.md) of ingested data. In this case, the ingested data contains 1 field of type `string`. The source data actually doesn't have a column name, for CSV ingestion, we can provide custom column names if source doesn't provide. + +Source data schema can be inferred if not provided. CSV fields will be inferred mostly as nullable strings. + +> ms.output.schema=[{"columnName":"path","isNullable":"true","dataType":{"type":"string"}}] + +## Define data lake properties of the ingested data (optional) + +These Gobblin Core parameters define where target files are published and state stores are kept. In this case, we will keep them all in local file system. + +> fs.uri=file://localhost/ +> state.store.fs.uri=file://localhost/ +> data.publisher.final.dir=/tmp/gobblin/job-output + +These Gobblin Core parameters define how the ingested data to be named. + +> extract.namespace=com.linkedin.test +> extract.table.name=test + +These Gobblin Core parameters define how the ingested data to be processed incrementally if repeating ingestion is incremental, i.e., each ingestion has only incremental changes since last ingestion. + +> extract.table.type=SNAPSHOT_ONLY +> extract.is.full=true + +## Define target file properties (optional) + +These Gobblin Core parameters define how the ingested data are to be stored. + +> writer.destination.type=HDFS +> writer.output.format=AVRO + +## Define job properties (required) + +Job.name is both a Gobblin Core and Azkaban properties. Job properties are required, and they are specific to your execution environment. In this case, we will execute the job through Gobblin Standalone. + +> job.name=testJob + diff --git a/docs/sample-configs/qualaroo-survey-api-ingestion.md b/docs/sample-configs/qualaroo-survey-api-ingestion.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/sample-configs/sfdc-bulk-api-ingestion.md b/docs/sample-configs/sfdc-bulk-api-ingestion.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/sample-configs/thematic-survey-api-ingestion.md b/docs/sample-configs/thematic-survey-api-ingestion.md new file mode 100644 index 0000000..e69de29 diff --git a/docs/sample-configs/zoom-webinar-api-ingestion.md b/docs/sample-configs/zoom-webinar-api-ingestion.md new file mode 100644 index 0000000..e69de29 diff --git a/gradle/scripts/dependencyDefinitions.gradle b/gradle/scripts/dependencyDefinitions.gradle index 3ff75d8..a2c6293 100644 --- a/gradle/scripts/dependencyDefinitions.gradle +++ b/gradle/scripts/dependencyDefinitions.gradle @@ -83,7 +83,6 @@ ext.externalDependency = [ "influxdbJava": "org.influxdb:influxdb-java:2.1", "kryo": "com.esotericsoftware.kryo:kryo:2.22", "libthrift":"org.apache.thrift:libthrift:0.9.3", - "lombok":"org.projectlombok:lombok:1.16.8", "mockRunnerJdbc":"com.mockrunner:mockrunner-jdbc:1.0.8", "xerces":"xerces:xercesImpl:2.11.0", "typesafeConfig": "com.typesafe:config:1.2.1", @@ -166,7 +165,6 @@ ext.externalDependency = [ "gobblin-crypto": "org.apache.gobblin:gobblin-crypto:" + gobblinVersion, "gobblin-crypto-provider": "org.apache.gobblin:gobblin-crypto-provider:" + gobblinVersion, "gobblin-data-management": "org.apache.gobblin:gobblin-data-management:" + gobblinVersion, - "gobblin-flow-config-service-api": "org.apache.gobblin:gobblin-data-management:" + gobblinVersion, "gobblin-helix": "org.apache.gobblin:gobblin-hive-registration:" + gobblinVersion, "gobblin-hive-registration": "org.apache.gobblin:gobblin-hive-registration:" + gobblinVersion, "gobblin-http": "org.apache.gobblin:gobblin-http:" + gobblinVersion, @@ -179,10 +177,7 @@ ext.externalDependency = [ "gobblin-rest-api": "org.apache.gobblin:gobblin-rest-api:" + gobblinVersion, "gobblin-rest-api-data-template": "org.apache.gobblin:gobblin-rest-api-data-template:" + gobblinVersion, "gobblin-runtime": "org.apache.gobblin:gobblin-runtime:" + gobblinVersion, - "gobblin-salesforce": "org.apache.gobblin:gobblin-runtime:" + gobblinVersion, - "gobblin-sql": "org.apache.gobblin:gobblin-runtime:" + gobblinVersion, + "gobblin-sql": "org.apache.gobblin:gobblin-sql:" + gobblinVersion, "gobblin-utility": "org.apache.gobblin:gobblin-utility:" + gobblinVersion, - - "okhttp": "com.squareup.okhttp3:okhttp:4.8.0", "commonsValidator": "commons-validator:commons-validator:1.6" ]