diff --git a/.gitignore b/.gitignore
new file mode 100644
index 0000000..82806b8
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1,90 @@
+# Compiled source #
+###################
+*.com
+*.class
+*.dll
+*.exe
+*.a
+*.o
+*.so
+*.node
+
+# Node Waf Byproducts #
+#######################
+.lock-wscript
+build/
+autom4te.cache/
+
+# Node Modules #
+################
+# Better to let npm install these from the package.json defintion
+# rather than maintain this manually
+node_modules/
+
+# Packages #
+############
+# it's better to unpack these files and commit the raw source
+# git has its own built in compression methods
+*.7z
+*.dmg
+*.gz
+*.iso
+*.jar
+*.rar
+*.tar
+*.zip
+
+# Logs and databases #
+######################
+*.log
+dump.rdb
+*.tap
+
+
+# OS generated files #
+######################
+.DS_Store?
+.DS_Store
+ehthumbs.db
+Icon?
+Thumbs.db
+
+# thrift generated files #
+##########################
+generated/
+
+# NodeJS Core Dump
+core
+
+# Jenkins build scripts
+rt-jenkins/
+
+# Coverage Reports
+coverage/
+
+# local docs, scratchboards
+localdocs/
+
+# vi temp files
+.*.swp
+
+# intelliJ
+.idea/
+*.iml
+
+# Project specific items (local conf, build dir)
+config/local.json
+maps-evidence/
+*.lst
+classes/
+target/
+*.dat
+
+# shaded jar pom file
+dependency-reduced-pom.xml
+
+# output of build plugin org.codehaus.mojo build-helper-maven-plugin
+test_properties.props
+
+# gradle generated logs
+.gradle
diff --git a/checkstyles/marmaray-0.0.1.xml b/checkstyles/marmaray-0.0.1.xml
new file mode 100644
index 0000000..fa3cf98
--- /dev/null
+++ b/checkstyles/marmaray-0.0.1.xml
@@ -0,0 +1,136 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/checkstyles/suppressions.xml b/checkstyles/suppressions.xml
new file mode 100644
index 0000000..f516bb7
--- /dev/null
+++ b/checkstyles/suppressions.xml
@@ -0,0 +1,13 @@
+
+
+
+
+
+
+
+
+
+
+
diff --git a/marmaray-tools/pom.xml b/marmaray-tools/pom.xml
new file mode 100644
index 0000000..1fc5334
--- /dev/null
+++ b/marmaray-tools/pom.xml
@@ -0,0 +1,22 @@
+
+ 4.0.0
+
+ 1.8
+ 1.8
+
+
+ com.uber.marmaray
+ marmaray-base
+ 1.0-SNAPSHOT
+
+ marmaray-tools
+ 1.0-SNAPSHOT
+
+
+
+ com.uber.marmaray
+ marmaray
+ 1.0-SNAPSHOT
+
+
+
diff --git a/marmaray-tools/src/main/cli/toggleHDFSMetadataFile.py b/marmaray-tools/src/main/cli/toggleHDFSMetadataFile.py
new file mode 100755
index 0000000..2e96826
--- /dev/null
+++ b/marmaray-tools/src/main/cli/toggleHDFSMetadataFile.py
@@ -0,0 +1,62 @@
+"""
+Copyright (c) 2018 Uber Technologies, Inc.
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions
+of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+IN THE SOFTWARE.
+"""
+
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+
+
+"""
+A script to either enable or disable a HDFS metadata file. A metadata file is disabled if the extension ".tmp"
+is appended at the end. Enabling a metadata file will remove the same extension. This tool will allow users
+to disable metadata files and associated watermarks inside these files used to determine which partition to process.
+
+Usage: toggleHDFSMetadataFile -f sample_file -d
+ toggleHDFSMetadataFile -f sample_file.tmp -e
+"""
+import argparse, sys
+import shutil
+import logging
+from os.path import basename, dirname, splitext, isfile
+
+def main(args):
+ logging.basicConfig(level=logging.INFO)
+ disabled_ext = ".tmp"
+
+ if not isfile(args.file):
+ logging.error("the file does not exist")
+ quit()
+
+ if args.disable:
+ if not args.file.endswith(disabled_ext):
+ logging.info("Disabling %s", args.file)
+ shutil.move(args.file, args.file + disabled_ext)
+ else:
+ logging.warning("the specified filename already ends with a .tmp extension")
+ if args.enable:
+ if args.file.endswith(disabled_ext):
+ logging.info("Enabling %s ", args.file)
+ shutil.move(args.file, dirname(args.file) + splitext(basename(args.file))[0])
+ else:
+ logging.error("the file must end with a .tmp extension")
+
+if __name__ == '__main__':
+ p= argparse.ArgumentParser()
+ p.add_argument('-f', '--file', help='File to toggle on/off as metadata file')
+ p.add_argument('-e', '--enable', help='Enable .tmp file as metadata file', action='store_true')
+ p.add_argument('-d', '--disable', help='disable metadata file', action='store_true')
+ args = p.parse_args()
+ main(args)
diff --git a/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPrinter.java b/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPrinter.java
new file mode 100644
index 0000000..8b13c25
--- /dev/null
+++ b/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPrinter.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.tools;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.metadata.HDFSMetadataManager;
+import com.uber.marmaray.common.metadata.StringValue;
+import com.uber.marmaray.utilities.CommandLineUtil;
+import com.uber.marmaray.utilities.FSUtils;
+import java.io.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.util.Map;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+@Slf4j
+public class HDFSMetadataPrinter {
+ private static final String METADATA_FILE_OPTION = "mfile";
+
+ public static void main(final String[] args) throws ParseException, IOException {
+ final CommandLineParser parser = new GnuParser();
+ final Options options = getCLIOptions();
+ final CommandLine cmd;
+
+ try {
+ cmd = parser.parse(options, args);
+ } catch (final ParseException e) {
+ final String cmdLineSyntax = "java -cp [jar_name] com.uber.marmaray.tools.HDFSMetadataPrinter "
+ + "-m [METADATA_FILE]";
+ final String header = "This tools prints out all the metadata contents of a HDFS metadata file.";
+ final String footer = "For help, please contact the Hadoop Data Platform team";
+ CommandLineUtil.printHelp(options, cmdLineSyntax, header, footer);
+ throw e;
+ }
+
+ final String metadataFilePath = cmd.getOptionValue(METADATA_FILE_OPTION);
+ Preconditions.checkState(!Strings.isNullOrEmpty(metadataFilePath));
+
+ log.info("Printing contents of metadata file: " + metadataFilePath);
+
+ final Configuration conf = new Configuration();
+ final FileSystem fs = FSUtils.getFs(conf);
+ try (final InputStream is = new BufferedInputStream(fs.open(new Path(metadataFilePath)))) {
+ try (final ObjectInputStream input = new ObjectInputStream(is)) {
+ final Map metadataMap = HDFSMetadataManager.deserialize(input);
+ metadataMap.entrySet()
+ .stream()
+ .forEach(entry ->
+ log.info("Key: " + entry.getKey() + " Value: " + entry.getValue().getValue()));
+ }
+ }
+ }
+
+ private static Options getCLIOptions() {
+ final Options options = new Options();
+ options.addOption(CommandLineUtil.generateOption("m", METADATA_FILE_OPTION, true, "HDFS metadata file", true));
+ return options;
+ }
+}
diff --git a/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPruner.java b/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPruner.java
new file mode 100644
index 0000000..b2bbe90
--- /dev/null
+++ b/marmaray-tools/src/main/java/com/uber/marmaray/tools/HDFSMetadataPruner.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.tools;
+
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.metadata.HDFSMetadataManager;
+import com.uber.marmaray.utilities.CommandLineUtil;
+import com.uber.marmaray.utilities.FSUtils;
+import java.io.IOException;
+import java.util.Comparator;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import parquet.Preconditions;
+
+@Slf4j
+public class HDFSMetadataPruner {
+ private static final String HDFS_PATH_LONG_OPTION = "path";
+ private static final String NUM_METADATA_FILES_RETAINED_LONG_OPTION = "numFiles";
+ private static final String FAKE_DELETE_LONG_OPTION = "fake";
+ private static final String HDFS_PATH_SHORT_OPTION = "p";
+ private static final String NUM_METADATA_FILES_RETAINED_SHORT_OPTION = "n";
+ private static final String FAKE_DELETE_SHORT_OPTION = "f";
+
+ // Metadata file names in HDFS = nanoseconds since epoch so we can sort by name
+ private static final Comparator byTimestampedNameAsc =
+ Comparator.comparingLong(f1 -> Long.parseLong(f1.getPath().getName()));
+
+ // Todo - consider putting main functionality in a static utility method
+ public static void main(final String[] args) throws ParseException, IOException {
+ final CommandLineParser parser = new GnuParser();
+ final Options options = getCLIOptions();
+ final CommandLine cmd;
+
+ try {
+ cmd = parser.parse(options, args);
+ } catch (final ParseException e) {
+ final String cmdLineSyntax =
+ String.format("java -cp [jar_name] com.uber.marmaray.tools.HDFSMetadataCuller "
+ + "-%s [METADATA_PATH] -%s [NUM_METADATA_FILES_RETAINED] -%s [FAKE_DELETE_BOOLEAN]",
+ HDFS_PATH_SHORT_OPTION, NUM_METADATA_FILES_RETAINED_SHORT_OPTION, FAKE_DELETE_SHORT_OPTION);
+ final String header = "This tool prunes metadata files for an HDFS path by modification time";
+ final String footer = "For help, please contact the Hadoop Data Platform team";
+ CommandLineUtil.printHelp(options, cmdLineSyntax, header, footer);
+ throw e;
+ }
+
+ final Path metadataPath = new Path(cmd.getOptionValue(HDFS_PATH_LONG_OPTION));
+
+ final int numFilesToRetain = cmd.hasOption(NUM_METADATA_FILES_RETAINED_LONG_OPTION)
+ ? Integer.parseInt(cmd.getOptionValue(NUM_METADATA_FILES_RETAINED_LONG_OPTION))
+ : HDFSMetadataManager.DEFAULT_NUM_METADATA_FILES_TO_RETAIN;
+
+ Preconditions.checkState(numFilesToRetain > 0, "Number of files to retain cannot be <= 0");
+
+ final boolean fakeDelete = cmd.hasOption(FAKE_DELETE_LONG_OPTION)
+ ? Boolean.parseBoolean(cmd.getOptionValue(FAKE_DELETE_LONG_OPTION))
+ : false;
+
+ final Configuration conf = new Configuration();
+ final FileSystem fs = FSUtils.getFs(conf);
+
+ if (fs.isDirectory(metadataPath)) {
+ final FileStatus[] fileStatuses = fs.listStatus(metadataPath);
+
+ if (fileStatuses.length < numFilesToRetain) {
+ log.info("No files were deleted. Number of files ({}) is less than number to retain ({})",
+ fileStatuses.length, numFilesToRetain);
+ return;
+ }
+
+ FSUtils.deleteHDFSMetadataFiles(fileStatuses, fs, numFilesToRetain, fakeDelete);
+ } else {
+ log.warn("Cannot prune any files, the path {} is not a directory", metadataPath);
+ }
+ }
+
+ private static Options getCLIOptions() {
+ final Options options = new Options();
+ options.addOption(CommandLineUtil.generateOption(HDFS_PATH_SHORT_OPTION,
+ HDFS_PATH_LONG_OPTION,
+ true,
+ "HDFS path",
+ true));
+
+ options.addOption(CommandLineUtil.generateOption(NUM_METADATA_FILES_RETAINED_SHORT_OPTION,
+ NUM_METADATA_FILES_RETAINED_LONG_OPTION,
+ true,
+ "number of metadata files to retain",
+ false));
+
+ options.addOption(CommandLineUtil.generateOption(FAKE_DELETE_SHORT_OPTION,
+ FAKE_DELETE_LONG_OPTION,
+ true,
+ "fake delete",
+ false));
+ return options;
+ }
+}
+
diff --git a/marmaray/build.gradle b/marmaray/build.gradle
new file mode 100644
index 0000000..015d697
--- /dev/null
+++ b/marmaray/build.gradle
@@ -0,0 +1,18 @@
+plugins {
+ id 'com.kageiit.jacobo' version '2.0.1'
+}
+
+description = 'translate jacoco to cobertura'
+
+dependencies {
+}
+
+task jacobo(type: com.kageiit.jacobo.JacoboTask) {
+ jacocoReport = file("./target/site/jacoco-ut/jacoco.xml")
+ coberturaReport = file("./target/site/cobertura/coverage.xml")
+ srcDirs = ["./src/main/java"]
+}
+
+task noop {
+ // noop task for when tests don't run
+}
diff --git a/marmaray/config/sample.yaml b/marmaray/config/sample.yaml
new file mode 100644
index 0000000..fc3e37c
--- /dev/null
+++ b/marmaray/config/sample.yaml
@@ -0,0 +1,45 @@
+---
+marmaray:
+ hadoop:
+ anything: "???"
+ hive:
+ dataPath: ""
+ jobName: ""
+ source:
+ saveCheckpoint: false
+ hoodie:
+ tables:
+ cell_table:
+ table_name: ""
+ base_path: ""
+ schema: ""
+ parallelism: 1024
+ row_table:
+ table_name: ""
+ base_bath: ""
+ schema: ""
+ default:
+ combine_before_insert: true
+ combine_before_upsert: true
+ parallelism: 512
+ kafka:
+ conn:
+ bootstrap:
+ servers: "???"
+ source:
+ topicName:
+ maxMessage:
+ readParallelism:
+ cassandra:
+ output:
+ native.port: ""
+ thrift.address: ""
+ keyspace: "keyspace"
+ tablename: "tableName"
+ cluster_name: "clusterName"
+ column_list: "columnList"
+ partition_keys: "partitionKeys"
+ clustering_keys: "clusteringKeys"
+ inputPath: "inputPath"
+ partitionType: "partitionType"
+ time_to_live: 0L
diff --git a/marmaray/pom.xml b/marmaray/pom.xml
new file mode 100644
index 0000000..1d5683d
--- /dev/null
+++ b/marmaray/pom.xml
@@ -0,0 +1,55 @@
+
+ 4.0.0
+
+ 2.7.4
+ 1.8
+ 1.8
+ 2.7.1
+
+
+
+
+
+ org.jacoco
+ jacoco-maven-plugin
+
+
+ org.fortasoft
+ gradle-maven-plugin
+
+
+ maven-assembly-plugin
+
+
+
+
+
+
+ com.fasterxml.jackson.dataformat
+ jackson-dataformat-yaml
+ ${jackson.dataformat.yaml}
+
+
+
+ org.apache.curator
+ curator-recipes
+ ${apache.curator}
+
+
+
+ org.apache.curator
+ curator-test
+ ${apache.curator}
+ test
+
+
+
+ com.uber.marmaray
+ marmaray-base
+ 1.0-SNAPSHOT
+
+ marmaray
+ 1.0-SNAPSHOT
+
+
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java b/marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java
new file mode 100644
index 0000000..d9214de
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/AvroPayload.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common;
+
+import com.google.common.base.Preconditions;
+import com.uber.marmaray.common.data.IData;
+import com.uber.marmaray.utilities.SparkUtil;
+import lombok.NonNull;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.hibernate.validator.constraints.NotEmpty;
+import scala.reflect.ClassManifestFactory;
+import scala.reflect.ClassTag;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.avro.Schema.Type.RECORD;
+
+/**
+ * This class contains the Avro data as payload with the schema
+ */
+// TODO (T962137)
+@ToString
+@Slf4j
+public class AvroPayload implements IPayload, IData, Serializable {
+
+ private static final ClassTag recordClassTag = ClassManifestFactory.fromClass(GenericRecord.class);
+ @NonNull
+ private final Map rootFields = new HashMap<>();
+ @NonNull
+ private final byte[] byteRecord;
+
+ public AvroPayload(@NonNull final GenericRecord record) {
+ this.byteRecord = SparkUtil.serialize(record, recordClassTag);
+ for (final Schema.Field f : record.getSchema().getFields()) {
+ if (!RECORD.equals(f.schema().getType())) {
+ this.rootFields.put(f.name(), record.get(f.name()));
+ }
+ }
+ }
+
+ public AvroPayload(@NonNull final GenericRecord record,
+ @NonNull final List fieldsToCache) {
+ this.byteRecord = SparkUtil.serialize(record, recordClassTag);
+ for (final String f : fieldsToCache) {
+ this.rootFields.put(f, record.get(f));
+ }
+ }
+
+ /**
+ * Avoid calling it to fetch top level record fields.
+ */
+ public GenericRecord getData() {
+ return SparkUtil.deserialize(this.byteRecord, recordClassTag);
+ }
+
+ /**
+ * It only supports fetching fields at the root level of the record which are of type other than
+ * {@link org.apache.avro.generic.GenericData.Record}.
+ *
+ * @param fieldName name of the field at the root level of the record.
+ */
+ public Object getField(@NotEmpty final String fieldName) {
+ Preconditions.checkState(this.rootFields.containsKey(fieldName),
+ "field is not cached at root level :" + fieldName);
+ return this.rootFields.get(fieldName);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/DispersalType.java b/marmaray/src/main/java/com/uber/marmaray/common/DispersalType.java
new file mode 100644
index 0000000..e6094cb
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/DispersalType.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common;
+
+/**
+ * {@link DispersalType} defines two dispersal types:
+ * version: append new file to path with version id
+ * overwrite: delete old files and then add new file to path
+ */
+public enum DispersalType {
+ VERSION,
+ OVERWRITE
+}
+
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/FileSinkType.java b/marmaray/src/main/java/com/uber/marmaray/common/FileSinkType.java
new file mode 100644
index 0000000..f9aa958
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/FileSinkType.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common;
+
+/**
+ * {@link FileSinkType} defines two options of file sink destinations
+ * 1. HDFS
+ * 2. S3: aws s3
+ */
+public enum FileSinkType {
+ HDFS,
+ S3
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/HoodieErrorPayload.java b/marmaray/src/main/java/com/uber/marmaray/common/HoodieErrorPayload.java
new file mode 100644
index 0000000..11e9551
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/HoodieErrorPayload.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common;
+
+import com.uber.hoodie.common.model.HoodieRecordPayload;
+import com.uber.hoodie.common.util.HoodieAvroUtils;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import lombok.NonNull;
+
+import java.io.IOException;
+import java.util.Optional;
+
+/**
+ * {@link HoodieErrorPayload} is a class to represent error payloads written by Hoodie.
+ */
+public class HoodieErrorPayload implements HoodieRecordPayload {
+ private final GenericRecord record;
+
+ public HoodieErrorPayload(@NonNull final GenericRecord record) {
+ this.record = record;
+ }
+
+ @Override
+ public Optional getInsertValue(final Schema schema) throws IOException {
+ final Optional record = getRecord();
+ return record.map(r -> HoodieAvroUtils.rewriteRecord(r, schema));
+ }
+
+ protected Optional getRecord() {
+ return Optional.of(this.record);
+ }
+
+ @Override
+ public HoodieErrorPayload preCombine(final HoodieErrorPayload hoodieErrorPayload) {
+ throw new JobRuntimeException("Not implemented yet!!");
+ }
+
+ @Override
+ public Optional combineAndGetUpdateValue(final IndexedRecord indexedRecord, final Schema schema)
+ throws IOException {
+ throw new JobRuntimeException("Not implemented yet!!");
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/IPayload.java b/marmaray/src/main/java/com/uber/marmaray/common/IPayload.java
new file mode 100644
index 0000000..49b6470
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/IPayload.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common;
+
+/**
+ * We explicitly model a generic payload here so that it gives us the flexibility to
+ * wrap the data with additional metadata as needed
+ *
+ * @param data type
+ */
+public interface IPayload {
+ D getData();
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/PartitionType.java b/marmaray/src/main/java/com/uber/marmaray/common/PartitionType.java
new file mode 100644
index 0000000..6d5c9f0
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/PartitionType.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common;
+
+/**
+ * {@link PartitionType} defines partition type for data
+ * normal: partition by some defined key
+ * date: partition by date
+ * none: no partition
+ */
+public enum PartitionType {
+ NORMAL,
+ DATE,
+ NONE
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/WorkUnit.java b/marmaray/src/main/java/com/uber/marmaray/common/WorkUnit.java
new file mode 100644
index 0000000..a4793e7
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/WorkUnit.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+
+@AllArgsConstructor
+public class WorkUnit {
+
+ @Getter
+ private final String workEntity;
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/actions/IJobDagAction.java b/marmaray/src/main/java/com/uber/marmaray/common/actions/IJobDagAction.java
new file mode 100644
index 0000000..8ec2426
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/actions/IJobDagAction.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.actions;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * {@link IJobDagAction} is interface to determine a generic action to execute.
+ * {@link JobDagActions} are completely independent and will determine if they should run based on success status.
+ */
+public interface IJobDagAction {
+ int DEFAULT_TIMEOUT_SECONDS = 120;
+ String ACTION_TYPE = "action_type";
+
+ /**
+ * Execute the action
+ *
+ * @param successful whether the job dag succeeded
+ * @return true if action succeeded
+ */
+ boolean execute(boolean successful);
+
+ /**
+ * Timeout to wait for the action to complete
+ * @return number of seconds to wait for task completion
+ */
+ default int getTimeoutSeconds() {
+ return DEFAULT_TIMEOUT_SECONDS;
+ }
+
+ /**
+ * @return metric tags to be used for reporting metrics.
+ */
+ default Map getMetricTags() {
+ final Map metricsTags = new HashMap<>();
+ metricsTags.put(ACTION_TYPE, this.getClass().getSimpleName());
+ return metricsTags;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/actions/JobDagActions.java b/marmaray/src/main/java/com/uber/marmaray/common/actions/JobDagActions.java
new file mode 100644
index 0000000..c9767bf
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/actions/JobDagActions.java
@@ -0,0 +1,141 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.actions;
+
+import com.uber.marmaray.common.job.ThreadPoolService;
+import com.uber.marmaray.common.job.ThreadPoolServiceTier;
+import com.uber.marmaray.common.metrics.LongMetric;
+import com.uber.marmaray.common.reporters.Reporters;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.util.Collection;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static com.uber.marmaray.common.metrics.DataFeedMetricNames.RESULT_FAILURE;
+import static com.uber.marmaray.common.metrics.DataFeedMetricNames.RESULT_SUCCESS;
+
+@Slf4j
+/**
+ * {@link JobDagActions} are actions that are run based on success status.
+ * This class is completely independent and NOT an implementation of the {@link IJobDagAction} interface
+ */
+public final class JobDagActions {
+ public static final String RESULT_METRIC = "result";
+ public static final String TIME_METRIC = "execution_time";
+
+ /**
+ * Class to hold and execute actions.
+ *
+ * Actions are executed in parallel; execution status will not affect the others.
+ */
+
+ public static final String DEFAULT_NAME = "anonymous";
+
+ @Getter
+ private final Queue actions;
+ private final Reporters reporters;
+
+ @Getter
+ private final String name;
+
+ public JobDagActions(@NonNull final Reporters reporters) {
+ this(reporters, DEFAULT_NAME);
+ }
+
+ public JobDagActions(@NonNull final Reporters reporters, @NotEmpty final String name) {
+ this.actions = new ConcurrentLinkedDeque<>();
+ this.name = name;
+ this.reporters = reporters;
+ }
+
+ /**
+ * Add an action to the container
+ * @param action IAction to hold
+ */
+ public void addAction(@NonNull final IJobDagAction action) {
+ this.actions.add(action);
+ }
+
+ /**
+ * Add a collection of actions to the container
+ * @param actions Collection of IActions to hold
+ */
+ public void addActions(@NonNull final Collection extends IJobDagAction> actions) {
+ this.actions.addAll(actions);
+ }
+
+ /**
+ * Execute all of the actions in parallel.
+ *
+ * Parallelism is managed by the ThreadPoolService. Throws IllegalStateException if any IAction
+ * throws an Exception during execution.
+ * @param dagSuccess whether the actions are responding to a successful dag run or a failed one
+ */
+ public boolean execute(final boolean dagSuccess) {
+ final AtomicBoolean successful = new AtomicBoolean(true);
+ final ConcurrentMap, IJobDagAction> futures = new ConcurrentHashMap<>();
+ this.actions.forEach(a ->
+ futures.put(ThreadPoolService.submit(() -> {
+ final long startTime = System.currentTimeMillis();
+ final boolean success;
+ try {
+ success = a.execute(dagSuccess);
+ } finally {
+ final long endTime = System.currentTimeMillis();
+ reportExecuteTime(a, endTime - startTime);
+ }
+ return success;
+ }, ThreadPoolServiceTier.ACTIONS_TIER, a.getTimeoutSeconds()),
+ a));
+ futures.forEach((future, action) -> {
+ final AtomicBoolean actionSuccess = new AtomicBoolean(true);
+ try {
+ actionSuccess.set(future.get());
+ } catch (Exception e) {
+ log.error("Error running JobDagAction {} for {}:", action.getClass(), this.getName(), e);
+ actionSuccess.set(false);
+ successful.set(false);
+ }
+ reportActionStatus(action, actionSuccess.get());
+ });
+ if (!successful.get()) {
+ log.warn("Errors encountered during JobDagActions execution");
+ }
+ return successful.get();
+ }
+
+ private void reportExecuteTime(@NonNull final IJobDagAction action, final long timeInMillis) {
+ final LongMetric timeMetric = new LongMetric(TIME_METRIC, TimeUnit.MILLISECONDS.toSeconds(timeInMillis));
+ timeMetric.addTags(action.getMetricTags());
+ this.reporters.getReporters().stream().forEach(r -> r.gauge(timeMetric));
+ }
+
+ private void reportActionStatus(@NonNull final IJobDagAction action, final boolean isSuccess) {
+ final LongMetric resultMetric = new LongMetric(RESULT_METRIC, isSuccess ? RESULT_SUCCESS : RESULT_FAILURE);
+ resultMetric.addTags(action.getMetricTags());
+ this.reporters.getReporters().stream().forEach(r -> r.gauge(resultMetric));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/actions/ReporterAction.java b/marmaray/src/main/java/com/uber/marmaray/common/actions/ReporterAction.java
new file mode 100644
index 0000000..d33e738
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/actions/ReporterAction.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.actions;
+
+import com.uber.marmaray.common.metrics.DataFeedMetrics;
+import com.uber.marmaray.common.metrics.JobMetrics;
+import com.uber.marmaray.common.reporters.Reporters;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+@AllArgsConstructor
+/**
+ * {@link ReporterAction} is an implemetation of the {@link IJobDagAction} interface and is used to
+ * report metrics
+ */
+public class ReporterAction implements IJobDagAction {
+
+ public static int DEFAULT_TIMEOUT_SECONDS = 120;
+
+ @Getter
+ private final Reporters reporters;
+ @Getter
+ private final JobMetrics jobMetrics;
+ private final DataFeedMetrics dataFeedMetrics;
+
+ @Override
+ public boolean execute(final boolean success) {
+ if (success) {
+ this.reporters.getReporters().forEach(reporter -> {
+ this.jobMetrics.gaugeAll(reporter);
+ this.dataFeedMetrics.gaugeAll(reporter);
+ });
+ } else {
+ log.warn("No metrics produced or actions being executed on reporter because errors were encountered");
+ }
+ return success;
+ }
+
+ @Override
+ public int getTimeoutSeconds() {
+ return DEFAULT_TIMEOUT_SECONDS;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/AwsConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/AwsConfiguration.java
new file mode 100644
index 0000000..fa83c80
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/AwsConfiguration.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common.configuration;
+
+import com.uber.marmaray.utilities.ConfigUtil;
+import lombok.Getter;
+import lombok.NonNull;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public class AwsConfiguration implements Serializable {
+ @Getter
+ private final String region;
+ @Getter
+ private final String awsAccessKeyId;
+ @Getter
+ private final String awsSecretAccessKey;
+ @Getter
+ private final String bucketName;
+ @Getter
+ private final String objectKey;
+ @Getter
+ private final String s3FilePrefix;
+ @Getter
+ private final String sourcePath;
+ @Getter
+ private final String fileSystemPrefix;
+ @Getter
+ private final String pathKey;
+
+ public AwsConfiguration(@NonNull final FileSinkConfiguration conf) {
+ ConfigUtil.checkMandatoryProperties(conf.getConf(), this.getMandatoryProperties());
+ this.region = conf.getAwsRegion().get();
+ this.awsAccessKeyId = conf.getAwsAccessKeyId().get();
+ this.awsSecretAccessKey = conf.getAwsSecretAccesskey().get();
+ this.bucketName = conf.getBucketName().get();
+ this.objectKey = conf.getObjectKey().get();
+ this.sourcePath = conf.getFullPath();
+ this.fileSystemPrefix = conf.getPathPrefix();
+
+ String s3FilePrefix;
+ if (conf.getSourcePartitionPath().isPresent()) {
+ s3FilePrefix = String.format("%s/%s/", this.objectKey, conf.getSourcePartitionPath().get());
+ } else {
+ s3FilePrefix = String.format("%s/", this.objectKey);
+ }
+ this.pathKey = s3FilePrefix;
+ s3FilePrefix += conf.getFileNamePrefix();
+ this.s3FilePrefix = s3FilePrefix;
+ }
+
+ private List getMandatoryProperties() {
+ return Collections.unmodifiableList(
+ Arrays.asList(
+ FileSinkConfiguration.AWS_REGION,
+ FileSinkConfiguration.BUCKET_NAME,
+ FileSinkConfiguration.OBJECT_KEY,
+ FileSinkConfiguration.AWS_ACCESS_KEY_ID,
+ FileSinkConfiguration.AWS_SECRET_ACCESS_KEY
+ ));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraSinkConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraSinkConfiguration.java
new file mode 100644
index 0000000..8c0e077
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/CassandraSinkConfiguration.java
@@ -0,0 +1,266 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+import com.uber.marmaray.common.PartitionType;
+import com.uber.marmaray.common.exceptions.MissingPropertyException;
+import com.uber.marmaray.common.schema.cassandra.ClusterKey;
+import com.uber.marmaray.utilities.ConfigUtil;
+import com.uber.marmaray.utilities.SchemaUtil;
+import com.uber.marmaray.utilities.StringTypes;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.hadoop.ConfigHelper;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * {@link CassandraSinkConfiguration} contains all relevant configuration properties to disperse data to Cassandra.
+ * Properties include but are not limited to MapReduce job settings, optimization parameters, etc.
+ *
+ * We explicitly assume all whitespace characters are trimmed from values in the yaml reader so we don't have
+ * to sanitize the values again here.
+ */
+@Slf4j
+public class CassandraSinkConfiguration implements Serializable {
+
+ public static final String CASSANDRA_PREFIX_ONLY = "cassandra.";
+ public static final String CASS_COMMON_PREFIX = Configuration.MARMARAY_PREFIX + CASSANDRA_PREFIX_ONLY;
+ public static final String HADOOP_COMMON_PREFIX = Configuration.MARMARAY_PREFIX + "hadoop.";
+ // *** Cassandra Configuration Settings ***
+ public static final String OUTPUT_THRIFT_PORT = CASS_COMMON_PREFIX + "output.thrift.port";
+
+ public static final String NATIVE_TRANSPORT_PORT =
+ HADOOP_COMMON_PREFIX + CASSANDRA_PREFIX_ONLY + "output.native.port";
+ // Note: CassandraUnit uses port 9142 so it won't disturb any local default cassandra installation
+ public static final String DEFAULT_OUTPUT_NATIVE_PORT = "9042";
+
+ public static final String INITIAL_HOSTS = HADOOP_COMMON_PREFIX + CASSANDRA_PREFIX_ONLY + "output.thrift.address";
+
+ // this setting is currently not used yet but will eventually be needed for throttling
+ public static final String STREAM_THROTTLE_MBITS = "mapreduce.output.bulkoutputformat.streamthrottlembits";
+
+ public static final String STORAGE_PORT = CASS_COMMON_PREFIX + "storage.port";
+ public static final String SSL_STORAGE_PORT = CASS_COMMON_PREFIX + "ssl.storage.port";
+
+ public static final String DISABLE_QUERY_UNS = CASS_COMMON_PREFIX + "disable_uns";
+
+ // *** End of Cassandra Configuration Settings ***1
+ public static final String DEFAULT_OUTPUT_RPC_PORT = "9160";
+ // *** End of Cassandra Configuration Settings ***
+
+ // *** Dispersal Job Settings ***
+ public static final String KEYSPACE = CASS_COMMON_PREFIX + "keyspace";
+ public static final String TABLE_NAME = CASS_COMMON_PREFIX + "tablename";
+ public static final String CLUSTER_NAME = CASS_COMMON_PREFIX + "cluster_name";
+ public static final String COLUMN_LIST = CASS_COMMON_PREFIX + "column_list";
+ public static final String PARTITION_KEYS = CASS_COMMON_PREFIX + "partition_keys";
+ public static final String CLUSTERING_KEYS = CASS_COMMON_PREFIX + "clustering_keys";
+ public static final String PARTITION_TYPE = CASS_COMMON_PREFIX + "partition_type";
+ public static final String USERNAME = CASS_COMMON_PREFIX + "username";
+ public static final String PASSWORD = CASS_COMMON_PREFIX + "password";
+ public static final String USE_CLIENT_SINK = CASS_COMMON_PREFIX + "use_client_sink";
+
+ // optional field for customers to timestamp their data
+ public static final String TIMESTAMP = CASS_COMMON_PREFIX + SchemaUtil.DISPERSAL_TIMESTAMP;
+ public static final String TIMESTAMP_IS_LONG_TYPE = CASS_COMMON_PREFIX + "timestamp_is_long_type";
+
+ // File path containing datacenter info on each machine
+ public static final String DC_PATH = "data_center_path";
+
+ public static final String DATACENTER = CASS_COMMON_PREFIX + "datacenter";
+
+ public static final String TIME_TO_LIVE = CASS_COMMON_PREFIX + "time_to_live";
+ private static final long DEFAULT_TIME_TO_LIVE = 0L;
+
+ private static final Splitter splitter = Splitter.on(StringTypes.COMMA);
+
+ @Getter
+ protected final Configuration conf;
+
+ @Getter
+ protected final List initialHosts;
+
+ /**
+ * Optional field. This functionality enables a subset of columns to be dispersed from the data source and
+ * not forcing the user to either disperse all or none of the data.
+ * This set must include all partition and clustering keys if defined or the job will fail.
+ */
+ @Getter
+ private final Optional> filteredColumns;
+
+ @Getter
+ private final List partitionKeys;
+
+ @Getter
+ private final List clusteringKeys;
+
+ @Getter
+ private final PartitionType partitionType;
+
+ @Getter
+ private final Optional writeTimestamp;
+
+ @Getter
+ private final boolean timestampIsLongType;
+
+ public CassandraSinkConfiguration(@NonNull final Configuration conf) {
+ this.conf = conf;
+ ConfigUtil.checkMandatoryProperties(this.conf, this.getMandatoryProperties());
+
+ this.partitionKeys = this.splitString(this.conf.getProperty(PARTITION_KEYS).get());
+
+ if (this.conf.getProperty(INITIAL_HOSTS).isPresent()) {
+ this.initialHosts = this.splitString(this.conf.getProperty(INITIAL_HOSTS).get());
+ } else {
+ this.initialHosts = new ArrayList<>();
+ }
+
+ // Source fields can be case insensitive so we convert everything to lower case for comparing
+ this.filteredColumns = this.conf.getProperty(COLUMN_LIST).isPresent()
+ ? Optional.of(new HashSet<>(this.splitString(this.conf.getProperty(COLUMN_LIST).get().toLowerCase())))
+ : Optional.absent();
+
+ this.clusteringKeys = this.conf.getProperty(CLUSTERING_KEYS).isPresent()
+ ? initClusterKeys(this.splitString(this.conf.getProperty(CLUSTERING_KEYS).get()))
+ : Collections.EMPTY_LIST;
+
+ if (this.conf.getProperty(PARTITION_TYPE).isPresent()) {
+ this.partitionType = PartitionType.valueOf(this.conf.getProperty(PARTITION_TYPE)
+ .get().trim().toUpperCase());
+ } else {
+ this.partitionType = PartitionType.NONE;
+ }
+
+ this.writeTimestamp = this.conf.getProperty(TIMESTAMP);
+ this.timestampIsLongType = this.conf.getBooleanProperty(TIMESTAMP_IS_LONG_TYPE, false);
+ }
+
+ /**
+ * Returns hadoop configuration.
+ * Some example of things that could be set are the output rpc port (cassandra.output.thrift.port)
+ * and other cassandra specific hadoop configuration settings.
+ */
+ public org.apache.hadoop.conf.Configuration getHadoopConf() {
+ final org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration();
+
+ if (this.getUserName().isPresent() && this.getPassword().isPresent()) {
+ ConfigHelper.setOutputKeyspaceUserNameAndPassword(
+ hadoopConf,
+ this.getUserName().get(),
+ this.getPassword().get()
+ );
+ }
+
+ if (!this.conf.getProperty(DATACENTER).isPresent()) {
+ // should never happen, but a sanity check
+ throw new MissingPropertyException("The datacenter information is missing");
+ }
+
+ // By default use Murmur3 Partitioner for now, we may want this to be changed in future
+ ConfigHelper.setOutputPartitioner(hadoopConf, Murmur3Partitioner.class.getName());
+
+ log.info("Setting output local DC only to true");
+ ConfigHelper.setOutputLocalDCOnly(hadoopConf, true);
+
+ this.conf.getPropertiesWithPrefix(CassandraSinkConfiguration.HADOOP_COMMON_PREFIX, true).forEach(
+ (key, value) -> {
+ log.info("hadoop-conf-update:key:[{}]:value:[{}]", key, value);
+ hadoopConf.set(key, value);
+ }
+ );
+ return hadoopConf;
+ }
+
+ public String getKeyspace() {
+ return this.getConf().getProperty(KEYSPACE).get().trim();
+ }
+
+ public String getTableName() {
+ return this.getConf().getProperty(TABLE_NAME).get().trim();
+ }
+
+ public String getClusterName() {
+ return this.getConf().getProperty(CLUSTER_NAME).get().trim();
+ }
+
+ public Boolean getUseClientSink() {
+ return this.getConf().getBooleanProperty(USE_CLIENT_SINK, false);
+ }
+
+ public Optional getUserName() {
+ return this.getConf().getProperty(USERNAME);
+ }
+
+ public Optional getPassword() {
+ return this.getConf().getProperty(PASSWORD);
+ }
+
+ public Optional getSSLStoragePort() {
+ return this.conf.getProperty(SSL_STORAGE_PORT).isPresent()
+ ? Optional.of(this.conf.getProperty(SSL_STORAGE_PORT).get())
+ : Optional.absent();
+ }
+
+ public Optional getStoragePort() {
+ return this.conf.getProperty(STORAGE_PORT).isPresent()
+ ? Optional.of(this.conf.getProperty(STORAGE_PORT).get())
+ : Optional.absent();
+ }
+
+ // TTL is optional. By default if you set it to 0 it means forever (confirm this)
+ public Optional getTimeToLive() {
+ return this.conf.getProperty(TIME_TO_LIVE).isPresent()
+ ? Optional.of(this.conf.getLongProperty(TIME_TO_LIVE, DEFAULT_TIME_TO_LIVE))
+ : Optional.absent();
+ }
+
+ // TODO - For these optional fields, consider adding a default value
+ public Optional getNativePort() {
+ return this.conf.getProperty(NATIVE_TRANSPORT_PORT);
+ }
+
+ private List getMandatoryProperties() {
+ return Collections.unmodifiableList(
+ Arrays.asList(
+ KEYSPACE,
+ TABLE_NAME,
+ CLUSTER_NAME,
+ PARTITION_KEYS
+ ));
+ }
+
+ protected List splitString(final String commaSeparatedValues) {
+ return Lists.newArrayList(splitter.split(commaSeparatedValues));
+ }
+
+ private List initClusterKeys(final List entries) {
+ return entries.stream().map(entry -> ClusterKey.parse(entry)).collect(Collectors.toList());
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/ConfigScopeResolver.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ConfigScopeResolver.java
new file mode 100644
index 0000000..1369dd6
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ConfigScopeResolver.java
@@ -0,0 +1,289 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common.configuration;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.hibernate.validator.constraints.NotEmpty;
+
+/**
+ *
+ * What qualify as scopes?
+ * Top-level nodes which appear under a tag dedicated to represent
+ * scope override qualify as scopes.
+ *
+ *
+ * How is scope override defined?
+ * Scope override definition is defined under a config driven tag dedicated
+ * for scope override. In the example below in (fig 1), the tag happens to be
+ * scope_override and scopes are x, y & z. Scopes y and z are the overrider
+ * scopes and x is the default scope.
+ *
+ *
+ * How is scope overriding done?
+ * There are not more than 2 scopes participating in a scope override at any time.
+ * That is, either the user chooses to use the scope y, in which case y overrides x
+ * OR the user chooses to use scope z, in which case z overrides x
+ * OR user chooses to not use any scope, in which case no overriding is done at all.
+ * Also note that y and z are totally unrelated and never participate together in
+ * scope overriding. Before explaining the logic of scope overriding, please note
+ * the convention in the example below
+ *
+ *
+ * Any node that starts with p represents a primitive node. In other
+ * words, it represents a config-key and a node that starts with v
+ * represents the config-value.
+ *
+ *
+ * Nodes x, y, z represent scopes as explained in the previous question.
+ *
+ *
+ * Rest of nodes [a-g] represent internal nodes.
+ *
+ *
+ *
+ * (fig 1)
+ *
+ * ├── scope_override
+ * │  ├── {y = x}
+ * │  └── {z = x}
+ * │   Â
+ * ├── x
+ * │  └── a
+ * │  ├── b
+ * │  │  ├── c
+ * │  │  │  ├── d
+ * │  │  │  │  ├── {p3 = v3_x}
+ * │  │  │  │  └── {p4 = v4_x}
+ * │  │  │  └── {p2 = v2_x}
+ * │  │  └── {p1 = v1_x}
+ * │  └── e
+ * │  └── f
+ * │  └── {p5 = v5_x}
+ * ├── y
+ * │  └── a
+ * │  └── b
+ * │  ├── c
+ * │  │  ├── d
+ * │  │  │  ├── {p3 = v3_y}
+ * │  │  │  └── {p4 = v4_y}
+ * │  │  └── {p2 = v2_y}
+ * │  └── {p1 = v1_y}
+ * └── z
+ * └── a
+ * └── b
+ * ├── c
+ * │  ├── d
+ * │  │  ├── {p3 = v3_z}
+ * │  │  └── {p7 = v7_z}
+ * │  └── {p2 = v2_z}
+ * ├── g
+ * │  └── {p6 = v6_z}
+ * └── {p1 = v1_z}
+ *
+ *
+ *
+ *
+ * Overriding is done by replacing the value of a config-key that appears
+ * in both the scopes, along the exact same hierarchy.
+ *
+ * When scope y overrides x, the final config-key values resolved are below.
+ * It inherits p5 as is from x. Overrides p1, p2, p3, p4.
+ * (fig 2)
+ *
+ * a
+ * ├── b
+ * │  ├── c
+ * │  │  ├── d
+ * │  │  │  ├── {p3 = v3_y}
+ * │  │  │  └── {p4 = v4_y}
+ * │  │  └── {p2 = v2_y}
+ * │  └── {p1 = v1_y}
+ * └── e
+ * └── f
+ * └── {p5 = v5_x}
+ *
+ * When scope z overrides x, the final config-key values resolved are below.
+ * It inherits p5 as is from x. Overrides p1, p2, p3, p4. Retains its p6.
+ * (fig 3)
+ *
+ * a
+ * ├── b
+ * │  ├── c
+ * │  │  ├── d
+ * │  │  │  ├── {p3 = v3_z}
+ * │  │  │  └── {p4 = v4_z}
+ * │  │  │  └── {p7 = v7_z}
+ * │  │  └── {p2 = v2_z}
+ * │ ├── g
+ * │ │  └── {p6 = v6_z}
+ * │  └── {p1 = v1_y}
+ * └── e
+ * └── f
+ * └── {p5 = v5_x}
+ *
+ *
+ */
+@Slf4j
+@RequiredArgsConstructor
+public class ConfigScopeResolver {
+
+ private final String scopeOverrideMappingKey;
+ private Map scopeOverrideMap;
+ private final ObjectMapper mapper = new ObjectMapper();
+
+ public JsonNode projectOverrideScopeOverDefault(
+ @NonNull final Optional scope, @NonNull final JsonNode rootJsonNode) {
+ if (!scope.isPresent() || !rootJsonNode.isContainerNode()) {
+ log.info("No scope overriding in effect. "
+ + "Either scope: {} is absent, or {} is not a container node",
+ scope, rootJsonNode);
+ return rootJsonNode;
+ }
+ Preconditions.checkState(rootJsonNode.has(scopeOverrideMappingKey),
+ String.format(
+ "scopeOverrideMappingKey: %s is not present in config but scoping is expected with scope: %s",
+ scopeOverrideMappingKey, scope));
+
+ final ObjectNode root = copyToObjectNode(rootJsonNode);
+ final JsonNode scopeOverrideDefinitionNodeVal = root.get(scopeOverrideMappingKey);
+ Preconditions.checkArgument(
+ scopeOverrideDefinitionNodeVal != null
+ && scopeOverrideDefinitionNodeVal.isContainerNode(),
+ String.format("Value for scopePrecedence %s should be a map, got null or primitive: ",
+ scopeOverrideDefinitionNodeVal));
+ this.scopeOverrideMap = mapper.convertValue(scopeOverrideDefinitionNodeVal, Map.class);
+ log.info("scopeOverrideMap is {} scope is {}", this.scopeOverrideMap, scope.get());
+
+ Preconditions.checkArgument(scopeOverrideMap.containsKey(scope.get()),
+ "Un-recognized scope passed for config resolving");
+
+ root.remove(scopeOverrideMappingKey);
+
+ final String overrideScope = scope.get();
+ final String defaultScope = scopeOverrideMap.get(overrideScope);
+ if (root.has(overrideScope) && root.has(defaultScope)) {
+ final JsonNode resolvedNode = handleScopeOverride(root, overrideScope,
+ defaultScope);
+ root.remove(overrideScope);
+ root.remove(defaultScope);
+ final Iterator fieldNamesOfResolvedNode = resolvedNode.fieldNames();
+ while (fieldNamesOfResolvedNode.hasNext()) {
+ final String fieldNameOfResolvedNode = fieldNamesOfResolvedNode.next();
+ root.put(fieldNameOfResolvedNode, resolvedNode.get(fieldNameOfResolvedNode));
+ }
+ } else {
+ log.info("No overriding done for scope combinations as one of them is missing."
+ + " IsOverrideScopePresent: {}, IsDefaultScopePresent: {} ",
+ root.has(overrideScope), root.has(defaultScope));
+ }
+ for (final Entry entry : scopeOverrideMap.entrySet()) {
+ // remove all scope definitions, now that resolving is done
+ root.remove(entry.getKey());
+ root.remove(entry.getValue());
+ }
+ return root;
+ }
+
+ private JsonNode handleScopeOverride(
+ @NonNull final ObjectNode root,
+ @NotEmpty final String overrideScope,
+ @NotEmpty final String defaultScope) {
+ final JsonNode overridingNode = root.get(overrideScope);
+ final JsonNode defaultNode = root.get(defaultScope);
+ final ObjectNode defaultNodeCopy;
+ defaultNodeCopy = copyToObjectNode(defaultNode);
+ // defaultNodeCopy will be updated by projecting overridingNode over it
+ projectOverrideNodeOverDefaultForField(null, defaultNodeCopy, overridingNode);
+ return defaultNodeCopy;
+
+ }
+
+ private ObjectNode copyToObjectNode(@NonNull final JsonNode defaultNode) {
+ final ObjectNode defaultNodeCopy;
+ try {
+ defaultNodeCopy = (ObjectNode) mapper
+ .readTree(mapper.writeValueAsString(defaultNode));
+ } catch (IOException e) {
+ log.error("Got exception", e);
+ throw new JobRuntimeException(e);
+ }
+ return defaultNodeCopy;
+ }
+
+ private void projectOverrideNodeOverDefaultForField(
+ @NotEmpty final String fieldName,
+ @NonNull final JsonNode parentDefaultNode,
+ @NonNull final JsonNode parentOverridingNode) {
+
+ final JsonNode defaultNode =
+ (fieldName == null) ? parentDefaultNode : parentDefaultNode.get(fieldName);
+ final JsonNode overridingNode =
+ (fieldName == null) ? parentOverridingNode : parentOverridingNode.get(fieldName);
+
+ if (fieldName != null) {
+ // not first time call to recursion
+ if (defaultNode == null || overridingNode == null) {
+ final JsonNode nodeToPutAtFieldName = java.util.Optional.ofNullable(defaultNode)
+ .orElse(overridingNode);
+ log.info("Copying fieldName: {} value: {}", fieldName, nodeToPutAtFieldName);
+ ((ObjectNode) parentDefaultNode).put(fieldName, nodeToPutAtFieldName);
+ return;
+ }
+ Preconditions
+ .checkState(
+ (defaultNode.isContainerNode() && overridingNode.isContainerNode())
+ || (!defaultNode.isContainerNode() && !overridingNode.isContainerNode()),
+ "Mismatch in node type between default node: {} and overriding node: {}."
+ + " One of them is a primitive node", defaultNode, overridingNode);
+ if (!overridingNode.isContainerNode()) {
+ // primitive node or TextNode since that is the only primitive node that appears here
+ // so blindly accept the value of the overriding node
+ log.info("Using value: {} of override node for fieldName: {}", overridingNode,
+ fieldName);
+ ((ObjectNode) parentDefaultNode).put(fieldName, overridingNode);
+ } else {
+ // both are container nodes
+ projectOverAllFields(defaultNode, overridingNode);
+ }
+ } else {
+ // first call to recursion, represents root default node and override node.
+ projectOverAllFields(defaultNode, overridingNode);
+ }
+ }
+
+ private void projectOverAllFields(
+ @NonNull final JsonNode defaultNode, @NonNull final JsonNode overridingNode) {
+ final Iterator childFieldNames = overridingNode.fieldNames();
+ while (childFieldNames.hasNext()) {
+ final String childFieldName = childFieldNames.next();
+ projectOverrideNodeOverDefaultForField(childFieldName, defaultNode, overridingNode);
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/Configuration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/Configuration.java
new file mode 100644
index 0000000..48dcfa0
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/Configuration.java
@@ -0,0 +1,227 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+
+import java.io.InputStream;
+import java.io.Serializable;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.hibernate.validator.constraints.NotEmpty;
+
+/**
+ * {@link Configuration} will be instantiated from a YAML based file
+ * and contain all the pertinent metadata to initialize and execute
+ * a data transfer job. Supports scopes and scope based config
+ * overriding. Refer to documentation of {@link ConfigScopeResolver}
+ * for more info on scope overriding
+ */
+@Slf4j
+public class Configuration implements Serializable {
+
+ public static final String MARMARAY_PREFIX = "marmaray.";
+ public static final String SCOPE_OVERRIDE_MAPPING_KEY = "scope_override_map";
+
+ private final Properties props = new Properties();
+
+ /**
+ * @deprecated todo: remove this constructor in a separate diff
+ * since callers will need to inject scope, so will need change in callers
+ */
+ @Deprecated
+ public Configuration() {
+
+ }
+
+ public Configuration(@NonNull final File yamlFile,
+ @NonNull final Optional scope) {
+ loadYamlFile(yamlFile, scope);
+ }
+
+ public Configuration(@NonNull final InputStream inputStream,
+ @NonNull final Optional scope) {
+ loadYamlStream(inputStream, scope);
+ }
+
+ /**
+ * @deprecated todo: remove this constructor in a separate diff
+ * since callers will need to inject scope, so will need change in callers
+ */
+ @Deprecated
+ public Configuration(@NonNull final Configuration conf) {
+ this.props.putAll(conf.props);
+ }
+
+ public void loadYamlFile(@NonNull final File yamlFile,
+ final Optional scope) {
+ try {
+ final FileSystem localFs = FileSystem.getLocal(
+ new HadoopConfiguration(new Configuration()).getHadoopConf());
+ final InputStream yamlInputStream = localFs.open(new Path(yamlFile.getPath()));
+ loadYamlStream(yamlInputStream, scope);
+ } catch (IOException e) {
+ final String errorMsg = String
+ .format("Error loading yaml config file %s", yamlFile.getAbsolutePath());
+ log.error(errorMsg, e);
+ throw new JobRuntimeException(errorMsg, e);
+ }
+ }
+
+ public void loadYamlStream(@NonNull final InputStream yamlStream,
+ @NonNull final Optional scope) {
+ try {
+ final ObjectMapper yamlReader = new ObjectMapper(new YAMLFactory());
+ final JsonNode jsonNode = yamlReader.readTree(yamlStream);
+ final JsonNode scopeOverriddenJsonNode = handleScopeOverriding(scope, jsonNode);
+ parseConfigJson(scopeOverriddenJsonNode, "");
+ } catch (IOException e) {
+ final String errorMsg = "Error loading yaml file ";
+ log.error(errorMsg, e);
+ throw new JobRuntimeException(errorMsg, e);
+ }
+ }
+
+ private JsonNode handleScopeOverriding(
+ @NonNull final Optional scope, @NonNull final JsonNode jsonNode) {
+ return new ConfigScopeResolver(SCOPE_OVERRIDE_MAPPING_KEY)
+ .projectOverrideScopeOverDefault(scope, jsonNode);
+ }
+
+ public String getProperty(final String key, final String defaultValue) {
+ return this.props.getProperty(key, defaultValue);
+ }
+
+ public Optional getProperty(final String key) {
+ Preconditions.checkArgument(!Strings.isNullOrEmpty(key));
+ final String val = this.props.getProperty(key);
+ return (val == null) ? Optional.absent() : Optional.of(val);
+ }
+
+ public void setProperty(final String key, final String value) {
+ this.props.setProperty(key, value);
+ }
+
+ /**
+ * Returns properties with or without prefix.
+ * @param prefix
+ * @param removePrefix if true it will remove prefix from properties.
+ * @return {@link Map} with properties.
+ */
+ public Map getPropertiesWithPrefix(final String prefix, final boolean removePrefix) {
+ Preconditions.checkState(!Strings.isNullOrEmpty(prefix));
+ final Map properties = new HashMap<>();
+ final int prefixLength = prefix.length();
+ this.props.entrySet().forEach(
+ entry -> {
+ final String key = (String) entry.getKey();
+ if (key.startsWith(prefix)) {
+ if (removePrefix) {
+ properties.put(key.substring(prefixLength), entry.getValue().toString());
+ } else {
+ properties.put(key, entry.getValue().toString());
+ }
+
+ }
+ });
+ return properties;
+ }
+
+ public static T getProperty(@NonNull final Configuration conf, @NotEmpty final String key,
+ @NonNull final T defaultValue) {
+ if (defaultValue instanceof Integer) {
+ return (T) new Integer(conf.getIntProperty(key, ((Integer) defaultValue).intValue()));
+ } else if (defaultValue instanceof Long) {
+ return (T) new Long(conf.getLongProperty(key, ((Long) defaultValue).longValue()));
+ } else if (defaultValue instanceof String) {
+ return (T) conf.getProperty(key, (String) defaultValue);
+ } else if (defaultValue instanceof Double) {
+ return (T) new Double(conf.getDoubleProperty(key, ((Double) defaultValue).doubleValue()));
+ } else if (defaultValue instanceof Boolean) {
+ return (T) new Boolean(conf.getBooleanProperty(key, ((Boolean) defaultValue).booleanValue()));
+ } else {
+ throw new IllegalArgumentException("Not supported :" + defaultValue.getClass());
+ }
+ }
+
+ public int getIntProperty(final String key, final int defaultValue) {
+ final Optional val = getProperty(key);
+ return val.isPresent() ? Integer.parseInt(val.get()) : defaultValue;
+ }
+
+ public long getLongProperty(final String key, final long defaultValue) {
+ final Optional val = getProperty(key);
+ return val.isPresent() ? Long.parseLong(val.get()) : defaultValue;
+ }
+
+ public double getDoubleProperty(final String key, final double defaultValue) {
+ final Optional val = getProperty(key);
+ return val.isPresent() ? Double.parseDouble(val.get()) : defaultValue;
+ }
+
+ public boolean getBooleanProperty(final String key, final boolean defaultValue) {
+ final Optional val = getProperty(key);
+ return val.isPresent() ? Boolean.parseBoolean(val.get()) : defaultValue;
+ }
+
+ private void parseConfigJson(final JsonNode jsonNode, final String prefix) {
+ final Iterator fieldNamesIt = jsonNode.fieldNames();
+ while (fieldNamesIt.hasNext()) {
+ final String fieldName = fieldNamesIt.next();
+ final String newPrefix = prefix.isEmpty() ? fieldName.trim() : prefix + "." + fieldName.trim();
+ final JsonNode newJsonNode = jsonNode.get(fieldName);
+ if (newJsonNode.isObject()) {
+ parseConfigJson(newJsonNode, newPrefix);
+ } else {
+ props.put(newPrefix, newJsonNode.asText().trim());
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder();
+ this.props.entrySet().forEach(
+ entry -> {
+ sb.append(entry.getKey() + "<=>" + entry.getValue() + "\n");
+ }
+ );
+ return sb.toString();
+ }
+
+ public Set getKeySet() {
+ return this.props.keySet();
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/ErrorTableConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ErrorTableConfiguration.java
new file mode 100644
index 0000000..3e741fc
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ErrorTableConfiguration.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.uber.marmaray.common.sinks.hoodie.HoodieWriteStatus;
+import com.uber.marmaray.utilities.ConfigUtil;
+import com.uber.marmaray.utilities.ErrorTableUtil;
+import lombok.Getter;
+import lombok.NonNull;
+import org.apache.hadoop.fs.Path;
+import org.hibernate.validator.constraints.NotEmpty;
+import scala.Serializable;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * {@link ErrorTableConfiguration} contains the configuration used to construct ErrorTable
+ */
+public class ErrorTableConfiguration implements Serializable {
+
+ public static final String ERROR_TABLE_PREFIX = Configuration.MARMARAY_PREFIX + "error_table.";
+ /**
+ * Flag to control whether error table is enabled
+ */
+ public static final String IS_ENABLED = ERROR_TABLE_PREFIX + "enabled";
+ public static final boolean DEFAULT_IS_ENABLED = false;
+ /**
+ * Parallelism for writing error table parquet files. Note: Larger value can result into large number of small
+ * files and HDFS Namenode performance may get affected.
+ */
+ public static final String WRITE_PARALLELISM = ERROR_TABLE_PREFIX + "parallelism";
+ public static final int DEFAULT_WRITE_PARALLELISM = 1;
+ /**
+ * Destination folder where error table files will be written
+ */
+ public static final String DESTINATION_PATH = ERROR_TABLE_PREFIX + "dest_path";
+ /**
+ * Flag to control whether error table is written to date partition
+ */
+ public static final String IS_DATE_PARTITIONED = ERROR_TABLE_PREFIX + "date_partitioned";
+ public static final boolean DEFAULT_IS_DATE_PARTITIONED = true;
+
+ @Getter
+ private final Configuration conf;
+ @Getter
+ private final boolean isEnabled;
+ @Getter
+ private Path destPath;
+ @Getter
+ private final int writeParallelism;
+ @Getter
+ private final boolean isDatePartitioned;
+
+ public ErrorTableConfiguration(@NonNull final Configuration conf) {
+ this.conf = conf;
+ this.isEnabled = conf.getBooleanProperty(IS_ENABLED, DEFAULT_IS_ENABLED);
+ if (this.isEnabled()) {
+ ConfigUtil.checkMandatoryProperties(this.conf, getMandatoryProperties());
+ this.destPath = new Path(conf.getProperty(DESTINATION_PATH).get());
+ }
+ this.isDatePartitioned = conf.getBooleanProperty(IS_DATE_PARTITIONED, DEFAULT_IS_DATE_PARTITIONED);
+ this.writeParallelism = conf.getIntProperty(WRITE_PARALLELISM, DEFAULT_WRITE_PARALLELISM);
+ }
+
+ /**
+ * @return hoodie configuration.
+ */
+ public HoodieConfiguration getHoodieConfiguration(@NonNull final Configuration conf,
+ @NotEmpty final String schema,
+ @NotEmpty final String tableKey,
+ @NotEmpty final String errorTableKey,
+ final boolean errorMetricsEnabled) {
+ final HoodieConfiguration hoodieConf = new HoodieConfiguration(conf, tableKey);
+ final String errorTableName = getErrorTableName(hoodieConf.getTableName());
+
+ final HoodieConfiguration.Builder builder = HoodieConfiguration.newBuilder(conf, errorTableKey)
+ .withSchema(schema)
+ .withTableName(errorTableName)
+ .withBasePath(this.getDestPath().toString())
+ .withBulkInsertParallelism(this.getWriteParallelism())
+ .enableMetrics(errorMetricsEnabled)
+ .withWriteStatusClass(HoodieWriteStatus.class);
+ // TODO T1793431 fix error metrics and enable metrics
+ if (errorMetricsEnabled) {
+ final String errorMetricsPrefix = getErrorMetricsPrefix(hoodieConf.getHoodieMetricsPrefix());
+ builder.withMetricsPrefix(errorMetricsPrefix);
+ }
+ return builder.build();
+ }
+
+ /**
+ * @return hoodie error table name.
+ */
+ private String getErrorTableName(@NotEmpty final String hoodieTableName) {
+ return hoodieTableName + ErrorTableUtil.ERROR_TABLE_SUFFIX;
+ }
+
+ /**
+ * @return hoodie error metrics prefix.
+ */
+ private String getErrorMetricsPrefix(@NotEmpty final String metricsPrefix) {
+ return metricsPrefix + ErrorTableUtil.ERROR_TABLE_SUFFIX;
+ }
+
+ public static List getMandatoryProperties() {
+ return Arrays.asList(DESTINATION_PATH);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSinkConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSinkConfiguration.java
new file mode 100644
index 0000000..75c6a67
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/FileSinkConfiguration.java
@@ -0,0 +1,240 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common.configuration;
+
+import com.uber.marmaray.common.DispersalType;
+import com.uber.marmaray.common.FileSinkType;
+import com.uber.marmaray.common.PartitionType;
+import com.uber.marmaray.common.exceptions.MissingPropertyException;
+import com.uber.marmaray.utilities.ConfigUtil;
+import com.uber.marmaray.utilities.SchemaUtil;
+import lombok.Getter;
+import lombok.NonNull;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Optional;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.lang3.EnumUtils;
+import org.apache.commons.lang3.StringUtils;
+
+@Slf4j
+public class FileSinkConfiguration implements Serializable {
+ public static final String FILE_PREFIX_ONLY = "file.";
+ public static final String FILE_COMM_PREFIX = Configuration.MARMARAY_PREFIX + FILE_PREFIX_ONLY;
+ public static final String FILE_NAME_PREFIX = "marmaray";
+ public static final String FS_PATH = FILE_COMM_PREFIX + "file_path";
+ public static final String DEFAULT_FS_PATH = "/dispersal_output";
+ public static final String PATH_PREFIX = FILE_COMM_PREFIX + "path_prefix";
+ public static final String FILE_TYPE = FILE_COMM_PREFIX + "file_type";
+ public static final String DEFAULT_FILE_TYPE = "csv";
+ public static final String CSV_COLUMN_HEADER = FILE_COMM_PREFIX + "with_column_header";
+ public static final Boolean DEFAULT_CSV_COLUMN_HEADER = false;
+ //Expected file size in MegaByte
+ public static final String FILE_SIZE_MEGABYTE = FILE_COMM_PREFIX + "file_size_megabyte";
+ //Default file size set output to one file.
+ public static final long DEFAULT_FILE_SIZE = -1;
+ public static final String SEPARATOR = FILE_COMM_PREFIX + "separator";
+ public static final char DEFAULT_SEPARATOR = ',';
+ //File name related setting
+ public static final String PARTITION_TYPE = FILE_COMM_PREFIX + "partition_type";
+ public static final PartitionType DEFAULT_PARTITION_TYPE = PartitionType.NONE;
+ public static final String TIMESTAMP = FILE_COMM_PREFIX + SchemaUtil.DISPERSAL_TIMESTAMP;
+ public static final String SOURCE_TYPE = FILE_COMM_PREFIX + "source_type";
+ public static final String SOURCE_NAME_PREFIX = FILE_COMM_PREFIX + "source_name_prefix";
+ public static final String SOURCE_PARTITION_PATH = FILE_COMM_PREFIX + "source_partition_path";
+ public static final String DISPERSAL_TYPE = FILE_COMM_PREFIX + "dispersal_type";
+ public static final DispersalType DEFAULT_DISPERSAL_TYPE = DispersalType.VERSION;
+ //aws s3 properties names
+ public static final String FILE_SINK_TYPE = FILE_COMM_PREFIX + "file_sink_type";
+ public static final FileSinkType DEFAULT_FILE_SINK_TYPE = FileSinkType.valueOf("HDFS");
+ public static final String AWS_REGION = FILE_COMM_PREFIX + "aws_region";
+ public static final String BUCKET_NAME = FILE_COMM_PREFIX + "bucket_name";
+ public static final String OBJECT_KEY = FILE_COMM_PREFIX + "object_key";
+ public static final String AWS_ACCESS_KEY_ID = FILE_COMM_PREFIX + "aws_access_key_id";
+ public static final String AWS_SECRET_ACCESS_KEY = FILE_COMM_PREFIX + "aws_secret_access_key";
+ public static final String AWS_LOCAL = FILE_COMM_PREFIX + "aws_local";
+ public static final String DEFAULT_AWS_LOCAL = "/aws_local_tmp";
+ public static final String AWS_JOB_PREFIX = FILE_COMM_PREFIX + "aws_job_prefix";
+
+ @Getter
+ private final char separator;
+ @Getter
+ private final String path;
+ @Getter
+ private final String pathPrefix;
+ @Getter
+ private final String fullPath;
+ @Getter
+ private final String fileType;
+ @Getter
+ private final double fileSizeMegaBytes;
+ @Getter
+ private final boolean columnHeader;
+ @Getter
+ private final FileSinkType sinkType;
+
+ //File name related setting
+ @Getter
+ private final String sourceType;
+ @Getter
+ private final String writeTimestamp;
+ @Getter
+ private final Optional sourcePartitionPath;
+ @Getter
+ private final DispersalType dispersalType;
+ @Getter
+ private final String sourceNamePrefix;
+ @Getter
+ private final String pathHdfs;
+ @Getter
+ private final String fileNamePrefix;
+ @Getter
+ private final PartitionType partitionType;
+
+ //aws s3 properties
+ @Getter
+ private final Optional awsRegion;
+ @Getter
+ private final Optional bucketName;
+ @Getter
+ private final Optional objectKey;
+ @Getter
+ private final Optional awsAccessKeyId;
+ @Getter
+ private final Optional awsSecretAccesskey;
+ @Getter
+ private final String awsLocal;
+ @Getter
+ private final Configuration conf;
+
+ public FileSinkConfiguration(@NonNull final Configuration conf)
+ throws MissingPropertyException, UnsupportedOperationException {
+ this.conf = conf;
+ ConfigUtil.checkMandatoryProperties(this.conf, this.getMandatoryProperties());
+ this.path = this.conf.getProperty(FS_PATH, DEFAULT_FS_PATH);
+ this.fileType = this.conf.getProperty(FILE_TYPE, DEFAULT_FILE_TYPE);
+ this.fileSizeMegaBytes = this.conf.getDoubleProperty(FILE_SIZE_MEGABYTE, DEFAULT_FILE_SIZE);
+ this.columnHeader = this.conf.getBooleanProperty(CSV_COLUMN_HEADER, DEFAULT_CSV_COLUMN_HEADER);
+
+ //File System Prefix
+ this.pathPrefix = this.conf.getProperty(PATH_PREFIX).get();
+
+ if (this.conf.getProperty(SEPARATOR).isPresent()) {
+ if (this.conf.getProperty(SEPARATOR).get().length() != 1) {
+ throw new UnsupportedOperationException("The separator should only contain one single character.");
+ } else {
+ this.separator = this.conf.getProperty(SEPARATOR).get().charAt(0);
+ }
+ } else {
+ this.separator = DEFAULT_SEPARATOR;
+ }
+
+ //File Partition Type
+ if (this.conf.getProperty(PARTITION_TYPE).isPresent()) {
+ this.partitionType = PartitionType.valueOf(this.conf.getProperty(PARTITION_TYPE)
+ .get().trim().toUpperCase());
+ } else {
+ this.partitionType = DEFAULT_PARTITION_TYPE;
+ }
+
+ //Data Dispersal Type: OverWrite or Version[default]
+ if (this.conf.getProperty(DISPERSAL_TYPE).isPresent()) {
+ final String dispersalType = this.conf.getProperty(DISPERSAL_TYPE).get().trim().toUpperCase();
+ final Boolean isValid = EnumUtils.isValidEnum(DispersalType.class, dispersalType);
+ if (isValid) {
+ this.dispersalType = DispersalType.valueOf(dispersalType);
+ } else {
+ final String errorMessage
+ = String.format("The data dispersal type: %s is not supported.", dispersalType);
+ throw new UnsupportedOperationException(errorMessage);
+ }
+ } else {
+ this.dispersalType = DEFAULT_DISPERSAL_TYPE;
+ }
+
+ //File Sink Type : HDFS(default) or AWS S3
+ if (this.conf.getProperty(FILE_SINK_TYPE).isPresent()) {
+ final String sinkName = this.conf.getProperty(FILE_SINK_TYPE).get().trim().toUpperCase();
+ final Boolean isValid = EnumUtils.isValidEnum(FileSinkType.class, sinkName);
+ if (isValid) {
+ this.sinkType = FileSinkType.valueOf(sinkName);
+ } else {
+ final String errorMessage = String.format("The file sink type: %s is not supported.", sinkName);
+ throw new UnsupportedOperationException(errorMessage);
+ }
+ } else {
+ this.sinkType = DEFAULT_FILE_SINK_TYPE;
+ }
+
+ //File Name and Path Configurations
+ this.sourceNamePrefix = this.conf.getProperty(SOURCE_NAME_PREFIX).get();
+
+ if (this.partitionType != PartitionType.NONE) {
+ if (!this.conf.getProperty(SOURCE_PARTITION_PATH).isPresent()) {
+ throw new MissingPropertyException(
+ "The source partition path is missing while partition type is not None.");
+ }
+ this.sourcePartitionPath = this.conf.getProperty(SOURCE_PARTITION_PATH);
+ } else {
+ if (this.conf.getProperty(HiveConfiguration.PARTITION_KEY_NAME).isPresent()) {
+ throw new UnsupportedOperationException(
+ "The partition type is none and there shouldn't be partition key name in Hive Configuration.");
+ }
+ this.sourcePartitionPath = Optional.absent();
+ }
+
+ this.writeTimestamp = this.conf.getProperty(TIMESTAMP).get();
+ this.sourceType = this.conf.getProperty(SOURCE_TYPE).get();
+
+ this.fileNamePrefix = String.format("%s_%s_%s_%s",
+ FILE_NAME_PREFIX, this.sourceType, this.sourceNamePrefix, this.writeTimestamp);
+
+ //Aws S3 configuration initialization
+ this.awsRegion = this.conf.getProperty(AWS_REGION);
+ this.bucketName = this.conf.getProperty(BUCKET_NAME);
+ this.objectKey = this.conf.getProperty(OBJECT_KEY);
+ this.awsAccessKeyId = this.conf.getProperty(AWS_ACCESS_KEY_ID);
+ this.awsSecretAccesskey = this.conf.getProperty(AWS_SECRET_ACCESS_KEY);
+ this.awsLocal = this.conf.getProperty(AWS_LOCAL, DEFAULT_AWS_LOCAL);
+ String fullPath = StringUtils.EMPTY;
+ if (this.sinkType == FileSinkType.HDFS) {
+ fullPath = String.format("%s%s", this.pathPrefix, this.path);
+ if (this.sourcePartitionPath.isPresent()) {
+ fullPath += String.format("/%s", this.sourcePartitionPath.get());
+ }
+ this.fullPath = String.format("%s/%s", fullPath, this.fileNamePrefix);
+ } else {
+ this.fullPath = String.format("%s%s", this.pathPrefix, this.awsLocal);
+ }
+ this.pathHdfs = fullPath;
+ }
+
+ private List getMandatoryProperties() {
+ return Collections.unmodifiableList(
+ Arrays.asList(
+ FileSinkConfiguration.SOURCE_TYPE,
+ FileSinkConfiguration.TIMESTAMP,
+ FileSinkConfiguration.PATH_PREFIX,
+ FileSinkConfiguration.SOURCE_NAME_PREFIX
+ ));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HadoopConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HadoopConfiguration.java
new file mode 100644
index 0000000..44198a6
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HadoopConfiguration.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * {@link HadoopConfiguration} uses system specific hadoop configurations and overrides them with the hadoop specific
+ * configs defined in {@link Configuration} which starts with {@link #HADOOP_COMMON_PROPERTY_PREFIX}.
+ */
+@Slf4j
+@RequiredArgsConstructor
+public class HadoopConfiguration {
+ public static final String HADOOP_COMMON_PROPERTY_PREFIX = Configuration.MARMARAY_PREFIX + "hadoop.";
+
+ // Hadoop properties not defined in our yaml configuration and set dynamically based on cluster topology
+ public static final String HADOOP_DEFAULT_FS = "fs.defaultFS";
+
+ @Getter
+ private final Configuration conf;
+
+ /**
+ * Returns hadoop configuration.
+ */
+ public org.apache.hadoop.conf.Configuration getHadoopConf() {
+ final org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration();
+ this.conf.getPropertiesWithPrefix(HADOOP_COMMON_PROPERTY_PREFIX, true).forEach(
+ (key, value) -> {
+ log.info("hadoop-conf-update:key:[{}]:value:[{}]", key, value);
+ hadoopConf.set(key, value);
+ }
+ );
+ return hadoopConf;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveConfiguration.java
new file mode 100644
index 0000000..ca812aa
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveConfiguration.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.google.common.base.Optional;
+import com.uber.marmaray.utilities.ConfigUtil;
+import lombok.Getter;
+import lombok.NonNull;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * {@link HiveConfiguration} contains all the generic metadata information for where Hive is either a source or
+ * sink for the data pipeline job. All HiveConfiguration properties starts with {@link #HIVE_PROPERTY_PREFIX}.
+ */
+public class HiveConfiguration implements Serializable {
+ public static final String HIVE_PROPERTY_PREFIX = Configuration.MARMARAY_PREFIX + "hive.";
+ public static final String HIVE_DATA_PATH = HIVE_PROPERTY_PREFIX + "data_path";
+ public static final String JOB_NAME = HIVE_PROPERTY_PREFIX + "job_name";
+ public static final String BASE_METADATA_PATH = HIVE_PROPERTY_PREFIX + "job_metadata";
+ public static final String PARTITION_KEY_NAME = HIVE_PROPERTY_PREFIX + "partition_key_name";
+
+ @Getter
+ private final Configuration conf;
+
+ /**
+ * This is the path where the data is either dispersed to (for sink) or read from (source) depending on context
+ */
+ @Getter
+ private final String dataPath;
+
+ @Getter
+ private final String jobName;
+
+ @Getter
+ private final String baseMetadataPath;
+
+ @Getter
+ private final Optional partitionKeyName;
+
+ public HiveConfiguration(@NonNull final Configuration conf) {
+ this.conf = conf;
+ ConfigUtil.checkMandatoryProperties(this.conf, getMandatoryProperties());
+
+ this.dataPath = this.getConf().getProperty(HIVE_DATA_PATH).get();
+ this.jobName = this.getConf().getProperty(JOB_NAME).get();
+ this.baseMetadataPath = this.getConf().getProperty(BASE_METADATA_PATH).get();
+ this.partitionKeyName = this.getConf().getProperty(PARTITION_KEY_NAME);
+ }
+
+ public static List getMandatoryProperties() {
+ return Collections.unmodifiableList(Arrays.asList(HIVE_DATA_PATH, JOB_NAME, BASE_METADATA_PATH));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveSourceConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveSourceConfiguration.java
new file mode 100644
index 0000000..16ba5c7
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HiveSourceConfiguration.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.google.common.base.Optional;
+import lombok.Getter;
+import lombok.NonNull;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+
+import java.io.Serializable;
+import java.util.Date;
+
+/**
+ * {@link HiveSourceConfiguration} class contains all the metadata for running of data pipeline job where Hive
+ * is the source of data. This class extends {@link HiveConfiguration}
+ *
+ * All properties start with {@link #HIVE_SOURCE_PREFIX}.
+ */
+public class HiveSourceConfiguration extends HiveConfiguration implements Serializable {
+
+ public static final String HIVE_SOURCE_PREFIX = HIVE_PROPERTY_PREFIX + "source.";
+ public static final String SAVE_CHECKPOINT = HIVE_SOURCE_PREFIX + "save_checkpoint";
+
+ /**
+ * Used for the very first run to determine first hive partition to disperse (if any).
+ */
+ public static final String HIVE_START_DATE = HIVE_SOURCE_PREFIX + "start_date";
+ public static final String HIVE_START_DATE_FORMAT = "yyyy-MM-dd";
+
+ @Getter
+ private final Optional startDate;
+
+ /**
+ * This allows the option to reprocess an old partition without the need to write a new checkpoint if the
+ * partition was processed in the past.
+ */
+ private final boolean saveCheckpoint;
+
+ public HiveSourceConfiguration(@NonNull final Configuration conf) {
+ super(conf);
+ this.saveCheckpoint = this.getConf().getBooleanProperty(SAVE_CHECKPOINT, true);
+
+ this.startDate = getConf().getProperty(HIVE_START_DATE).isPresent()
+ ? Optional.of(DateTime.parse(getConf().getProperty(HIVE_START_DATE).get(),
+ DateTimeFormat.forPattern(HIVE_START_DATE_FORMAT).withZoneUTC()).toDate())
+ : Optional.absent();
+ }
+
+ public boolean shouldSaveCheckpoint() {
+ return this.saveCheckpoint;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieConfiguration.java
new file mode 100644
index 0000000..71fa778
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieConfiguration.java
@@ -0,0 +1,524 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.google.common.base.Optional;
+import com.uber.hoodie.WriteStatus;
+import com.uber.hoodie.common.model.HoodieCleaningPolicy;
+import com.uber.hoodie.common.table.HoodieTableConfig;
+import com.uber.hoodie.config.HoodieCompactionConfig;
+import com.uber.hoodie.config.HoodieIndexConfig;
+import com.uber.hoodie.config.HoodieMetricsConfig;
+import com.uber.hoodie.config.HoodieWriteConfig;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.common.exceptions.MissingPropertyException;
+import com.uber.marmaray.utilities.ConfigUtil;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.io.FileUtils;
+import org.hibernate.validator.constraints.NotEmpty;
+import scala.Serializable;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * {@link HoodieConfiguration} class holds hoodie configurations.
+ *
+ * All common properties start with {@link #HOODIE_COMMON_PROPERTY_PREFIX}.
+ * All table properties start with {@link #HOODIE_TABLES_PREFIX}.
+ */
+@Slf4j
+public class HoodieConfiguration implements Serializable {
+
+ public static final String HOODIE_COMMON_PROPERTY_PREFIX = Configuration.MARMARAY_PREFIX + "hoodie.%s.";
+ public static final String HOODIE_TABLES_PREFIX = "tables.%s";
+ public static final String HOODIE_DEFAULT_PREFIX = "default";
+ /**
+ * Hoodie table name
+ */
+ public static final String HOODIE_TABLE_NAME = HOODIE_COMMON_PROPERTY_PREFIX + "table_name";
+ /**
+ * Hoodie base path
+ */
+ public static final String HOODIE_BASE_PATH = HOODIE_COMMON_PROPERTY_PREFIX + "base_path";
+ /**
+ * Schema for Hoodie dataset
+ */
+ public static final String HOODIE_AVRO_SCHEMA = HOODIE_COMMON_PROPERTY_PREFIX + "schema";
+ /**
+ * Flag to control whether it should combine before insert
+ */
+ public static final String HOODIE_COMBINE_BEFORE_INSERT = HOODIE_COMMON_PROPERTY_PREFIX + "combine_before_insert";
+ public static final boolean DEFAULT_HOODIE_COMBINE_BEFORE_INSERT = false;
+ /**
+ * Flag to control whether it should combine before upsert
+ */
+ public static final String HOODIE_COMBINE_BEFORE_UPSERT = HOODIE_COMMON_PROPERTY_PREFIX + "combine_before_upsert";
+ public static final boolean DEFAULT_HOODIE_COMBINE_BEFORE_UPSERT = false;
+ /**
+ * Hoodie bulk_insert, insert & upsert parallelism
+ * The default value is the same as HoodieWriteConfig's default (a private variable)
+ */
+ public static final String HOODIE_BULKINSERT_PARALLELISM
+ = HOODIE_COMMON_PROPERTY_PREFIX + "bulkinsert_parallelism";
+ public static final String HOODIE_INSERT_PARALLELISM = HOODIE_COMMON_PROPERTY_PREFIX + "insert_parallelism";
+ public static final String HOODIE_UPSERT_PARALLELISM = HOODIE_COMMON_PROPERTY_PREFIX + "upsert_parallelism";
+ public static final int DEFAULT_HOODIE_PARALLELISM = 200;
+ /**
+ * Auto tune insert parallelism for bulk insert
+ */
+ public static final String HOODIE_AUTO_TUNE_PARALLELISM =
+ HOODIE_COMMON_PROPERTY_PREFIX + "auto_tune_parallelism";
+ public static final boolean DEFAULT_AUTO_TUNE_PARALLELISM = true;
+ /**
+ * Target file size if auto tuning is enabled for insert parallelism.
+ */
+ public static final String HOODIE_TARGET_FILE_SIZE =
+ HOODIE_COMMON_PROPERTY_PREFIX + "auto_target_file_size";
+ // default is set to 1GB which is between HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT and
+ // HOODIE_PARQUET_MAX_FILE_SIZE.
+ public static final long DEFAULT_HOODIE_TARGET_FILE_SIZE = FileUtils.ONE_GB;
+ /**
+ * Write buffer limit in bytes to be used for bulk insert
+ */
+ public static final String HOODIE_INSERT_BUFFER_MEMORY_BYTES =
+ HOODIE_COMMON_PROPERTY_PREFIX + "insert_buffer_memory_bytes";
+ public static final int DEFAULT_HOODIE_INSERT_BUFFER_MEMORY_BYTES = (int) (32 * FileUtils.ONE_MB);
+
+ // Hoodie Compaction parameters
+ /**
+ * Hoodie enable auto clean
+ */
+ public static final String HOODIE_ENABLE_AUTO_CLEAN = HOODIE_COMMON_PROPERTY_PREFIX + "enable_auto_clean";
+ public static final boolean DEFAULT_HOODIE_ENABLE_AUTO_CLEAN = true;
+ /**
+ * Hoodie cleaner policy
+ */
+ public static final String HOODIE_CLEANER_POLICY = HOODIE_COMMON_PROPERTY_PREFIX + "cleaner_policy";
+ public static final String DEFAULT_HOODIE_CLEANER_POLICY = "KEEP_LATEST_COMMITS";
+ /**
+ * Hoodie cleaner commits retained
+ */
+ public static final String HOODIE_CLEANER_COMMITS_RETAINED =
+ HOODIE_COMMON_PROPERTY_PREFIX + "cleaner_commits_retained";
+ public static final int DEFAULT_HOODIE_CLEANER_COMMITS_RETAINED = 10;
+ /**
+ * Hoodie cleaner versions retained
+ */
+ public static final String HOODIE_CLEANER_VERSIONS_RETAINED =
+ HOODIE_COMMON_PROPERTY_PREFIX + "cleaner_versions_retained";
+ public static final int DEFAULT_HOODIE_CLEANER_VERSIONS_RETAINED = 3;
+ /**
+ * Hoodie compaction small file size
+ */
+ public static final String HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT =
+ HOODIE_COMMON_PROPERTY_PREFIX + "compaction_small_file_size_limit";
+ public static final long DEFAULT_HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT = 80 * FileUtils.ONE_MB;
+ /**
+ * Hoodie Storage file size.
+ */
+ /**
+ * Range for maximum parquet file size (uncompressed) is between {@link #HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT}
+ * and 6GB. Default is set to 4GB. Set this value to at least > 2.5 times
+ * {@link #HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT}.
+ */
+ public static final String HOODIE_PARQUET_MAX_FILE_SIZE =
+ HOODIE_COMMON_PROPERTY_PREFIX + "parquet_max_file_size";
+ public static final long DEFAULT_HOODIE_PARQUET_MAX_FILE_SIZE = 4 * FileUtils.ONE_GB;
+ /**
+ * Hoodie insert split size
+ */
+ public static final String HOODIE_INSERT_SPLIT_SIZE = HOODIE_COMMON_PROPERTY_PREFIX + "insert_split_size";
+
+ // Hoodie bloom index properties
+ /**
+ * Hoodie bloom index filter FPP
+ */
+ public static final String HOODIE_BLOOM_FILTER_FPP = HOODIE_COMMON_PROPERTY_PREFIX + "bloom_filter_fpp";
+ public static final double DEFAULT_HOODIE_BLOOM_FILTER_FPP =
+ Double.parseDouble(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP);
+ /**
+ * Hoodie bloom filter num entries
+ */
+ public static final String HOODIE_BLOOM_FILTER_NUM_ENTRIES =
+ HOODIE_COMMON_PROPERTY_PREFIX + "bloom_filter_num_entries";
+ public static final int DEFAULT_HOODIE_BLOOM_FILTER_NUM_ENTRIES =
+ Integer.parseInt(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
+ /**
+ * Hoodie bloom index parallelism
+ */
+ public static final String HOODIE_BLOOM_INDEX_PARALLELISM =
+ HOODIE_COMMON_PROPERTY_PREFIX + "bloom_index_parallelism";
+ public static final int DEFAULT_HOODIE_BLOOM_INDEX_PARALLELISM = 1024;
+
+ // Hoodie Write Status config.
+ /**
+ * Hoodie Write status class
+ */
+ public static final String HOODIE_WRITE_STATUS_CLASS = HOODIE_COMMON_PROPERTY_PREFIX + "write_status_class";
+ public static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getCanonicalName();
+ // Hoodie metrics config.
+ /**
+ * Hoodie metrics prefix
+ */
+ public static final String HOODIE_METRICS_PREFIX = HOODIE_COMMON_PROPERTY_PREFIX + "metrics_prefix";
+ /**
+ * Hoodie enable metrics
+ */
+ public static final String HOODIE_ENABLE_METRICS = HOODIE_COMMON_PROPERTY_PREFIX + "enable_metrics";
+ public static final boolean DEFAULT_HOODIE_ENABLE_METRICS = true;
+
+ /**
+ * Rollback inflight commits.
+ */
+ public static final String HOODIE_ROLLBACK_INFLIGHT_COMMITS =
+ HOODIE_COMMON_PROPERTY_PREFIX + "rollback_inflight_commits";
+ public static final boolean DEFAULT_HOODIE_ROLLBACK_INFLIGHT_COMMITS = true;
+
+ @Getter
+ private final Configuration conf;
+ @Getter
+ private final String tableKey;
+ @Getter
+ private final Optional version;
+
+ public HoodieConfiguration(@NonNull final Configuration conf, @NotEmpty final String tableKey,
+ @NonNull final Optional version) {
+ this.conf = conf;
+ this.tableKey = tableKey;
+ this.version = version;
+ ConfigUtil.checkMandatoryProperties(this.conf, getMandatoryProperties());
+ }
+
+ public HoodieConfiguration(@NonNull final Configuration conf, @NotEmpty final String tableKey) {
+ this(conf, tableKey, Optional.absent());
+ }
+
+ /**
+ * @return List of mandatory properties.
+ */
+ public List getMandatoryProperties() {
+ return Collections.unmodifiableList(Arrays.asList(getTablePropertyKey(HOODIE_TABLE_NAME, this.tableKey),
+ getTablePropertyKey(HOODIE_BASE_PATH, this.tableKey)));
+ }
+
+ /**
+ * @return hoodie base path directory
+ */
+ public String getBasePath() {
+ // HOODIE_BASE_PATH is a mandatory property. Please check {#getMandatoryProperties()}.
+ return this.getConf().getProperty(getTablePropertyKey(HOODIE_BASE_PATH, this.tableKey)).get();
+ }
+
+ /**
+ * @return hoodie table name.
+ */
+ public String getTableName() {
+ return this.getConf().getProperty(getTablePropertyKey(HOODIE_TABLE_NAME, this.tableKey)).get();
+ }
+
+ /**
+ * @return hoodie metrics prefix.
+ * */
+ public String getHoodieMetricsPrefix() {
+ return this.getConf().getProperty(getTablePropertyKey(HOODIE_METRICS_PREFIX, this.tableKey)).get();
+ }
+
+ /**
+ * @return true if {@link com.uber.hoodie.HoodieWriteClient} should rollback inflight commits from previous write
+ * call.
+ */
+ public boolean shouldRollbackInFlight() {
+ return getProperty(HOODIE_ROLLBACK_INFLIGHT_COMMITS, DEFAULT_HOODIE_ROLLBACK_INFLIGHT_COMMITS);
+ }
+
+ /**
+ * @return true if auto-clean is enabled.
+ */
+ public boolean shouldAutoClean() {
+ return getProperty(HOODIE_ENABLE_AUTO_CLEAN, DEFAULT_HOODIE_ENABLE_AUTO_CLEAN);
+ }
+
+ /**
+ * @return true if insert parallelism needs to be auto tuned.
+ */
+ public boolean shouldAutoTuneParallelism() {
+ return getProperty(HOODIE_AUTO_TUNE_PARALLELISM, DEFAULT_AUTO_TUNE_PARALLELISM);
+ }
+
+ /**
+ * @return expected target file size. Needs to be set if {@link #HOODIE_AUTO_TUNE_PARALLELISM} is enabled.
+ */
+ public long getTargetFileSize() {
+ return getProperty(HOODIE_TARGET_FILE_SIZE, DEFAULT_HOODIE_TARGET_FILE_SIZE);
+ }
+
+ /**
+ * Used for updating table property
+ */
+ public void setTableProperty(@NotEmpty final String tablePropertyKey, @NotEmpty final String value) {
+ this.conf.setProperty(getTablePropertyKey(tablePropertyKey, this.tableKey), value);
+ }
+
+ /**
+ * @return returns hoodie properties
+ */
+ public Properties getHoodieInitProperties() {
+ final Properties props = new Properties();
+ props.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, this.getTableName());
+ return props;
+ }
+
+ /**
+ * @return hoodie bulk insert parallelism
+ */
+ public int getBulkInsertParallelism() {
+ return getProperty(HOODIE_BULKINSERT_PARALLELISM, DEFAULT_HOODIE_PARALLELISM);
+ }
+
+ /**
+ * @return hoodie insert parallelism
+ */
+ public int getInsertParallelism() {
+ return getProperty(HOODIE_INSERT_PARALLELISM, DEFAULT_HOODIE_PARALLELISM);
+ }
+
+ /**
+ * @return hoodie upsert parallelism
+ */
+ public int getUpsertParallelism() {
+ return getProperty(HOODIE_UPSERT_PARALLELISM, DEFAULT_HOODIE_PARALLELISM);
+ }
+
+ /**
+ * @return {@link HoodieWriteConfig}. It uses {@link #conf} to create {@link HoodieWriteConfig}. If any property is
+ * missing then it will throw {@link MissingPropertyException}.
+ */
+ public HoodieWriteConfig getHoodieWriteConfig() {
+ final HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder();
+ try {
+ builder.forTable(getTableName());
+ builder.withPath(getBasePath());
+ final boolean combineBeforeInsert =
+ getProperty(HOODIE_COMBINE_BEFORE_INSERT, DEFAULT_HOODIE_COMBINE_BEFORE_INSERT);
+ final boolean combineBeforeUpsert =
+ getProperty(HOODIE_COMBINE_BEFORE_UPSERT, DEFAULT_HOODIE_COMBINE_BEFORE_UPSERT);
+ builder.combineInput(combineBeforeInsert, combineBeforeUpsert);
+ final String schemaPropertyKey = getTablePropertyKey(HOODIE_AVRO_SCHEMA, this.tableKey);
+ final Optional schema = this.conf.getProperty(schemaPropertyKey);
+ if (!schema.isPresent()) {
+ throw new MissingPropertyException(schemaPropertyKey);
+ }
+ builder.withSchema(schema.get());
+ builder.withParallelism(this.getInsertParallelism(), this.getUpsertParallelism())
+ .withBulkInsertParallelism(this.getBulkInsertParallelism());
+ builder.withAutoCommit(false);
+
+ // Date partitioning.
+ builder.withAssumeDatePartitioning(true);
+
+ // Hoodie compaction config.
+ final HoodieCompactionConfig.Builder compactionConfigBuilder = HoodieCompactionConfig.newBuilder();
+ compactionConfigBuilder.withCleanerPolicy(HoodieCleaningPolicy
+ .valueOf(getProperty(HOODIE_CLEANER_POLICY, DEFAULT_HOODIE_CLEANER_POLICY)));
+ compactionConfigBuilder.retainCommits(
+ getProperty(HOODIE_CLEANER_COMMITS_RETAINED, DEFAULT_HOODIE_CLEANER_COMMITS_RETAINED));
+ compactionConfigBuilder.retainFileVersions(
+ getProperty(HOODIE_CLEANER_VERSIONS_RETAINED, DEFAULT_HOODIE_CLEANER_VERSIONS_RETAINED));
+ final Integer insertSplitSize = getProperty(HOODIE_INSERT_SPLIT_SIZE, -1);
+ if (insertSplitSize > 0) {
+ compactionConfigBuilder.autoTuneInsertSplits(false);
+ compactionConfigBuilder.insertSplitSize(insertSplitSize);
+ } else {
+ compactionConfigBuilder.autoTuneInsertSplits(true);
+ }
+ compactionConfigBuilder.compactionSmallFileSize(
+ getProperty(HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT,
+ DEFAULT_HOODIE_COMPACTION_SMALL_FILE_SIZE_LIMIT));
+ compactionConfigBuilder.withAutoClean(shouldAutoClean());
+ builder.withCompactionConfig(compactionConfigBuilder.build());
+
+ // Hoodie index config
+ builder.withIndexConfig(new HoodieIndexConfiguration(getConf(), getTableKey()).configureHoodieIndex());
+
+ // Hoodie metrics config
+ final boolean enableMetrics = getProperty(HOODIE_ENABLE_METRICS, DEFAULT_HOODIE_ENABLE_METRICS);
+ if (enableMetrics) {
+ final HoodieMetricsConfig.Builder hoodieMetricsConfigBuilder = HoodieMetricsConfig.newBuilder();
+ final String hoodieMetricsPropertyKey = getTablePropertyKey(HOODIE_METRICS_PREFIX, this.tableKey);
+ final Optional hoodieMetricsProperty = this.conf.getProperty(hoodieMetricsPropertyKey);
+ if (!hoodieMetricsProperty.isPresent()) {
+ throw new MissingPropertyException(hoodieMetricsPropertyKey);
+ }
+ hoodieMetricsConfigBuilder.usePrefix(hoodieMetricsProperty.get());
+ hoodieMetricsConfigBuilder.on(getProperty(HOODIE_ENABLE_METRICS, DEFAULT_HOODIE_ENABLE_METRICS));
+ builder.withMetricsConfig(hoodieMetricsConfigBuilder.build());
+ }
+ // Write status StorageLevel.
+ builder.withWriteStatusStorageLevel("DISK_ONLY");
+ final String writeStatusClassName =
+ getProperty(HOODIE_WRITE_STATUS_CLASS, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
+ try {
+ builder.withWriteStatusClass(
+ (Class extends WriteStatus>) Class.forName(writeStatusClassName));
+ } catch (ClassNotFoundException e) {
+ final String errorStr =
+ String.format("error loading hoodie write status class :{}", writeStatusClassName);
+ log.error(errorStr);
+ throw new JobRuntimeException(errorStr, e);
+ }
+
+ // enable tmp directory writes for hoodie.
+ builder.withUseTempFolderCopyOnWriteForCreate(true);
+
+ return builder.build();
+ } catch (IllegalArgumentException e) {
+ throw new MissingPropertyException(e.getMessage(), e);
+ }
+ }
+
+ /**
+ * It will read property value from table and default namespace. Value will be returned in following order.
+ * For example for propertyKey ("common.hoodie.%s.insert_split_size")
+ * 1) table specific value ("common.hoodie.tables.table1.insert_split_size" defined in {@link Configuration})
+ * 2) default hoodie property value ("common.hoodie.default.insert_split_size" defined in {@link Configuration})
+ * 3) default value specified. (passed in as an argument).
+ *
+ * @param propertyKey hoodie property key
+ * @param defaultValue default value of the property
+ * @param DataType of the property
+ */
+ public T getProperty(@NotEmpty final String propertyKey,
+ @NonNull final T defaultValue) {
+ final String defaultKey = getDefaultPropertyKey(propertyKey);
+ final String tableKey = getTablePropertyKey(propertyKey, this.tableKey);
+ final T retValue = Configuration.getProperty(this.conf, defaultKey, defaultValue);
+ return Configuration.getProperty(this.conf, tableKey, retValue);
+ }
+
+ public static String getTablePropertyKey(@NotEmpty final String propertyKey, @NotEmpty final String tableKey) {
+ return String.format(propertyKey, String.format(HOODIE_TABLES_PREFIX, tableKey));
+ }
+
+ public static String getDefaultPropertyKey(@NotEmpty final String propertyKey) {
+ return String.format(propertyKey, HOODIE_DEFAULT_PREFIX);
+ }
+
+ public static Builder newBuilder(@NotEmpty final String tableKey) {
+ return newBuilder(new Configuration(), tableKey);
+ }
+
+ public static Builder newBuilder(@NonNull final Configuration conf, @NotEmpty final String tableKey) {
+ return new Builder(conf, tableKey);
+ }
+
+ /**
+ * Builder class to build {@link HoodieConfiguration}.
+ */
+ public static final class Builder {
+
+ private final Configuration conf;
+ private final String tableKey;
+ private Optional version = Optional.absent();
+
+ private Builder(@NonNull final Configuration conf, @NotEmpty final String tableKey) {
+ this.conf = conf;
+ this.tableKey = tableKey;
+ }
+
+ public Builder withTableName(@NotEmpty final String tableName) {
+ this.conf.setProperty(getTablePropertyKey(HOODIE_TABLE_NAME, this.tableKey), tableName);
+ return this;
+ }
+
+ public Builder withBasePath(@NotEmpty final String basePath) {
+ this.conf.setProperty(getTablePropertyKey(HOODIE_BASE_PATH, this.tableKey), basePath);
+ return this;
+ }
+
+ public Builder withSchema(@NotEmpty final String schema) {
+ this.conf.setProperty(getTablePropertyKey(HOODIE_AVRO_SCHEMA, this.tableKey), schema);
+ return this;
+ }
+
+ public Builder withBulkInsertParallelism(final int parallelism) {
+ this.conf.setProperty(
+ getTablePropertyKey(HOODIE_BULKINSERT_PARALLELISM, this.tableKey), Integer.toString(parallelism));
+ return this;
+ }
+
+ public Builder withInsertParallelism(final int parallelism) {
+ this.conf.setProperty(
+ getTablePropertyKey(HOODIE_INSERT_PARALLELISM, this.tableKey), Integer.toString(parallelism));
+ return this;
+ }
+
+ public Builder withUpsertParallelism(final int parallelism) {
+ this.conf.setProperty(
+ getTablePropertyKey(HOODIE_UPSERT_PARALLELISM, this.tableKey), Integer.toString(parallelism));
+ return this;
+ }
+
+ public Builder withMetricsPrefix(@NotEmpty final String metricsPrefix) {
+ this.conf.setProperty(getTablePropertyKey(HOODIE_METRICS_PREFIX, tableKey), metricsPrefix);
+ return this;
+ }
+
+ public Builder withCombineBeforeInsert(final boolean combineBeforeInsert) {
+ this.conf.setProperty(getTablePropertyKey(HOODIE_COMBINE_BEFORE_INSERT, tableKey),
+ Boolean.toString(combineBeforeInsert));
+ return this;
+ }
+
+ public Builder withCombineBeforeUpsert(final boolean combineBeforeUpsert) {
+ this.conf.setProperty(getTablePropertyKey(HOODIE_COMBINE_BEFORE_UPSERT, tableKey),
+ Boolean.toString(combineBeforeUpsert));
+ return this;
+ }
+
+ public Builder enableMetrics(final boolean enableMetrics) {
+ this.conf.setProperty(
+ getTablePropertyKey(HOODIE_ENABLE_METRICS, tableKey), Boolean.toString(enableMetrics));
+ return this;
+ }
+
+ public Builder autoTuneParallelism(final boolean autoTuneParallelism) {
+ this.conf.setProperty(
+ getTablePropertyKey(HOODIE_AUTO_TUNE_PARALLELISM, tableKey), Boolean.toString(autoTuneParallelism));
+ return this;
+ }
+
+ public Builder withWriteStatusClass(@NotEmpty final Class writeStatusClass) {
+ this.conf.setProperty(
+ getTablePropertyKey(HOODIE_WRITE_STATUS_CLASS, tableKey), writeStatusClass.getCanonicalName());
+ return this;
+ }
+
+ public Builder withVersion(@NotEmpty final String version) {
+ this.version = Optional.of(version);
+ return this;
+ }
+
+ public HoodieConfiguration build() {
+ return new HoodieConfiguration(this.conf, this.tableKey, this.version);
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieIndexConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieIndexConfiguration.java
new file mode 100644
index 0000000..d73c8dd
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/HoodieIndexConfiguration.java
@@ -0,0 +1,207 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common.configuration;
+
+import com.google.common.base.Preconditions;
+import com.uber.hoodie.config.HoodieIndexConfig;
+import com.uber.hoodie.index.HoodieIndex;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.utilities.StringTypes;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.io.IOException;
+
+@Slf4j
+public class HoodieIndexConfiguration extends HoodieConfiguration {
+
+ // Hoodie Index config
+ public static final String HOODIE_INDEX_PROPERTY_PREFIX =
+ HoodieConfiguration.HOODIE_COMMON_PROPERTY_PREFIX + "index.";
+ /**
+ * Hoodie index types
+ */
+ public static final String HOODIE_BLOOM_INDEX = "bloom";
+ public static final String HOODIE_HBASE_INDEX = "hbase";
+ public static final String HOODIE_HBASE_INDEX_PREFIX = "hbase.";
+ public static final String HOODIE_INDEX_TYPE = HOODIE_INDEX_PROPERTY_PREFIX + "type";
+ public static final String HOODIE_INDEX_ZKNODE = "zknode.";
+ public static final String DEFAULT_HOODIE_INDEX_TYPE = HOODIE_BLOOM_INDEX;
+ /**
+ * Hoodie index zookeeper
+ */
+ public static final String HOODIE_INDEX_ZOOKEEPER_QUORUM =
+ HOODIE_INDEX_PROPERTY_PREFIX + "zookeeper_quorum";
+ public static final String HOODIE_INDEX_ZOKEEPER_PORT = HOODIE_INDEX_PROPERTY_PREFIX + "zookeeper_port";
+ public static final String HOODIE_INDEX_HBASE_ZK_ZNODEPARENT =
+ HOODIE_INDEX_PROPERTY_PREFIX + HOODIE_HBASE_INDEX_PREFIX + HOODIE_INDEX_ZKNODE + "path";
+ /**
+ * Hoodie index get batch size
+ */
+ public static final String HOODIE_INDEX_GET_BATCH_SIZE =
+ HOODIE_INDEX_PROPERTY_PREFIX + "get_batch_size";
+ public static final int DEFAULT_HOODIE_INDEX_GET_BATCH_SIZE = 1000;
+ /**
+ * Hoodie index QPS fraction
+ */
+ public static final String HOODIE_INDEX_QPS_FRACTION = HOODIE_INDEX_PROPERTY_PREFIX + "qps_fraction";
+ public static final double DEFAULT_HOODIE_INDEX_QPS_FRACTION = 0.125f;
+ /**
+ * Hoodie index max QPS per region server
+ */
+ public static final String HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER =
+ HOODIE_INDEX_PROPERTY_PREFIX + "max_qps_per_region_server";
+ public static final int DEFAULT_HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER = 400;
+ public static final String DEFAULT_VERSION = "";
+
+ /**
+ * Hoodie HBase index table name. Required if the index type is hbase.
+ */
+ public static final String HOODIE_HBASE_INDEX_TABLE_NAME =
+ HOODIE_INDEX_PROPERTY_PREFIX + "hbase_index_table";
+
+ @Getter
+ private final Configuration conf;
+ @Getter
+ private final String tableKey;
+
+ public HoodieIndexConfiguration(@NonNull final Configuration conf, @NotEmpty final String tableKey) {
+ super(conf, tableKey);
+ this.conf = conf;
+ this.tableKey = tableKey;
+ }
+
+ public HoodieIndex.IndexType getHoodieIndexType() {
+ final String indexName = getProperty(HOODIE_INDEX_TYPE, DEFAULT_HOODIE_INDEX_TYPE);
+ if (HOODIE_BLOOM_INDEX.equals(indexName.toLowerCase())) {
+ return HoodieIndex.IndexType.BLOOM;
+ } else if (HOODIE_HBASE_INDEX.equals(indexName.toLowerCase())) {
+ return HoodieIndex.IndexType.HBASE;
+ } else {
+ throw new IllegalStateException("Unsupported index type " + indexName);
+ }
+ }
+
+ public String getHoodieIndexZookeeperQuorum() {
+ final String value = getProperty(HOODIE_INDEX_ZOOKEEPER_QUORUM, StringTypes.EMPTY);
+ Preconditions.checkState(!value.isEmpty(), "%s must not be empty", HOODIE_INDEX_ZOOKEEPER_QUORUM);
+ return value;
+ }
+
+ public String getHoodieHbaseIndexTableName() {
+ final String value = getProperty(HOODIE_HBASE_INDEX_TABLE_NAME, StringTypes.EMPTY);
+ Preconditions.checkState(!value.isEmpty(), "%s must not be empty", HOODIE_HBASE_INDEX_TABLE_NAME);
+ return value;
+ }
+
+ public int getHoodieIndexZookeeperPort() {
+ final int value = getProperty(HOODIE_INDEX_ZOKEEPER_PORT, 0);
+ Preconditions.checkState(value > 0, "%s must be greater than zero", HOODIE_INDEX_ZOKEEPER_PORT);
+ return value;
+ }
+
+ public String getZkZnodeParent() {
+ final String value = getProperty(HOODIE_INDEX_HBASE_ZK_ZNODEPARENT, StringTypes.EMPTY);
+ Preconditions.checkState(!value.isEmpty(), "%s must always be set", HOODIE_INDEX_HBASE_ZK_ZNODEPARENT);
+ return value;
+ }
+
+ public int getHoodieIndexMaxQpsPerRegionServer() {
+ final int value = getProperty(HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER,
+ DEFAULT_HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER);
+ Preconditions.checkState(value > 0, "%s must be greater than zero", HOODIE_INDEX_MAX_QPS_PER_REGION_SERVER);
+ return value;
+ }
+
+ public double getHoodieIndexQPSFraction() {
+ final double value = getProperty(HOODIE_INDEX_QPS_FRACTION, DEFAULT_HOODIE_INDEX_QPS_FRACTION);
+ Preconditions.checkState(value > 0 && value <= 1, "%s must be between 0 and 1", HOODIE_INDEX_QPS_FRACTION);
+ return value;
+ }
+
+ public int getHoodieIndexGetBatchSize() {
+ final int value = getProperty(HOODIE_INDEX_GET_BATCH_SIZE, DEFAULT_HOODIE_INDEX_GET_BATCH_SIZE);
+ Preconditions.checkState(value > 0, "%s must be greater than zero", HOODIE_INDEX_GET_BATCH_SIZE);
+ return value;
+ }
+
+ /**
+ * Configure the Hoodie HBase index.
+ */
+ public HoodieIndexConfig configureHoodieIndex() {
+ final String version;
+ if (getVersion().isPresent()) {
+ version = getVersion().get();
+ } else {
+ version = DEFAULT_VERSION;
+ }
+ final String topicName = getTableName();
+ final HoodieIndexConfig.Builder builder = HoodieIndexConfig.newBuilder()
+ .withIndexType(getHoodieIndexType());
+
+ if (HoodieIndex.IndexType.HBASE.equals(getHoodieIndexType())) {
+ final String quorum = getHoodieIndexZookeeperQuorum();
+ final Integer port = getHoodieIndexZookeeperPort();
+ final String zkZnodeParent = getZkZnodeParent();
+ createHbaseIndexTableIfNotExists(topicName, quorum, port.toString(), zkZnodeParent,
+ version);
+ }
+
+ return builder.build();
+ }
+
+ public void createHbaseIndexTableIfNotExists(@NotEmpty final String dataFeed, @NotEmpty final String zkQuorum,
+ @NotEmpty final String zkPort, @NotEmpty final String zkZnodeParent, @NotEmpty final String version) {
+ final String tableName = getHoodieHbaseIndexTableName();
+ final String family = "_s";
+ final org.apache.hadoop.conf.Configuration hbaseConfig = new org.apache.hadoop.conf.Configuration();
+ hbaseConfig.set("hbase.zookeeper.quorum", zkQuorum);
+ hbaseConfig.set("hbase.zookeeper.property.clientPort", zkPort);
+ hbaseConfig.set("zookeeper.znode.parent", zkZnodeParent);
+ try {
+ try (final Connection connection = ConnectionFactory.createConnection(hbaseConfig)) {
+ if (!connection.getAdmin().tableExists(TableName.valueOf(tableName))) {
+ final HTableDescriptor tableDesc = new HTableDescriptor(TableName.valueOf(tableName));
+ final HColumnDescriptor familyDesc = new HColumnDescriptor(Bytes.toBytes(family));
+ familyDesc.setBloomFilterType(BloomType.ROW);
+ familyDesc.setCompressionType(Compression.Algorithm.SNAPPY);
+ tableDesc.addFamily(familyDesc);
+ connection.getAdmin().createTable(tableDesc);
+ log.info("Created HBase table {} with family {}", tableName, family);
+ } else {
+ log.debug("HBase table {} exists", tableName);
+ }
+ }
+ } catch (IOException e) {
+ //todo: better handle try catch
+ log.error("Error creating HBase table {} ", tableName, e);
+ throw new JobRuntimeException(e);
+ }
+
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaConfiguration.java
new file mode 100644
index 0000000..bd64947
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaConfiguration.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.uber.marmaray.utilities.ConfigUtil;
+import lombok.Getter;
+import lombok.NonNull;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import scala.Serializable;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * {@link KafkaConfiguration} holds all the common kafka configurations.
+ *
+ * All common properties start with {@link #KAFKA_COMMON_PROPERTY_PREFIX}.
+ */
+public class KafkaConfiguration implements Serializable {
+
+ public static final String KAFKA_COMMON_PROPERTY_PREFIX = Configuration.MARMARAY_PREFIX + "kafka.";
+ public static final String KAFKA_CONNECTION_PREFIX = KAFKA_COMMON_PROPERTY_PREFIX + "conn.";
+ public static final String KAFKA_BROKER_LIST = KAFKA_CONNECTION_PREFIX + "bootstrap.servers";
+ public static final String KAFKA_GROUP_ID = KAFKA_CONNECTION_PREFIX + "group.id";
+ public static final String KEY_DESERIALIZER = "key.deserializer";
+ public static final String ENABLE_AUTO_COMMIT = "enable.auto.commit";
+ public static final String VALUE_DESERIALIZER = "value.deserializer";
+ public static final String GROUP_ID = "group.id";
+ public static final String DEFAULT_GROUP_ID = "marmaray_group";
+ public static final String ENABLE_AUTO_COMMIT_VALUE = "false";
+ public static final String KAFKA_BACKOFF_MS_CONFIG = "retry.backoff.ms";
+ public static final String DEFAULT_KAFKA_BACKOFF_MS_CONFIG = "20";
+
+ @Getter
+ private final Configuration conf;
+
+ /**
+ * It holds the connection related parameters required for connecting to kafka broker.
+ */
+ @Getter
+ private final Map kafkaParams;
+
+ public KafkaConfiguration(@NonNull final Configuration conf) {
+ this.conf = conf;
+ ConfigUtil.checkMandatoryProperties(this.conf, getMandatoryProperties());
+ this.kafkaParams = new HashMap<>();
+ this.kafkaParams.put(GROUP_ID, DEFAULT_GROUP_ID);
+ this.kafkaParams.putAll(getConf().getPropertiesWithPrefix(KAFKA_CONNECTION_PREFIX, true));
+ this.kafkaParams.put(KEY_DESERIALIZER, ByteArrayDeserializer.class.getCanonicalName());
+ this.kafkaParams.put(VALUE_DESERIALIZER, ByteArrayDeserializer.class.getCanonicalName());
+ this.kafkaParams.put(ENABLE_AUTO_COMMIT, ENABLE_AUTO_COMMIT_VALUE);
+ // If retry backoff is not set then we would want to reduce it to lower values. Default value is 400ms.
+ if (!kafkaParams.containsKey(KAFKA_BACKOFF_MS_CONFIG)) {
+ kafkaParams.put(KAFKA_BACKOFF_MS_CONFIG, DEFAULT_KAFKA_BACKOFF_MS_CONFIG);
+ }
+ }
+
+ public List getMandatoryProperties() {
+ return Arrays.asList(KAFKA_BROKER_LIST);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaSourceConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaSourceConfiguration.java
new file mode 100644
index 0000000..b5525f1
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/KafkaSourceConfiguration.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.uber.marmaray.utilities.NumberConstants;
+import lombok.Getter;
+import lombok.NonNull;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * {@link KafkaSourceConfiguration} defines configurations for Kafka source and extends {@link KafkaConfiguration}.
+ *
+ * All properties start with {@link #KAFKA_COMMON_PROPERTY_PREFIX}.
+ */
+public class KafkaSourceConfiguration extends KafkaConfiguration {
+
+ public static final String KAFKA_PROPERTY_PREFIX = KAFKA_COMMON_PROPERTY_PREFIX + "source.";
+ public static final String KAFKA_TOPIC_NAME = KAFKA_PROPERTY_PREFIX + "topic_name";
+ public static final String KAFKA_CLUSTER_NAME = KAFKA_PROPERTY_PREFIX + "cluster_name";
+ public static final String KAFKA_MAX_MESSAGES_TO_READ = KAFKA_PROPERTY_PREFIX + "max_messages";
+ public static final long DEFAULT_KAFKA_MAX_MESSAGES_TO_READ = NumberConstants.ONE_MILLION;
+ public static final String KAFKA_READ_PARALLELISM = KAFKA_PROPERTY_PREFIX + "read_parallelism";
+ /**
+ * It is used for the very first run to set partition offsets for kafka topic. Expected format is "yyyy-MM-dd".
+ */
+ public static final String KAFKA_START_DATE = KAFKA_PROPERTY_PREFIX + "start_date";
+ public static final String KAFKA_START_DATE_FORMAT = "yyyy-MM-dd";
+
+ @Getter
+ private final String topicName;
+ @Getter
+ private final String clusterName;
+ /**
+ * start time in millis. (inclusive).
+ */
+ @Getter
+ private final long startTime;
+
+ public KafkaSourceConfiguration(@NonNull final Configuration conf) {
+ super(conf);
+ this.topicName = getConf().getProperty(KAFKA_TOPIC_NAME).get();
+ this.clusterName = getConf().getProperty(KAFKA_CLUSTER_NAME).get();
+ this.startTime =
+ DateTime.parse(getConf().getProperty(KAFKA_START_DATE).get(),
+ DateTimeFormat.forPattern(KafkaSourceConfiguration.KAFKA_START_DATE_FORMAT).withZoneUTC()
+ ).toDate().getTime();
+ }
+
+ public List getMandatoryProperties() {
+ final List ret = new LinkedList<>();
+ ret.addAll(super.getMandatoryProperties());
+ ret.add(KAFKA_TOPIC_NAME);
+ ret.add(KAFKA_CLUSTER_NAME);
+ ret.add(KAFKA_START_DATE);
+ return ret;
+ }
+
+ public int getReadParallelism() {
+ return Math.max(1, getConf().getIntProperty(KAFKA_READ_PARALLELISM, 1));
+ }
+
+ public long getMaxMessagesToRead() {
+ return getConf().getLongProperty(KAFKA_MAX_MESSAGES_TO_READ, DEFAULT_KAFKA_MAX_MESSAGES_TO_READ);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/LockManagerConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/LockManagerConfiguration.java
new file mode 100644
index 0000000..a64ff12
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/LockManagerConfiguration.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.uber.marmaray.utilities.ConfigUtil;
+
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+/**
+ * {@link LockManagerConfiguration} defines configurations taking locks on jobs via ZooKeeper
+ *
+ * All properties start with {@link #LOCK_MANAGER_PREFIX}.
+ */
+public class LockManagerConfiguration {
+
+ public static final String LOCK_MANAGER_PREFIX = Configuration.MARMARAY_PREFIX + "lock_manager.";
+
+ public static final String IS_ENABLED = LOCK_MANAGER_PREFIX + "is_enabled";
+ public static final boolean DEFAULT_IS_ENABLED = true;
+
+ public static final String ZK_BASE_PATH = LOCK_MANAGER_PREFIX + "zk_base_path";
+
+ public static final String ZK_SESSION_TIMEOUT_MS = LOCK_MANAGER_PREFIX + "zk_session_timeout_ms";
+ public static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 60 * 1000;
+
+ public static final String ZK_CONNECTION_TIMEOUT_MS = LOCK_MANAGER_PREFIX + "zk_connection_timeout_ms";
+ public static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 15 * 1000;
+
+ public static final String ACQUIRE_LOCK_TIME_MS = LOCK_MANAGER_PREFIX + "acquire_lock_time_ms";
+ public static final int DEFAULT_ACQUIRE_LOCK_TIME_MS = 60 * 1000;
+
+ @Getter
+ private final Configuration conf;
+
+ @Getter
+ private final boolean isEnabled;
+
+ @Getter
+ private final String zkBasePath;
+
+ @Getter
+ private final int zkSessionTimeoutMs;
+
+ @Getter
+ private final int zkConnectionTimeoutMs;
+
+ @Getter
+ private final int acquireLockTimeMs;
+
+ public LockManagerConfiguration(@NonNull final Configuration conf) {
+ this.conf = conf;
+ this.isEnabled = this.getConf().getBooleanProperty(IS_ENABLED, DEFAULT_IS_ENABLED);
+ if (this.isEnabled()) {
+ ConfigUtil.checkMandatoryProperties(conf, getMandatoryProperties());
+ this.zkBasePath = cleanZkBasePath(this.getConf().getProperty(ZK_BASE_PATH).get());
+ } else {
+ this.zkBasePath = null;
+ }
+ this.zkSessionTimeoutMs = this.getConf().getIntProperty(ZK_SESSION_TIMEOUT_MS, DEFAULT_ZK_SESSION_TIMEOUT_MS);
+ this.zkConnectionTimeoutMs = this.getConf().getIntProperty(ZK_CONNECTION_TIMEOUT_MS,
+ DEFAULT_ZK_CONNECTION_TIMEOUT_MS);
+ this.acquireLockTimeMs = this.getConf().getIntProperty(ACQUIRE_LOCK_TIME_MS, DEFAULT_ACQUIRE_LOCK_TIME_MS);
+ }
+
+ private String cleanZkBasePath(@NotEmpty final String orig) {
+ final String cleaned = orig.replaceAll("//*", "/").replaceAll("^/", "").replaceAll("/$", "");
+ return cleaned;
+ }
+
+ private static List getMandatoryProperties() {
+ return Collections.unmodifiableList(Arrays.asList(ZK_BASE_PATH));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/RetryStrategyConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/RetryStrategyConfiguration.java
new file mode 100644
index 0000000..1467ebe
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/RetryStrategyConfiguration.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.common.retry.IRetryStrategy;
+import com.uber.marmaray.common.retry.SimpleRetryStrategy;
+
+import java.io.Serializable;
+
+import lombok.NonNull;
+
+/**
+ * {@link RetryStrategyConfiguration} defines configurations related to the retry strategy.
+ *
+ * All properties start with {@link #RETRY_STRATEGY_PREFIX}.
+ */
+public class RetryStrategyConfiguration implements Serializable {
+ public static final String RETRY_STRATEGY_PREFIX = Configuration.MARMARAY_PREFIX + "retry_strategy.";
+ public static final String DEFAULT_STRATEGY = RETRY_STRATEGY_PREFIX + "default_strategy";
+
+ public static final String SIMPLE_RETRY_STRATEGY = "SimpleRetryStrategy";
+
+ private final Configuration conf;
+
+ public RetryStrategyConfiguration(@NonNull final Configuration conf) {
+ this.conf = conf;
+ }
+
+ public IRetryStrategy getRetryStrategy() {
+ final String strategy = conf.getProperty(DEFAULT_STRATEGY, SIMPLE_RETRY_STRATEGY);
+
+ switch (strategy) {
+ case SIMPLE_RETRY_STRATEGY:
+ return new SimpleRetryStrategy(new SimpleRetryStrategyConfiguration(conf));
+ default:
+ throw new JobRuntimeException(String.format("Retry strategy %s is not supported.", strategy));
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/SimpleRetryStrategyConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/SimpleRetryStrategyConfiguration.java
new file mode 100644
index 0000000..0f0b00a
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/SimpleRetryStrategyConfiguration.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.uber.marmaray.common.retry.SimpleRetryStrategy;
+
+import java.io.Serializable;
+
+import lombok.NonNull;
+
+/**
+ * {@link SimpleRetryStrategyConfiguration} defines configurations related to a retry strategy based on
+ * total number of retries and time to wait in between retries.
+ *
+ * All properties start with {@link #SIMPLE_RETRY_PREFIX}.
+ */
+public class SimpleRetryStrategyConfiguration implements Serializable {
+ private static final String SIMPLE_RETRY_PREFIX = RetryStrategyConfiguration.RETRY_STRATEGY_PREFIX + "simple.";
+ private static final String NUM_OF_RETRIES = SIMPLE_RETRY_PREFIX + "num_of_retries";
+ private static final String WAIT_TIME_IN_MS = SIMPLE_RETRY_PREFIX + "wait_time_in_ms";
+
+ private final Configuration conf;
+
+ public SimpleRetryStrategyConfiguration(@NonNull final Configuration conf) {
+ this.conf = conf;
+ }
+
+ public int getNumRetries() {
+ return conf.getIntProperty(NUM_OF_RETRIES, SimpleRetryStrategy.DEFAULT_NUMBER_OF_RETRIES);
+ }
+
+ public long getWaitTimeInMs() {
+ return conf.getLongProperty(WAIT_TIME_IN_MS, SimpleRetryStrategy.DEFAULT_WAIT_TIME_IN_MS);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/configuration/ZookeeperConfiguration.java b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ZookeeperConfiguration.java
new file mode 100644
index 0000000..a7c6cbc
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/configuration/ZookeeperConfiguration.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.configuration;
+
+import com.uber.marmaray.utilities.ConfigUtil;
+import lombok.Getter;
+import lombok.NonNull;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * {@link ZookeeperConfiguration} defines zookeeper related configuration
+ *
+ * All properties start with {@link #ZK_PROPERTY_PREFIX}.
+ */
+public class ZookeeperConfiguration {
+ public static final String ZK_PROPERTY_PREFIX = Configuration.MARMARAY_PREFIX + "zookeeper.";
+ public static final String ZK_QUORUM = ZK_PROPERTY_PREFIX + "quorum";
+ public static final String ZK_PORT = ZK_PROPERTY_PREFIX + "port";
+
+ @Getter
+ private final Configuration conf;
+ @Getter
+ private final String zkQuorum;
+ @Getter
+ private final String zkPort;
+
+ public ZookeeperConfiguration(@NonNull final Configuration conf) {
+ this.conf = conf;
+ ConfigUtil.checkMandatoryProperties(this.getConf(), getMandatoryProperties());
+
+ this.zkQuorum = this.getConf().getProperty(ZK_QUORUM).get();
+ this.zkPort = this.getConf().getProperty(ZK_PORT).get();
+ }
+
+ public static List getMandatoryProperties() {
+ return Collections.unmodifiableList(Arrays.asList(ZK_QUORUM, ZK_PORT));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/converterresult/ConverterResult.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/converterresult/ConverterResult.java
new file mode 100644
index 0000000..d9c355c
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/converterresult/ConverterResult.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common.converters.converterresult;
+
+import com.google.common.base.Optional;
+import com.uber.marmaray.common.data.ErrorData;
+import com.uber.marmaray.common.data.RawData;
+import com.uber.marmaray.common.data.ValidData;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.io.Serializable;
+
+/**
+ * {@link ConverterResult} associates data of an input type with the converted data with an output type
+ *
+ * User has option to implement a conversion data of type OD to type ID in a one-way transformation as needed
+ *
+ * This class is used to maintain parity between the ID data that was converted to OD data as needed for
+ * any reporting and error handling purposes.
+ *
+ * If ID is empty we assume that the OD data was valid and schema-conforming and the original input data is
+ * no longer needed. If ID is non-empty there was an issue with the converted data and it was non-schema conforming.
+ *
+ * @param Data Type to Convert to
+ * @param Original Data Type to Convert from
+ */
+@Slf4j
+public class ConverterResult implements Serializable {
+
+ @NonNull
+ @Getter
+ protected Optional> successData;
+
+ @NonNull
+ @Getter
+ protected Optional> errorData;
+
+ /**
+ * Constructor for case that OD is schema conforming
+ * @param successData
+ */
+ public ConverterResult(@NonNull final OD successData) {
+ this.successData = Optional.of(new ValidData<>(successData));
+ this.errorData = Optional.absent();
+ }
+
+ public ConverterResult(@NonNull final ID errorData, @NotEmpty final String errorMessage) {
+ this.successData = Optional.absent();
+ this.errorData = Optional.of(new ErrorData<>(errorMessage, new RawData<>(errorData)));
+
+ }
+
+ public ConverterResult(@NonNull final ID errorData,
+ @NotEmpty final String errorMessage,
+ @NonNull final OD successData) {
+ this.successData = Optional.of(new ValidData<>(successData));
+ this.errorData = Optional.of(new ErrorData<>(errorMessage, new RawData<>(errorData)));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/AbstractDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/AbstractDataConverter.java
new file mode 100644
index 0000000..42c808d
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/AbstractDataConverter.java
@@ -0,0 +1,160 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.data;
+
+import com.google.common.base.Optional;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.converters.converterresult.ConverterResult;
+import com.uber.marmaray.common.data.ErrorData;
+import com.uber.marmaray.common.data.ForkData;
+import com.uber.marmaray.common.data.IData;
+import com.uber.marmaray.common.data.RDDWrapper;
+import com.uber.marmaray.common.data.RawData;
+import com.uber.marmaray.common.data.RawDataHelper;
+import com.uber.marmaray.common.data.ValidData;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.common.forkoperator.ForkFunction;
+import com.uber.marmaray.common.forkoperator.ForkOperator;
+import com.uber.marmaray.utilities.ErrorExtractor;
+import com.uber.marmaray.utilities.ErrorTableUtil;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.Setter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.VoidFunction;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Implementations of the {@link AbstractDataConverter} interface will convert data records from one schema type
+ * to another
+ * @param inputSchema
+ * @param outputSchema
+ * @param inputDataType
+ * @param outputDataType
+ */
+@Slf4j
+public abstract class AbstractDataConverter implements Serializable {
+ public static final long serialVersionUID = 1L;
+ public static final Integer VALID_RECORD = 0;
+ public static final Integer ERROR_RECORD = 1;
+ private static final String SUCCESS = "SUCCESS";
+ private static final String FAILURE = "FAILURE";
+
+ @Getter @NonNull
+ protected Configuration conf;
+ /**
+ * If defined then {@link #failureRecordHandler} will be invoked with input record in case of any exception from
+ * {@link #convert(Object)}.
+ */
+ @NonNull
+ @Setter
+ protected Optional> failureRecordHandler = Optional.absent();
+ /**
+ * If defined then {@link #successRecordHandler} will be invoked with output record if record conversion succeeds
+ * with no exceptions from {@link #convert(Object)}.
+ */
+ @NonNull
+ @Setter
+ protected Optional> successRecordHandler = Optional.absent();
+
+ @NonNull
+ protected ErrorExtractor errorExtractor;
+
+ public AbstractDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) {
+ this.conf = conf;
+ this.errorExtractor = errorExtractor;
+ }
+
+ public final RDDWrapper map(@NonNull final JavaRDD data) {
+ final ForkOperator converter =
+ new ForkOperator<>(data.map(r -> RawDataHelper.getRawData(r)),
+ new DataConversionFunction(), this.conf);
+ converter.execute();
+
+ // Write error records.
+ ErrorTableUtil.writeErrorRecordsToErrorTable(data.context(), this.conf, Optional.absent(),
+ new RDDWrapper<>(converter.getRDD(ERROR_RECORD).map(r -> (ErrorData) r), converter.getCount(ERROR_RECORD)),
+ errorExtractor);
+
+ return new RDDWrapper<>(converter.getRDD(VALID_RECORD).map(r -> ((ValidData) r).getData()),
+ converter.getCount(VALID_RECORD));
+ }
+
+ protected abstract List> convert(@NonNull ID data) throws Exception;
+
+ public class DataConversionFunction extends ForkFunction {
+
+ public DataConversionFunction() {
+ registerKeys(Arrays.asList(VALID_RECORD, ERROR_RECORD));
+ }
+
+ @Override
+ protected List> process(final IData record) {
+ if (!(record instanceof RawData)) {
+ throw new JobRuntimeException("Illegal data type :" + record.getClass());
+ }
+
+ final RawData rawData = (RawData) record;
+
+ List> results;
+
+ try {
+ results = convert(rawData.getData());
+ } catch (RuntimeException re) {
+ throw new JobRuntimeException(re);
+ } catch (Exception e) {
+ results = Collections.singletonList(new ConverterResult(rawData.getData(), e.getMessage()));
+ }
+
+ final List> forkData = new ArrayList<>();
+
+ results.stream().forEach(t -> {
+ if (t.getErrorData().isPresent()) {
+ processRecordHandler(AbstractDataConverter.this.failureRecordHandler,
+ t.getErrorData().get().getRawData().getData(), FAILURE);
+
+ forkData.add(new ForkData<>(Arrays.asList(ERROR_RECORD), t.getErrorData().get()));
+ }
+
+ if (t.getSuccessData().isPresent()) {
+ processRecordHandler(AbstractDataConverter.this.successRecordHandler,
+ t.getSuccessData().get().getData(), SUCCESS);
+ forkData.add(new ForkData<>(Arrays.asList(VALID_RECORD), t.getSuccessData().get()));
+ }
+ });
+ return forkData;
+ }
+
+ private void processRecordHandler(@NonNull final Optional> recordHandler,
+ @NonNull final T data, @NotEmpty final String recordHandlerType) {
+ if (recordHandler.isPresent()) {
+ try {
+ recordHandler.get().call(data);
+ } catch (Exception fe) {
+ log.error("exception in :" + recordHandlerType , fe);
+ }
+ }
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkCQLDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkCQLDataConverter.java
new file mode 100644
index 0000000..7abf969
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkCQLDataConverter.java
@@ -0,0 +1,200 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.data;
+
+import com.datastax.driver.core.querybuilder.Insert;
+import com.datastax.driver.core.querybuilder.QueryBuilder;
+import com.google.common.base.Joiner;
+import com.google.common.base.Optional;
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.configuration.CassandraSinkConfiguration;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.converters.converterresult.ConverterResult;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.common.exceptions.MissingPropertyException;
+import com.uber.marmaray.common.schema.cassandra.CassandraSchema;
+import com.uber.marmaray.utilities.ByteBufferUtil;
+import com.uber.marmaray.utilities.ErrorExtractor;
+import com.uber.marmaray.utilities.GenericRecordUtil;
+import com.uber.marmaray.utilities.SchemaUtil;
+import com.uber.marmaray.utilities.TimestampInfo;
+import lombok.NonNull;
+import lombok.Setter;
+import org.apache.avro.Schema;
+import org.apache.cassandra.db.marshal.LongType;
+import com.datastax.driver.core.Statement;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * {@link CassandraSinkCQLDataConverter} converts data from intermediate Avro payload to the Cassandra statement which
+ * can be executed by Cassandra client. This class is only to be used where the sink of the data migration is
+ * Cassandra. The main convert method of this class will return a RDD of Cassandra statement to the caller.
+ */
+public class CassandraSinkCQLDataConverter extends SinkDataConverter {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String inputSchemaJson;
+
+ @Setter
+ private String keyspaceName;
+
+ @Setter
+ private String tableName;
+
+ /*
+ * The fields to convert are defined in the job configuration from the user. This can be all or a subset of fields
+ * from the schema.
+ */
+ private final Optional> fieldsToConvert;
+
+ /*
+ * The required fields that must be populated in the schema. These keys form the primary/partition/clustering
+ * keys in the Cassandra schema and are defined in the job configuration.
+ */
+ private final List requiredFields;
+
+ private Optional inputSchema = Optional.absent();
+
+ private final TimestampInfo timestampInfo;
+
+ /**
+ * This constructor gives the option to only convert certain fields from the schema
+ * @param inputSchema
+ * @param conf
+ * @param fieldsToConvert
+ * @param requiredFields
+ */
+ public CassandraSinkCQLDataConverter(@NonNull final Schema inputSchema,
+ @NonNull final Configuration conf,
+ @NonNull final Optional> fieldsToConvert,
+ @NonNull final List requiredFields,
+ @NonNull final TimestampInfo timestampInfo,
+ @NonNull final ErrorExtractor errorExtractor) {
+ super(conf, errorExtractor);
+ if (fieldsToConvert.isPresent()) {
+ validate(fieldsToConvert.get(), requiredFields);
+ }
+ this.inputSchemaJson = inputSchema.toString();
+ this.requiredFields = Collections.unmodifiableList(requiredFields);
+ this.fieldsToConvert = fieldsToConvert;
+ this.timestampInfo = timestampInfo;
+ this.keyspaceName = conf.getProperty(CassandraSinkConfiguration.KEYSPACE, "");
+ this.tableName = conf.getProperty(CassandraSinkConfiguration.TABLE_NAME, "");
+ }
+
+ @Override
+ public List> convert(final AvroPayload avroPayload) throws Exception {
+ final Insert insertStatement = QueryBuilder.insertInto(keyspaceName, tableName);
+ final Set requiredKeysToFind = new HashSet<>(this.requiredFields);
+ if (!this.inputSchema.isPresent()) {
+ this.inputSchema = Optional.of(new Schema.Parser().parse(inputSchemaJson));
+ }
+
+ for (int i = 0; i < this.inputSchema.get().getFields().size(); i++) {
+ final Schema.Field field = this.inputSchema.get().getFields().get(i);
+ if (!this.fieldsToConvert.isPresent()
+ || this.fieldsToConvert.isPresent()
+ && this.fieldsToConvert.get().contains(field.name().toLowerCase())) {
+ final Object rawData = avroPayload.getData().get(field.name());
+
+ if (rawData != null) {
+
+ final Schema nonNullSchema = GenericRecordUtil.isOptional(field.schema())
+ ? GenericRecordUtil.getNonNull(field.schema())
+ : field.schema();
+ final Schema.Type type = nonNullSchema.getType();
+
+ switch (type) {
+ case BOOLEAN:
+ final Boolean boolData = (Boolean) rawData;
+ insertStatement.value(field.name(), boolData);
+ break;
+ case INT:
+ final Integer intData = (Integer) rawData;
+ insertStatement.value(field.name(), intData);
+ break;
+ case LONG:
+ final Long longData = (Long) rawData;
+ insertStatement.value(field.name(), longData);
+ break;
+ case DOUBLE:
+ final Double doubleData = (Double) rawData;
+ insertStatement.value(field.name(), doubleData);
+ break;
+ case STRING:
+ final String strData = rawData.toString();
+ insertStatement.value(field.name(), strData);
+ break;
+ case FLOAT:
+ final Float floatData = (Float) rawData;
+ insertStatement.value(field.name(), floatData);
+ break;
+ // todo(T936057) - add support for non-primitive types
+ default:
+ throw new JobRuntimeException("Type " + field.schema().getType() + " not supported");
+ }
+ } else {
+ if (requiredKeysToFind.contains(field.name())) {
+ throw new JobRuntimeException("Data for a required key is missing. Key: " + field.name());
+ }
+ }
+ requiredKeysToFind.remove(field.name());
+ }
+ }
+
+ if (this.timestampInfo.hasTimestamp()) {
+ final ByteBuffer bb = this.timestampInfo.isSaveAsLongType()
+ ? LongType.instance.decompose(Long.parseLong(this.timestampInfo.getTimestamp().get()))
+ : ByteBufferUtil.wrap(this.timestampInfo.getTimestamp().get());
+ insertStatement.value(SchemaUtil.DISPERSAL_TIMESTAMP, bb);
+ }
+
+ if (!requiredKeysToFind.isEmpty()) {
+ final Joiner joiner = Joiner.on(",");
+ throw new MissingPropertyException(joiner.join(requiredKeysToFind));
+ }
+
+ return Collections.singletonList(new ConverterResult<>(insertStatement));
+ }
+
+ /**
+ * Not all the fields in the Avro Schema will be converted to Cassandra fields.
+ *
+ * All required fields listed, however, must exist in the fields to convert
+ * @param allFieldsToConvert
+ * @param requiredFields
+ */
+ private void validate(final Set allFieldsToConvert, final List requiredFields) {
+ if (!allFieldsToConvert.containsAll(requiredFields)) {
+ final List missingFields = requiredFields.
+ stream()
+ .filter(rf -> allFieldsToConvert.contains(this.requiredFields))
+ .collect(Collectors.toList());
+ final Joiner joiner = Joiner.on(",");
+ final String errMsg = String.format("Listed required fields are missing from the list of fields to convert."
+ + " Please check your job configuration. Missing fields are: %s", joiner.join(missingFields));
+ throw new JobRuntimeException(errMsg);
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkDataConverter.java
new file mode 100644
index 0000000..5fe15f0
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/CassandraSinkDataConverter.java
@@ -0,0 +1,201 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.data;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Optional;
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.converters.converterresult.ConverterResult;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.common.exceptions.MissingPropertyException;
+import com.uber.marmaray.common.schema.cassandra.CassandraDataField;
+import com.uber.marmaray.common.schema.cassandra.CassandraPayload;
+import com.uber.marmaray.common.schema.cassandra.CassandraSchema;
+import com.uber.marmaray.utilities.ByteBufferUtil;
+import com.uber.marmaray.utilities.ErrorExtractor;
+import com.uber.marmaray.utilities.GenericRecordUtil;
+import com.uber.marmaray.utilities.SchemaUtil;
+import com.uber.marmaray.utilities.TimestampInfo;
+import lombok.NonNull;
+import org.apache.avro.Schema;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.FloatType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.LongType;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * {@link CassandraSinkDataConverter} extends {@link SinkDataConverter}
+ * This class converts data from intermediate Avro schema to the Cassandra schema. This
+ * class is only to be used where the sink of the data migration is Cassandra. The main convert method of this
+ * class will return a RDD of Cassandra payloads to the caller. All keys & values will be serialized as
+ * ByteBuffers as required by Cassandra. All strings are encoded using UTF-8.
+ */
+public class CassandraSinkDataConverter extends SinkDataConverter {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String inputSchemaJson;
+
+ /*
+ * The fields to convert are defined in the job configuration from the user. This can be all or a subset of fields
+ * from the schema.
+ */
+ private final Optional> fieldsToConvert;
+
+ /*
+ * The required fields that must be populated in the schema. These keys form the primary/partition/clustering
+ * keys in the Cassandra schema and are defined in the job configuration.
+ */
+ private final List requiredFields;
+
+ private Optional inputSchema = Optional.absent();
+
+ private final TimestampInfo timestampInfo;
+
+ /**
+ * This constructor gives the option to only convert certain fields from the schema
+ * @param inputSchema
+ * @param conf
+ * @param fieldsToConvert
+ * @param requiredFields
+ */
+ public CassandraSinkDataConverter(@NonNull final Schema inputSchema,
+ @NonNull final Configuration conf,
+ @NonNull final Optional> fieldsToConvert,
+ @NonNull final List requiredFields,
+ @NonNull final TimestampInfo timestampInfo,
+ @NonNull final ErrorExtractor errorExtractor) {
+ super(conf, errorExtractor);
+ if (fieldsToConvert.isPresent()) {
+ validate(fieldsToConvert.get(), requiredFields);
+ }
+ this.inputSchemaJson = inputSchema.toString();
+ this.requiredFields = Collections.unmodifiableList(requiredFields);
+ this.fieldsToConvert = fieldsToConvert;
+ this.timestampInfo = timestampInfo;
+ }
+
+ @Override
+ public List> convert(final AvroPayload avroPayload)
+ throws Exception {
+ final CassandraPayload row = new CassandraPayload();
+ final Set requiredKeysToFind = new HashSet<>(this.requiredFields);
+ if (!this.inputSchema.isPresent()) {
+ this.inputSchema = Optional.of(new Schema.Parser().parse(inputSchemaJson));
+ }
+
+ for (int i = 0; i < this.inputSchema.get().getFields().size(); i++) {
+ final Schema.Field field = this.inputSchema.get().getFields().get(i);
+ if (!this.fieldsToConvert.isPresent()
+ || this.fieldsToConvert.isPresent()
+ && this.fieldsToConvert.get().contains(field.name().toLowerCase())) {
+ final Object rawData = avroPayload.getData().get(field.name());
+
+ final ByteBuffer bb;
+
+ if (rawData != null) {
+
+ final Schema nonNullSchema = GenericRecordUtil.isOptional(field.schema())
+ ? GenericRecordUtil.getNonNull(field.schema())
+ : field.schema();
+ final Schema.Type type = nonNullSchema.getType();
+
+ switch (type) {
+ case BOOLEAN:
+ final Boolean boolData = (Boolean) rawData;
+ bb = BooleanType.instance.decompose(boolData);
+ break;
+ case INT:
+ final Integer intData = (Integer) rawData;
+ bb = Int32Type.instance.decompose(intData);
+ break;
+ case LONG:
+ final Long longData = (Long) rawData;
+ bb = LongType.instance.decompose(longData);
+ break;
+ case DOUBLE:
+ final Double doubleData = (Double) rawData;
+ bb = DoubleType.instance.decompose(doubleData);
+ break;
+ case STRING:
+ final String strData = rawData.toString();
+ bb = ByteBufferUtil.wrap(strData);
+ break;
+ case FLOAT:
+ final Float floatData = (Float) rawData;
+ bb = FloatType.instance.decompose(floatData);
+ break;
+ // todo(T936057) - add support for non-primitive types
+ default:
+ throw new JobRuntimeException("Type " + field.schema().getType() + " not supported");
+ }
+ } else {
+ if (requiredKeysToFind.contains(field.name())) {
+ throw new JobRuntimeException("Data for a required key is missing. Key: " + field.name());
+ }
+ bb = null;
+ }
+ requiredKeysToFind.remove(field.name());
+ row.addField(new CassandraDataField(ByteBufferUtil.wrap(field.name()), bb));
+ }
+ }
+
+ if (this.timestampInfo.hasTimestamp()) {
+ final ByteBuffer bb = this.timestampInfo.isSaveAsLongType()
+ ? LongType.instance.decompose(Long.parseLong(this.timestampInfo.getTimestamp().get()))
+ : ByteBufferUtil.wrap(this.timestampInfo.getTimestamp().get());
+ row.addField(
+ new CassandraDataField(ByteBufferUtil.wrap(SchemaUtil.DISPERSAL_TIMESTAMP), bb));
+ }
+
+ if (!requiredKeysToFind.isEmpty()) {
+ final Joiner joiner = Joiner.on(",");
+ throw new MissingPropertyException(joiner.join(requiredKeysToFind));
+ }
+
+ return Collections.singletonList(new ConverterResult<>(row));
+ }
+
+ /**
+ * Not all the fields in the Avro Schema will be converted to Cassandra fields.
+ *
+ * All required fields listed, however, must exist in the fields to convert
+ * @param allFieldsToConvert
+ * @param requiredFields
+ */
+ private void validate(final Set allFieldsToConvert, final List requiredFields) {
+ if (!allFieldsToConvert.containsAll(requiredFields)) {
+ final List missingFields = requiredFields.
+ stream()
+ .filter(rf -> allFieldsToConvert.contains(this.requiredFields))
+ .collect(Collectors.toList());
+ final Joiner joiner = Joiner.on(",");
+ final String errMsg = String.format("Listed required fields are missing from the list of fields to convert."
+ + " Please check your job configuration. Missing fields are: %s", joiner.join(missingFields));
+ throw new JobRuntimeException(errMsg);
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/DummyHoodieSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/DummyHoodieSinkDataConverter.java
new file mode 100644
index 0000000..54224e5
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/DummyHoodieSinkDataConverter.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common.converters.data;
+
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.utilities.ErrorExtractor;
+
+import lombok.NonNull;
+
+/**
+ * {@link DummyHoodieSinkDataConverter} is necessary where you do not want payloads to be
+ * processed by the data converter.
+ */
+public class DummyHoodieSinkDataConverter extends HoodieSinkDataConverter {
+ public DummyHoodieSinkDataConverter() {
+ super(new Configuration(), new ErrorExtractor());
+ }
+
+ @Override
+ protected String getRecordKey(@NonNull final AvroPayload payload) throws Exception {
+ return null;
+ }
+
+ @Override
+ protected String getPartitionPath(@NonNull final AvroPayload payload) throws Exception {
+ return null;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverter.java
new file mode 100644
index 0000000..7736235
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/FileSinkDataConverter.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common.converters.data;
+
+import com.opencsv.CSVWriter;
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.configuration.FileSinkConfiguration;
+import com.uber.marmaray.common.converters.converterresult.ConverterResult;
+import com.uber.marmaray.utilities.ErrorExtractor;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.io.StringWriter;
+import java.util.List;
+import java.util.Collections;
+
+/**
+ * {@link FileSinkDataConverter} extends {@link SinkDataConverter}
+ * This class converts data from intermediate Avro schema to string with csv format.
+ * This class is only to be used where the sink of the data migration is FileSink.
+ * The main convertAll method of this class will return a RDD of String with csv format to caller.
+ * The getHeader method will return a String of column header for the csv file.
+ */
+@Slf4j
+public class FileSinkDataConverter extends SinkDataConverter {
+ public static final String CSV = "csv";
+ public final String fileType;
+ public final char separator;
+
+ public FileSinkDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) {
+ super(conf, errorExtractor);
+ final FileSinkConfiguration fsConf = new FileSinkConfiguration(conf);
+ this.fileType = fsConf.getFileType();
+ this.separator = fsConf.getSeparator();
+ }
+
+ /**
+ * This method converts RDD of AvroPayload data to RDD of String with specified file type.
+ * Currently supports csv file only.
+ * @param data
+ * @return
+ * @throws UnsupportedOperationException
+ */
+ public JavaRDD convertAll(@NonNull final JavaRDD data) throws UnsupportedOperationException {
+ final JavaRDD lines = data.map(row -> {
+ final String line = this.convert(row).get(0).getSuccessData().get().getData();
+ log.debug("Line: {}", line);
+ return line;
+ });
+ return lines;
+ }
+
+ @Override
+ public List> convert(@NonNull final AvroPayload data)
+ throws UnsupportedOperationException {
+ String line = "";
+ if (this.fileType.equals(this.CSV)) {
+ final GenericRecord r = data.getData();
+ final String [] tmp = r.getSchema().getFields().stream().map(f ->r.get(f.name())
+ .toString()).toArray(String[]::new);
+ final StringWriter sw = new StringWriter();
+ final CSVWriter writer = new CSVWriter(sw
+ , this.separator, '\"', '\\', "");
+ writer.writeNext(tmp, false);
+ line = sw.toString();
+ } else {
+ //Todo:Add more file type options.
+ final String errorMessage = "Format " + this.fileType + " not supported yet.";
+ throw new UnsupportedOperationException(errorMessage);
+ }
+ return Collections.singletonList(new ConverterResult<>(line));
+ }
+
+ /**
+ * This methods get the column header of data.
+ * It specifically works for file type: csv.
+ * @param data
+ * @return String of column header separated by separator.
+ */
+ public String getHeader(@NonNull final JavaRDD data) {
+ final AvroPayload line = data.first();
+ final String[] headList
+ = line.getData().getSchema().getFields().stream().map(f -> f.name()).toArray(String[]::new);
+ final StringWriter sw = new StringWriter();
+ final CSVWriter writer = new CSVWriter(sw
+ , this.separator, '\"', '\\', "");
+ writer.writeNext(headList, false);
+ return sw.toString();
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/HoodieSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/HoodieSinkDataConverter.java
new file mode 100644
index 0000000..8487b6d
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/HoodieSinkDataConverter.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.data;
+
+import com.uber.hoodie.common.model.HoodieAvroPayload;
+import com.uber.hoodie.common.model.HoodieKey;
+import com.uber.hoodie.common.model.HoodieRecord;
+import com.uber.hoodie.common.model.HoodieRecordPayload;
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.converters.converterresult.ConverterResult;
+import com.uber.marmaray.common.sinks.hoodie.HoodieSink;
+import com.uber.marmaray.utilities.ErrorExtractor;
+
+import lombok.NonNull;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * {@link HoodieSinkDataConverter} extends {@link SinkDataConverter}
+ * This class is used by {@link HoodieSink} to generate {@link com.uber.hoodie.common.model.HoodieRecord} from
+ * {@link com.uber.marmaray.common.AvroPayload}.
+ */
+public abstract class HoodieSinkDataConverter extends SinkDataConverter> {
+
+ // store the schema as a string since Schema doesn't serialize. Used in extended classes.
+ protected String schema;
+ private final ErrorExtractor errorExtractor;
+
+ public HoodieSinkDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) {
+ super(conf, errorExtractor);
+ this.errorExtractor = errorExtractor;
+ }
+
+ public HoodieSinkDataConverter(@NonNull final Configuration conf, final String schema,
+ @NonNull final ErrorExtractor errorExtractor) {
+ super(conf, errorExtractor);
+ this.schema = schema;
+ this.errorExtractor = errorExtractor;
+ }
+
+ @Override
+ protected final List>> convert(
+ @NonNull final AvroPayload payload) throws Exception {
+ final HoodieKey hoodieKey = new HoodieKey(getRecordKey(payload), getPartitionPath(payload));
+ final HoodieRecordPayload hoodiePayload = getPayload(payload);
+ return Collections.singletonList(new ConverterResult<>((new HoodieRecord<>(hoodieKey, hoodiePayload))));
+ }
+
+ /**
+ * The implementation of it should use fields from {@link GenericRecord} to generate record key which is needed for
+ * {@link HoodieKey}.
+ *
+ * @param payload {@link AvroPayload}.
+ */
+ protected abstract String getRecordKey(@NonNull final AvroPayload payload) throws Exception;
+
+ /**
+ * The implementation of it should use fields from {@link AvroPayload} to generate partition path which is needed
+ * for {@link HoodieKey}.
+ *
+ * @param payload {@link AvroPayload}.
+ */
+ protected abstract String getPartitionPath(@NonNull final AvroPayload payload) throws Exception;
+
+ protected HoodieRecordPayload getPayload(@NonNull final AvroPayload payload) {
+ return new HoodieAvroPayload(java.util.Optional.of(payload.getData()));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/KafkaSourceDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/KafkaSourceDataConverter.java
new file mode 100644
index 0000000..11ef4fd
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/KafkaSourceDataConverter.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.data;
+
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.converters.converterresult.ConverterResult;
+import com.uber.marmaray.common.schema.ISchemaService.ISchemaServiceReader;
+import com.uber.marmaray.utilities.ErrorExtractor;
+import lombok.Getter;
+import lombok.NonNull;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.api.java.function.Function;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * {@link KafkaSourceDataConverter} extends {@link SourceDataConverter}
+ *
+ * This class converts kafka messages to {@link AvroPayload}. It uses {@link ISchemaServiceReader} for decoding kafka
+ * message bytes.
+ */
+public class KafkaSourceDataConverter extends SourceDataConverter {
+ @NonNull
+ @Getter
+ private final ISchemaServiceReader schemaServiceReader;
+ @NonNull
+ @Getter
+ private final List fieldsToCache;
+
+ /**
+ * List of {@Link Function} to apply to the record between reading from kafka and
+ * transferring to the ISource
+ */
+ @NonNull
+ @Getter
+ private final List> updateFunctions;
+
+ public KafkaSourceDataConverter(@NonNull final ISchemaServiceReader schemaServiceReader,
+ @NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) {
+ this(schemaServiceReader, conf, Collections.emptyList(), Collections.emptyList(), errorExtractor);
+ }
+
+ public KafkaSourceDataConverter(@NonNull final ISchemaServiceReader schemaServiceReader,
+ @NonNull final Configuration conf,
+ @NonNull final List> updateFunctions,
+ @NonNull final List fieldsToCache,
+ @NonNull final ErrorExtractor errorExtractor) {
+ super(conf, errorExtractor);
+ this.schemaServiceReader = schemaServiceReader;
+ this.updateFunctions = updateFunctions;
+ this.fieldsToCache = fieldsToCache;
+ }
+
+ @Override
+ public List> convert(@NonNull final byte[] data) throws Exception {
+ GenericRecord genericRecord = this.schemaServiceReader.read(data);
+ for (Function func : this.updateFunctions) {
+ genericRecord = func.call(genericRecord);
+ }
+ return Collections.singletonList(new ConverterResult(new AvroPayload(genericRecord, this.fieldsToCache)));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SinkDataConverter.java
new file mode 100644
index 0000000..178f1e5
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SinkDataConverter.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.data;
+
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.utilities.ErrorExtractor;
+import lombok.NonNull;
+import org.apache.avro.Schema;
+
+/**
+ * {@link SinkDataConverter} extends {@link AbstractDataConverter}
+ * This class converts records from ({@link Schema}, {@link AvroPayload}) to (OS, OD).
+ * @param output schema type
+ * @param output data type
+ */
+public abstract class SinkDataConverter extends AbstractDataConverter {
+ private static final long serialVersionUID = 1L;
+
+ public SinkDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) {
+ super(conf, errorExtractor);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SourceDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SourceDataConverter.java
new file mode 100644
index 0000000..55dda51
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SourceDataConverter.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.data;
+
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.utilities.ErrorExtractor;
+import lombok.NonNull;
+import org.apache.avro.Schema;
+
+/**
+ * {@link SourceDataConverter} extends {@link AbstractDataConverter}
+ *
+ * This class converts records from (IS, ID) to ({@link Schema}, {@link AvroPayload}).
+ * @param input schema type
+ * @param input data type
+ */
+public abstract class SourceDataConverter extends AbstractDataConverter {
+ public SourceDataConverter(@NonNull final Configuration conf, @NonNull final ErrorExtractor errorExtractor) {
+ super(conf, errorExtractor);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SparkSourceDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SparkSourceDataConverter.java
new file mode 100644
index 0000000..657f402
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/SparkSourceDataConverter.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.data;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.converters.converterresult.ConverterResult;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.utilities.ErrorExtractor;
+import com.uber.marmaray.utilities.SparkUtil;
+import com.uber.marmaray.utilities.StringTypes;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * {@link SparkSourceDataConverter} extends {@link SourceDataConverter}
+ *
+ * This converts data from Spark format to the common intermediate {@link AvroPayload} format
+ */
+@Slf4j
+public class SparkSourceDataConverter extends SourceDataConverter {
+ private static final Set supportedDataTypes = SparkUtil.getSupportedDataTypes();
+
+ private final String jsonOutputSchema;
+ private final StructField[] fields;
+ private final Set requiredKeys;
+ private Optional outputSchema = Optional.absent();
+
+ public SparkSourceDataConverter(@NonNull final StructType inputSchema,
+ @NonNull final Schema outputSchema,
+ @NonNull final Configuration conf,
+ @NonNull final Set requiredKeys,
+ @NonNull final ErrorExtractor errorExtractor) {
+ super(conf, errorExtractor);
+ this.fields = Arrays.stream(inputSchema.fields())
+ .filter(f -> !f.name().startsWith("_")).toArray(StructField[]::new);
+ this.jsonOutputSchema = outputSchema.toString();
+ this.requiredKeys = requiredKeys;
+ }
+
+ @Override
+ public List> convert(@NonNull final Row row) throws Exception {
+ if (!this.outputSchema.isPresent()) {
+ this.outputSchema = Optional.of((new Schema.Parser().parse(this.jsonOutputSchema)));
+ }
+
+ Preconditions.checkNotNull(row.schema());
+
+ final GenericRecord gr = new GenericData.Record(this.outputSchema.get());
+
+ final Set required = this.requiredKeys.stream().map(String::new).collect(Collectors.toSet());
+
+ // todo: think about generalizing this, the pattern is the same
+ for (int i = 0; i < this.fields.length; i++) {
+ required.remove(this.fields[i].name());
+ final DataType dt = this.fields[i].dataType();
+
+ try {
+ final Object data = row.getAs(this.fields[i].name());
+ if (supportedDataTypes.contains(dt)) {
+ gr.put(this.fields[i].name(), data);
+ } else {
+ throw new JobRuntimeException(dt.toString() + " field type is not supported at this time");
+ }
+ } catch (final IllegalArgumentException e) {
+ // the fieldname did not exist in the row which is ok, skip it
+ continue;
+ }
+ }
+
+ if (!required.isEmpty()) {
+ final Joiner joiner = Joiner.on(StringTypes.COMMA);
+ final String errMsg = String.format("Required fields were missing. Fields: {}", joiner.join(required));
+ throw new JobRuntimeException(errMsg);
+ }
+
+ return Collections.singletonList(new ConverterResult<>(new AvroPayload(gr)));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/data/TSBasedHoodieSinkDataConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/TSBasedHoodieSinkDataConverter.java
new file mode 100644
index 0000000..44bea41
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/data/TSBasedHoodieSinkDataConverter.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.data;
+
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.exceptions.InvalidDataException;
+import com.uber.hoodie.common.model.HoodieKey;
+import com.uber.marmaray.utilities.HoodieSinkConverterErrorExtractor;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.concurrent.TimeUnit;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import static com.uber.marmaray.utilities.DateUtil.DATE_PARTITION_FORMAT;
+
+/**
+ * {@link TSBasedHoodieSinkDataConverter} extends {@link HoodieSinkDataConverter}
+ *
+ * This class generates {@link HoodieKey} from given {@link AvroPayload}. The passed in {@link AvroPayload} requires
+ * {@link #partitionFieldName} with timestamp in {@link #timeUnit}.
+ *
+ * {@link AvroPayload} also requires a {@link #recordKeyFieldName} which should be the primary key for the record.
+ */
+@Slf4j
+public class TSBasedHoodieSinkDataConverter extends HoodieSinkDataConverter {
+
+ public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat(DATE_PARTITION_FORMAT);
+ @NotEmpty
+ private final String recordKeyFieldName;
+ @NotEmpty
+ private final String partitionFieldName;
+ @NonNull
+ private final TimeUnit timeUnit;
+
+ public TSBasedHoodieSinkDataConverter(@NonNull final Configuration conf,
+ @NotEmpty final String recordKeyFieldName, @NotEmpty final String partitionFieldName,
+ @NonNull final TimeUnit timeUnit) {
+ super(conf, new HoodieSinkConverterErrorExtractor());
+ this.recordKeyFieldName = recordKeyFieldName;
+ this.partitionFieldName = partitionFieldName;
+ this.timeUnit = timeUnit;
+ }
+
+ @Override
+ protected String getRecordKey(@NonNull final AvroPayload payload) throws Exception {
+ final Object recordKeyFieldVal = payload.getField(recordKeyFieldName);
+ if (recordKeyFieldVal == null) {
+ throw new InvalidDataException("required field is missing:" + recordKeyFieldName);
+ }
+ return recordKeyFieldVal.toString();
+ }
+
+ @Override
+ protected String getPartitionPath(final AvroPayload payload) throws Exception {
+ final Object partitionFieldVal = payload.getField(partitionFieldName);
+ if (partitionFieldVal == null) {
+ throw new InvalidDataException("required field is missing:" + partitionFieldName);
+ }
+ final Date date = new Date(this.timeUnit.toMillis((long) Double.parseDouble(partitionFieldVal.toString())));
+ return PARTITION_FORMATTER.format(date);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/AbstractSchemaConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/AbstractSchemaConverter.java
new file mode 100644
index 0000000..1584a3f
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/AbstractSchemaConverter.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.schema;
+
+import org.apache.avro.Schema;
+
+/**
+ * {@link AbstractSchemaConverter} is responsible for 2 way conversion to convert an external schema to a common schema
+ * extending {@link Schema} as well as vice versa
+ * @param external schema
+ * @param common schema
+ */
+public abstract class AbstractSchemaConverter {
+ public abstract ES convertToExternalSchema(CS commonSchema);
+ public abstract CS convertToCommonSchema(ES externalSchema);
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/CassandraSchemaConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/CassandraSchemaConverter.java
new file mode 100644
index 0000000..2b8ee14
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/CassandraSchemaConverter.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.schema;
+
+import com.google.common.base.Optional;
+import com.uber.marmaray.common.schema.cassandra.CassandraSchema;
+import com.uber.marmaray.common.schema.cassandra.CassandraSchemaField;
+import com.uber.marmaray.utilities.SchemaUtil;
+import com.uber.marmaray.utilities.StringTypes;
+import com.uber.marmaray.utilities.TimestampInfo;
+import lombok.Getter;
+import lombok.NonNull;
+import org.apache.avro.Schema;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.util.Set;
+
+/**
+ * {@link CassandraSchemaConverter} extends {@AbstractSchemaConverter} and converts {@link CassandraSchema}
+ * to {@link Schema} and vice versa
+ *
+ * In Cassandra, all keys and values are generic ByteBuffers.
+ *
+ * The actual cassandra schema would just be a list of ByteBuffer that describes each of the field names
+ * In order to encode the values of each row properly as ByteBuffers properly, however, we need to know
+ * the type of field from the common schema
+ */
+public class CassandraSchemaConverter extends AbstractSchemaConverter {
+ @Getter
+ final String keySpace;
+
+ @Getter
+ final String tableName;
+
+ final TimestampInfo tsInfo;
+
+ /**
+ * This optional field is only populated when a job is configured to only handle a subset of fields
+ * and not all the available fields from the source data.
+ */
+ final Optional> filteredFields;
+
+ public CassandraSchemaConverter(@NotEmpty final String keySpace,
+ @NotEmpty final String tableName,
+ @NonNull final TimestampInfo timestampInfo,
+ @NonNull final Optional> filteredFields) {
+ this.keySpace = keySpace;
+ this.tableName = tableName;
+ this.tsInfo = timestampInfo;
+ this.filteredFields = filteredFields;
+ }
+
+ public CassandraSchemaConverter(@NotEmpty final String keySpace,
+ @NotEmpty final String tableName,
+ @NonNull final Optional> filteredFields) {
+ this(keySpace, tableName, TimestampInfo.generateEmptyTimestampInfo(), filteredFields);
+ }
+
+ @Override
+ public CassandraSchema convertToExternalSchema(final Schema commonSchema) {
+ // todo T936057 - Need to handle more complex schemas (i.e Record inside record)
+ final CassandraSchema cs = new CassandraSchema(this.keySpace, this.tableName);
+
+ for (final Schema.Field field : commonSchema.getFields()) {
+ // Cassandra does not support field names starting with _
+ if (this.shouldIncludeField(field.name())) {
+ final String cassFieldType = CassandraSchemaField.convertFromAvroType(field.schema());
+ cs.addField(new CassandraSchemaField(field.name(), cassFieldType));
+ }
+ }
+
+ if (this.tsInfo.hasTimestamp()) {
+ if (this.tsInfo.isSaveAsLongType()) {
+ cs.addField(new CassandraSchemaField(SchemaUtil.DISPERSAL_TIMESTAMP, CassandraSchemaField.LONG_TYPE));
+ } else {
+ cs.addField(new CassandraSchemaField(SchemaUtil.DISPERSAL_TIMESTAMP, CassandraSchemaField.STRING_TYPE));
+ }
+ }
+
+ return cs;
+ }
+
+ @Override
+ public Schema convertToCommonSchema(final CassandraSchema schema) {
+ throw new UnsupportedOperationException();
+ }
+ private boolean shouldIncludeField(@NotEmpty final String fieldName) {
+ boolean shouldInclude = false;
+ if (!fieldName.startsWith(StringTypes.UNDERSCORE)) {
+ if (this.filteredFields.isPresent()) {
+ if (this.filteredFields.get().contains(fieldName.toLowerCase())) {
+ shouldInclude = true;
+ }
+ } else {
+ shouldInclude = true;
+ }
+ }
+ return shouldInclude;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/DataFrameSchemaConverter.java b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/DataFrameSchemaConverter.java
new file mode 100644
index 0000000..a918cd0
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/converters/schema/DataFrameSchemaConverter.java
@@ -0,0 +1,105 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.converters.schema;
+
+import com.google.common.base.Preconditions;
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link DataFrameSchemaConverter} extends {@AbstractSchemaConverter} and converts {@link StructType}
+ * to {@link Schema}
+ *
+ * Conversion from {@link Schema} to {@link StructType} is currently not supported.
+ *
+ */
+public class DataFrameSchemaConverter extends AbstractSchemaConverter {
+
+ @Override
+ public Schema convertToCommonSchema(final StructType dataFrameSchema) {
+ Preconditions.checkNotNull(dataFrameSchema);
+ final SchemaBuilder.FieldAssembler fieldAssembler = SchemaBuilder.record("commonSchema").fields();
+ for (final StructField sf : dataFrameSchema.fields()) {
+ addField(fieldAssembler, sf);
+ }
+
+ return fieldAssembler.endRecord();
+ }
+
+ @Override
+ public StructType convertToExternalSchema(final Schema commonSchema) {
+ // We don't anticipate needing this currently but if needed we can implement
+ throw new UnsupportedOperationException();
+ }
+
+ private void addField(final SchemaBuilder.FieldAssembler fieldAssembler, final StructField structField) {
+ final SchemaBuilder.FieldBuilder fieldBuilder = fieldAssembler.name(structField.name());
+
+ final DataType dt = structField.dataType().defaultConcreteType();
+
+ /**
+ * Can't use switch statement here because this is not a compile time constant expression
+ * and I'd rather compare it to a concrete type rather than use strings
+ *
+ * For now, we just handle atomic, fractional, or integral types.
+ * Todo: Handle Map/Array/Struct types etc
+ */
+ final SchemaBuilder.BaseFieldTypeBuilder fieldType =
+ structField.nullable() ? fieldBuilder.type().nullable() : fieldBuilder.type();
+ if (dt.equals(DataTypes.TimestampType)) {
+ fieldType.stringType().noDefault();
+ } else if (dt.equals(DataTypes.StringType)) {
+ fieldType.stringType().noDefault();
+ } else if (dt.equals(DataTypes.BooleanType)) {
+ fieldType.booleanType().noDefault();
+ } else if (dt.equals(DataTypes.DateType)) {
+ fieldType.stringType().noDefault();
+ } else if (dt.equals(DataTypes.BinaryType)) {
+ // validate this is correct
+ fieldType.bytesType().noDefault();
+ } else if (dt.equals(DataTypes.DoubleType)) {
+ fieldType.doubleType().noDefault();
+ } else if (dt.equals(DataTypes.FloatType)) {
+ fieldType.floatType().noDefault();
+ } else if (dt.equals(DataTypes.ByteType)) {
+ fieldType.bytesType().noDefault();
+ } else if (dt.equals(DataTypes.IntegerType)) {
+ fieldType.intType().noDefault();
+ } else if (dt.equals(DataTypes.LongType)) {
+ fieldType.longType().noDefault();
+ } else if (dt.equals(DataTypes.ShortType)) {
+ // no corresponding short type in DataTypes
+ // we can make this int and lose no precision
+ fieldType.intType().noDefault();
+ } else {
+ throw new RuntimeException("The field type " + dt + " is not supported");
+ }
+ /**
+ * Todo: Handle following types
+ * CalendarIntervalType
+ * StructType
+ * MapType
+ * ArrayType
+ * NullType
+ *
+ */
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/BinaryRawData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/BinaryRawData.java
new file mode 100644
index 0000000..a20b50a
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/data/BinaryRawData.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.data;
+
+import javax.xml.bind.DatatypeConverter;
+
+/**
+ * Convenience class for wrapping byte[].
+ */
+public class BinaryRawData extends RawData {
+
+ public BinaryRawData(final byte[] data) {
+ super(data);
+ }
+
+ @Override
+ public String toString() {
+ return DatatypeConverter.printHexBinary(getData());
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/ErrorData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/ErrorData.java
new file mode 100644
index 0000000..3247420
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/data/ErrorData.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.data;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+
+/**
+ * It holds the input {@link RawData} record and an error message describing which error check condition failed. This
+ * will be useful information for debugging error records.
+ */
+@AllArgsConstructor
+public class ErrorData implements IData {
+
+ /**
+ * Error message.
+ */
+ @Getter
+ private final String errMessage;
+
+ @Getter
+ private final RawData rawData;
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/ForkData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/ForkData.java
new file mode 100644
index 0000000..5d852f6
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/data/ForkData.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.data;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.NonNull;
+
+/**
+ * It is used to hold forked data. It wraps individual record entry with forked pipeline keys.
+ */
+@AllArgsConstructor
+public class ForkData implements Serializable {
+ @Getter
+ private final List keys = new ArrayList<>();
+ @Getter
+ private final DI record;
+
+ public ForkData(@NonNull final List keys, @NonNull final DI record) {
+ this.keys.addAll(keys);
+ this.record = record;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/IData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/IData.java
new file mode 100644
index 0000000..d5b6e12
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/data/IData.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.data;
+
+import java.io.Serializable;
+
+public interface IData extends Serializable {
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/RDDWrapper.java b/marmaray/src/main/java/com/uber/marmaray/common/data/RDDWrapper.java
new file mode 100644
index 0000000..6a7b5b5
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/data/RDDWrapper.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.data;
+
+import lombok.Getter;
+import lombok.NonNull;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.Optional;
+
+/**
+ * Convenience class to wrap RDD of records with its count to avoid multiple calls for count operation. If we need to
+ * use count operation for a given RDD more than once in the form of isEmpty or actual count; then this wrapper will be
+ * useful.
+ * T dataType of RDD records. {@link #data} for more details.
+ */
+public class RDDWrapper {
+
+ @Getter
+ @NonNull
+ final JavaRDD data;
+
+ Optional count;
+
+ public RDDWrapper(@NonNull final JavaRDD data) {
+ this.data = data;
+ this.count = Optional.absent();
+ }
+
+ public RDDWrapper(@NonNull final JavaRDD data, final long count) {
+ this.data = data;
+ this.count = Optional.of(count);
+ }
+
+ public long getCount() {
+ if (!count.isPresent()) {
+ this.count = Optional.of(this.data.count());
+ }
+ return count.get();
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/RawData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/RawData.java
new file mode 100644
index 0000000..1ffc7b1
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/data/RawData.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.data;
+
+import java.io.Serializable;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.ToString;
+
+@AllArgsConstructor
+@ToString
+public class RawData implements IData, Serializable {
+
+ @Getter
+ private final T data;
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/RawDataHelper.java b/marmaray/src/main/java/com/uber/marmaray/common/data/RawDataHelper.java
new file mode 100644
index 0000000..4c00f8b
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/data/RawDataHelper.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.data;
+
+import lombok.NonNull;
+
+/**
+ * Helper class to return appropriate RawData subclass.
+ */
+public class RawDataHelper {
+
+ public static RawData getRawData(@NonNull final T data) {
+ if (data instanceof byte[]) {
+ return new BinaryRawData((byte[]) data);
+ }
+ return new RawData(data);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/data/ValidData.java b/marmaray/src/main/java/com/uber/marmaray/common/data/ValidData.java
new file mode 100644
index 0000000..8998a3b
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/data/ValidData.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.data;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+
+@AllArgsConstructor
+public class ValidData implements IData {
+ @Getter
+ private final T data;
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/dataset/ErrorRecord.java b/marmaray/src/main/java/com/uber/marmaray/common/dataset/ErrorRecord.java
new file mode 100644
index 0000000..53afc2b
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/dataset/ErrorRecord.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.dataset;
+
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import org.hibernate.validator.constraints.NotEmpty;
+
+/**
+ * Simple Java Bean used to construct {@link UtilTable} of {@ErrorRecord}
+ */
+@Data
+@EqualsAndHashCode(callSuper = true)
+public class ErrorRecord extends UtilRecord {
+ private String table_name;
+ private String row_key;
+ private String column_name;
+ private String content;
+ private String error_type;
+
+ public ErrorRecord(@NotEmpty final String applicationId,
+ @NotEmpty final String jobName,
+ final long jobStartTimestamp,
+ final long timestamp,
+ @NotEmpty final String tableName,
+ @NotEmpty final String rowKey,
+ @NotEmpty final String columnName,
+ @NotEmpty final String content,
+ @NotEmpty final String errorType) {
+ super(applicationId, jobName, jobStartTimestamp, timestamp);
+ this.table_name = tableName;
+ this.row_key = rowKey;
+ this.column_name = columnName;
+ this.content = content;
+ this.error_type = errorType;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/dataset/ExceptionRecord.java b/marmaray/src/main/java/com/uber/marmaray/common/dataset/ExceptionRecord.java
new file mode 100644
index 0000000..c9d6a47
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/dataset/ExceptionRecord.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.dataset;
+
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import org.hibernate.validator.constraints.NotEmpty;
+
+/**
+ * Simple Java Bean used to construct {@link UtilTable} of {@ExceptionRecord}
+ */
+@Data
+@EqualsAndHashCode(callSuper = true)
+public class ExceptionRecord extends UtilRecord {
+ private String exception;
+ private String exception_message;
+ private String stacktrace;
+ private boolean detected_on_driver;
+
+ public ExceptionRecord(@NotEmpty final String applicationId,
+ @NotEmpty final String jobName,
+ final long jobStartTimestamp,
+ final long timestamp,
+ @NotEmpty final String exception,
+ @NotEmpty final String exceptionMessage,
+ @NotEmpty final String stacktrace,
+ final boolean isDriver) {
+ super(applicationId, jobName, jobStartTimestamp, timestamp);
+ this.exception = exception;
+ this.exception_message = exceptionMessage;
+ this.stacktrace = stacktrace;
+ this.detected_on_driver = isDriver;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/dataset/MetricRecord.java b/marmaray/src/main/java/com/uber/marmaray/common/dataset/MetricRecord.java
new file mode 100644
index 0000000..b27f3f0
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/dataset/MetricRecord.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.dataset;
+
+import lombok.Data;
+import lombok.EqualsAndHashCode;
+import org.hibernate.validator.constraints.NotEmpty;
+
+/**
+ * Simple Java Bean used to construct {@link UtilTable} of {@MetricRecord}
+ */
+@Data
+@EqualsAndHashCode(callSuper = true)
+public class MetricRecord extends UtilRecord {
+ private String metric_name;
+ private Long metric_value;
+ private String tags;
+
+ public MetricRecord(@NotEmpty final String applicationId,
+ @NotEmpty final String jobName,
+ final long jobStartTimestamp,
+ final long timestamp,
+ @NotEmpty final String metricName,
+ final long metricValue,
+ @NotEmpty final String tags) {
+ super(applicationId, jobName, jobStartTimestamp, timestamp);
+ this.metric_name = metricName;
+ this.metric_value = metricValue;
+ this.tags = tags;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilRecord.java b/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilRecord.java
new file mode 100644
index 0000000..13f8510
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilRecord.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.dataset;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+
+/**
+ * {@link UtilRecord} is the member type of {@link UtilTable} collections.
+ * Subclasses of {@link UtilRecord} must conform to the requirements of a
+ * simple Java Bean so they can be converted to {@link org.apache.spark.sql.Dataset},
+ * which are:
+ * 1) Have primitive field types
+ * 2) Have default values for instance fields
+ * 3) Have getter and setters for all fields
+ * 4) Have a constructor with no arguments
+ */
+@AllArgsConstructor
+@Data
+public abstract class UtilRecord implements Serializable {
+ private String application_id;
+ private String job_name;
+ private long job_start_timestamp;
+ private long timestamp;
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilTable.java b/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilTable.java
new file mode 100644
index 0000000..838228f
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/dataset/UtilTable.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.dataset;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.uber.marmaray.utilities.SparkUtil;
+import lombok.AllArgsConstructor;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.rdd.RDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.SparkSession;
+
+import java.io.IOException;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+
+import static com.uber.marmaray.utilities.DateUtil.DATE_PARTITION_FORMAT;
+
+/**
+ * {@link UtilTable} is represented as a {@link Dataset} of {@link UtilRecord},
+ * which is converted from a {@link JavaRDD} of {@link UtilRecord}. We can extend
+ * the functionality of {@link UtilTable} same as that of {@link Dataset}
+ */
+@Slf4j
+@AllArgsConstructor
+public class UtilTable {
+ private final SparkSession spark;
+ private final Dataset dataset;
+ private final Path destPath;
+ private final boolean isDatePartitioned;
+
+ public UtilTable(@NonNull final Class type,
+ @NonNull final JavaRDD javaRDD,
+ @NonNull final Path destPath,
+ final boolean isDatePartitioned) {
+ this(type, javaRDD, destPath, isDatePartitioned, SparkUtil.getOrCreateSparkSession());
+ }
+
+ public UtilTable(@NonNull final Class type,
+ @NonNull final JavaRDD javaRDD,
+ @NonNull final Path destPath,
+ final boolean isDatePartitioned,
+ @NonNull final SparkSession sparkSession) {
+ this.spark = sparkSession;
+ final RDD rdd = javaRDD.rdd();
+ final Encoder bean = Encoders.bean(type);
+ this.dataset = this.spark.createDataset(rdd, bean);
+ this.destPath = destPath;
+ this.isDatePartitioned = isDatePartitioned;
+ }
+
+ public void writeParquet() throws IOException {
+ // TODO: Consider having a configuration to limit number records written out
+ this.dataset.write().mode(SaveMode.Append).parquet(getDestWritePath().toString());
+ }
+
+ public Long size() {
+ return this.dataset.count();
+ }
+
+ @VisibleForTesting
+ public void show() {
+ this.dataset.show();
+ }
+
+ public Path getDestWritePath() {
+ return this.isDatePartitioned ? getDestDatePartitionedPath() : this.destPath;
+ }
+
+ private Path getDestDatePartitionedPath() {
+ final ZonedDateTime date = ZonedDateTime.now(ZoneOffset.UTC);
+ final DateTimeFormatter formatter = DateTimeFormatter.ofPattern(DATE_PARTITION_FORMAT);
+ return new Path(this.destPath, date.format(formatter));
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/ForkOperationException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/ForkOperationException.java
new file mode 100644
index 0000000..f85e953
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/ForkOperationException.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.exceptions;
+
+/**
+ * Thrown when ForkOperation fails to succeed.
+ */
+public class ForkOperationException extends JobRuntimeException {
+
+ public ForkOperationException(final String message) {
+ super(message);
+ }
+
+ public ForkOperationException(final String message, final Throwable t) {
+ super(message, t);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/InvalidDataException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/InvalidDataException.java
new file mode 100644
index 0000000..6c2c441
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/InvalidDataException.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.exceptions;
+
+import com.uber.marmaray.common.converters.data.HoodieSinkDataConverter;
+import lombok.NonNull;
+import org.apache.avro.generic.GenericRecord;
+import org.hibernate.validator.constraints.NotEmpty;
+
+/**
+ * It is a checked exception and should be thrown when there is either missing or invalid user defined field in
+ * data. Check {@link HoodieSinkDataConverter#getKey(GenericRecord)}
+ * for an example.
+ */
+public class InvalidDataException extends Exception {
+
+ public InvalidDataException(@NotEmpty final String message) {
+ super(message);
+ }
+
+ public InvalidDataException(@NotEmpty final String message, @NonNull final Throwable t) {
+ super(message, t);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/JobRuntimeException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/JobRuntimeException.java
new file mode 100644
index 0000000..aa29687
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/JobRuntimeException.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.exceptions;
+
+import lombok.NonNull;
+import org.hibernate.validator.constraints.NotEmpty;
+
+/**
+ * This is the parent runtime exception thrown whenever job encounters unrecoverable exception.
+ */
+
+public class JobRuntimeException extends RuntimeException {
+ public JobRuntimeException(@NotEmpty final String message) {
+ super(message);
+ }
+
+ public JobRuntimeException(@NonNull final Throwable t) {
+ super(t);
+ }
+
+ public JobRuntimeException(@NotEmpty final String message, @NonNull final Throwable t) {
+ super(message, t);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MetadataException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MetadataException.java
new file mode 100644
index 0000000..16aaefe
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MetadataException.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.exceptions;
+
+/**
+ * This exception is thrown anytime we encounter an exception with reading or writing metadata
+ */
+public class MetadataException extends JobRuntimeException {
+ public MetadataException(final String message) {
+ super(message);
+ }
+
+ public MetadataException(final String message, final Throwable t) {
+ super(message, t);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MissingPropertyException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MissingPropertyException.java
new file mode 100644
index 0000000..3fe938a
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/MissingPropertyException.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.exceptions;
+
+/**
+ * Thrown when any required property for a module is not specified.
+ */
+
+public class MissingPropertyException extends JobRuntimeException {
+ public MissingPropertyException(final String propertyName) {
+ super("property:" + propertyName);
+ }
+
+ public MissingPropertyException(final String message, final Throwable t) {
+ super(message, t);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/exceptions/RetryException.java b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/RetryException.java
new file mode 100644
index 0000000..ecaea6b
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/exceptions/RetryException.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.exceptions;
+
+/**
+ * This exception is thrown when the task runs out of retries.
+ */
+public class RetryException extends JobRuntimeException {
+ public RetryException(final String message) {
+ super(message);
+ }
+
+ public RetryException(final String message, final Throwable t) {
+ super(message, t);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/FilterFunction.java b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/FilterFunction.java
new file mode 100644
index 0000000..fa90ad5
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/FilterFunction.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.forkoperator;
+
+import com.uber.marmaray.common.data.ForkData;
+import lombok.AllArgsConstructor;
+import org.apache.spark.api.java.function.Function;
+
+/**
+ * This is used for filtering result RDD. The passed in filterkey should be same as the one used
+ * in ForkFunction.
+ * @param
+ */
+@AllArgsConstructor
+public class FilterFunction implements Function, Boolean> {
+
+ private final Integer filterKey;
+
+ @Override
+ public final Boolean call(final ForkData forkData) {
+ return execute(forkData);
+ }
+
+ /**
+ * It is used for filtering out tupleEntries. If it returns true then tupleEntry will be
+ * filtered out. It will have same set of keys as defined by corresponding ForkFunction.
+ *
+ * @param forkData : forkData to be filtered out or retained.
+ */
+ protected Boolean execute(final ForkData forkData) {
+ return forkData.getKeys().contains(this.filterKey);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkFunction.java b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkFunction.java
new file mode 100644
index 0000000..11630c2
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkFunction.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.forkoperator;
+
+import com.google.common.base.Optional;
+import com.uber.marmaray.common.data.ForkData;
+import com.uber.marmaray.common.exceptions.ForkOperationException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.spark.SparkContext;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.util.LongAccumulator;
+
+/**
+ * It associates every record with a set of fork-keys. This is an abstract class and subclass of
+ * it should implement {@link ForkFunction#process(Object)} method. All the keys used by
+ * {@link ForkFunction} should be registered by calling {@link ForkFunction#registerKeys(List)}. If
+ * any non registered key is returned by {@link ForkFunction#process(Object)} then it will
+ * raise {@link ForkOperationException}.
+ * @param
+ */
+@Slf4j
+public abstract class ForkFunction implements FlatMapFunction> {
+
+ private final Map> registeredKeys = new HashMap<>();
+
+ public void registerKeys(final List keys) {
+ keys.stream().forEach(key -> {
+ if (this.registeredKeys.containsKey(key)) {
+ throw new ForkOperationException("Duplicate key found :" + key);
+ }
+ this.registeredKeys.put(key, Optional.absent());
+ });
+ }
+
+ public long getRecordCount(final int key) {
+ final Optional keyCount = this.registeredKeys.get(key);
+ final long ret = keyCount.isPresent() ? keyCount.get().count() : 0;
+ log.info("{} : key :{}: count :{}", this.getClass().getName(), key, ret);
+ return ret;
+ }
+
+ public void registerAccumulators(@NonNull final SparkContext sparkContext) {
+ this.registeredKeys.entrySet().forEach(
+ entry -> {
+ entry.setValue(Optional.of(sparkContext.longAccumulator()));
+ }
+ );
+ }
+
+ @Override
+ public final Iterator> call(final DI di) {
+ final List> forkData = process(di);
+ forkData.stream().forEach(fd -> verifyKeys(fd.getKeys(), di));
+ return forkData.iterator();
+ }
+
+ private void verifyKeys(final List keys, final DI di) {
+ keys.stream().forEach(key -> {
+ if (!this.registeredKeys.containsKey(key)) {
+ log.error("Invalid key:{}: in keys:{}:for record:{}", key, keys, di);
+ throw new ForkOperationException("Using unregistered key :" + key);
+ }
+ this.registeredKeys.get(key).get().add(1);
+ });
+ }
+
+ /**
+ * This method should be implemented by subclass. This method is suppose to associate every
+ * record with set of registered keys.
+ * @return {@link ForkData} with set of fork keys which should be associated with the record.
+ * @param record element from {@link ForkOperator#inputRDD}.
+ * @throws Exception
+ */
+ protected abstract List> process(final DI record);
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkOperator.java b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkOperator.java
new file mode 100644
index 0000000..b5f4689
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/forkoperator/ForkOperator.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.forkoperator;
+
+import com.google.common.base.Optional;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.data.ForkData;
+import com.uber.marmaray.common.exceptions.ForkOperationException;
+import com.uber.marmaray.utilities.SparkUtil;
+import com.uber.marmaray.utilities.listener.SparkJobTracker;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.storage.RDDInfo;
+import org.apache.spark.storage.StorageLevel;
+
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * {@link ForkOperator} is to be used when single RDD needs to be sent to multiple forked
+ * pipelines. Each forked pipeline is associated with unique integer key and check
+ * {@link ForkFunction#registerKeys(List)} more details. It calls {@link ForkFunction} for every
+ * record and expects it to associate it with set of forked pipeline keys. Once entire
+ * {@link ForkOperator#inputRDD}; its result is persisted using persistence level provided. If
+ * nothing is specified then it will use {@link ForkOperator#DEFAULT_PERSIST_LEVEL}. Make sure to
+ * call {@link ForkOperator#execute()} before calling {@link ForkOperator#getRDD(int)} or {@link #getCount(int)}.
+ * Once all the data is retrieved call {@link ForkOperator#close()} to unpersist the result RDD.
+ * @param
+ */
+@Slf4j
+public class ForkOperator implements Serializable {
+
+ public static final String PERSIST_LEVEL = Configuration.MARMARAY_PREFIX + "fork.persist_level";
+ public static final String DEFAULT_PERSIST_LEVEL = "DISK_ONLY";
+
+ private final JavaRDD inputRDD;
+ private final ForkFunction forkFunction;
+ private Optional>> groupRDD = Optional.absent();
+ @Getter
+ private final StorageLevel persistLevel;
+
+ public ForkOperator(@NonNull final JavaRDD inputRDD, @NonNull final ForkFunction forkFunction,
+ @NonNull final Configuration conf) {
+ this.inputRDD = inputRDD;
+ this.forkFunction = forkFunction;
+ this.persistLevel = StorageLevel
+ .fromString(conf.getProperty(PERSIST_LEVEL, DEFAULT_PERSIST_LEVEL));
+ }
+
+ public final void execute() {
+ this.forkFunction.registerAccumulators(this.inputRDD.rdd().sparkContext());
+ // Converts JavaRDD -> JavaRDD, T>
+ JavaRDD> forkedData = this.inputRDD.flatMap(this.forkFunction)
+ .persist(this.persistLevel);
+ final String jobName = SparkJobTracker.getJobName(this.inputRDD.rdd().sparkContext());
+ forkedData.setName(String.format("%s-%s", jobName, forkedData.id()));
+ // deliberately calling count so that DAG gets executed.
+ final long processedRecords = forkedData.count();
+ final Optional rddInfo = SparkUtil.getRddInfo(forkedData.context(), forkedData.id());
+ log.info("#processed records :{} name:{}", processedRecords, forkedData.name());
+ if (rddInfo.isPresent()) {
+ final long size = rddInfo.get().diskSize() + rddInfo.get().memSize();
+ log.info("rddInfo -> name:{} partitions:{} size:{}", forkedData.name(), rddInfo.get().numPartitions(),
+ size);
+ }
+ this.groupRDD = Optional.of(forkedData);
+ }
+
+ public long getCount(final int filterKey) {
+ return this.forkFunction.getRecordCount(filterKey);
+ }
+
+ public JavaRDD getRDD(final int filterKey) {
+ final long count = getCount(filterKey);
+ log.info("#records for :{} = {}", filterKey, count);
+ if (count > 0) {
+ return getRDD(new FilterFunction<>(filterKey));
+ } else {
+ return (new JavaSparkContext(inputRDD.rdd().sparkContext())).emptyRDD();
+ }
+ }
+
+ public JavaRDD getRDD(final FilterFunction filterFunction) {
+ if (!this.groupRDD.isPresent()) {
+ throw new ForkOperationException("No RDD is found");
+ }
+ return this.groupRDD.get().filter(filterFunction).map(record -> record.getRecord());
+ }
+
+ public void close() {
+ if (this.groupRDD.isPresent()) {
+ this.groupRDD.get().unpersist();
+ this.groupRDD = Optional.absent();
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/DagPayload.java b/marmaray/src/main/java/com/uber/marmaray/common/job/DagPayload.java
new file mode 100644
index 0000000..e21d2f9
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/job/DagPayload.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.job;
+
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.IPayload;
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.NonNull;
+import org.apache.spark.api.java.JavaRDD;
+
+/**
+ * Helper class to pass Payload to child dag.
+ */
+@AllArgsConstructor
+public class DagPayload implements IPayload> {
+
+ @NonNull
+ @Getter
+ private final JavaRDD data;
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/ExecutionTimeJobExecutionStrategy.java b/marmaray/src/main/java/com/uber/marmaray/common/job/ExecutionTimeJobExecutionStrategy.java
new file mode 100644
index 0000000..504d299
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/job/ExecutionTimeJobExecutionStrategy.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common.job;
+
+import com.google.common.base.Optional;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.common.metadata.JobManagerMetadataTracker;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import scala.Tuple2;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.TimeUnit;
+
+@Slf4j
+/**
+ * {@link ExecutionTimeJobExecutionStrategy} is a {@link IJobExecutionStrategy} that sorts the {@link JobDag}s by
+ * execution time and latest completion, so DAGs that have not finished will run first, followed by jobs sorted by
+ * execution time descending.
+ * This order should help prevent individual long-running dags from keeping the execution from completing on time.
+ */
+public class ExecutionTimeJobExecutionStrategy implements IJobExecutionStrategy {
+
+ public static final int DEFAULT_LAST_EXECUTION_TIME_THRESHOLD_HOURS = 6;
+
+ private final int lastExecutionTimeThresholdHours;
+ private final long currentTime;
+
+ @NonNull
+ private final JobManagerMetadataTracker tracker;
+
+ public ExecutionTimeJobExecutionStrategy(@NonNull final JobManagerMetadataTracker tracker) {
+ this(tracker, DEFAULT_LAST_EXECUTION_TIME_THRESHOLD_HOURS);
+ }
+
+ public ExecutionTimeJobExecutionStrategy(@NonNull final JobManagerMetadataTracker tracker,
+ final int lastExecutionTimeThresholdHours) {
+ this.lastExecutionTimeThresholdHours = lastExecutionTimeThresholdHours;
+ this.tracker = tracker;
+ this.currentTime = System.currentTimeMillis();
+ }
+
+ @Override
+ public List sort(@NonNull final Queue inputJobDags) {
+ final List result = new ArrayList<>(inputJobDags.size());
+ final long lastExecutionTimeThresholdMillis = TimeUnit.HOURS.toMillis(this.lastExecutionTimeThresholdHours);
+ inputJobDags.stream().map(dag -> {
+ try {
+ final Optional> contents = this.tracker.get(dag.getDataFeedName());
+ if (contents.isPresent() && contents.get().containsKey(JobDag.LAST_RUNTIME_METADATA_KEY)) {
+ long lastExecutionTime = contents.get().containsKey(JobDag.LAST_EXECUTION_METADATA_KEY)
+ ? Long.parseLong(contents.get().get(JobDag.LAST_EXECUTION_METADATA_KEY))
+ : Long.MIN_VALUE;
+ if (this.currentTime - lastExecutionTime > lastExecutionTimeThresholdMillis) {
+ return new Tuple2<>(dag, Long.MAX_VALUE);
+ }
+ return new Tuple2<>(dag, Long.valueOf(contents.get().get(JobDag.LAST_RUNTIME_METADATA_KEY)));
+ } else {
+ return new Tuple2<>(dag, Long.MAX_VALUE);
+ }
+ } catch (IOException e) {
+ throw new JobRuntimeException(String.format(
+ "Unable to get metadata for dag %s : ", dag.getDataFeedName()), e);
+ }
+ }).sorted((o1, o2) -> o2._2().compareTo(o1._2()))
+ .forEach(tuple -> result.add(tuple._1()));
+ return result;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/IJobExecutionStrategy.java b/marmaray/src/main/java/com/uber/marmaray/common/job/IJobExecutionStrategy.java
new file mode 100644
index 0000000..6f82fd5
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/job/IJobExecutionStrategy.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+
+package com.uber.marmaray.common.job;
+
+import lombok.NonNull;
+
+import java.util.List;
+import java.util.Queue;
+
+/**
+ * {@link IJobExecutionStrategy} determines the order for {@link JobDag} execution rather than just relying on
+ * submission order.
+ */
+public interface IJobExecutionStrategy {
+
+ List sort(@NonNull final Queue inputJobDags);
+
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/Job.java b/marmaray/src/main/java/com/uber/marmaray/common/job/Job.java
new file mode 100644
index 0000000..78fc5e6
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/job/Job.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.job;
+
+import com.uber.marmaray.common.configuration.Configuration;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.AllArgsConstructor;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+@AllArgsConstructor
+public class Job {
+
+ @NonNull
+ private final JobDag jobDag;
+ @NonNull
+ private final Configuration conf;
+
+ public void run() {
+ final AtomicBoolean isSuccess = new AtomicBoolean(true);
+ try {
+ ThreadPoolService.init(this.conf);
+ jobDag.execute();
+ } catch (final Throwable t) {
+ isSuccess.set(false);
+ throw t;
+ } finally {
+ ThreadPoolService.shutdown(!isSuccess.get());
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/JobDag.java b/marmaray/src/main/java/com/uber/marmaray/common/job/JobDag.java
new file mode 100644
index 0000000..7f5f1ce
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/job/JobDag.java
@@ -0,0 +1,189 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.job;
+
+import com.google.common.base.Optional;
+import com.uber.marmaray.common.AvroPayload;
+import com.uber.marmaray.common.actions.IJobDagAction;
+import com.uber.marmaray.common.actions.JobDagActions;
+import com.uber.marmaray.common.actions.ReporterAction;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.common.metadata.AbstractValue;
+import com.uber.marmaray.common.metadata.IMetadataManager;
+import com.uber.marmaray.common.metrics.DataFeedMetricNames;
+import com.uber.marmaray.common.metrics.DataFeedMetrics;
+import com.uber.marmaray.common.metrics.JobMetrics;
+import com.uber.marmaray.common.metrics.LongMetric;
+import com.uber.marmaray.common.metrics.TimerMetric;
+import com.uber.marmaray.common.reporters.Reporters;
+import com.uber.marmaray.common.sinks.ISink;
+import com.uber.marmaray.common.sources.IRunState;
+import com.uber.marmaray.common.sources.ISource;
+import com.uber.marmaray.common.sources.IWorkUnitCalculator;
+import com.uber.marmaray.common.sources.IWorkUnitCalculator.IWorkUnitCalculatorResult;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.Setter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.spark.api.java.JavaRDD;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+@Slf4j
+public class JobDag, C extends IWorkUnitCalculator,
+ K extends IWorkUnitCalculatorResult> {
+
+ public static final String LAST_RUNTIME_METADATA_KEY = "runtime";
+ public static final String LAST_EXECUTION_METADATA_KEY = "last_execution";
+ @NonNull
+ private final ISource source;
+ @NonNull
+ private final JobSubDag sinkDag;
+ @NonNull
+ private final IMetadataManager metadataManager;
+ @NonNull
+ private final IWorkUnitCalculator workUnitCalculator;
+ @NonNull @Getter
+ private final String jobName;
+ @NonNull @Getter
+ private final String dataFeedName;
+
+ @Getter @Setter
+ private Map jobManagerMetadata;
+ private final Reporters reporters;
+ private final JobDagActions postJobDagActions;
+
+ @NonNull
+ private final JobMetrics jobMetrics;
+ @Getter
+ private final DataFeedMetrics dataFeedMetrics;
+
+ public JobDag(@NonNull final ISource source,
+ @NonNull final ISink sink,
+ @NonNull final IMetadataManager metadataManager,
+ @NonNull final IWorkUnitCalculator workUnitCalculator,
+ @NotEmpty final String jobName,
+ @NotEmpty final String dataFeedName,
+ @NonNull final JobMetrics jobMetrics,
+ @NonNull final Reporters reporters) {
+ this(source, new SingleSinkSubDag(sink), metadataManager,
+ workUnitCalculator, jobName, dataFeedName, jobMetrics, reporters);
+ }
+
+ public JobDag(@NonNull final ISource source,
+ @NonNull final JobSubDag sinkDag,
+ @NonNull final IMetadataManager metadataManager,
+ @NonNull final IWorkUnitCalculator workUnitCalculator,
+ @NotEmpty final String jobName,
+ @NotEmpty final String dataFeedName,
+ @NonNull final JobMetrics jobMetrics,
+ @NonNull final Reporters reporters) {
+ this.source = source;
+ this.sinkDag = sinkDag;
+ this.metadataManager = metadataManager;
+ this.workUnitCalculator = workUnitCalculator;
+ this.jobName = jobName;
+ this.dataFeedName = dataFeedName;
+ this.reporters = reporters;
+ this.postJobDagActions = new JobDagActions(this.reporters, dataFeedName);
+ this.jobMetrics = jobMetrics;
+ this.dataFeedMetrics = new DataFeedMetrics(this.jobName,
+ Collections.singletonMap(DataFeedMetrics.DATA_FEED_NAME, this.dataFeedName));
+ }
+
+ /**
+ * Add an action on success
+ * @param action to execute after job is successful
+ */
+ public void addAction(final IJobDagAction action) {
+ this.postJobDagActions.addAction(action);
+ }
+
+ public void execute() {
+ log.info("Starting job dag for {}", this.jobName);
+ final AtomicBoolean successful = new AtomicBoolean(true);
+ final ReporterAction reporterAction = new ReporterAction(this.reporters, this.jobMetrics, this.dataFeedMetrics);
+ this.addAction(reporterAction);
+ final TimerMetric timerMetric = this.dataFeedMetrics.createTimerMetric(
+ DataFeedMetricNames.TOTAL_LATENCY_MS, new HashMap<>(), Optional.absent());
+ try {
+ // set up metrics for downstreams
+ Arrays.asList(this.workUnitCalculator, this.sinkDag, this.source).forEach(metricable -> {
+ metricable.setDataFeedMetrics(this.dataFeedMetrics);
+ metricable.setJobMetrics(this.jobMetrics);
+ });
+ // initialize previous run state.
+ this.workUnitCalculator.initPreviousRunState(this.metadataManager);
+ // compute work units.
+ final K workUnitCalculatorResult = this.workUnitCalculator.computeWorkUnits();
+ log.info("Work unit calculator result :{}", workUnitCalculatorResult);
+ // save run state for next processing
+ this.workUnitCalculator.saveNextRunState(this.metadataManager, workUnitCalculatorResult.getNextRunState());
+ if (workUnitCalculatorResult.hasWorkUnits()) {
+ // read source rdd.
+ final JavaRDD sourceRDD = this.source.getData(workUnitCalculatorResult);
+
+ // execute sink dag.
+ this.sinkDag.execute(Optional.of(new DagPayload(sourceRDD)));
+
+ // commit sink dag
+ this.sinkDag.commit();
+ }
+
+ try {
+ this.metadataManager.saveChanges();
+ } catch (IOException e) {
+ final String msg = "Failed to save metadata changes" + e.getMessage();
+ log.error(msg, e);
+ throw new JobRuntimeException(msg, e);
+ }
+ } catch (Exception e) {
+ // set failure status if there was an error
+ successful.set(false);
+ throw e;
+ } finally {
+ // execute all actions at the last minute
+ timerMetric.stop();
+ reportStatus(successful.get());
+ if (getJobManagerMetadata() != null && successful.get()) {
+ this.getJobManagerMetadata().put(LAST_RUNTIME_METADATA_KEY, timerMetric.getMetricValue().toString());
+ this.getJobManagerMetadata().put(LAST_EXECUTION_METADATA_KEY,
+ String.valueOf(TimeUnit.SECONDS.toMillis(timerMetric.getStartTime().getEpochSecond())));
+ }
+ this.dataFeedMetrics.createLongMetric(DataFeedMetricNames.RESULT,
+ successful.get() ? DataFeedMetricNames.RESULT_SUCCESS : DataFeedMetricNames.RESULT_FAILURE,
+ Collections.emptyMap());
+ this.postJobDagActions.execute(successful.get());
+ }
+ }
+
+ private void reportStatus(final boolean successful) {
+ final long statusValue =
+ successful ? DataFeedMetricNames.RESULT_SUCCESS : DataFeedMetricNames.RESULT_FAILURE;
+ final LongMetric successMetric = new LongMetric(
+ DataFeedMetricNames.RESULT, statusValue);
+ successMetric.addTags(this.dataFeedMetrics.getBaseTags());
+ this.reporters.report(successMetric);
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/JobManager.java b/marmaray/src/main/java/com/uber/marmaray/common/job/JobManager.java
new file mode 100644
index 0000000..dc46233
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/job/JobManager.java
@@ -0,0 +1,368 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.job;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.uber.marmaray.common.actions.IJobDagAction;
+import com.uber.marmaray.common.actions.JobDagActions;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.configuration.HadoopConfiguration;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.common.exceptions.MetadataException;
+import com.uber.marmaray.common.metadata.JobManagerMetadataTracker;
+import com.uber.marmaray.common.metrics.JobMetricNames;
+import com.uber.marmaray.common.metrics.JobMetrics;
+import com.uber.marmaray.common.metrics.TimerMetric;
+import com.uber.marmaray.common.reporters.ConsoleReporter;
+import com.uber.marmaray.common.reporters.IReporter;
+import com.uber.marmaray.common.reporters.Reporters;
+import com.uber.marmaray.utilities.LockManager;
+import com.uber.marmaray.utilities.SparkUtil;
+import com.uber.marmaray.utilities.listener.SparkEventListener;
+import com.uber.marmaray.utilities.listener.SparkJobTracker;
+import com.uber.marmaray.utilities.listener.TimeoutManager;
+
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.Setter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.avro.Schema;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * JobManager
+ *
+ * This class is responsible for running the entire spark application that will be created. It schedules an runs
+ * the {@link JobDag}s that are registered in the application, as well as any {@link IJobDagAction}s that are
+ * registered.
+ *
+ * This class is also responsible for the {@link JavaSparkContext}.
+ */
+@Slf4j
+public final class JobManager {
+
+ public static final boolean DEFAULT_LOCK_FREQUENCY = true;
+
+ private static JobManager instance;
+ private static final Object lock = new Object();
+
+ @NonNull
+ private final Queue jobDags = new ConcurrentLinkedDeque<>();
+ private final JobDagActions postJobManagerActions;
+ @Getter
+ private final Configuration conf;
+
+ @NotEmpty
+ private final String appName;
+ private String appId;
+ private Optional sparkContext = Optional.absent();
+ private final JobLockManager jobLockManager;
+
+ @Getter
+ private final List schemas = new ArrayList<>();
+
+ @Getter
+ private final List serializationClasses = new LinkedList<>();
+
+ @Getter
+ private final JobMetrics jobMetrics;
+
+ @Getter
+ private final Reporters reporters;
+
+ @Getter @Setter
+ private JobManagerMetadataTracker tracker;
+
+ @Getter @Setter
+ private boolean jobManagerMetadataEnabled;
+
+ @Setter
+ private Optional jobExecutionStrategy = Optional.absent();
+
+ private JobManager(@NonNull final Configuration conf, @NotEmpty final String appName,
+ @NotEmpty final String frequency, final boolean shouldLockFrequency) {
+ this.conf = conf;
+ this.appName = appName;
+ this.jobMetrics = new JobMetrics(appName);
+ this.reporters = new Reporters();
+ this.reporters.addReporter(new ConsoleReporter());
+ this.jobLockManager = new JobLockManager(conf, frequency, shouldLockFrequency);
+ this.postJobManagerActions = new JobDagActions(this.reporters, "jobManager");
+ }
+
+ /**
+ * Create the JobManager. Will fail if the job manager has already been created.
+ * @param conf Configuration for the job manager, used to determine parallelism of execution
+ * @param appName Name of the application, used in the SparkContext
+ * @param frequency name of the frequency, used to lock entire frequencies
+ * @param lockFrequency whether the frequency should be locked
+ */
+ public static JobManager createJobManager(@NonNull final Configuration conf, @NotEmpty final String appName,
+ @NotEmpty final String frequency, final boolean lockFrequency) {
+ synchronized (lock) {
+ Preconditions.checkState(instance == null,
+ "JobManager was already created");
+ instance = new JobManager(conf, appName, frequency, lockFrequency);
+ }
+ return instance;
+ }
+
+ /**
+ * Create the JobManager. Will fail if the job manager has already been created.
+ * @param conf Configuration for the job manager, used to determine parallelism of execution
+ * @param appName Name of the application, used in the SparkContext
+ * @param frequency name of the frequency, used to lock entire frequencies
+ */
+ public static JobManager createJobManager(@NonNull final Configuration conf, @NotEmpty final String appName,
+ @NotEmpty final String frequency) {
+ return createJobManager(conf, appName, frequency, DEFAULT_LOCK_FREQUENCY);
+ }
+
+ /**
+ * Creates JavaSparkContext if its hasn't been created yet, or returns the instance. {@link #addSchema(Schema)} and
+ * {@link #addSchemas(Collection)} must not be called once the JavaSparkContext has been created
+ * @return the JavaSparkContext that will be used to execute the JobDags
+ */
+ public JavaSparkContext getOrCreateSparkContext() {
+ if (!this.sparkContext.isPresent()) {
+ this.sparkContext = Optional.of(new JavaSparkContext(
+ SparkUtil.getSparkConf(
+ this.appName, Optional.of(this.schemas), this.serializationClasses, this.conf)));
+ this.sparkContext.get().sc().addSparkListener(new SparkEventListener());
+ // Adding hadoop configuration to default
+ this.sparkContext.get().sc().hadoopConfiguration().addResource(
+ new HadoopConfiguration(conf).getHadoopConf());
+ this.appId = this.sparkContext.get().sc().applicationId();
+ }
+ return this.sparkContext.get();
+ }
+
+ /**
+ * Execute all registered {@link JobDag}, then perform all registered {@link IJobDagAction}
+ */
+ public void run() {
+ final Queue futures = new ConcurrentLinkedDeque<>();
+ final AtomicBoolean isSuccess = new AtomicBoolean(true);
+ // ensure the SparkContext has been created
+ final JavaSparkContext sc = getOrCreateSparkContext();
+ Preconditions.checkState(!this.jobDags.isEmpty(), "No job dags to execute");
+ TimeoutManager.init(this.conf, sc.sc());
+ final boolean hasMultipleDags = this.jobDags.size() > 1;
+ final Queue runtimeJobDagOrder;
+ if (hasMultipleDags && this.jobExecutionStrategy.isPresent()) {
+ runtimeJobDagOrder = new ConcurrentLinkedDeque<>(this.jobExecutionStrategy.get().sort(this.jobDags));
+ } else {
+ runtimeJobDagOrder = this.jobDags;
+ }
+ try {
+ ThreadPoolService.init(this.conf);
+ runtimeJobDagOrder.forEach(jobDag ->
+ futures.add(ThreadPoolService.submit(
+ () -> {
+ SparkJobTracker.setJobName(sc.sc(), jobDag.getDataFeedName());
+ if (hasMultipleDags) {
+ setSparkStageName(sc, jobDag.getDataFeedName());
+ }
+ jobDag.execute();
+ return null;
+ }, ThreadPoolServiceTier.JOB_DAG_TIER)));
+ TimeoutManager.getInstance().startMonitorThread();
+ futures.forEach(future -> {
+ try {
+ future.get();
+ } catch (Exception e) {
+ log.error("Error running job", e);
+ isSuccess.set(false);
+ }
+ }
+ );
+
+ if (!isSuccess.get()) {
+ throw new JobRuntimeException("Error while running job. Look at previous log entries for detail");
+ }
+ } catch (final Throwable t) {
+ isSuccess.set(false);
+ throw t;
+ } finally {
+ this.postJobManagerActions.execute(isSuccess.get());
+ ThreadPoolService.shutdown(!isSuccess.get());
+ if (this.isJobManagerMetadataEnabled()) {
+ jobDags.forEach(jobDag -> this.getTracker().set(jobDag.getDataFeedName(),
+ jobDag.getJobManagerMetadata()));
+ try {
+ this.getTracker().writeJobManagerMetadata();
+ } catch (MetadataException e) {
+ log.error("Unable to save metadata: {}", e.getMessage());
+ }
+ }
+ sc.stop();
+ this.jobLockManager.stop();
+ this.reporters.getReporters().forEach(IReporter::finish);
+ }
+ }
+
+ /**
+ * Add {@link JobDag} to be executed on {@link #run()}
+ * @param jobDag JobDag to be added
+ */
+ public void addJobDag(@NonNull final JobDag jobDag) {
+ if (jobLockManager.lockDag(jobDag.getJobName(), jobDag.getDataFeedName())) {
+ this.jobDags.add(jobDag);
+ } else {
+ log.warn("Failed to obtain lock for JobDag {} - {}", jobDag.getJobName(), jobDag.getDataFeedName());
+ }
+ }
+
+ /**
+ * Add collection of {@link JobDag} to be executed on {@link #run()}
+ * @param jobDags collection of JobDags to be added
+ */
+ public void addJobDags(@NonNull final Collection extends JobDag> jobDags) {
+ jobDags.forEach(this::addJobDag);
+ }
+
+ /**
+ * Add {@link IJobDagAction} to be executed after all {@link JobDag} have completed
+ * @param action action to add
+ */
+ public void addPostJobManagerAction(@NonNull final IJobDagAction action) {
+ this.postJobManagerActions.addAction(action);
+ }
+
+ /**
+ * Add collection of {@link IJobDagAction} to be executed after all {@link JobDag} have completed
+ * @param actions action to add
+ */
+ public void addPostJobManagerActions(@NonNull final Collection extends IJobDagAction> actions) {
+ actions.forEach(this::addPostJobManagerAction);
+ }
+
+ /**
+ * Add schema for registration into {@link JavaSparkContext}. Must not be called after the JavaSparkContext has been
+ * created.
+ * @param schema schema to register into spark context
+ */
+ public void addSchema(@NonNull final Schema schema) {
+ Preconditions.checkState(!this.sparkContext.isPresent(),
+ "Schemas must be added before sparkContext is instantiated");
+ this.schemas.add(schema);
+ }
+
+ /**
+ * Add serialization classes for registration into {@link JavaSparkContext}. Must not be called after the
+ * {@link JavaSparkContext} have been created.
+ * @param serializationClasses serialization classes
+ */
+ public void addSerializationClasses(@NonNull final List serializationClasses) {
+ Preconditions.checkState(!this.sparkContext.isPresent(),
+ "Serialization classes must be added before sparkContext is instantiated");
+ this.serializationClasses.addAll(serializationClasses);
+ }
+
+ /**
+ * Add Collection of schemas for registration into {@link JavaSparkContext}. Must not be called after the
+ * JavaSparkContext has been created.
+ * @param schemas collection of schemas to register
+ */
+ public void addSchemas(@NonNull final Collection extends Schema> schemas) {
+ schemas.forEach(this::addSchema);
+ }
+
+ private static void setSparkStageName(@NonNull final JavaSparkContext jsc, @NotEmpty final String dataFeedName) {
+ // For now we will only set stageName as "dataFeedName" but long term we would want to also include spark's
+ // action name in it; which will need support from spark.
+ jsc.setCallSite(dataFeedName);
+ }
+
+ private final class JobLockManager {
+ private static final String MANAGER_LOCK_KEY = "JOBMANAGER";
+ private static final String DAG_LOCK_KEY = "JOBDAGS";
+
+ private static final String JOB_FREQUENCY_TAG = "job_frequency";
+ private static final String JOB_NAME_TAG = "job_name";
+ private static final String DATA_FEED_TAG = "data_feed_name";
+
+ @NonNull
+ private final LockManager lockManager;
+ @NonNull
+ private final String jobFrequency;
+
+ @NonNull
+ private final TimerMetric managerTimerMetric;
+ @NonNull
+ private final HashMap dagTimerMetricMap;
+
+ private JobLockManager(@NonNull final Configuration conf, @NotEmpty final String frequency,
+ final boolean shouldLockFrequency) {
+ this.lockManager = new LockManager(conf);
+ this.jobFrequency = frequency;
+
+ final String key = LockManager.getLockKey(MANAGER_LOCK_KEY, jobFrequency);
+ this.managerTimerMetric = new TimerMetric(JobMetricNames.JOB_MANAGER_LOCK_TIME_MS,
+ ImmutableMap.of(JOB_FREQUENCY_TAG, jobFrequency,
+ JOB_NAME_TAG, appName));
+
+ if (shouldLockFrequency) {
+ final boolean success = lockManager.lock(key,
+ String.format("JobManager %s AppId %s", jobFrequency, appId));
+ this.managerTimerMetric.stop();
+ if (!success) {
+ lockManager.close();
+ throw new IllegalStateException("Failed to obtain lock for JobManager " + jobFrequency);
+ }
+ } else {
+ managerTimerMetric.stop();
+ log.info("Frequency lock disabled");
+ }
+ this.dagTimerMetricMap = new HashMap<>();
+ }
+
+ private boolean lockDag(@NotEmpty final String jobName, @NotEmpty final String dagName) {
+ final String key = LockManager.getLockKey(DAG_LOCK_KEY, dagName);
+ final TimerMetric timerMetric = new TimerMetric(JobMetricNames.JOB_DAG_LOCK_TIME_MS,
+ ImmutableMap.of(
+ JOB_FREQUENCY_TAG, jobFrequency,
+ JOB_NAME_TAG, jobName,
+ DATA_FEED_TAG, dagName));
+ final boolean success = lockManager.lock(key,
+ String.format("JobDag %s AppId %s", dagName, appId));
+ timerMetric.stop();
+ dagTimerMetricMap.put(dagName, timerMetric);
+ return success;
+ }
+
+ private void stop() {
+ log.info("Closing the LockManager in the JobManager.");
+ this.lockManager.close();
+ reporters.report(managerTimerMetric);
+ dagTimerMetricMap.forEach((dagName, timerMetric) -> reporters.report(timerMetric));
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/JobSubDag.java b/marmaray/src/main/java/com/uber/marmaray/common/job/JobSubDag.java
new file mode 100644
index 0000000..edf6ec3
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/job/JobSubDag.java
@@ -0,0 +1,236 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.job;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.uber.marmaray.common.IPayload;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.common.metrics.DataFeedMetrics;
+import com.uber.marmaray.common.metrics.IMetricable;
+import com.uber.marmaray.common.metrics.JobMetrics;
+import lombok.Data;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+/**
+ * {@link JobSubDag} is useful when we need to run custom job dag which can be represented in the form of an n-way
+ * tree. User should implement {@link #executeNode(Optional)}, {@link #commitNode()} and
+ * {@link #getDataForChild(JobSubDag)} for every node in the jobDag. The execution sequence is as follows.
+ * Consider an example Dag.
+ * # Note :- Numbers in the bracket denote their priority. Priority is only compared between children of same parent
+ * # node.
+ * # N1
+ * # / \
+ * # / \
+ * # (0)N2 (1)N3
+ * # /
+ * # /
+ * # (0)N4
+ * Caller should call below methods for complete execution of the above dag. [N1.execute() and N1.commit()].
+ * This is how the job dag will get executed.
+ * 1) N1.execute() <- user calls this.
+ * 1-a) N1.executeNode() [Step1]
+ * 2-a) N2.execute() [Step2]
+ * 2-a-a) N2.executeNode() [Step3]
+ * 2-a-b) N4.execute() [Step4]
+ * 2-a-b-a) N4.executeNode() [Step5]
+ * 2-a) N3.execute() [Step2]
+ * 2-a-a) N3.executeNode() [Step3]
+ * // It will wait for all tasks to finish.
+ * 2) N1.commit() <- user calls this.
+ * 2-a) N2.commit() [Step1]
+ * 2-a-a) N4.commit() [Step2]
+ * 2-a-b) N4.commitNode() [Step3]
+ * 2-a-c) N2.commitNode() [Step4]
+ * 2-b) N3.commit() [Step5]
+ * 2-b-a) N3.commitNode() [Step6]
+ * 2-c) N1.commitNode() [Step7]
+ */
+@Slf4j
+public abstract class JobSubDag implements IMetricable {
+
+ @NotEmpty
+ @Getter
+ // Logical name of the job node.
+ private final String name;
+
+ @Getter
+ private Optional jobMetrics = Optional.absent();
+
+ @Getter
+ private Optional dataFeedMetrics = Optional.absent();
+
+ /**
+ * Holds child nodes with their priority. ChildNodes are committed from lower to higher priority. All nodes at the
+ * same priority are committed in parallel (where ordering is not guaranteed). However
+ * {@link #executeNode(Optional)} method for all child nodes is called in parallel.
+ */
+ private final TreeMap> childNodes = new TreeMap<>();
+
+ protected JobSubDag(@NotEmpty final String name) {
+ this.name = name;
+ }
+
+ public void addSubDag(final int priority, @NonNull final JobSubDag subDag) {
+ if (!this.childNodes.containsKey(priority)) {
+ this.childNodes.put(priority, new LinkedList<>());
+ }
+ this.childNodes.get(priority).add(subDag);
+ }
+
+ @Override
+ public void setJobMetrics(@NonNull final JobMetrics jobMetrics) {
+ this.jobMetrics = Optional.of(jobMetrics);
+ }
+
+ @Override
+ public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) {
+ this.dataFeedMetrics = Optional.of(dataFeedMetrics);
+ }
+
+ /**
+ * Should pre process input and create appropriate {@link IPayload} for child nodes. Later when childNode's
+ * {@link #execute(Optional)} method is called it will call {@link #getDataForChild(JobSubDag)} to retrieve payload
+ * for it.
+ *
+ * @param data input payload
+ */
+ protected abstract void executeNode(@NonNull final Optional data);
+
+ /**
+ * An optional method which user can implement. Ideally this should be used to call atomic operations.
+ */
+ protected void commitNode() {
+ }
+
+ /**
+ * Called to retrieve payload for child dag.
+ *
+ * @param childSubDag childSubDag for which {@link IPayload} data needs to be retrieved
+ * @return payload for child dag
+ */
+ protected Optional getDataForChild(@NonNull final JobSubDag childSubDag) {
+ return Optional.absent();
+ }
+
+ /**
+ * Will execute current node's {@link #executeNode(Optional)} followed by parallel execution of {@link
+ * #childNodes}'s {@link #execute(Optional)} method.
+ */
+ public final void execute(@NonNull final Optional data) {
+ Preconditions.checkState(this.dataFeedMetrics.isPresent() && this.jobMetrics.isPresent(),
+ "Missing dataFeed or job metrics");
+ log.info("running : executeNode {}", this.name);
+ // first call Current node's executeNode().
+ executeNode(data);
+
+ // setup metrics for child nodes.
+ setupChildMetrics();
+
+ final Queue statuses = new LinkedList<>();
+ this.childNodes.entrySet().stream().forEach(
+ childNodesAtSamePriority -> {
+ childNodesAtSamePriority.getValue().stream().forEach(
+ childNode -> statuses.add(
+ new SubDagExecutionStatus(childNode,
+ ThreadPoolService.submit(
+ () -> {
+ childNode.execute(getDataForChild(childNode));
+ return 0;
+ }, ThreadPoolServiceTier.JOB_DAG_TIER
+ )))
+ );
+ }
+ );
+ waitForTasksToFinish(statuses);
+ }
+
+ private void setupChildMetrics() {
+
+ this.childNodes.entrySet().stream().forEach(
+ entry -> entry.getValue().stream().forEach(
+ jobSubDag -> {
+ jobSubDag.setJobMetrics(this.jobMetrics.get());
+ jobSubDag.setDataFeedMetrics(this.dataFeedMetrics.get());
+ }
+ ));
+ }
+
+ // Helper method to wait for parallel tasks to finish execution.
+ private void waitForTasksToFinish(@NonNull final Queue statuses) {
+ while (!statuses.isEmpty()) {
+ final SubDagExecutionStatus status = statuses.poll();
+ while (!status.getStatus().isDone()) {
+ try {
+ status.getStatus().get();
+ } catch (InterruptedException e) {
+ log.error("interrupted {} {}", status.getSubDag().getName(), e);
+ throw new JobRuntimeException("dag execution interrupted", e);
+ } catch (ExecutionException e) {
+ log.error("failed to execute subdag {} {}", status.getSubDag().getName(), e.getCause());
+ throw new JobRuntimeException("failed to execute subDag", e.getCause());
+ }
+ }
+ }
+ }
+
+ /**
+ * Will execute childNode's {@link #commitNode()} method from lower priority to higher priority.
+ * {@link #commitNode()} method of the nodes at same priority level will get executed in parallel.
+ */
+ public final void commit() {
+ log.info("calling {}'s childNodes' commit", this.name);
+ this.childNodes.entrySet().stream().forEach(
+ childNodesAtSamePriority -> {
+ final Queue statuses = new LinkedList<>();
+ childNodesAtSamePriority.getValue().stream().forEach(
+ childNode -> statuses.add(
+ new SubDagExecutionStatus(childNode,
+ ThreadPoolService.submit(
+ () -> {
+ childNode.commit();
+ return 0;
+ }, ThreadPoolServiceTier.JOB_DAG_TIER
+ )))
+ );
+ waitForTasksToFinish(statuses);
+ }
+ );
+ log.info("calling {}'s commitNode", this.name);
+ commitNode();
+ }
+
+ /**
+ * Helper class to wrap {@link JobSubDag} with it's run status.
+ */
+ @Data
+ class SubDagExecutionStatus {
+
+ private final JobSubDag subDag;
+ private final Future status;
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/SingleSinkSubDag.java b/marmaray/src/main/java/com/uber/marmaray/common/job/SingleSinkSubDag.java
new file mode 100644
index 0000000..c8482f2
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/job/SingleSinkSubDag.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.job;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.uber.marmaray.common.IPayload;
+import com.uber.marmaray.common.sinks.ISink;
+import javax.annotation.Nonnull;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * {@link SingleSinkSubDag} is used when {@link JobDag} is configured to run for single
+ * {@link com.uber.marmaray.common.sinks.ISink}.
+ */
+@Slf4j
+public class SingleSinkSubDag extends JobSubDag {
+
+ @NonNull
+ @Getter
+ private final ISink sink;
+
+ public SingleSinkSubDag(@Nonnull final ISink sink) {
+ super(String.format("%s:%s", SingleSinkSubDag.class.getName(), sink.getClass().getName()));
+ this.sink = sink;
+ }
+
+ @Override
+ protected void executeNode(@NonNull final Optional data) {
+ Preconditions.checkState(data.isPresent() && (data.get() instanceof DagPayload),
+ "Invalid payload :" + (data.isPresent() ? data.get().getClass() : null));
+
+ // setup job and topic metrics.
+ setupMetrics();
+ this.sink.write(((DagPayload) data.get()).getData());
+ }
+
+ private void setupMetrics() {
+ this.sink.setJobMetrics(getJobMetrics().get());
+ this.sink.setDataFeedMetrics(getDataFeedMetrics().get());
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolService.java b/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolService.java
new file mode 100644
index 0000000..8821a28
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolService.java
@@ -0,0 +1,424 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.job;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.uber.marmaray.common.configuration.Configuration;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+
+import java.util.Queue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Helper class to manage thread pool service. This is a singleton service and number of threads are controlled by
+ * {@link #NUM_THREADS} configuration.
+ */
+@Slf4j
+public final class ThreadPoolService {
+
+ public static final String THREAD_POOL_SERVICE_PREFIX = Configuration.MARMARAY_PREFIX + "thread_pool.";
+ public static final String NUM_THREADS = THREAD_POOL_SERVICE_PREFIX + "num_threads";
+ public static final String JOB_DAG_THREADS = THREAD_POOL_SERVICE_PREFIX + "job_dag_threads";
+ public static final String ACTIONS_THREADS = THREAD_POOL_SERVICE_PREFIX + "actions_threads";
+
+ public static final int DEFAULT_NUM_THREADS = 8;
+ public static final int AWAIT_TERMINATION_ATTEMPTS = 30;
+ public static final int AWAIT_TERMINATION_CHECK_INTERVAL_MS = 1000;
+ public static final int DEFAULT_JOB_DAG_THREADS = 6;
+ public static final int DEFAULT_ACTIONS_THREADS = 2;
+ public static final long NO_TIMEOUT = -1;
+ // Singleton service.
+ private static Optional service = Optional.absent();
+
+ @NonNull
+ private final ExecutorService threadPool;
+
+ private final int reservedJobDagThreads;
+ private final int reservedActionsThreads;
+ private final int numThreads;
+
+ private boolean isShutdown = false;
+
+ private final AtomicInteger currentThreads = new AtomicInteger();
+ private final AtomicInteger currentJobDagThreads = new AtomicInteger();
+ private final AtomicInteger currentActionsThreads = new AtomicInteger();
+
+ private final Queue jobDagQueue;
+ private final Queue actionsQueue;
+
+ private ThreadPoolService(final int numThreads, final int reservedJobDagThreads, final int reservedActionsThreads) {
+ log.info("Starting thread pool service numThreads:{} numJobDagThreads:{}", numThreads, reservedJobDagThreads);
+ Preconditions.checkState(numThreads > 0 && reservedJobDagThreads > 0 && reservedActionsThreads > 0,
+ String.format("Number of threads should be positive: total: %d, jobDag: %d, actions: %d",
+ numThreads, reservedJobDagThreads, reservedActionsThreads));
+ Preconditions.checkState(numThreads >= reservedJobDagThreads + reservedActionsThreads,
+ String.format(
+ "Total threads must be at least equal to reserved threads: total: %d, jobDag: %d, actions: %d ",
+ numThreads, reservedJobDagThreads, reservedActionsThreads));
+ this.reservedActionsThreads = reservedActionsThreads;
+ this.reservedJobDagThreads = reservedJobDagThreads;
+ this.threadPool = Executors.newFixedThreadPool(numThreads);
+ this.actionsQueue = new ConcurrentLinkedDeque<>();
+ this.jobDagQueue = new ConcurrentLinkedDeque<>();
+ this.numThreads = numThreads;
+ }
+
+ /**
+ * Initialize the ThreadPoolService
+ *
+ * @param conf Configuration from which to load the properties.
+ */
+ public static synchronized void init(@NonNull final Configuration conf) {
+ final int numThreads = conf.getIntProperty(NUM_THREADS, DEFAULT_NUM_THREADS);
+ final int reservedJobDagThreads = conf.getIntProperty(JOB_DAG_THREADS,
+ DEFAULT_JOB_DAG_THREADS);
+ final int reservedActionsThreads = conf.getIntProperty(ACTIONS_THREADS,
+ DEFAULT_ACTIONS_THREADS);
+ if (service.isPresent()) {
+ throw new JobRuntimeException("Re-initializing thread pool service.");
+ }
+ service = Optional.of(new ThreadPoolService(numThreads, reservedJobDagThreads, reservedActionsThreads));
+ }
+
+ /**
+ * Submit a callable to the pool, in the correct tier.
+ *
+ * @param callable callable to submit
+ * @param tier tier to submit to
+ * @return Future tied to the callable's execution
+ */
+ public static Future submit(@NonNull final Callable callable, @NonNull final ThreadPoolServiceTier tier) {
+ return submit(callable, tier, NO_TIMEOUT);
+ }
+
+ /**
+ * Submit a callable to the pool, in the correct tier.
+ * Callable has a timeout of timeoutSeconds, which starts once the callable has been added to the pool.
+ *
+ * @param callable callable to submit
+ * @param tier tier to submit to
+ * @param timeoutSeconds timeout in seconds of the callable
+ * @return Future tied to the callable's execution
+ */
+ public static synchronized Future submit(@NonNull final Callable callable,
+ @NonNull final ThreadPoolServiceTier tier, final long timeoutSeconds) {
+ Preconditions.checkState(service.isPresent(), "No thread pool service is running");
+ Preconditions.checkState(!service.get().isShutdown, "Service is shutting down");
+ final ThreadPoolService service = ThreadPoolService.service.get();
+ final ThreadPoolServiceCallable threadPoolServiceCallable = service
+ .new ThreadPoolServiceCallable(callable, tier);
+ final ThreadPoolServiceFuture future = service
+ .new ThreadPoolServiceFuture(threadPoolServiceCallable, timeoutSeconds,
+ TimeUnit.SECONDS);
+ threadPoolServiceCallable.setFuture(future);
+ synchronized (service) {
+ if (tier.equals(ThreadPoolServiceTier.JOB_DAG_TIER)) {
+ if (service.canScheduleJobDag()) {
+ future.addWrappedFuture(service.scheduleJobDag(threadPoolServiceCallable));
+ } else {
+ service.queueJobDag(future);
+ }
+ } else if (tier.equals(ThreadPoolServiceTier.ACTIONS_TIER)) {
+ if (service.canScheduleAction()) {
+ future.addWrappedFuture(service.scheduleAction(threadPoolServiceCallable));
+ } else {
+ service.queueAction(future);
+ }
+ } else {
+ throw new JobRuntimeException("Trying to submit to illegal tier " + ThreadPoolServiceTier.JOB_DAG_TIER);
+ }
+ }
+ return future;
+ }
+
+ /**
+ * Check if the service is already initialized
+ * @return true if the service is ready to submit
+ */
+ public static boolean isInitialized() {
+ return service.isPresent();
+ }
+
+ private void queueAction(final ThreadPoolServiceFuture future) {
+ this.actionsQueue.add(future);
+ }
+
+ private Future scheduleAction(final Callable callable) {
+ this.currentActionsThreads.incrementAndGet();
+ this.currentThreads.incrementAndGet();
+ return this.threadPool.submit(callable);
+ }
+
+ private boolean canScheduleAction() {
+ return (this.currentThreads.get() < this.numThreads
+ // if we have fewer than reserved job dag threads, we can schedule
+ && (this.currentActionsThreads.get() < this.reservedActionsThreads
+ // if we have room to schedule a job dag without reaching into the actions reserve, we can schedule
+ || this.currentActionsThreads.get() < this.numThreads - this.reservedJobDagThreads));
+ }
+
+ private void queueJobDag(@NonNull final ThreadPoolServiceFuture future) {
+ this.jobDagQueue.add(future);
+ }
+
+ private Future scheduleJobDag(final Callable callable) {
+ this.currentThreads.incrementAndGet();
+ this.currentJobDagThreads.incrementAndGet();
+ return this.threadPool.submit(callable);
+ }
+
+ private boolean canScheduleJobDag() {
+ return (this.currentThreads.get() < this.numThreads
+ // if we have fewer than reserved job dag threads, we can schedule
+ && (this.currentJobDagThreads.get() < this.reservedJobDagThreads
+ // if we have room to schedule a job dag without reaching into the actions reserve, we can schedule
+ || this.currentJobDagThreads.get() < this.numThreads - this.reservedActionsThreads));
+ }
+
+ /**
+ * Will wait for all threads to finish their task unless forceShutdown is set to true in which case service will
+ * be forcefully shutdown.
+ */
+ public static void shutdown(final boolean forceShutdown) {
+ ThreadPoolService currentService = null;
+ synchronized (ThreadPoolService.class) {
+ if (!service.isPresent()) {
+ return;
+ }
+ currentService = service.get();
+ if (currentService.isShutdown) {
+ return;
+ }
+ log.info("Shutting down thread pool service");
+ currentService.isShutdown = true;
+ service = Optional.absent();
+ }
+ if (forceShutdown) {
+ log.error("forcefully shutting down waiting threads");
+ currentService.threadPool.shutdownNow();
+ } else {
+ currentService.shutdown();
+ }
+ }
+
+ private synchronized boolean removeFromQueue(@NonNull final ThreadPoolServiceFuture future) {
+ return this.actionsQueue.remove(future) || this.jobDagQueue.remove(future);
+ }
+
+ private void shutdown() {
+ int terminationChecks = 0;
+ while (terminationChecks < AWAIT_TERMINATION_ATTEMPTS && (!this.jobDagQueue.isEmpty()
+ || !this.actionsQueue.isEmpty())) {
+ log.info("waiting for tasks to clear out of queue.");
+ try {
+ Thread.sleep(AWAIT_TERMINATION_CHECK_INTERVAL_MS);
+ terminationChecks += 1;
+ } catch (InterruptedException e) {
+ // pass
+ }
+ }
+ this.threadPool.shutdown();
+ while (!this.threadPool.isTerminated()) {
+ try {
+ log.info("waiting for tasks to stop.");
+ if (!this.threadPool.awaitTermination(
+ (AWAIT_TERMINATION_ATTEMPTS - terminationChecks) * AWAIT_TERMINATION_CHECK_INTERVAL_MS,
+ TimeUnit.MILLISECONDS)) {
+ this.threadPool.shutdownNow();
+ }
+ } catch (InterruptedException e) {
+ throw new JobRuntimeException("Error waiting for threadpool to stop", e);
+ }
+ }
+ }
+
+ private synchronized void finishExecution(@NonNull final ThreadPoolServiceTier tier) {
+ if (tier.equals(ThreadPoolServiceTier.ACTIONS_TIER)) {
+ if (this.actionsQueue.isEmpty()) {
+ if (this.jobDagQueue.isEmpty() || !canScheduleJobDag()) {
+ // nothing else to run, just reduce number of threads
+ this.currentThreads.decrementAndGet();
+ this.currentActionsThreads.decrementAndGet();
+ } else {
+ // can now move a jobDag thread to an actions thread
+ submitFuture(this.jobDagQueue.poll());
+ this.currentActionsThreads.decrementAndGet();
+ this.currentJobDagThreads.incrementAndGet();
+ }
+ } else {
+ submitFuture(this.actionsQueue.poll());
+ }
+ } else if (tier.equals(ThreadPoolServiceTier.JOB_DAG_TIER)) {
+ if (this.jobDagQueue.isEmpty()) {
+ if (this.actionsQueue.isEmpty() || !canScheduleAction()) {
+ // nothing else to run, reduce count of running threads
+ this.currentThreads.decrementAndGet();
+ this.currentJobDagThreads.decrementAndGet();
+ } else {
+ // can now move an actions thread to a jobDag thread
+ submitFuture(this.actionsQueue.poll());
+ this.currentJobDagThreads.decrementAndGet();
+ this.currentActionsThreads.incrementAndGet();
+ }
+ } else {
+ submitFuture(this.jobDagQueue.poll());
+ }
+ } else {
+ throw new JobRuntimeException(String.format("Attempting to finish illegal tier %s", tier.toString()));
+ }
+ }
+
+ private void submitFuture(@NonNull final ThreadPoolServiceFuture future) {
+ future.addWrappedFuture(this.threadPool.submit(future.getWrappedCallable()));
+ }
+
+ private final class ThreadPoolServiceFuture implements Future {
+
+ @Getter
+ private final Callable wrappedCallable;
+ private Future wrappedFuture = null;
+ private LinkedBlockingQueue wrappedFutureWaitQ = new LinkedBlockingQueue<>();
+ private final long timeout;
+ private final TimeUnit timeUnit;
+ private boolean cancelled = false;
+ private boolean done = false;
+
+ private ThreadPoolServiceFuture(@NonNull final Callable wrappedCallable, final long timeout,
+ @NonNull final TimeUnit timeUnit) {
+ this.wrappedCallable = wrappedCallable;
+ this.timeout = timeout;
+ this.timeUnit = timeUnit;
+ }
+
+ public void addWrappedFuture(@NonNull final Future wrappedFuture) {
+ this.wrappedFutureWaitQ.offer(wrappedFuture);
+ }
+
+ private void waitForWrappedFuture(final long timeout,
+ @NonNull final TimeUnit timeUnit) throws TimeoutException {
+ if (this.wrappedFuture == null) {
+ try {
+ this.wrappedFuture = this.wrappedFutureWaitQ.poll(timeout, timeUnit);
+ if (this.wrappedFuture == null) {
+ throw new TimeoutException("no wrapped future received");
+ }
+ } catch (InterruptedException e) {
+ throw new JobRuntimeException(e);
+ }
+ }
+ }
+
+ private void waitForWrappedFuture() {
+ if (this.wrappedFuture == null) {
+ try {
+ this.wrappedFuture = this.wrappedFutureWaitQ.take();
+ } catch (InterruptedException e) {
+ throw new JobRuntimeException(e);
+ }
+ }
+ }
+
+ @Override
+ public boolean cancel(final boolean mayInterruptIfRunning) {
+ this.cancelled = removeFromQueue(this);
+ if (!this.wrappedFutureWaitQ.isEmpty()) {
+ waitForWrappedFuture();
+ }
+ if (!this.cancelled && this.wrappedFuture != null) {
+ this.cancelled = this.wrappedFuture.cancel(mayInterruptIfRunning);
+ }
+ return this.cancelled;
+ }
+
+ @Override
+ public boolean isCancelled() {
+ return this.cancelled;
+ }
+
+ /**
+ * @return true if the operation finished (with or without error) or it was cancelled else false.
+ */
+ @Override
+ public boolean isDone() {
+ return this.done || this.cancelled;
+ }
+
+ @Override
+ public Object get() throws InterruptedException, ExecutionException {
+ waitForWrappedFuture();
+ if (this.timeout != ThreadPoolService.NO_TIMEOUT) {
+ try {
+ return this.wrappedFuture.get(this.timeout, this.timeUnit);
+ } catch (TimeoutException e) {
+ throw new JobRuntimeException(e);
+ }
+ } else {
+ return this.wrappedFuture.get();
+ }
+ }
+
+ @Override
+ public Object get(final long timeout, @NonNull final TimeUnit timeUnit)
+ throws InterruptedException, ExecutionException, TimeoutException {
+ final long endTimeMs = System.currentTimeMillis() + timeUnit.toMillis(timeout);
+ waitForWrappedFuture(timeout, timeUnit);
+ return this.wrappedFuture.get(Math.max(1, endTimeMs - System.currentTimeMillis()), TimeUnit.MILLISECONDS);
+ }
+ }
+
+ private final class ThreadPoolServiceCallable implements Callable {
+
+ private final Callable wrappedCallable;
+ private final ThreadPoolServiceTier tier;
+ private ThreadPoolServiceFuture future;
+
+ private ThreadPoolServiceCallable(@NonNull final Callable wrappedCallable,
+ @NonNull final ThreadPoolServiceTier tier) {
+ this.wrappedCallable = wrappedCallable;
+ this.tier = tier;
+ }
+
+ @Override
+ public Object call() throws Exception {
+ try {
+ final Object result = this.wrappedCallable.call();
+ return result;
+ } finally {
+ finishExecution(this.tier);
+ this.future.done = true;
+ }
+ }
+
+ private void setFuture(@NonNull final ThreadPoolServiceFuture future) {
+ this.future = future;
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolServiceTier.java b/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolServiceTier.java
new file mode 100644
index 0000000..50e03bb
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/job/ThreadPoolServiceTier.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.job;
+
+public enum ThreadPoolServiceTier {
+ JOB_DAG_TIER,
+ ACTIONS_TIER
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/AbstractValue.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/AbstractValue.java
new file mode 100644
index 0000000..3d6a9c0
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/AbstractValue.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.metadata;
+
+import java.io.Serializable;
+
+/**
+ * {@link AbstractValue} wraps a generic data value type that is required to be serializable. The data represents
+ * some metadata that will be stored for a job.
+ * @param
+ */
+public abstract class AbstractValue implements Serializable {
+ abstract D getValue();
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSDatePartitionManager.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSDatePartitionManager.java
new file mode 100644
index 0000000..18b5c24
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSDatePartitionManager.java
@@ -0,0 +1,137 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.metadata;
+
+import com.google.common.base.Optional;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.utilities.DateUtil;
+import com.uber.marmaray.utilities.StringTypes;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.fs.FileSystem;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.ZoneId;
+import java.util.Date;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * {@link HDFSDatePartitionManager} is an extension of {@link HDFSPartitionManager}
+ * and provides functionality to specifically handle date partitions.
+ * All date times should be in UTC time and in the format YYYY-MM-DD
+ */
+@Slf4j
+public class HDFSDatePartitionManager extends HDFSPartitionManager {
+
+ private static final String DEFAULT_START_DATE = "1970-01-01";
+ private final String partitionKeyName;
+ private final boolean hasPartitionKeyInHDFSPartitionPath;
+ private final Optional startDate;
+
+ public HDFSDatePartitionManager(@NotEmpty final String metadataKey,
+ @NotEmpty final String genericBaseMetadataPath,
+ @NotEmpty final String genericBaseDataPath,
+ @NotEmpty final String partitionKeyName,
+ @NonNull final Optional startDate,
+ @NonNull final FileSystem fileSystem) throws IOException {
+ super(metadataKey, genericBaseMetadataPath, genericBaseDataPath, fileSystem);
+ this.partitionKeyName = partitionKeyName + StringTypes.EQUAL;
+ this.hasPartitionKeyInHDFSPartitionPath = hasPartitionKeyNameInPartition();
+ this.startDate = startDate;
+ log.info("HDFSDatePartitionManager has partitionKey in HDFS path: {}", this.hasPartitionKeyInHDFSPartitionPath);
+ }
+
+ @Override
+ public Optional getNextPartition() throws IOException {
+
+ if (this.isSinglePartition()) {
+ log.info("Next partition: {}", this.rawDataRootPath);
+ return Optional.of(this.rawDataRootPath);
+ } else {
+ final Optional lastCheckPoint = calculateLastCheckpoint();
+
+ if (lastCheckPoint.isPresent()) {
+ log.info("Last checkpoint: {}", lastCheckPoint.get().getValue());
+ } else {
+ log.info("No last checkpoint found");
+ }
+
+ final LocalDate startDate = getDefaultStartDate();
+
+ final Optional dt = lastCheckPoint.isPresent()
+ ? Optional.of(DateUtil.convertToUTCDate(
+ lastCheckPoint.get().getValue().replace(this.partitionKeyName, StringTypes.EMPTY)))
+ : Optional.absent();
+
+ final LocalDate compareDate = !dt.isPresent() || startDate.isAfter(dt.get()) ? startDate : dt.get();
+
+ final List existingPartitions = listSortedPartitionsAfterDate(compareDate);
+ if (!existingPartitions.isEmpty()) {
+ // get first partition after the checkpoint
+ final String nextPartition = this.hasPartitionKeyInHDFSPartitionPath
+ ? this.partitionKeyName + existingPartitions.get(0).toString()
+ : existingPartitions.get(0).toString();
+
+ log.info("Next partition to process: {}", nextPartition);
+ return Optional.of(nextPartition);
+ } else {
+ log.info("No partitions found to be processed");
+ return Optional.absent();
+ }
+ }
+ }
+
+ private LocalDate getDefaultStartDate() {
+ final ZoneId UTC = ZoneId.of("Z");
+ final LocalDate ld = this.startDate.isPresent() ? this.startDate.get().toInstant().atZone(UTC).toLocalDate()
+ : LocalDate.parse(DEFAULT_START_DATE);
+ log.info("Default start date: {}", ld.toString());
+ return ld;
+ }
+ /**
+ * Returns the partitions in sorted ascending order which are after the date value
+ * @param localDate
+ * @return
+ * @throws IOException
+ */
+ private List listSortedPartitionsAfterDate(final LocalDate localDate) throws IOException {
+ final LocalDate startDate = localDate.plusDays(1);
+
+ final List partitions = getExistingPartitions()
+ .stream()
+ .map(dt -> DateUtil.convertToUTCDate(dt.replace(this.partitionKeyName, StringTypes.EMPTY)))
+ .filter(dt -> dt.compareTo(startDate) >= 0)
+ .collect(Collectors.toList());
+
+ return partitions;
+ }
+
+ private boolean hasPartitionKeyNameInPartition() {
+ try {
+ final boolean hasPartitionKey = getExistingPartitions()
+ .stream()
+ .anyMatch(partition -> partition.startsWith(this.partitionKeyName));
+ return hasPartitionKey;
+ } catch (IOException e) {
+ throw new JobRuntimeException(String.format("Unable to read existing partitions in the HDFS Path {}",
+ this.rawDataRootPath));
+ }
+ }
+}
diff --git a/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSMetadataManager.java b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSMetadataManager.java
new file mode 100644
index 0000000..65f8ae0
--- /dev/null
+++ b/marmaray/src/main/java/com/uber/marmaray/common/metadata/HDFSMetadataManager.java
@@ -0,0 +1,278 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc.
+ * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+ * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
+ * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
+ * permit persons to whom the Software is furnished to do so, subject to the following conditions:
+ *
+ * The above copyright notice and this permission notice shall be included in all copies or substantial portions
+ * of the Software.
+ *
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO
+ * THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+ * TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+ * IN THE SOFTWARE.
+ */
+package com.uber.marmaray.common.metadata;
+
+import com.github.rholder.retry.RetryException;
+import com.github.rholder.retry.Retryer;
+import com.github.rholder.retry.RetryerBuilder;
+import com.github.rholder.retry.StopStrategies;
+import com.github.rholder.retry.WaitStrategies;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.uber.marmaray.common.exceptions.JobRuntimeException;
+import com.uber.marmaray.common.exceptions.MetadataException;
+import com.uber.marmaray.utilities.FSUtils;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Strings;
+import org.hibernate.validator.constraints.NotEmpty;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+@Slf4j
+/**
+ * {@link HDFSMetadataManager} implements the {@link IMetadataManager} interface, adding the capability
+ * to put and retrieve generic metadata from HDFS. All metadata will be stored under a single filename
+ * with the name = System.currentTimeMillis() when {@link IMetadataManager#saveChanges()} is invoked
+ */
+public class HDFSMetadataManager implements IMetadataManager {
+ public static final int DEFAULT_NUM_METADATA_FILES_TO_RETAIN = 5;
+ private static final int SERIALIZATION_VERSION = 1;
+ private static final Comparator byDateAsc =
+ Comparator.comparingLong(f1 -> Long.parseLong(f1.getPath().getName()));
+
+ // Using a thread-safe HashMap doesn't really provide any protection against jobs from other or same
+ // customers running jobs against the same metadata directory. We eventually want to take locks on
+ // a directory (possivly via ZooKeeper) so only one job can operate at a time per job name.
+ private final Map metadataMap;
+
+ @NonNull
+ private final FileSystem fileSystem;
+
+ @NotEmpty @Getter
+ private final String baseMetadataPath;
+
+ /*
+ * If it is able to update {@link #shouldSaveChanges} from true to false; then only it will create new
+ * metadata file and will save information in it.
+ */
+ @Getter
+ private final AtomicBoolean shouldSaveChanges;
+
+ public HDFSMetadataManager(@NonNull final FileSystem fs, @NotEmpty final String baseMetadataPath,
+ @NonNull final AtomicBoolean shouldSaveChanges)
+ throws IOException {
+ this.fileSystem = fs;
+ this.baseMetadataPath = baseMetadataPath;
+ this.shouldSaveChanges = shouldSaveChanges;
+ this.metadataMap = loadMetadata();
+ }
+
+ @Override
+ public void set(@NotEmpty final String key, @NonNull final StringValue value) throws MetadataException {
+ Preconditions.checkArgument(!Strings.isNullOrEmpty(key));
+ this.metadataMap.put(key, value);
+ }
+
+ @Override
+ public Optional remove(@NotEmpty final String key) {
+ return Optional.fromNullable(this.metadataMap.remove(key));
+ }
+
+ @Override
+ public Optional get(@NotEmpty final String key) throws MetadataException {
+ Preconditions.checkArgument(!Strings.isNullOrEmpty(key));
+ return this.metadataMap.containsKey(key) ? Optional.of(this.metadataMap.get(key)) : Optional.absent();
+ }
+
+ @Override
+ public Set getAllKeys() {
+ return this.metadataMap.keySet();
+ }
+
+ /**
+ * This method will first attempt to save the metadata file to a temp file. Once that succeeds it will
+ * copy the file to it's permanent location without the temp extension.
+ *
+ * @return
+ * @throws IOException
+ */
+ @Override
+ public void saveChanges() {
+ if (this.shouldSaveChanges.compareAndSet(true, false)) {
+ log.info("Saving checkpoint information");
+ } else {
+ log.info("Checkpoint info is already saved. Not saving it again.");
+ return;
+ }
+
+ final Callable callable = () -> {
+ writeMetadataFile();
+ pruneMetadataFiles();
+ return null;
+ };
+
+ final Retryer