You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gobblin.apache.org by ab...@apache.org on 2018/03/21 08:30:19 UTC

[01/50] incubator-gobblin git commit: [GOBBLIN-361] Support Nested nullable Record type for JDBCWriter

Repository: incubator-gobblin
Updated Branches:
  refs/heads/0.12.0 2951fd267 -> b7f123f77


[GOBBLIN-361] Support Nested nullable Record type for JDBCWriter

Closes #2233 from jinhyukchang/master


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/c6b3824a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/c6b3824a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/c6b3824a

Branch: refs/heads/0.12.0
Commit: c6b3824aac01a61cb492a5cd5b672fc2fea1b09f
Parents: ec85298
Author: Jin Hyuk Chang <jn...@linkedin.com>
Authored: Tue Jan 23 10:17:10 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Tue Jan 23 10:17:10 2018 -0800

----------------------------------------------------------------------
 .../filter/AvroFieldsPickConverter.java         |  55 +++++-
 .../filter/AvroFieldsPickConverterTest.java     |  37 +++-
 .../converted_pickfields_nested_with_union.avro | Bin 0 -> 678 bytes
 .../converted_pickfields_nested_with_union.avsc |  47 +++++
 .../converter/pickfields_nested_with_union.avro | Bin 0 -> 1264 bytes
 .../converter/pickfields_nested_with_union.avsc |  33 ++++
 .../jdbc/AvroToJdbcEntryConverter.java          | 150 ++++++++------
 .../jdbc/AvroToJdbcEntryConverterTest.java      | 121 ++++++++++++
 .../converter/pickfields_nested_with_union.avro | Bin 0 -> 1264 bytes
 .../converter/pickfields_nested_with_union.avsc |  33 ++++
 .../converter/pickfields_nested_with_union.json | 194 +++++++++++++++++++
 11 files changed, 608 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-core/src/main/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverter.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverter.java b/gobblin-core/src/main/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverter.java
index c7e2db5..74ed3f3 100644
--- a/gobblin-core/src/main/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverter.java
+++ b/gobblin-core/src/main/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverter.java
@@ -21,8 +21,10 @@ import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
+import org.apache.avro.Schema.Type;
 import org.apache.avro.generic.GenericRecord;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -31,6 +33,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+
 import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.WorkUnitState;
 import org.apache.gobblin.converter.AvroToAvroConverterBase;
@@ -133,29 +136,71 @@ public class AvroFieldsPickConverter extends AvroToAvroConverterBase {
     return createSchemaHelper(schema, root);
   }
 
-  private static Schema createSchemaHelper(Schema inputSchema, TrieNode node) {
-    Schema newRecord = Schema.createRecord(inputSchema.getName(), inputSchema.getDoc(), inputSchema.getNamespace(),
-        inputSchema.isError());
+  private static Schema createSchemaHelper(final Schema inputSchema, TrieNode node) {
     List<Field> newFields = Lists.newArrayList();
     for (TrieNode child : node.children.values()) {
-      Field innerSrcField = inputSchema.getField(child.val);
-      Preconditions.checkNotNull(innerSrcField, child.val + " does not exist under " + inputSchema);
+      Schema recordSchema = getActualRecord(inputSchema);
+      Field innerSrcField = recordSchema.getField(child.val);
+      Preconditions.checkNotNull(innerSrcField, child.val + " does not exist under " + recordSchema);
 
       if (child.children.isEmpty()) { //Leaf
         newFields.add(
             new Field(innerSrcField.name(), innerSrcField.schema(), innerSrcField.doc(), innerSrcField.defaultValue()));
       } else {
         Schema innerSrcSchema = innerSrcField.schema();
+
         Schema innerDestSchema = createSchemaHelper(innerSrcSchema, child); //Recurse of schema
         Field innerDestField =
             new Field(innerSrcField.name(), innerDestSchema, innerSrcField.doc(), innerSrcField.defaultValue());
         newFields.add(innerDestField);
       }
     }
+
+    if (Type.UNION.equals(inputSchema.getType())) {
+      Preconditions.checkArgument(inputSchema.getTypes().size() <= 2,
+          "For union type in nested record, it should only have NULL and Record type");
+
+      Schema recordSchema = getActualRecord(inputSchema);
+      Schema newRecord = Schema.createRecord(recordSchema.getName(), recordSchema.getDoc(), recordSchema.getNamespace(),
+          recordSchema.isError());
+      newRecord.setFields(newFields);
+      if (inputSchema.getTypes().size() == 1) {
+        return Schema.createUnion(newRecord);
+      }
+      return Schema.createUnion(Lists.newArrayList(Schema.create(Type.NULL), newRecord));
+    }
+
+    Schema newRecord = Schema.createRecord(inputSchema.getName(), inputSchema.getDoc(), inputSchema.getNamespace(),
+        inputSchema.isError());
     newRecord.setFields(newFields);
     return newRecord;
   }
 
+  /**
+   * For the schema that is a UNION type with NULL and Record type, it provides Records type.
+   * @param inputSchema
+   * @return
+   */
+  private static Schema getActualRecord(Schema inputSchema) {
+    if (Type.RECORD.equals(inputSchema.getType())) {
+      return inputSchema;
+    }
+
+    Preconditions.checkArgument(Type.UNION.equals(inputSchema.getType()), "Nested schema is only support with either record or union type of null with record");
+    Preconditions.checkArgument(inputSchema.getTypes().size() <= 2,
+        "For union type in nested record, it should only have NULL and Record type");
+
+    for (Schema inner : inputSchema.getTypes()) {
+      if (Type.NULL.equals(inner.getType())) {
+        continue;
+      }
+      Preconditions.checkArgument(Type.RECORD.equals(inner.getType()), "For union type in nested record, it should only have NULL and Record type");
+      return inner;
+
+    }
+    throw new IllegalArgumentException(inputSchema + " is not supported.");
+  }
+
   private static TrieNode buildTrie(List<String> fqns) {
     TrieNode root = new TrieNode(null);
     for (String fqn : fqns) {

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-core/src/test/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverterTest.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverterTest.java b/gobblin-core/src/test/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverterTest.java
index 009bcc7..a2f71f5 100644
--- a/gobblin-core/src/test/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverterTest.java
+++ b/gobblin-core/src/test/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverterTest.java
@@ -17,12 +17,18 @@
 
 package org.apache.gobblin.converter.filter;
 
+import java.io.File;
+
 import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.WorkUnitState;
 import org.apache.gobblin.converter.SchemaConversionException;
-
 import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.commons.io.FileUtils;
 import org.skyscreamer.jsonassert.JSONAssert;
+import org.testng.Assert;
 import org.testng.annotations.Test;
 
 @Test(groups = { "gobblin.converter.filter" })
@@ -59,4 +65,33 @@ public class AvroFieldsPickConverterTest {
       JSONAssert.assertEquals(expected.toString(), converted.toString(), false);
     }
   }
+
+  @Test
+  public void testFieldsPickWithNestedRecord() throws Exception {
+    Schema inputSchema = new Schema.Parser().parse(getClass().getResourceAsStream("/converter/pickfields_nested_with_union.avsc"));
+
+    WorkUnitState workUnitState = new WorkUnitState();
+    workUnitState.setProp(ConfigurationKeys.CONVERTER_AVRO_FIELD_PICK_FIELDS, "name,favorite_number,nested1.nested1_string,nested1.nested2_union.nested2_string");
+
+    try (AvroFieldsPickConverter converter = new AvroFieldsPickConverter()) {
+      Schema convertedSchema = converter.convertSchema(inputSchema, workUnitState);
+      Schema expectedSchema = new Schema.Parser().parse(getClass().getResourceAsStream("/converter/converted_pickfields_nested_with_union.avsc"));
+      JSONAssert.assertEquals(expectedSchema.toString(), convertedSchema.toString(), false);
+
+      try (DataFileReader<GenericRecord> srcDataFileReader = new DataFileReader<GenericRecord>(
+              new File(getClass().getResource("/converter/pickfields_nested_with_union.avro").toURI()),
+                  new GenericDatumReader<GenericRecord>(inputSchema));
+          DataFileReader<GenericRecord> expectedDataFileReader = new DataFileReader<GenericRecord>(
+              new File(getClass().getResource("/converter/converted_pickfields_nested_with_union.avro").toURI()),
+                  new GenericDatumReader<GenericRecord>(expectedSchema));) {
+
+        while (expectedDataFileReader.hasNext()) {
+          GenericRecord expected = expectedDataFileReader.next();
+          GenericRecord actual = converter.convertRecord(convertedSchema, srcDataFileReader.next(), workUnitState).iterator().next();
+          Assert.assertEquals(actual, expected);
+        }
+        Assert.assertTrue(!srcDataFileReader.hasNext());
+      }
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-core/src/test/resources/converter/converted_pickfields_nested_with_union.avro
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/converter/converted_pickfields_nested_with_union.avro b/gobblin-core/src/test/resources/converter/converted_pickfields_nested_with_union.avro
new file mode 100644
index 0000000..5d63a1a
Binary files /dev/null and b/gobblin-core/src/test/resources/converter/converted_pickfields_nested_with_union.avro differ

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-core/src/test/resources/converter/converted_pickfields_nested_with_union.avsc
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/converter/converted_pickfields_nested_with_union.avsc b/gobblin-core/src/test/resources/converter/converted_pickfields_nested_with_union.avsc
new file mode 100644
index 0000000..cdfc283
--- /dev/null
+++ b/gobblin-core/src/test/resources/converter/converted_pickfields_nested_with_union.avsc
@@ -0,0 +1,47 @@
+{
+  "type": "record",
+  "name": "User",
+  "namespace": "example.avro",
+  "fields": [
+    {
+      "name": "name",
+      "type": "string"
+    },
+    {
+      "name": "favorite_number",
+      "type": [
+        "int",
+        "null"
+      ]
+    },
+    {
+      "name": "nested1",
+      "type": {
+        "type": "record",
+        "name": "dummy_nested1",
+        "fields": [
+          {
+            "name": "nested1_string",
+            "type": "string"
+          },
+          {
+            "name": "nested2_union",
+            "type": [
+              "null",
+              {
+                "type": "record",
+                "name": "dummy_nested2",
+                "fields": [
+                  {
+                    "name": "nested2_string",
+                    "type": "string"
+                  }
+                ]
+              }
+            ]
+          }
+        ]
+      }
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-core/src/test/resources/converter/pickfields_nested_with_union.avro
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/converter/pickfields_nested_with_union.avro b/gobblin-core/src/test/resources/converter/pickfields_nested_with_union.avro
new file mode 100644
index 0000000..b6e8d63
Binary files /dev/null and b/gobblin-core/src/test/resources/converter/pickfields_nested_with_union.avro differ

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-core/src/test/resources/converter/pickfields_nested_with_union.avsc
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/converter/pickfields_nested_with_union.avsc b/gobblin-core/src/test/resources/converter/pickfields_nested_with_union.avsc
new file mode 100644
index 0000000..bbe402d
--- /dev/null
+++ b/gobblin-core/src/test/resources/converter/pickfields_nested_with_union.avsc
@@ -0,0 +1,33 @@
+{"namespace": "example.avro",
+ "type": "record",
+ "name": "User",
+ "fields": [
+     {"name": "name", "type": "string"},
+     {"name": "favorite_number",  "type": ["int", "null"]},
+     {"name": "favorite_color", "type": ["string", "null"]},
+     {"name": "date_of_birth", "type": "long"},
+     {"name": "last_modified", "type": "long"},
+     {"name": "created", "type": "long"},
+     {"name": "nested1",
+      "type" : {
+                   "type": "record",
+                   "name": "dummy_nested1",
+                   "fields": [
+                       {"name": "nested1_string", "type": "string"},
+                       {"name": "nested1_int",  "type": ["int", "null"]},
+                       {"name": "nested2_union", "type": ["null", {
+                                                               "type" : "record",
+                                                               "name" : "dummy_nested2",
+                                                               "fields": [
+                                                                   {"name": "nested2_string", "type": "string"},
+                                                                   {"name": "nested2_int",  "type": ["int", "null"]}
+                                                               ]
+                                                           }
+                                                       ]
+                       }
+                   ]
+               }
+     }
+ ]
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-modules/gobblin-sql/src/main/java/org/apache/gobblin/converter/jdbc/AvroToJdbcEntryConverter.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-sql/src/main/java/org/apache/gobblin/converter/jdbc/AvroToJdbcEntryConverter.java b/gobblin-modules/gobblin-sql/src/main/java/org/apache/gobblin/converter/jdbc/AvroToJdbcEntryConverter.java
index b787de6..5b7e89d 100644
--- a/gobblin-modules/gobblin-sql/src/main/java/org/apache/gobblin/converter/jdbc/AvroToJdbcEntryConverter.java
+++ b/gobblin-modules/gobblin-sql/src/main/java/org/apache/gobblin/converter/jdbc/AvroToJdbcEntryConverter.java
@@ -20,9 +20,12 @@ package org.apache.gobblin.converter.jdbc;
 import java.sql.Date;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.avro.Schema;
@@ -36,6 +39,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
@@ -68,6 +72,11 @@ import org.apache.gobblin.writer.commands.JdbcWriterCommandsFactory;
 public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema, GenericRecord, JdbcEntryData> {
 
   public static final String CONVERTER_AVRO_JDBC_DATE_FIELDS = "converter.avro.jdbc.date_fields";
+  private static final String AVRO_NESTED_COLUMN_DELIMITER = ".";
+  private static final String JDBC_FLATTENED_COLUMN_DELIMITER = "_";
+  private static final String AVRO_NESTED_COLUMN_DELIMITER_REGEX_COMPATIBLE = "\\.";
+  private static final Splitter AVRO_RECORD_LEVEL_SPLITTER = Splitter.on(AVRO_NESTED_COLUMN_DELIMITER).omitEmptyStrings();
+
 
   private static final Logger LOG = LoggerFactory.getLogger(AvroToJdbcEntryConverter.class);
   private static final Map<Type, JdbcType> AVRO_TYPE_JDBC_TYPE_MAPPING =
@@ -83,6 +92,7 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
       ImmutableSet.<Type> builder()
         .addAll(AVRO_TYPE_JDBC_TYPE_MAPPING.keySet())
         .add(Type.UNION)
+        .add(Type.RECORD)
         .build();
   private static final Set<JdbcType> JDBC_SUPPORTED_TYPES =
       ImmutableSet.<JdbcType> builder()
@@ -93,7 +103,7 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
         .build();
 
   private Optional<Map<String, String>> avroToJdbcColPairs = Optional.absent();
-  private Optional<Map<String, String>> jdbcToAvroColPairs = Optional.absent();
+  private Map<String, String> jdbcToAvroColPairs = new HashMap<>();
 
   public AvroToJdbcEntryConverter() {
     super();
@@ -128,7 +138,6 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
           jdbcToAvroBuilder.put(entry.getValue().getAsString(), entry.getKey());
         }
         this.avroToJdbcColPairs = Optional.of((Map<String, String>) avroToJdbcBuilder.build());
-        this.jdbcToAvroColPairs = Optional.of((Map<String, String>) jdbcToAvroBuilder.build());
       }
     }
     return this;
@@ -139,7 +148,7 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
    *
    * Few precondition to the Avro schema
    * 1. Avro schema should have one entry type record at first depth.
-   * 2. Avro schema can recurse by having record inside record. As RDBMS structure is not recursive, this is not allowed.
+   * 2. Avro schema can recurse by having record inside record.
    * 3. Supported Avro primitive types and conversion
    *  boolean --> java.lang.Boolean
    *  int --> java.lang.Integer
@@ -150,9 +159,9 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
    *  string --> java.lang.String
    *  null: only allowed if it's within union (see complex types for more details)
    * 4. Supported Avro complex types
-   *  Records: Only first level depth can have Records type. Basically converter will peel out Records type and start with 2nd level.
+   *  Records: Supports nested record type as well.
    *  Enum --> java.lang.String
-   *  Unions --> Only allowed if it have one primitive type in it or null type with one primitive type where null will be ignored.
+   *  Unions --> Only allowed if it have one primitive type in it, along with Record type, or null type with one primitive type where null will be ignored.
    *  Once Union is narrowed down to one primitive type, it will follow conversion of primitive type above.
    * {@inheritDoc}
    *
@@ -167,6 +176,10 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
   @Override
   public JdbcEntrySchema convertSchema(Schema inputSchema, WorkUnitState workUnit) throws SchemaConversionException {
     LOG.info("Converting schema " + inputSchema);
+    Preconditions.checkArgument(Type.RECORD.equals(inputSchema.getType()),
+        "%s is expected for the first level element in Avro schema %s",
+        Type.RECORD, inputSchema);
+
     Map<String, Type> avroColumnType = flatten(inputSchema);
     String jsonStr = Preconditions.checkNotNull(workUnit.getProp(CONVERTER_AVRO_JDBC_DATE_FIELDS));
     java.lang.reflect.Type typeOfMap = new TypeToken<Map<String, JdbcType>>() {}.getType();
@@ -175,7 +188,8 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
 
     List<JdbcEntryMetaDatum> jdbcEntryMetaData = Lists.newArrayList();
     for (Map.Entry<String, Type> avroEntry : avroColumnType.entrySet()) {
-      String colName = tryConvertColumn(avroEntry.getKey(), this.avroToJdbcColPairs);
+      String colName = tryConvertAvroColNameToJdbcColName(avroEntry.getKey());
+
       JdbcType JdbcType = dateColumnMapping.get(colName);
       if (JdbcType == null) {
         JdbcType = AVRO_TYPE_JDBC_TYPE_MAPPING.get(avroEntry.getValue());
@@ -190,15 +204,36 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
     return converted;
   }
 
-  private static String tryConvertColumn(String key, Optional<Map<String, String>> mapping) {
-    if (!mapping.isPresent()) {
-      return key;
+  /**
+   * Convert Avro column name to JDBC column name. If name mapping is defined, follow it. Otherwise, just return avro column name,
+   * while replacing nested column delimiter, dot, to underscore.
+   * This method also updates, mapping from JDBC column name to Avro column name for reverse look up.
+   * @param avroColName
+   * @return
+   */
+  private String tryConvertAvroColNameToJdbcColName(String avroColName) {
+    if (!avroToJdbcColPairs.isPresent()) {
+      String converted = avroColName.replaceAll(AVRO_NESTED_COLUMN_DELIMITER_REGEX_COMPATIBLE, JDBC_FLATTENED_COLUMN_DELIMITER);
+      jdbcToAvroColPairs.put(converted, avroColName);
+      return converted;
     }
 
-    String converted = mapping.get().get(key);
-    return converted != null ? converted : key;
+    String converted = avroToJdbcColPairs.get().get(avroColName);
+    converted = converted != null ? converted : avroColName;
+    jdbcToAvroColPairs.put(converted, avroColName);
+    return converted;
+  }
+
+  /**
+   * Provides JDBC column name based on Avro column name. It's a one liner method but contains knowledge on where the mapping is.
+   * @param colName
+   * @return
+   */
+  private String convertJdbcColNameToAvroColName(String colName) {
+    return Preconditions.checkNotNull(jdbcToAvroColPairs.get(colName));
   }
 
+
   /**
    * Flattens Avro's (possibly recursive) structure and provides field name and type.
    * It assumes that the leaf level field name has unique name.
@@ -208,41 +243,44 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
    */
   private static Map<String, Type> flatten(Schema schema) throws SchemaConversionException {
     Map<String, Type> flattened = new LinkedHashMap<>();
-    if (!Type.RECORD.equals(schema.getType())) {
-      throw new SchemaConversionException(
-          Type.RECORD + " is expected for the first level element in Avro schema " + schema);
-    }
+    Schema recordSchema = determineType(schema);
 
-    for (Field f : schema.getFields()) {
-      produceFlattenedHelper(f.schema(), f, flattened);
+    Preconditions.checkArgument(Type.RECORD.equals(recordSchema.getType()), "%s is expected. Schema: %s",
+        Type.RECORD, recordSchema);
+
+    for (Field f : recordSchema.getFields()) {
+      produceFlattenedHelper(f, flattened);
     }
     return flattened;
   }
 
-  private static void produceFlattenedHelper(Schema schema, Field field, Map<String, Type> flattened)
+  private static void produceFlattenedHelper(Field field, Map<String, Type> flattened)
       throws SchemaConversionException {
-    if (Type.RECORD.equals(schema.getType())) {
-      throw new SchemaConversionException(Type.RECORD + " is only allowed for first level.");
+    Schema actualSchema = determineType(field.schema());
+    if (Type.RECORD.equals(actualSchema.getType())) {
+      Map<String, Type> map = flatten(actualSchema);
+      for (Entry<String, Type> entry : map.entrySet()) {
+        String key = String.format("%s" + AVRO_NESTED_COLUMN_DELIMITER + "%s", field.name(), entry.getKey());
+        Type existing = flattened.put(key, entry.getValue());
+        Preconditions.checkArgument(existing == null, "Duplicate name detected in Avro schema. Field: " + key);
+      }
+      return;
     }
 
-    Type t = determineType(schema);
-    if (field == null) {
-      throw new IllegalArgumentException("Invalid Avro schema, no name has been assigned to " + schema);
-    }
-    Type existing = flattened.put(field.name(), t);
+    Type existing = flattened.put(field.name(), actualSchema.getType());
     if (existing != null) {
       //No duplicate name allowed when flattening (not considering name space we don't have any assumption between namespace and actual database field name)
       throw new SchemaConversionException("Duplicate name detected in Avro schema. " + field.name());
     }
   }
 
-  private static Type determineType(Schema schema) throws SchemaConversionException {
+  private static Schema determineType(Schema schema) throws SchemaConversionException {
     if (!AVRO_SUPPORTED_TYPES.contains(schema.getType())) {
       throw new SchemaConversionException(schema.getType() + " is not supported");
     }
 
     if (!Type.UNION.equals(schema.getType())) {
-      return schema.getType();
+      return schema;
     }
 
     //For UNION, only supported avro type with NULL is allowed.
@@ -251,20 +289,13 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
       throw new SchemaConversionException("More than two types are not supported " + schemas);
     }
 
-    Type t = null;
     for (Schema s : schemas) {
       if (Type.NULL.equals(s.getType())) {
         continue;
       }
-      if (t == null) {
-        t = s.getType();
-      } else {
-        throw new SchemaConversionException("Union type of " + schemas + " is not supported.");
-      }
-    }
-    if (t != null) {
-      return t;
+      return s;
     }
+
     throw new SchemaConversionException("Cannot determine type of " + schema);
   }
 
@@ -276,12 +307,14 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
     }
     List<JdbcEntryDatum> jdbcEntryData = Lists.newArrayList();
     for (JdbcEntryMetaDatum entry : outputSchema) {
-      final String colName = entry.getColumnName();
+      final String jdbcColName = entry.getColumnName();
       final JdbcType jdbcType = entry.getJdbcType();
-      final Object val = record.get(tryConvertColumn(colName, this.jdbcToAvroColPairs));
+
+      String avroColName = convertJdbcColNameToAvroColName(jdbcColName);
+      final Object val = avroRecordValueGet(record, AVRO_RECORD_LEVEL_SPLITTER.split(avroColName).iterator());
 
       if (val == null) {
-        jdbcEntryData.add(new JdbcEntryDatum(colName, null));
+        jdbcEntryData.add(new JdbcEntryDatum(jdbcColName, null));
         continue;
       }
 
@@ -291,35 +324,23 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
 
       switch (jdbcType) {
         case VARCHAR:
-          jdbcEntryData.add(new JdbcEntryDatum(colName, val.toString()));
+          jdbcEntryData.add(new JdbcEntryDatum(jdbcColName, val.toString()));
           continue;
         case INTEGER:
         case BOOLEAN:
         case BIGINT:
         case FLOAT:
         case DOUBLE:
-          jdbcEntryData.add(new JdbcEntryDatum(colName, val));
+          jdbcEntryData.add(new JdbcEntryDatum(jdbcColName, val));
           continue;
-        //        case BOOLEAN:
-        //          jdbcEntryData.add(new JdbcEntryDatum(colName, Boolean.valueOf((boolean) val)));
-        //          continue;
-        //        case BIGINT:
-        //          jdbcEntryData.add(new JdbcEntryDatum(colName, Long.valueOf((long) val)));
-        //          continue;
-        //        case FLOAT:
-        //          jdbcEntryData.add(new JdbcEntryDatum(colName, Float.valueOf((float) val)));
-        //          continue;
-        //        case DOUBLE:
-        //          jdbcEntryData.add(new JdbcEntryDatum(colName, Double.valueOf((double) val)));
-        //          continue;
         case DATE:
-          jdbcEntryData.add(new JdbcEntryDatum(colName, new Date((long) val)));
+          jdbcEntryData.add(new JdbcEntryDatum(jdbcColName, new Date((long) val)));
           continue;
         case TIME:
-          jdbcEntryData.add(new JdbcEntryDatum(colName, new Time((long) val)));
+          jdbcEntryData.add(new JdbcEntryDatum(jdbcColName, new Time((long) val)));
           continue;
         case TIMESTAMP:
-          jdbcEntryData.add(new JdbcEntryDatum(colName, new Timestamp((long) val)));
+          jdbcEntryData.add(new JdbcEntryDatum(jdbcColName, new Timestamp((long) val)));
           continue;
         default:
           throw new DataConversionException(jdbcType + " is not supported");
@@ -332,6 +353,23 @@ public class AvroToJdbcEntryConverter extends Converter<Schema, JdbcEntrySchema,
     return new SingleRecordIterable<>(converted);
   }
 
+  private Object avroRecordValueGet(GenericRecord record, Iterator<String> recordNameIterator) {
+    String name = recordNameIterator.next();
+    Object val = record.get(name);
+    if (val == null) {
+      //Either leaf value is null or nested Record (represented as UNION) is null
+      return null;
+    }
+    if (!recordNameIterator.hasNext()) {
+      //Leaf
+      return val;
+    }
+
+    //Recurse
+    return avroRecordValueGet((GenericRecord) val, recordNameIterator);
+  }
+
+  @Override
   public ConverterInitializer getInitializer(State state, WorkUnitStream workUnits, int branches, int branchId) {
     JdbcWriterCommandsFactory factory = new JdbcWriterCommandsFactory();
     if (workUnits.isSafeToMaterialize()) {

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-modules/gobblin-sql/src/test/java/org/apache/gobblin/converter/jdbc/AvroToJdbcEntryConverterTest.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-sql/src/test/java/org/apache/gobblin/converter/jdbc/AvroToJdbcEntryConverterTest.java b/gobblin-modules/gobblin-sql/src/test/java/org/apache/gobblin/converter/jdbc/AvroToJdbcEntryConverterTest.java
index de1f0a3..a835d89 100644
--- a/gobblin-modules/gobblin-sql/src/test/java/org/apache/gobblin/converter/jdbc/AvroToJdbcEntryConverterTest.java
+++ b/gobblin-modules/gobblin-sql/src/test/java/org/apache/gobblin/converter/jdbc/AvroToJdbcEntryConverterTest.java
@@ -18,29 +18,50 @@
 package org.apache.gobblin.converter.jdbc;
 
 import static org.mockito.Mockito.*;
+
 import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.State;
 import org.apache.gobblin.configuration.WorkUnitState;
+import org.apache.gobblin.converter.DataConversionException;
 import org.apache.gobblin.converter.SchemaConversionException;
 import org.apache.gobblin.publisher.JdbcPublisher;
 import org.apache.gobblin.writer.Destination.DestinationType;
 import org.apache.gobblin.writer.commands.JdbcWriterCommands;
 import org.apache.gobblin.writer.commands.JdbcWriterCommandsFactory;
 
+import java.io.File;
 import java.io.IOException;
+import java.io.InputStreamReader;
+import java.lang.reflect.Type;
+import java.net.URISyntaxException;
 import java.sql.Connection;
+import java.sql.Date;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 import com.google.common.collect.Maps;
 import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonArray;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+import com.google.gson.JsonPrimitive;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
 
 @Test(groups = {"gobblin.converter"})
 public class AvroToJdbcEntryConverterTest {
@@ -130,4 +151,104 @@ public class AvroToJdbcEntryConverterTest {
 
     Assert.assertEquals(expected, actual);
   }
+
+  @Test
+  public void testFlattening() throws IOException, SchemaConversionException, SQLException, URISyntaxException, DataConversionException {
+    final String db = "db";
+    final String table = "users";
+    Map<String, JdbcType> dateColums = new HashMap<>();
+    dateColums.put("date_of_birth", JdbcType.DATE);
+    dateColums.put("last_modified", JdbcType.TIME);
+    dateColums.put("created", JdbcType.TIMESTAMP);
+
+    JdbcWriterCommands mockWriterCommands = mock(JdbcWriterCommands.class);
+    when(mockWriterCommands.retrieveDateColumns(db, table)).thenReturn(dateColums);
+
+    JdbcWriterCommandsFactory factory = mock(JdbcWriterCommandsFactory.class);
+    when(factory.newInstance(any(State.class), any(Connection.class))).thenReturn(mockWriterCommands);
+
+    List<JdbcEntryMetaDatum> jdbcEntryMetaData = new ArrayList<>();
+    jdbcEntryMetaData.add(new JdbcEntryMetaDatum("name", JdbcType.VARCHAR));
+    jdbcEntryMetaData.add(new JdbcEntryMetaDatum("favorite_number", JdbcType.VARCHAR));
+    jdbcEntryMetaData.add(new JdbcEntryMetaDatum("favorite_color", JdbcType.VARCHAR));
+    jdbcEntryMetaData.add(new JdbcEntryMetaDatum("date_of_birth", JdbcType.DATE));
+    jdbcEntryMetaData.add(new JdbcEntryMetaDatum("last_modified", JdbcType.TIME));
+    jdbcEntryMetaData.add(new JdbcEntryMetaDatum("created", JdbcType.TIMESTAMP));
+    jdbcEntryMetaData.add(new JdbcEntryMetaDatum("nested1_nested1_string", JdbcType.VARCHAR));
+    jdbcEntryMetaData.add(new JdbcEntryMetaDatum("nested1_nested1_int", JdbcType.INTEGER));
+    jdbcEntryMetaData.add(new JdbcEntryMetaDatum("nested1_nested2_union_nested2_string", JdbcType.VARCHAR));
+    jdbcEntryMetaData.add(new JdbcEntryMetaDatum("nested1_nested2_union_nested2_int", JdbcType.INTEGER));
+    JdbcEntrySchema expected = new JdbcEntrySchema(jdbcEntryMetaData);
+
+    Schema inputSchema = new Schema.Parser().parse(getClass().getResourceAsStream("/converter/pickfields_nested_with_union.avsc"));
+    WorkUnitState workUnitState = new WorkUnitState();
+    workUnitState.appendToListProp(JdbcPublisher.JDBC_PUBLISHER_FINAL_TABLE_NAME, table);
+    AvroToJdbcEntryConverter converter = new AvroToJdbcEntryConverter(workUnitState);
+
+    Map<String, JdbcType> dateColumnMapping = Maps.newHashMap();
+    dateColumnMapping.put("date_of_birth", JdbcType.DATE);
+    dateColumnMapping.put("last_modified", JdbcType.TIME);
+    dateColumnMapping.put("created", JdbcType.TIMESTAMP);
+    workUnitState.appendToListProp(AvroToJdbcEntryConverter.CONVERTER_AVRO_JDBC_DATE_FIELDS,
+                                   new Gson().toJson(dateColumnMapping));
+
+    JdbcEntrySchema actualSchema = converter.convertSchema(inputSchema, workUnitState);
+    Assert.assertEquals(expected, actualSchema);
+
+    try (
+        DataFileReader<GenericRecord> srcDataFileReader =
+            new DataFileReader<GenericRecord>(new File(getClass().getResource(
+                "/converter/pickfields_nested_with_union.avro").toURI()), new GenericDatumReader<GenericRecord>(
+                inputSchema))) {
+
+      List<JdbcEntryData> entries = new ArrayList<>();
+      while (srcDataFileReader.hasNext()) {
+        JdbcEntryData actualData = converter.convertRecord(actualSchema, srcDataFileReader.next(), workUnitState).iterator().next();
+        entries.add(actualData);
+      }
+
+      final JsonSerializer<JdbcEntryDatum> datumSer = new JsonSerializer<JdbcEntryDatum>() {
+        @Override
+        public JsonElement serialize(JdbcEntryDatum datum, Type typeOfSrc, JsonSerializationContext context) {
+          JsonObject jso = new JsonObject();
+          if (datum.getVal() == null) {
+            jso.add(datum.getColumnName(), null);
+            return jso;
+          }
+
+          if (datum.getVal() instanceof Date) {
+            jso.addProperty(datum.getColumnName(), ((Date) datum.getVal()).getTime());
+          } else if (datum.getVal() instanceof Timestamp) {
+            jso.addProperty(datum.getColumnName(), ((Timestamp) datum.getVal()).getTime());
+          } else if (datum.getVal() instanceof Time) {
+            jso.addProperty(datum.getColumnName(), ((Time) datum.getVal()).getTime());
+          } else {
+            jso.addProperty(datum.getColumnName(), datum.getVal().toString());
+          }
+          return jso;
+        }
+      };
+
+      JsonSerializer<JdbcEntryData> serializer = new JsonSerializer<JdbcEntryData>() {
+        @Override
+        public JsonElement serialize(JdbcEntryData src, Type typeOfSrc, JsonSerializationContext context) {
+          JsonArray arr = new JsonArray();
+          for (JdbcEntryDatum datum : src) {
+            arr.add(datumSer.serialize(datum, datum.getClass(), context));
+          }
+          return arr;
+        }
+      };
+
+      Gson gson = new GsonBuilder().registerTypeAdapter(JdbcEntryData.class, serializer).serializeNulls().create();
+
+      JsonElement actualSerialized = gson.toJsonTree(entries);
+      JsonElement expectedSerialized =
+          new JsonParser().parse(new InputStreamReader(getClass().getResourceAsStream("/converter/pickfields_nested_with_union.json")));
+
+      Assert.assertEquals(actualSerialized, expectedSerialized);
+    }
+
+    converter.close();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.avro
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.avro b/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.avro
new file mode 100644
index 0000000..b6e8d63
Binary files /dev/null and b/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.avro differ

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.avsc
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.avsc b/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.avsc
new file mode 100644
index 0000000..bbe402d
--- /dev/null
+++ b/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.avsc
@@ -0,0 +1,33 @@
+{"namespace": "example.avro",
+ "type": "record",
+ "name": "User",
+ "fields": [
+     {"name": "name", "type": "string"},
+     {"name": "favorite_number",  "type": ["int", "null"]},
+     {"name": "favorite_color", "type": ["string", "null"]},
+     {"name": "date_of_birth", "type": "long"},
+     {"name": "last_modified", "type": "long"},
+     {"name": "created", "type": "long"},
+     {"name": "nested1",
+      "type" : {
+                   "type": "record",
+                   "name": "dummy_nested1",
+                   "fields": [
+                       {"name": "nested1_string", "type": "string"},
+                       {"name": "nested1_int",  "type": ["int", "null"]},
+                       {"name": "nested2_union", "type": ["null", {
+                                                               "type" : "record",
+                                                               "name" : "dummy_nested2",
+                                                               "fields": [
+                                                                   {"name": "nested2_string", "type": "string"},
+                                                                   {"name": "nested2_int",  "type": ["int", "null"]}
+                                                               ]
+                                                           }
+                                                       ]
+                       }
+                   ]
+               }
+     }
+ ]
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c6b3824a/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.json
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.json b/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.json
new file mode 100644
index 0000000..42db8d6
--- /dev/null
+++ b/gobblin-modules/gobblin-sql/src/test/resources/converter/pickfields_nested_with_union.json
@@ -0,0 +1,194 @@
+[
+   [
+      {
+         "created":-8205952289873597770
+      },
+      {
+         "date_of_birth":4924540963523509391
+      },
+      {
+         "favorite_color":"�hxqpwvxrf"
+      },
+      {
+         "favorite_number":"826032918"
+      },
+      {
+         "last_modified":5532738755424028468
+      },
+      {
+         "name":"btnzlrfptk"
+      },
+      {
+         "nested1_nested1_int":"-36788251"
+      },
+      {
+         "nested1_nested1_string":"gji\u001a\u0000sujkj"
+      },
+      {
+         "nested1_nested2_union_nested2_int":"1026040670"
+      },
+      {
+         "nested1_nested2_union_nested2_string":"yobzdadkgk"
+      }
+   ],
+   [
+      {
+         "created":-1393624224378683129
+      },
+      {
+         "date_of_birth":-1175814878817216371
+      },
+      {
+         "favorite_color":"x\n\tjhrpgyd"
+      },
+      {
+         "favorite_number":"1816171539"
+      },
+      {
+         "last_modified":8404219756951923781
+      },
+      {
+         "name":"\u000f\rpkgrlgio"
+      },
+      {
+         "nested1_nested1_int":"-50507635"
+      },
+      {
+         "nested1_nested1_string":"q\u000f?uspbscf"
+      },
+      {
+         "nested1_nested2_union_nested2_int":null
+      },
+      {
+         "nested1_nested2_union_nested2_string":null
+      }
+   ],
+   [
+      {
+         "created":-7739579554682470032
+      },
+      {
+         "date_of_birth":-607816151590576707
+      },
+      {
+         "favorite_color":"lyuuuympyg"
+      },
+      {
+         "favorite_number":"1866476467"
+      },
+      {
+         "last_modified":-941580389512399179
+      },
+      {
+         "name":"g?pbkpjrxh"
+      },
+      {
+         "nested1_nested1_int":"1327904823"
+      },
+      {
+         "nested1_nested1_string":"kxqmrenntu"
+      },
+      {
+         "nested1_nested2_union_nested2_int":null
+      },
+      {
+         "nested1_nested2_union_nested2_string":null
+      }
+   ],
+   [
+      {
+         "created":-6056615843637407776
+      },
+      {
+         "date_of_birth":-1429852167829293715
+      },
+      {
+         "favorite_color":"�dgasutgtx"
+      },
+      {
+         "favorite_number":"-1553608691"
+      },
+      {
+         "last_modified":450932180461066816
+      },
+      {
+         "name":"gqmcmimbhp"
+      },
+      {
+         "nested1_nested1_int":"-351781782"
+      },
+      {
+         "nested1_nested1_string":"o\u001ac\u0000bmefwh"
+      },
+      {
+         "nested1_nested2_union_nested2_int":"-1596923241"
+      },
+      {
+         "nested1_nested2_union_nested2_string":"dbcczapozw"
+      }
+   ],
+   [
+      {
+         "created":-4666560421015124885
+      },
+      {
+         "date_of_birth":-8070729844977385755
+      },
+      {
+         "favorite_color":"pfzharskmy"
+      },
+      {
+         "favorite_number":"-170051651"
+      },
+      {
+         "last_modified":-7703151747036814734
+      },
+      {
+         "name":"f\u000fwszbxhzm"
+      },
+      {
+         "nested1_nested1_int":"-1126087353"
+      },
+      {
+         "nested1_nested1_string":"mjwmnevxer"
+      },
+      {
+         "nested1_nested2_union_nested2_int":"-1722304492"
+      },
+      {
+         "nested1_nested2_union_nested2_string":"h\rwdawizsu"
+      }
+   ],
+   [
+      {
+         "created":6548727010966246856
+      },
+      {
+         "date_of_birth":8554093846897734514
+      },
+      {
+         "favorite_color":"\u000fcgsqjdabu"
+      },
+      {
+         "favorite_number":"-2132346518"
+      },
+      {
+         "last_modified":3298280474340398482
+      },
+      {
+         "name":"k\n\tngmvhpe"
+      },
+      {
+         "nested1_nested1_int":"-1330607161"
+      },
+      {
+         "nested1_nested1_string":"ubbhpssdeh"
+      },
+      {
+         "nested1_nested2_union_nested2_int":"992907867"
+      },
+      {
+         "nested1_nested2_union_nested2_string":"jk?jknvxkw"
+      }
+   ]
+]
\ No newline at end of file


[17/50] incubator-gobblin git commit: [GOBBLIN-387] pick job files in FIFO order

Posted by ab...@apache.org.
[GOBBLIN-387] pick job files in FIFO order

pick job files in FIFO order

delete temp file on exit

dummy commit

fix findBugsMain

Closes #2264 from arjun4084346/jobOrder


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/cd9447a5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/cd9447a5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/cd9447a5

Branch: refs/heads/0.12.0
Commit: cd9447a58c0bc66bc24d223803bd8f33dda31887
Parents: 94bcc16
Author: Arjun <ab...@linkedin.com>
Authored: Mon Feb 5 11:44:14 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Mon Feb 5 11:44:14 2018 -0800

----------------------------------------------------------------------
 .../org/apache/gobblin/util/PullFileLoader.java | 37 +++++++++++++++-----
 .../apache/gobblin/util/PullFileLoaderTest.java | 36 ++++++++++++++++++-
 2 files changed, 64 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/cd9447a5/gobblin-utility/src/main/java/org/apache/gobblin/util/PullFileLoader.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/util/PullFileLoader.java b/gobblin-utility/src/main/java/org/apache/gobblin/util/PullFileLoader.java
index a68f9ac..210615c 100644
--- a/gobblin-utility/src/main/java/org/apache/gobblin/util/PullFileLoader.java
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/util/PullFileLoader.java
@@ -23,6 +23,8 @@ import java.io.InputStreamReader;
 import java.io.Reader;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
 import java.util.Properties;
 import java.util.Set;
@@ -74,7 +76,7 @@ public class PullFileLoader {
 
   public static final String PROPERTY_DELIMITER_PARSING_ENABLED_KEY = "property.parsing.enablekey";
   public static final boolean DEFAULT_PROPERTY_DELIMITER_PARSING_ENABLED_KEY = false;
-  
+
   private final Path rootDirectory;
   private final FileSystem fs;
   private final ExtensionFilter javaPropsPullFileFilter;
@@ -145,28 +147,36 @@ public class PullFileLoader {
   }
 
   /**
-   * Find and load all pull files under a base {@link Path} recursively.
+   * Find and load all pull files under a base {@link Path} recursively in an order sorted by last modified date.
    * @param path base {@link Path} where pull files should be found recursively.
    * @param sysProps A {@link Config} used as fallback.
    * @param loadGlobalProperties if true, will also load at most one *.properties file per directory from the
    *          {@link #rootDirectory} to the pull file {@link Path} for each pull file.
    * @return The loaded {@link Config}s.
    */
-  public Collection<Config> loadPullFilesRecursively(Path path, Config sysProps, boolean loadGlobalProperties) {
+  public List<Config> loadPullFilesRecursively(Path path, Config sysProps, boolean loadGlobalProperties) {
     try {
       Config fallback = sysProps;
       if (loadGlobalProperties && PathUtils.isAncestor(this.rootDirectory, path.getParent())) {
         fallback = loadAncestorGlobalConfigs(path.getParent(), fallback);
       }
-      return loadPullFilesRecursivelyHelper(path, fallback, loadGlobalProperties);
+      return getSortedConfigs(loadPullFilesRecursivelyHelper(path, fallback, loadGlobalProperties));
     } catch (IOException ioe) {
       return Lists.newArrayList();
     }
   }
 
-  private Collection<Config> loadPullFilesRecursivelyHelper(Path path, Config fallback, boolean loadGlobalProperties) {
-    List<Config> pullFiles = Lists.newArrayList();
+  private List<Config> getSortedConfigs(List<ConfigWithTimeStamp> configsWithTimeStamps) {
+    List<Config> sortedConfigs = Lists.newArrayList();
+    Collections.sort(configsWithTimeStamps, (config1, config2) -> (config1.timeStamp > config2.timeStamp) ? 1 : -1);
+    for (ConfigWithTimeStamp configWithTimeStamp : configsWithTimeStamps) {
+      sortedConfigs.add(configWithTimeStamp.config);
+    }
+    return sortedConfigs;
+  }
 
+  private List<ConfigWithTimeStamp> loadPullFilesRecursivelyHelper(Path path, Config fallback, boolean loadGlobalProperties) {
+    List<ConfigWithTimeStamp> pullFiles = Lists.newArrayList();
     try {
       if (loadGlobalProperties) {
         fallback = findAndLoadGlobalConfigInDirectory(path, fallback);
@@ -183,9 +193,11 @@ public class PullFileLoader {
           if (status.isDirectory()) {
             pullFiles.addAll(loadPullFilesRecursivelyHelper(status.getPath(), fallback, loadGlobalProperties));
           } else if (this.javaPropsPullFileFilter.accept(status.getPath())) {
-            pullFiles.add(loadJavaPropsWithFallback(status.getPath(), fallback).resolve());
+            log.debug("modification time of {} is {}", status.getPath(), status.getModificationTime());
+            pullFiles.add(new ConfigWithTimeStamp(status.getModificationTime(), loadJavaPropsWithFallback(status.getPath(), fallback).resolve()));
           } else if (this.hoconPullFileFilter.accept(status.getPath())) {
-            pullFiles.add(loadHoconConfigAtPath(status.getPath()).withFallback(fallback).resolve());
+            log.debug("modification time of {} is {}", status.getPath(), status.getModificationTime());
+            pullFiles.add(new ConfigWithTimeStamp(status.getModificationTime(), loadHoconConfigAtPath(status.getPath()).withFallback(fallback).resolve()));
           }
         } catch (IOException ioe) {
           // Failed to load specific subpath, try with the other subpaths in this directory
@@ -298,4 +310,13 @@ public class PullFileLoader {
     }
   }
 
+  private static class ConfigWithTimeStamp {
+    long timeStamp;
+    Config config;
+
+    public ConfigWithTimeStamp(long timeStamp, Config config) {
+      this.timeStamp = timeStamp;
+      this.config = config;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/cd9447a5/gobblin-utility/src/test/java/org/apache/gobblin/util/PullFileLoaderTest.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/test/java/org/apache/gobblin/util/PullFileLoaderTest.java b/gobblin-utility/src/test/java/org/apache/gobblin/util/PullFileLoaderTest.java
index 2560206..1c8bff8 100644
--- a/gobblin-utility/src/test/java/org/apache/gobblin/util/PullFileLoaderTest.java
+++ b/gobblin-utility/src/test/java/org/apache/gobblin/util/PullFileLoaderTest.java
@@ -17,8 +17,11 @@
 
 package org.apache.gobblin.util;
 
+import java.io.File;
 import java.io.IOException;
+import java.io.PrintWriter;
 import java.util.Collection;
+import java.util.List;
 import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
@@ -28,6 +31,7 @@ import org.testng.Assert;
 import org.testng.annotations.Test;
 
 import com.google.common.collect.ImmutableMap;
+import com.google.common.io.Files;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
 
@@ -139,6 +143,36 @@ public class PullFileLoaderTest {
     Assert.assertEquals(pullFile.entrySet().size(), 4);
   }
 
+  /**
+   * Tests to verify job written first to the job catalog is picked up first.
+   * @throws Exception
+   */
+  @Test void testJobLoadingOrder() throws Exception {
+    Properties sysProps = new Properties();
+    FileSystem fs = FileSystem.getLocal(new Configuration());
+    File tmpDir = Files.createTempDir();
+    tmpDir.deleteOnExit();
+    Path localBasePath = new Path(tmpDir.getAbsolutePath(), "PullFileLoaderTestDir");
+    fs.mkdirs(localBasePath);
+
+    for (int i=5; i>0; i--) {
+      String job = localBasePath.toString() + "/job" + i + ".conf";
+      PrintWriter writer = new PrintWriter(job, "UTF-8");
+      writer.println("key=job" + i + "_val");
+      writer.close();
+      Thread.sleep(1000);
+    }
+
+    List<Config> configs =
+        loader.loadPullFilesRecursively(localBasePath, ConfigUtils.propertiesToConfig(sysProps), false);
+
+    int i = 5;
+    for (Config config : configs) {
+      Assert.assertEquals(config.getString("key"), "job" + i + "_val");
+      i--;
+    }
+  }
+
   @Test
   public void testJobLoadingWithSysPropsAndGlobalProps() throws Exception {
     Path path;
@@ -230,7 +264,7 @@ public class PullFileLoaderTest {
     pullFile = loader.loadPullFile(path, cfg, false);
     Assert.assertEquals(pullFile.getString("json.property.key"), pullFile.getString("json.property.key1"));
   }
-  
+
   private Config pullFileFromPath(Collection<Config> configs, Path path) throws IOException {
     for (Config config : configs) {
       if (config.getString(ConfigurationKeys.JOB_CONFIG_FILE_PATH_KEY).equals(path.toString())) {


[40/50] incubator-gobblin git commit: [GOBBLIN-417] Pass in the correct protocol version in the AvroR2JoinConverter

Posted by ab...@apache.org.
[GOBBLIN-417] Pass in the correct protocol version in the AvroR2JoinConverter

Closes #2295 from
ragepati/ragepati-r2joinconverter-protocolversion


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/4c15fde8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/4c15fde8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/4c15fde8

Branch: refs/heads/0.12.0
Commit: 4c15fde85d8ec46b9948438824ef3bc33b34182b
Parents: 0d89640
Author: Raul Agepati <ra...@users.noreply.github.com>
Authored: Tue Feb 27 12:02:15 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Tue Feb 27 12:02:15 2018 -0800

----------------------------------------------------------------------
 .../org/apache/gobblin/converter/AvroR2JoinConverter.java    | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/4c15fde8/gobblin-modules/gobblin-http/src/main/java/org/apache/gobblin/converter/AvroR2JoinConverter.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-http/src/main/java/org/apache/gobblin/converter/AvroR2JoinConverter.java b/gobblin-modules/gobblin-http/src/main/java/org/apache/gobblin/converter/AvroR2JoinConverter.java
index b1f76ef..b2adcd6 100644
--- a/gobblin-modules/gobblin-http/src/main/java/org/apache/gobblin/converter/AvroR2JoinConverter.java
+++ b/gobblin-modules/gobblin-http/src/main/java/org/apache/gobblin/converter/AvroR2JoinConverter.java
@@ -40,12 +40,15 @@ import org.apache.gobblin.r2.R2Client;
 import org.apache.gobblin.r2.R2ResponseStatus;
 import org.apache.gobblin.r2.R2RestRequestBuilder;
 import org.apache.gobblin.r2.R2RestResponseHandler;
+import org.apache.gobblin.util.ConfigUtils;
 import org.apache.gobblin.utils.HttpConstants;
 
 
 @Slf4j
 public class AvroR2JoinConverter extends AvroHttpJoinConverter<RestRequest, RestResponse>{
 
+  public static final String DEFAULT_PROTOCOL_VERSION = "1.0.0";
+
   @Override
   protected void fillHttpOutputData(Schema schema, GenericRecord outputRecord, RestRequest restRequest,
       ResponseStatus status)
@@ -84,9 +87,10 @@ public class AvroR2JoinConverter extends AvroHttpJoinConverter<RestRequest, Rest
   protected AsyncRequestBuilder<GenericRecord, RestRequest> createRequestBuilder(Config config) {
     String urlTemplate = config.getString(HttpConstants.URL_TEMPLATE);
     String verb = config.getString(HttpConstants.VERB);
-    String contentType = config.getString(HttpConstants.CONTENT_TYPE);
+    String protocolVersion = ConfigUtils.getString(config,
+        HttpConstants.PROTOCOL_VERSION, DEFAULT_PROTOCOL_VERSION);
 
-    return new R2RestRequestBuilder(urlTemplate, verb, contentType);
+    return new R2RestRequestBuilder(urlTemplate, verb, protocolVersion);
   }
 
 }


[05/50] incubator-gobblin git commit: [GOBBLIN-388] Allow classpath to be configured for JVM based task execution in gobblin cluster

Posted by ab...@apache.org.
[GOBBLIN-388] Allow classpath to be configured for JVM based task execution in gobblin cluster

Closes #2265 from yukuai518/classpath


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/6a31ef84
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/6a31ef84
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/6a31ef84

Branch: refs/heads/0.12.0
Commit: 6a31ef845bd9617cb5cb8fa8ef53f184c3d6dd88
Parents: 11abf9f
Author: Kuai Yu <ku...@linkedin.com>
Authored: Wed Jan 24 21:27:03 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Wed Jan 24 21:27:03 2018 -0800

----------------------------------------------------------------------
 .../cluster/GobblinClusterConfigurationKeys.java       |  3 +++
 .../org/apache/gobblin/cluster/GobblinTaskRunner.java  |  2 +-
 .../org/apache/gobblin/cluster/HelixTaskFactory.java   |  5 +++--
 .../org/apache/gobblin/cluster/SingleTaskLauncher.java | 13 +++++++++++--
 .../apache/gobblin/cluster/SingleTaskLauncherTest.java |  4 +++-
 5 files changed, 21 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/6a31ef84/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
index 4e78078..de501f1 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
@@ -30,8 +30,11 @@ public class GobblinClusterConfigurationKeys {
 
   public static final String GOBBLIN_CLUSTER_PREFIX = "gobblin.cluster.";
 
+  // Task separation properties
   public static final String ENABLE_TASK_IN_SEPARATE_PROCESS =
       GOBBLIN_CLUSTER_PREFIX + "enableTaskInSeparateProcess";
+  public static final String TASK_CLASSPATH =
+      GOBBLIN_CLUSTER_PREFIX + "task.classpath";
 
   // General Gobblin Cluster application configuration properties.
   public static final String APPLICATION_NAME_OPTION_NAME = "app_name";

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/6a31ef84/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
index dead73b..8816457 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
@@ -192,7 +192,7 @@ public class GobblinTaskRunner {
     TaskFactory taskFactory;
     if (isRunTaskInSeparateProcessEnabled) {
       logger.info("Running a task in a separate process is enabled.");
-      taskFactory = new HelixTaskFactory(this.containerMetrics, CLUSTER_CONF_PATH);
+      taskFactory = new HelixTaskFactory(this.containerMetrics, CLUSTER_CONF_PATH, config);
     } else {
       taskFactory = getInProcessTaskFactory();
     }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/6a31ef84/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixTaskFactory.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixTaskFactory.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixTaskFactory.java
index ecb97d5..96ecffc 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixTaskFactory.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixTaskFactory.java
@@ -29,6 +29,7 @@ import org.slf4j.LoggerFactory;
 
 import com.codahale.metrics.Counter;
 import com.google.common.base.Optional;
+import com.typesafe.config.Config;
 
 import org.apache.gobblin.util.GobblinProcessBuilder;
 import org.apache.gobblin.util.SystemPropertiesWrapper;
@@ -49,7 +50,7 @@ public class HelixTaskFactory implements TaskFactory {
   private final Optional<Counter> newTasksCounter;
   private final SingleTaskLauncher launcher;
 
-  public HelixTaskFactory(Optional<ContainerMetrics> containerMetrics, Path clusterConfPath) {
+  public HelixTaskFactory(Optional<ContainerMetrics> containerMetrics, Path clusterConfPath, Config sysConfig) {
     this.containerMetrics = containerMetrics;
     if (this.containerMetrics.isPresent()) {
       this.newTasksCounter = Optional
@@ -58,7 +59,7 @@ public class HelixTaskFactory implements TaskFactory {
       this.newTasksCounter = Optional.absent();
     }
     launcher = new SingleTaskLauncher(new GobblinProcessBuilder(), new SystemPropertiesWrapper(),
-        clusterConfPath);
+        clusterConfPath, sysConfig);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/6a31ef84/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
index 10bad09..1fe3eaf 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
@@ -29,6 +29,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.gobblin.util.GobblinProcessBuilder;
 import org.apache.gobblin.util.SystemPropertiesWrapper;
 
+import com.typesafe.config.Config;
+
 import static org.apache.gobblin.cluster.SingleTaskRunnerMainOptions.CLUSTER_CONFIG_FILE_PATH;
 import static org.apache.gobblin.cluster.SingleTaskRunnerMainOptions.JOB_ID;
 import static org.apache.gobblin.cluster.SingleTaskRunnerMainOptions.WORK_UNIT_FILE_PATH;
@@ -40,12 +42,14 @@ class SingleTaskLauncher {
   private final GobblinProcessBuilder processBuilder;
   private final SystemPropertiesWrapper propertiesWrapper;
   private final Path clusterConfigFilePath;
+  private final Config sysConfig;
 
   SingleTaskLauncher(final GobblinProcessBuilder processBuilder,
-      final SystemPropertiesWrapper propertiesWrapper, final Path clusterConfigFilePath) {
+      final SystemPropertiesWrapper propertiesWrapper, final Path clusterConfigFilePath, Config sysConfig) {
     this.processBuilder = processBuilder;
     this.propertiesWrapper = propertiesWrapper;
     this.clusterConfigFilePath = clusterConfigFilePath;
+    this.sysConfig = sysConfig;
   }
 
   Process launch(final String jobId, final Path workUnitFilePath)
@@ -94,7 +98,12 @@ class SingleTaskLauncher {
 
     private void addClassPath() {
       this.cmd.add("-cp");
-      final String classPath = SingleTaskLauncher.this.propertiesWrapper.getJavaClassPath();
+      String classPath;
+      if (sysConfig.hasPath(GobblinClusterConfigurationKeys.TASK_CLASSPATH)) {
+        classPath = sysConfig.getString(GobblinClusterConfigurationKeys.TASK_CLASSPATH);
+      } else {
+        classPath = SingleTaskLauncher.this.propertiesWrapper.getJavaClassPath();
+      }
       this.cmd.add(classPath);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/6a31ef84/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/SingleTaskLauncherTest.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/SingleTaskLauncherTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/SingleTaskLauncherTest.java
index a8a361c..afa933d 100644
--- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/SingleTaskLauncherTest.java
+++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/SingleTaskLauncherTest.java
@@ -28,6 +28,8 @@ import org.testng.annotations.Test;
 import org.apache.gobblin.util.GobblinProcessBuilder;
 import org.apache.gobblin.util.SystemPropertiesWrapper;
 
+import com.typesafe.config.ConfigFactory;
+
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
@@ -56,7 +58,7 @@ public class SingleTaskLauncherTest {
 
     final Path clusterConfPath = Paths.get(CLUSTER_CONFIG_CONF_PATH);
     final SingleTaskLauncher launcher =
-        new SingleTaskLauncher(processBuilder, propertiesWrapper, clusterConfPath);
+        new SingleTaskLauncher(processBuilder, propertiesWrapper, clusterConfPath, ConfigFactory.empty());
 
     final Path workUnitPath = Paths.get(WORK_UNIT_PATH);
     final Process process = launcher.launch(JOB_ID, workUnitPath);


[18/50] incubator-gobblin git commit: Don't start services if non is needed

Posted by ab...@apache.org.
Don't start services if non is needed

When process isolation feature is enabled, there
may be no services
required to start in the worker process.

In this case, we will get a waring as below:

"Jan 23, 2018 10:43:50 AM
com.google.common.util.concurrent.ServiceManager
<init>
 WARNING: ServiceManager configured with no
services.  Is your application configured
properly?
 com.google.common.util.concurrent.ServiceManager$E
mptyServiceManagerWarning
 	at com.google.common.util.concurrent.ServiceManage
r.<init>(ServiceManager.java:168)
 	at org.apache.gobblin.cluster.GobblinTaskRunner.<i
nit>(GobblinTaskRunner.java:167)
 	at org.apache.gobblin.cluster.ClusterIntegrationTe
st.startWorker(ClusterIntegrationTest.java:196)
 	at org.apache.gobblin.cluster.ClusterIntegrationTe
st.startCluster(ClusterIntegrationTest.java:189)
 	at org.apache.gobblin.cluster.ClusterIntegrationTe
st.runSimpleJobAndVerifyResult(ClusterIntegrationT
est.java:93)
 	at org.apache.gobblin.cluster.ClusterIntegrationTe
st.simpleJobShouldCompleteInTaskIsolationMode(Clus
terIntegrationTest.java:87)
 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Na
tive Method)
 	at sun.reflect.NativeMethodAccessorImpl.invoke(Nat
iveMethodAccessorImpl.java:62)
 	at sun.reflect.DelegatingMethodAccessorImpl.invoke
(DelegatingMethodAccessorImpl.java:43)
 	at
java.lang.reflect.Method.invoke(Method.java:498)
 	at org.testng.internal.MethodInvocationHelper.invo
keMethod(MethodInvocationHelper.java:86)
 	at org.testng.internal.Invoker.invokeMethod(Invoke
r.java:643)
 	at org.testng.internal.Invoker.invokeTestMethod(In
voker.java:820)
 	at org.testng.internal.Invoker.invokeTestMethods(I
nvoker.java:1128)
 	at org.testng.internal.TestMethodWorker.invokeTest
Methods(TestMethodWorker.java:129)
 	at org.testng.internal.TestMethodWorker.run(TestMe
thodWorker.java:112)
 	at org.testng.TestRunner.privateRun(TestRunner.jav
a:782)
 	at org.testng.TestRunner.run(TestRunner.java:632)
 	at org.testng.SuiteRunner.runTest(SuiteRunner.java
:366)
 	at org.testng.SuiteRunner.runSequentially(SuiteRun
ner.java:361)
 	at org.testng.SuiteRunner.privateRun(SuiteRunner.j
ava:319)
 	at
org.testng.SuiteRunner.run(SuiteRunner.java:268)
 	at org.testng.SuiteRunnerWorker.runSuite(SuiteRunn
erWorker.java:52)
 	at org.testng.SuiteRunnerWorker.run(SuiteRunnerWor
ker.java:86)
 	at org.testng.TestNG.runSuitesSequentially(TestNG.
java:1244)
 	at org.testng.TestNG.runSuitesLocally(TestNG.java:
1169)
 	at org.testng.TestNG.run(TestNG.java:1064)
 	at org.testng.IDEARemoteTestNG.run(IDEARemoteTestN
G.java:72)
 	at org.testng.RemoteTestNGStarter.main(RemoteTestN
GStarter.java:123)
"

Fix:
Don't use serviceManager if no service is needed.

Testing:
Ran with the basic integration test and verified
the warning is gone is
the log.

Closes #2259 from HappyRay/do-not-start-services-
if-none-is-needed


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/9e788351
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/9e788351
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/9e788351

Branch: refs/heads/0.12.0
Commit: 9e788351e37fc0c1da2874bd4c7816240990a1a5
Parents: cd9447a
Author: Ray Yang <ru...@gmail.com>
Authored: Mon Feb 5 11:50:15 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Mon Feb 5 11:50:15 2018 -0800

----------------------------------------------------------------------
 .../gobblin/cluster/GobblinTaskRunner.java      | 28 +++++++++++++++-----
 1 file changed, 21 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/9e788351/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
index 8816457..a3fddab 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
@@ -164,7 +164,11 @@ public class GobblinTaskRunner {
     this.taskStateModelFactory = registerHelixTaskFactory();
 
     services.addAll(getServices());
-    this.serviceManager = new ServiceManager(services);
+    if (services.isEmpty()) {
+      this.serviceManager = null;
+    } else {
+      this.serviceManager = new ServiceManager(services);
+    }
 
     logger.debug("GobblinTaskRunner: applicationName {}, helixInstanceName {}, applicationId {}, taskRunnerId {}, config {}, appWorkDir {}",
         applicationName, helixInstanceName, applicationId, taskRunnerId, config, appWorkDirOptional);
@@ -263,8 +267,10 @@ public class GobblinTaskRunner {
               this.taskRunnerId);
     }
 
-    this.serviceManager.startAsync();
-    this.serviceManager.awaitStopped();
+    if (this.serviceManager != null) {
+      this.serviceManager.startAsync();
+      this.serviceManager.awaitStopped();
+    }
   }
 
   public synchronized void stop() {
@@ -282,10 +288,7 @@ public class GobblinTaskRunner {
     }
 
     try {
-      // Give the services 5 minutes to stop to ensure that we are responsive to shutdown requests
-      this.serviceManager.stopAsync().awaitStopped(5, TimeUnit.MINUTES);
-    } catch (TimeoutException te) {
-      logger.error("Timeout in stopping the service manager", te);
+      stopServices();
     } finally {
       this.taskStateModelFactory.shutdown();
 
@@ -295,6 +298,17 @@ public class GobblinTaskRunner {
     this.isStopped = true;
   }
 
+  private void stopServices() {
+    if (this.serviceManager != null) {
+      try {
+        // Give the services 5 minutes to stop to ensure that we are responsive to shutdown requests
+        this.serviceManager.stopAsync().awaitStopped(5, TimeUnit.MINUTES);
+      } catch (TimeoutException te) {
+        logger.error("Timeout in stopping the service manager", te);
+      }
+    }
+  }
+
   /**
    * Creates and returns a {@link List} of additional {@link Service}s that should be run in this
    * {@link GobblinTaskRunner}. Sub-classes that need additional {@link Service}s to run, should override this method


[24/50] incubator-gobblin git commit: [GOBBLIN-403] Fix null pointer issue due to kafkajobmonitor metrics is not initialized in the constructor

Posted by ab...@apache.org.
[GOBBLIN-403] Fix null pointer issue due to kafkajobmonitor metrics is not initialized in the constructor

Closes #2277 from yukuai518/fix


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/19b2d81b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/19b2d81b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/19b2d81b

Branch: refs/heads/0.12.0
Commit: 19b2d81b9539207beaca70c2efb36f258160fa27
Parents: 34de6bf
Author: Kuai Yu <ku...@linkedin.com>
Authored: Tue Feb 6 14:16:03 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Tue Feb 6 14:16:03 2018 -0800

----------------------------------------------------------------------
 .../service/StreamingKafkaSpecConsumer.java     | 25 +++++++++++++++++---
 1 file changed, 22 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/19b2d81b/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java b/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
index 4764603..5fd5413 100644
--- a/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
+++ b/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
@@ -203,6 +203,8 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
     private ContextAwareGauge<Long> jobSpecEnq;
     private ContextAwareGauge<Long> jobSpecDeq;
     private ContextAwareGauge<Long> jobSpecConsumed;
+    private ContextAwareGauge<Long> jobSpecParseFailures;
+
     private AtomicLong jobSpecEnqCount = new AtomicLong(0);
     private AtomicLong jobSpecDeqCount = new AtomicLong(0);
 
@@ -210,14 +212,30 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
     public static final String SPEC_CONSUMER_JOB_SPEC_ENQ = "specConsumerJobSpecEnq";
     public static final String SPEC_CONSUMER_JOB_SPEC_DEQ = "specConsumerJobSpecDeq";
     public static final String SPEC_CONSUMER_JOB_SPEC_CONSUMED = "specConsumerJobSpecConsumed";
-
+    public static final String SPEC_CONSUMER_JOB_SPEC_PARSE_FAILURES = "specConsumerJobSpecParseFailures";
 
     public Metrics(MetricContext context) {
       this.jobSpecQueueSize = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_QUEUE_SIZE, ()->StreamingKafkaSpecConsumer.this._jobSpecQueue.size());
       this.jobSpecEnq = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_ENQ, ()->jobSpecEnqCount.get());
       this.jobSpecDeq = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_DEQ, ()->jobSpecDeqCount.get());
       this.jobSpecConsumed = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_CONSUMED,
-          ()->StreamingKafkaSpecConsumer.this._jobMonitor.getNewSpecs().getCount() + StreamingKafkaSpecConsumer.this._jobMonitor.getRemmovedSpecs().getCount());
+          ()->getNewSpecs() + getRemovedSpecs() + getMessageParseFailures());
+      this.jobSpecParseFailures = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_PARSE_FAILURES, ()->getMessageParseFailures());
+    }
+
+    private long getNewSpecs() {
+      return StreamingKafkaSpecConsumer.this._jobMonitor.getNewSpecs() != null?
+          StreamingKafkaSpecConsumer.this._jobMonitor.getNewSpecs().getCount() : 0;
+    }
+
+    private long getRemovedSpecs() {
+      return StreamingKafkaSpecConsumer.this._jobMonitor.getRemmovedSpecs() != null?
+          StreamingKafkaSpecConsumer.this._jobMonitor.getRemmovedSpecs().getCount() : 0;
+    }
+
+    private long getMessageParseFailures() {
+      return StreamingKafkaSpecConsumer.this._jobMonitor.getMessageParseFailures() != null?
+          StreamingKafkaSpecConsumer.this._jobMonitor.getMessageParseFailures().getCount():0;
     }
 
     public Collection<ContextAwareGauge<?>> getGauges() {
@@ -226,13 +244,14 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
       list.add(jobSpecEnq);
       list.add(jobSpecDeq);
       list.add(jobSpecConsumed);
+      list.add(jobSpecParseFailures);
       return list;
     }
   }
 
   @Override
   public StandardMetrics getStandardMetrics() {
-    throw new UnsupportedOperationException("Implemented in sub class");
+    return this._metrics;
   }
 
   @Nonnull


[35/50] incubator-gobblin git commit: [GOBBLIN-413] Use same compaction start time for time lookback check during compaction

Posted by ab...@apache.org.
[GOBBLIN-413] Use same compaction start time for time lookback check during compaction

Closes #2289 from yukuai518/compacttime


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/a7a85e15
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/a7a85e15
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/a7a85e15

Branch: refs/heads/0.12.0
Commit: a7a85e150474b8911b0b92114781a30105b77822
Parents: a3189d7
Author: Kuai Yu <ku...@linkedin.com>
Authored: Wed Feb 21 14:14:48 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Wed Feb 21 14:14:48 2018 -0800

----------------------------------------------------------------------
 .../compaction/source/CompactionSource.java     |   5 +-
 .../verify/CompactionTimeRangeVerifier.java     |   7 +-
 .../gobblin/azkaban/AzkabanJobLauncher.java     |  24 +++-
 .../runtime/listeners/CompositeJobListener.java | 133 +++++++++++++++++++
 .../listeners/EmailNotificationJobListener.java |   2 +
 5 files changed, 166 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/a7a85e15/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/source/CompactionSource.java
----------------------------------------------------------------------
diff --git a/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/source/CompactionSource.java b/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/source/CompactionSource.java
index 4e8d3e0..f11378f 100644
--- a/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/source/CompactionSource.java
+++ b/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/source/CompactionSource.java
@@ -75,6 +75,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
+import org.joda.time.DateTimeUtils;
 
 import java.io.IOException;
 import java.net.URI;
@@ -94,6 +95,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
  */
 @Slf4j
 public class CompactionSource implements WorkUnitStreamSource<String, String> {
+  public static final String COMPACTION_INIT_TIME = "compaction.init.time";
   private CompactionSuite suite;
   private Path tmpJobDir;
   private FileSystem fs;
@@ -108,6 +110,7 @@ public class CompactionSource implements WorkUnitStreamSource<String, String> {
   public WorkUnitStream getWorkunitStream(SourceState state) {
     try {
       fs = getSourceFileSystem(state);
+      state.setProp(COMPACTION_INIT_TIME, DateTimeUtils.currentTimeMillis());
       suite = CompactionSuiteUtils.getCompactionSuiteFactory(state).createSuite(state);
 
       initRequestAllocator(state);
@@ -433,7 +436,7 @@ public class CompactionSource implements WorkUnitStreamSource<String, String> {
     }
   }
 
-  protected FileSystem getSourceFileSystem(State state)
+  public static FileSystem getSourceFileSystem(State state)
           throws IOException {
     Configuration conf = HadoopUtils.getConfFromState(state);
     String uri = state.getProp(ConfigurationKeys.SOURCE_FILEBASED_FS_URI, ConfigurationKeys.LOCAL_FS_URI);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/a7a85e15/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/verify/CompactionTimeRangeVerifier.java
----------------------------------------------------------------------
diff --git a/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/verify/CompactionTimeRangeVerifier.java b/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/verify/CompactionTimeRangeVerifier.java
index 85eca40..a267ab5 100644
--- a/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/verify/CompactionTimeRangeVerifier.java
+++ b/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/verify/CompactionTimeRangeVerifier.java
@@ -21,6 +21,7 @@ import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.gobblin.compaction.dataset.TimeBasedSubDirDatasetsFinder;
 import org.apache.gobblin.compaction.mapreduce.MRCompactor;
 import org.apache.gobblin.compaction.parser.CompactionPathParser;
+import org.apache.gobblin.compaction.source.CompactionSource;
 import org.apache.gobblin.configuration.State;
 import org.apache.gobblin.dataset.FileSystemDataset;
 import lombok.AllArgsConstructor;
@@ -50,19 +51,19 @@ public class CompactionTimeRangeVerifier implements CompactionVerifier<FileSyste
       CompactionPathParser.CompactionParserResult result = new CompactionPathParser(state).parse(dataset);
       DateTime folderTime = result.getTime();
       DateTimeZone timeZone = DateTimeZone.forID(this.state.getProp(MRCompactor.COMPACTION_TIMEZONE, MRCompactor.DEFAULT_COMPACTION_TIMEZONE));
-      DateTime current = new DateTime(timeZone);
+      DateTime compactionStartTime = new DateTime(this.state.getPropAsLong(CompactionSource.COMPACTION_INIT_TIME), timeZone);
       PeriodFormatter formatter = new PeriodFormatterBuilder().appendMonths().appendSuffix("m").appendDays().appendSuffix("d").appendHours()
               .appendSuffix("h").toFormatter();
 
       // get earliest time
       String maxTimeAgoStr = this.state.getProp(TimeBasedSubDirDatasetsFinder.COMPACTION_TIMEBASED_MAX_TIME_AGO, TimeBasedSubDirDatasetsFinder.DEFAULT_COMPACTION_TIMEBASED_MAX_TIME_AGO);
       Period maxTimeAgo = formatter.parsePeriod(maxTimeAgoStr);
-      earliest = current.minus(maxTimeAgo);
+      earliest = compactionStartTime.minus(maxTimeAgo);
 
       // get latest time
       String minTimeAgoStr = this.state.getProp(TimeBasedSubDirDatasetsFinder.COMPACTION_TIMEBASED_MIN_TIME_AGO, TimeBasedSubDirDatasetsFinder.DEFAULT_COMPACTION_TIMEBASED_MIN_TIME_AGO);
       Period minTimeAgo = formatter.parsePeriod(minTimeAgoStr);
-      latest = current.minus(minTimeAgo);
+      latest = compactionStartTime.minus(minTimeAgo);
 
       if (earliest.isBefore(folderTime) && latest.isAfter(folderTime)) {
         log.debug("{} falls in the user defined time range", dataset.datasetRoot());

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/a7a85e15/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java b/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java
index 45b2f40..2a7d311 100644
--- a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java
+++ b/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java
@@ -34,6 +34,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.gobblin.runtime.job_catalog.PackagedTemplatesJobCatalogDecorator;
+import org.apache.gobblin.runtime.listeners.CompositeJobListener;
+import org.apache.gobblin.util.ClassAliasResolver;
 import org.apache.gobblin.util.ConfigUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
@@ -98,6 +100,7 @@ public class AzkabanJobLauncher extends AbstractJob implements ApplicationLaunch
   private static final Logger LOG = Logger.getLogger(AzkabanJobLauncher.class);
 
   public static final String GOBBLIN_LOG_LEVEL_KEY = "gobblin.log.levelOverride";
+  public static final String GOBBLIN_CUSTOM_JOB_LISTENERS = "gobblin.custom.job.listeners";
   public static final String TEMPLATE_KEY = "gobblin.template.uri";
 
   private static final String HADOOP_FS_DEFAULT_NAME = "fs.default.name";
@@ -115,7 +118,8 @@ public class AzkabanJobLauncher extends AbstractJob implements ApplicationLaunch
 
   private final Closer closer = Closer.create();
   private final JobLauncher jobLauncher;
-  private final JobListener jobListener = new EmailNotificationJobListener();
+  private final JobListener jobListener;
+
   private final Properties props;
   private final ApplicationLauncher applicationLauncher;
   private final long ownAzkabanSla;
@@ -134,6 +138,9 @@ public class AzkabanJobLauncher extends AbstractJob implements ApplicationLaunch
     this.props = new Properties();
     this.props.putAll(props);
 
+    // initialize job listeners after properties has been initialized
+    this.jobListener = initJobListener();
+
     // load dynamic configuration and add them to the job properties
     Config propsAsConfig = ConfigUtils.propertiesToConfig(props);
     DynamicConfigGenerator dynamicConfigGenerator =
@@ -217,6 +224,21 @@ public class AzkabanJobLauncher extends AbstractJob implements ApplicationLaunch
         this.closer.register(new ServiceBasedAppLauncher(jobProps, "Azkaban-" + UUID.randomUUID()));
   }
 
+  private JobListener initJobListener() {
+    CompositeJobListener compositeJobListener = new CompositeJobListener();
+    List<String> listeners = new State(props).getPropAsList(GOBBLIN_CUSTOM_JOB_LISTENERS, EmailNotificationJobListener.class.getSimpleName());
+    try {
+      for (String listenerAlias: listeners) {
+        ClassAliasResolver<JobListener> conditionClassAliasResolver = new ClassAliasResolver<>(JobListener.class);
+        compositeJobListener.addJobListener(conditionClassAliasResolver.resolveClass(listenerAlias).newInstance());
+      }
+    } catch (IllegalAccessException | InstantiationException | ClassNotFoundException e) {
+      throw new IllegalArgumentException(e);
+    }
+
+    return compositeJobListener;
+  }
+
   @Override
   public void run()
       throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/a7a85e15/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/listeners/CompositeJobListener.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/listeners/CompositeJobListener.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/listeners/CompositeJobListener.java
new file mode 100644
index 0000000..bfdc4c8
--- /dev/null
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/listeners/CompositeJobListener.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.runtime.listeners;
+
+import java.util.List;
+
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.gobblin.runtime.JobContext;
+
+import com.google.common.collect.Lists;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
+
+@Slf4j
+@AllArgsConstructor
+public class CompositeJobListener extends AbstractJobListener {
+  private List<JobListener> listeners = Lists.newArrayList();
+
+  public CompositeJobListener() {
+  }
+
+  public void addJobListener(JobListener listener) {
+    this.listeners.add(listener);
+  }
+
+
+  @Override
+  public void onJobPrepare(JobContext jobContext) throws Exception {
+    StringBuffer buf = new StringBuffer();
+    for (JobListener listener: listeners) {
+      try {
+        listener.onJobPrepare(jobContext);
+      } catch (Exception e) {
+        buf.append(listener.getClass().getName() + ":" + e.toString());
+        log.error(ExceptionUtils.getFullStackTrace(e));
+      }
+    }
+
+    String exceptions = buf.toString();
+    if (!exceptions.isEmpty()) {
+      throw new RuntimeException(exceptions);
+    }
+  }
+
+  @Override
+  public void onJobStart(JobContext jobContext) throws Exception {
+    StringBuffer buf = new StringBuffer();
+    for (JobListener listener: listeners) {
+      try {
+        listener.onJobStart(jobContext);
+      } catch (Exception e) {
+        buf.append(listener.getClass().getName() + ":" + e.toString());
+        log.error(ExceptionUtils.getFullStackTrace(e));
+      }
+    }
+
+    String exceptions = buf.toString();
+    if (!exceptions.isEmpty()) {
+      throw new RuntimeException(exceptions);
+    }
+  }
+
+  @Override
+  public void onJobCompletion(JobContext jobContext) throws Exception {
+    StringBuffer buf = new StringBuffer();
+    for (JobListener listener: listeners) {
+      try {
+        listener.onJobCompletion(jobContext);
+      } catch (Exception e) {
+        buf.append(listener.getClass().getName() + ":" + e.toString());
+        log.error(ExceptionUtils.getFullStackTrace(e));
+      }
+    }
+
+    String exceptions = buf.toString();
+    if (!exceptions.isEmpty()) {
+      throw new RuntimeException(exceptions);
+    }
+  }
+
+  @Override
+  public void onJobCancellation(JobContext jobContext) throws Exception {
+    StringBuffer buf = new StringBuffer();
+    for (JobListener listener: listeners) {
+      try {
+        listener.onJobCancellation(jobContext);
+      } catch (Exception e) {
+        buf.append(listener.getClass().getName() + ":" + e.toString());
+        log.error(ExceptionUtils.getFullStackTrace(e));
+      }
+    }
+
+    String exceptions = buf.toString();
+    if (!exceptions.isEmpty()) {
+      throw new RuntimeException(exceptions);
+    }
+  }
+
+  @Override
+  public void onJobFailure(JobContext jobContext) throws Exception {
+    StringBuffer buf = new StringBuffer();
+    for (JobListener listener: listeners) {
+      try {
+        listener.onJobFailure(jobContext);
+      } catch (Exception e) {
+        buf.append(listener.getClass().getName() + ":" + e.toString());
+        log.error(ExceptionUtils.getFullStackTrace(e));
+      }
+    }
+
+    String exceptions = buf.toString();
+    if (!exceptions.isEmpty()) {
+      throw new RuntimeException(exceptions);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/a7a85e15/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/listeners/EmailNotificationJobListener.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/listeners/EmailNotificationJobListener.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/listeners/EmailNotificationJobListener.java
index 3106f4d..feb0d92 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/listeners/EmailNotificationJobListener.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/listeners/EmailNotificationJobListener.java
@@ -18,6 +18,7 @@
 package org.apache.gobblin.runtime.listeners;
 
 import org.apache.commons.mail.EmailException;
+import org.apache.gobblin.annotation.Alias;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -32,6 +33,7 @@ import org.apache.gobblin.util.EmailUtils;
  *
  * @author Yinan Li
  */
+@Alias("EmailNotificationJobListener")
 public class EmailNotificationJobListener extends AbstractJobListener {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(EmailNotificationJobListener.class);


[21/50] incubator-gobblin git commit: [GOBBLIN-249] Add docs for writing source schema

Posted by ab...@apache.org.
[GOBBLIN-249] Add docs for writing source schema

Closes #2221 from tilakpatidar/source_schema_docs


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/3094fe56
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/3094fe56
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/3094fe56

Branch: refs/heads/0.12.0
Commit: 3094fe56c22a8eec3caf29393f4b07ab5f7831b1
Parents: 7d11c65
Author: tilakpatidar <ti...@gmail.com>
Authored: Mon Feb 5 12:02:32 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Mon Feb 5 12:02:32 2018 -0800

----------------------------------------------------------------------
 .../user-guide/Source-schema-and-Converters.md  | 372 +++++++++++++++++++
 mkdocs.yml                                      |   1 +
 2 files changed, 373 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/3094fe56/gobblin-docs/user-guide/Source-schema-and-Converters.md
----------------------------------------------------------------------
diff --git a/gobblin-docs/user-guide/Source-schema-and-Converters.md b/gobblin-docs/user-guide/Source-schema-and-Converters.md
new file mode 100644
index 0000000..0bc1475
--- /dev/null
+++ b/gobblin-docs/user-guide/Source-schema-and-Converters.md
@@ -0,0 +1,372 @@
+Table of Contents
+--------------------
+
+[TOC]
+
+## Source schema
+A source schema has to be declared before extracting the data from the source. 
+To define the source schema `source.schema` property is available which takes a JSON value defining the source schema. 
+This schema is used by Converters to perform data type or data format conversions. 
+The java class representation of a source schema can be found here [Schema.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/schema/Schema.java).
+
+## Converters
+In Gobblin library a Converter is an interface for classes that implement data transformations, e.g., data type conversions,
+schema projections, data manipulations, data filtering, etc. This interface is responsible for 
+converting both schema and data records. Classes implementing this interface are composible and 
+can be chained together to achieve more complex data transformations.
+
+A converter basically needs four inputs:
+- Input schema
+- Output schema type
+- Input data
+- Output data type
+
+There are various inbuilt Converters available within gobblin-core. However, you can also implement your own converter 
+by extending abstract class ```org.apache.gobblin.converter.Converter```. Below, is example of such a custom implementation 
+of Gobblin Converter which replaces multiple newlines and spaces from JSON values.
+
+``` java
+
+package org.apache.gobblin.example.sample;
+import org.apache.gobblin.configuration.WorkUnitState;
+import org.apache.gobblin.converter.Converter;
+import org.apache.gobblin.converter.DataConversionException;
+import org.apache.gobblin.converter.SchemaConversionException;
+import org.apache.gobblin.converter.SingleRecordIterable;
+
+import com.google.gson.JsonArray;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+
+
+public class FilterSpacesConverter extends Converter<JsonArray, JsonArray, JsonObject, JsonObject> {
+  @Override
+  public JsonArray convertSchema(JsonArray inputSchema, WorkUnitState workUnit)
+      throws SchemaConversionException {
+    return inputSchema; //We are not doing any schema conversion
+  }
+
+  @Override
+  public Iterable<JsonObject> convertRecord(JsonArray outputSchema, JsonObject inputRecord, WorkUnitState workUnit)
+      throws DataConversionException {
+    String jsonStr = inputRecord.toString().replaceAll("\\s{2,}", " ");
+    return new SingleRecordIterable<>(new JsonParser().parse(jsonStr).getAsJsonObject());
+  }
+}
+```
+The converters can also be chained to perform sequential conversion on each input record. 
+To chain converters use the property ```converter.classes``` and provide a list of comma separated 
+converters with full reference name of converters. The execution order of the converters is same as 
+defined in the comma separated list. 
+
+For example:
+If you are reading data from a JsonSource and you want to write data into Avro format. 
+For this you can chain the converters to convert from Json string to Json and the convert Json into 
+Avro. By using the following property in your .pull file.
+```converter.classes="org.apache.gobblin.converter.json.JsonStringToJsonIntermediateConverter,org.apache.gobblin.converter.avro.JsonIntermediateToAvroConverter"```
+
+## Converters available in Gobblin
+- [AvroFieldRetrieverConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/AvroFieldRetrieverConverter.java)
+- [AvroRecordToAvroWritableConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/AvroRecordToAvroWritableConverter.java)
+- [AvroToAvroCopyableConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/AvroToAvroCopyableConverter.java)
+- [AvroToBytesConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/AvroToBytesConverter.java)
+- [BytesToAvroConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/BytesToAvroConverter.java)
+- [FlattenNestedKeyConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/FlattenNestedKeyConverter.java)
+- [JsonIntermediateToAvroConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/JsonIntermediateToAvroConverter.java)
+- [JsonRecordAvroSchemaToAvroConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverter.java)
+- [CsvToJsonConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/csv/CsvToJsonConverter.java)
+- [CsvToJsonConverterV2.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/csv/CsvToJsonConverterV2.java)
+- [AvroFieldsPickConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/filter/AvroFieldsPickConverter.java)
+- [AvroFilterConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/filter/AvroFilterConverter.java)
+- [AvroToRestJsonEntryConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/http/AvroToRestJsonEntryConverter.java)
+- [BytesToJsonConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/json/BytesToJsonConverter.java)
+- [JsonStringToJsonIntermediateConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/json/JsonStringToJsonIntermediateConverter.java)
+- [JsonToStringConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/json/JsonToStringConverter.java)
+- [ObjectStoreConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/objectstore/ObjectStoreConverter.java)
+- [ObjectStoreDeleteConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/objectstore/ObjectStoreDeleteConverter.java)
+- [HiveSerDeConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/serde/HiveSerDeConverter.java)
+- [ObjectToStringConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/string/ObjectToStringConverter.java)
+- [StringFilterConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/string/StringFilterConverter.java)
+- [StringSplitterConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/string/StringSplitterConverter.java)
+- [StringSplitterToListConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/string/StringSplitterToListConverter.java)
+- [StringToBytesConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/string/StringToBytesConverter.java)
+- [TextToStringConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/string/TextToStringConverter.java)
+- [GobblinMetricsPinotFlattenerConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/GobblinMetricsPinotFlattenerConverter.java)
+
+
+## Schema specification
+  The following section discusses the specification to define source schema using a JSON format.
+
+| Key Name     			| Value data type   	| Description                                             |
+|-----------------------|-----------------------|---------------------------------------------------------|
+| columnName			| String            	| The name of the JSON key which will contain the data.   |
+| isNullable			| Boolean				| Can data be null?                         |
+| comment				| String				| Field description just for documentation purpose.|
+| dataType				| JSON					| Provides more information about the data type.                   |
+| dataType.type			| String				| Type of data to store. ex: int, long etc                |
+| dataType.name			| String				| Provide a name to your data type.                       |
+| dataType.items		| String/JSON			| Used for array type to define the data type of items contained by the array. If data type of array items is primitive the String is used as value otherwise for complex type dataType JSON should be used as a value to provide further information on complex array items.  |
+| dataType.values		| String/JSON/Array		| Used by map and record types to define the data type of the values. In case of records it will always be Array type defining fields. In case of map it could be String or JSON based on primitive or complex data type involved.|
+| dataype.symbols		| Array<String>			| Array of strings to define the enum symbols. |
+| watermark				| Boolean				| To specify if the key is used as a watermark. Or use `extract.delta.fields` property to define comma separated list of watermark fields. |
+| unique			| Boolean				| To specify if the key should be unique set of records. |
+| defaultValue			| Object				| To specify the default value. |
+
+## Supported data types by different converters
+The converters which perform data format conversions such as CSV to JSON, JSON to AVRO etc. will have to perform data type conversions. Below, is the list of such converters and the data types they support.
+
+| Converter  | Data types  |
+|---|---|
+| [JsonIntermediateToAvroConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/JsonIntermediateToAvroConverter.java)  | <ul><li>DATE</li><li>TIMESTAMP</li><li>TIME</li><li>STRING</li><li>BYTES</li><li>INT</li><li>LONG</li><li>FLOAT</li><li>DOUBLE</li><li>BOOLEAN</li><li>ARRAY</li><li>MAP</li><li>ENUM</li></ul>|
+| [JsonIntermediateToParquetGroupConverter.java](https://github.com/apache/incubator-gobblin/blob/master/gobblin-modules/gobblin-parquet/src/main/java/org/apache/gobblin/converter/parquet/JsonIntermediateToParquetGroupConverter.java)  | <ul><li>DATE</li><li>TIMESTAMP</li><li>TIME</li><li>STRING</li><li>BYTES</li><li>INT</li><li>LONG</li><li>FLOAT</li><li>DOUBLE</li><li>BOOLEAN</li><li>ARRAY</li><li>MAP</li><li>ENUM</li></ul>|
+
+
+### Primitive types 
+ The following primitive types are available int, float, string, double, long, null, boolean.
+ 
+**Sample data**
+
+```js
+{
+	"jobRoles": 42,
+	"peopleWeightAvg": 50.5,
+	"peopleOrg": "EvilCorp",
+	"peopleAvgSal": 342222.65,
+        "peopleCount": 8344242342,
+	"peopleBrain": null,
+	"public": false
+}
+```
+**Sample schema**
+```js
+[
+    {
+        "columnName": "jobRoles",
+        "isNullable": false,
+        "comment": "Number of roles in the org"
+        "dataType": {
+                "type": "int"
+            }
+    },
+    {
+        "columnName": "peopleWeightAvg",
+        "isNullable": false,
+        "comment": "Avg weight of people in org"
+        "dataType": {
+                "type": "float"
+            }
+    },
+    {
+        "columnName": "peopleOrg",
+        "isNullable": false,
+        "comment": "Name of org people works for"
+        "dataType": {
+                "type": "string"
+            }
+    },
+    {
+        "columnName": "peopleAvgSal",
+        "isNullable": false,
+        "comment": "Avg salary of people in org"
+        "dataType": {
+                "type": "double"
+            }
+    },
+    {
+        "columnName": "peopleCount",
+        "isNullable": false,
+        "comment": "Count of people in org"
+        "dataType": {
+                "type": "long"
+            }
+    },
+    {
+        "columnName": "peopleBrain",
+        "comment": "Brain obj of people"
+        "dataType": {
+                "type": "null"
+            }
+    },
+    {
+        "columnName": "public",
+        "isNullable": false,
+        "comment": "Is data public"
+        "dataType": {
+                "type": "boolean"
+            }
+    }
+]
+```
+
+
+### Complex types
+#### Array
+
+**Sample data**
+```js
+{
+	"arrayOfInts": [25, 50, 75]
+}
+```
+**Sample schema**
+```js
+[
+    {
+        "columnName": "arrayOfInts",
+        "isNullable": false,
+        "comment": "Items in array have same data type as defined in dataType."
+        "dataType": {
+                "type": "array",
+                "items": "int"
+            }
+    }
+]
+```
+#### Map
+Maps can contain n number of key value pairs with constraint of same data type for values and keys are always string.
+**Sample data**
+```js
+{
+	"bookDetails":{
+		"harry potter and the deathly hallows": 10245,
+		"harry potter and the cursed child": 20362
+	}
+}
+```
+
+**Sample schema**
+
+```js
+[
+    {
+        "columnName": "bookDetails",
+        "isNullable": false,
+        "comment": "Maps always have string as keys and all values have same type as defined in dataType"
+        "dataType": {
+                "type": "map",
+                "values": "long"
+            }
+    }
+]
+```
+
+#### Record
+Unlike map, values in record type are not bound by single value type. Keys and values have to be declared in the schema with data type.
+**Sample data**
+```js
+{
+	"userDetails": {
+		"userName": "anonyoumous",
+		"userAge": 50,
+	}
+}
+```
+**Sample schema**
+```js
+[
+    {
+        "columnName": "userDetails",
+        "isNullable": false,
+        "comment": "user detail"
+        "dataType": {
+                "type": "record",
+                "values": [
+                    {
+                        "columnName": "userName",
+                        "dataType":{
+                            "type":"string"
+                        }
+                    },
+                    {
+                        "columnName": "userAge",
+                        "dataType":{
+                            "type":"int"
+                        }
+                    }
+                ]
+            }
+    }
+]
+```
+
+#### Enum
+**Sample data**
+```js
+{
+	"userStatus": "ACTIVE"
+}
+```
+**Sample schema**
+```js
+[
+    {
+        "columnName": "userStatus",
+        "dataType":{
+            "type": "enum",
+            "symbols":[
+                "ACTIVE", "INACTIVE"
+            ]
+        }
+    }
+]
+```
+
+### Nesting types
+Complex types can be used to created nested schemas.
+**Array, Map and Record can have complex items instead of just primitive types.**
+
+Few of the examples to show how nested schema is written
+
+**Array with nested record**
+```js
+[
+  {
+    "columnName": "userName",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "purchase",
+    "dataType": {
+      "type": "array",
+      "items": {
+        "dataType": {
+          "type": "record",
+          "values": [
+            {
+              "columnName": "ProductName",
+              "dataType": {
+                "type": "string"
+              }
+            },
+            {
+              "columnName": "ProductPrice",
+              "dataType": {
+                "type": "long"
+              }
+            }
+          ]
+        }
+      }
+    }
+  }
+]
+```
+**Map with nested array**
+```js
+[
+  {
+    "columnName": "persons",
+    "dataType": {
+      "type": "map",
+      "values": {
+        "dataType": {
+          "type": "array",
+          "items": "int"
+        }
+      }
+    }
+  }
+]
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/3094fe56/mkdocs.yml
----------------------------------------------------------------------
diff --git a/mkdocs.yml b/mkdocs.yml
index b3e770c..f3486b4 100644
--- a/mkdocs.yml
+++ b/mkdocs.yml
@@ -28,6 +28,7 @@ pages:
         - State Management and Watermarks: user-guide/State-Management-and-Watermarks.md
         - Fork Operator: user-guide/Working-with-the-ForkOperator.md
         - Configuration Glossary: user-guide/Configuration-Properties-Glossary.md
+        - Source schema and Converters: user-guide/Source-schema-and-Converters.md
         - Partitioned Writers: user-guide/Partitioned-Writers.md
         - Monitoring: user-guide/Monitoring.md
         - Template: user-guide/Gobblin-template.md


[16/50] incubator-gobblin git commit: [GOBBLIN-404] Disable immediate execution of all flows in FlowCatalog on Gobblin Service restart[]

Posted by ab...@apache.org.
[GOBBLIN-404] Disable immediate execution of all flows in FlowCatalog on Gobblin Service restart[]

Closes #2279 from sv2000/gaasScheduler


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/94bcc169
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/94bcc169
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/94bcc169

Branch: refs/heads/0.12.0
Commit: 94bcc1694ae5575cc1dcfcba12b0efab3ec8ac4e
Parents: de83a3f
Author: suvasude <su...@linkedin.biz>
Authored: Mon Feb 5 11:40:08 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Mon Feb 5 11:40:08 2018 -0800

----------------------------------------------------------------------
 .../scheduler/GobblinServiceJobScheduler.java   | 67 ++++++++++++++------
 .../GobblinServiceJobSchedulerTest.java         | 59 +++++++++++++++++
 2 files changed, 106 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/94bcc169/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
index 5c26445..9cb39fb 100644
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
@@ -23,8 +23,10 @@ import java.util.Map;
 import java.util.Properties;
 
 import java.util.UUID;
+
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.helix.HelixManager;
 import org.apache.helix.InstanceType;
@@ -37,9 +39,11 @@ import org.quartz.UnableToInterruptJobException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.collect.Maps;
 import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
 
 import org.apache.gobblin.annotation.Alpha;
 import org.apache.gobblin.configuration.ConfigurationKeys;
@@ -77,9 +81,10 @@ public class GobblinServiceJobScheduler extends JobScheduler implements SpecCata
   @Getter
   protected volatile boolean isActive;
 
-  public GobblinServiceJobScheduler(Config config, Optional<HelixManager> helixManager, Optional<FlowCatalog> flowCatalog,
-      Optional<TopologyCatalog> topologyCatalog, Orchestrator orchestrator, SchedulerService schedulerService,
-      Optional<Logger> log) throws Exception {
+  public GobblinServiceJobScheduler(Config config, Optional<HelixManager> helixManager,
+      Optional<FlowCatalog> flowCatalog, Optional<TopologyCatalog> topologyCatalog, Orchestrator orchestrator,
+      SchedulerService schedulerService, Optional<Logger> log)
+      throws Exception {
     super(ConfigUtils.configToProperties(config), schedulerService);
 
     _log = log.isPresent() ? log.get() : LoggerFactory.getLogger(getClass());
@@ -90,13 +95,14 @@ public class GobblinServiceJobScheduler extends JobScheduler implements SpecCata
     this.scheduledFlowSpecs = Maps.newHashMap();
   }
 
-  public GobblinServiceJobScheduler(Config config, Optional<HelixManager> helixManager, Optional<FlowCatalog> flowCatalog,
-      Optional<TopologyCatalog> topologyCatalog, SchedulerService schedulerService, Optional<Logger> log) throws Exception {
+  public GobblinServiceJobScheduler(Config config, Optional<HelixManager> helixManager,
+      Optional<FlowCatalog> flowCatalog, Optional<TopologyCatalog> topologyCatalog, SchedulerService schedulerService,
+      Optional<Logger> log)
+      throws Exception {
     this(config, helixManager, flowCatalog, topologyCatalog, new Orchestrator(config, topologyCatalog, log),
         schedulerService, log);
   }
 
-
   public synchronized void setActive(boolean isActive) {
     if (this.isActive == isActive) {
       // No-op if already in correct state
@@ -111,7 +117,13 @@ public class GobblinServiceJobScheduler extends JobScheduler implements SpecCata
       if (this.flowCatalog.isPresent()) {
         Collection<Spec> specs = this.flowCatalog.get().getSpecsWithTimeUpdate();
         for (Spec spec : specs) {
-          onAddSpec(spec);
+          //Disable FLOW_RUN_IMMEDIATELY on service startup or leadership change
+          if (spec instanceof FlowSpec) {
+            Spec modifiedSpec = disableFlowRunImmediatelyOnStart((FlowSpec) spec);
+            onAddSpec(modifiedSpec);
+          } else {
+            onAddSpec(spec);
+          }
         }
       }
     }
@@ -126,8 +138,19 @@ public class GobblinServiceJobScheduler extends JobScheduler implements SpecCata
     }
   }
 
+  @VisibleForTesting
+  protected static Spec disableFlowRunImmediatelyOnStart(FlowSpec spec) {
+    Properties properties = spec.getConfigAsProperties();
+    properties.setProperty(ConfigurationKeys.FLOW_RUN_IMMEDIATELY, "false");
+    Config config = ConfigFactory.parseProperties(properties);
+    FlowSpec flowSpec = new FlowSpec(spec.getUri(), spec.getVersion(), spec.getDescription(), config, properties,
+        spec.getTemplateURIs(), spec.getChildSpecs());
+    return flowSpec;
+  }
+
   @Override
-  protected void startUp() throws Exception {
+  protected void startUp()
+      throws Exception {
     super.startUp();
   }
 
@@ -135,7 +158,8 @@ public class GobblinServiceJobScheduler extends JobScheduler implements SpecCata
    * Synchronize the job scheduling because the same flowSpec can be scheduled by different threads.
    */
   @Override
-  public synchronized void scheduleJob(Properties jobProps, JobListener jobListener) throws JobException {
+  public synchronized void scheduleJob(Properties jobProps, JobListener jobListener)
+      throws JobException {
     Map<String, Object> additionalJobDataMap = Maps.newHashMap();
     additionalJobDataMap.put(ServiceConfigKeys.GOBBLIN_SERVICE_FLOWSPEC,
         this.scheduledFlowSpecs.get(jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY)));
@@ -148,7 +172,8 @@ public class GobblinServiceJobScheduler extends JobScheduler implements SpecCata
   }
 
   @Override
-  public void runJob(Properties jobProps, JobListener jobListener) throws JobException {
+  public void runJob(Properties jobProps, JobListener jobListener)
+      throws JobException {
     try {
       Spec flowSpec = this.scheduledFlowSpecs.get(jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY));
       this.orchestrator.orchestrate(flowSpec);
@@ -185,10 +210,11 @@ public class GobblinServiceJobScheduler extends JobScheduler implements SpecCata
         jobConfig.setProperty(ConfigurationKeys.JOB_GROUP_KEY,
             ((FlowSpec) addedSpec).getConfig().getValue(ConfigurationKeys.FLOW_GROUP_KEY).toString());
         jobConfig.setProperty(ConfigurationKeys.FLOW_RUN_IMMEDIATELY,
-            ConfigUtils.getString(((FlowSpec) addedSpec).getConfig(), ConfigurationKeys.FLOW_RUN_IMMEDIATELY,"false"));
-        if (flowSpecProperties.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)
-            && StringUtils.isNotBlank(flowSpecProperties.getProperty(ConfigurationKeys.JOB_SCHEDULE_KEY))) {
-          jobConfig.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, flowSpecProperties.getProperty(ConfigurationKeys.JOB_SCHEDULE_KEY));
+            ConfigUtils.getString(((FlowSpec) addedSpec).getConfig(), ConfigurationKeys.FLOW_RUN_IMMEDIATELY, "false"));
+        if (flowSpecProperties.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY) && StringUtils
+            .isNotBlank(flowSpecProperties.getProperty(ConfigurationKeys.JOB_SCHEDULE_KEY))) {
+          jobConfig.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY,
+              flowSpecProperties.getProperty(ConfigurationKeys.JOB_SCHEDULE_KEY));
         }
 
         this.scheduledFlowSpecs.put(addedSpec.getUri().toString(), addedSpec);
@@ -223,8 +249,9 @@ public class GobblinServiceJobScheduler extends JobScheduler implements SpecCata
 
     if (!isActive && helixManager.isPresent()) {
       _log.info("Scheduler running in slave mode, forward Spec delete via Helix message to master: " + deletedSpecURI);
-      HelixUtils.sendUserDefinedMessage(ServiceConfigKeys.HELIX_FLOWSPEC_REMOVE, deletedSpecURI.toString() + ":" +
-              deletedSpecVersion, UUID.randomUUID().toString(), InstanceType.CONTROLLER, helixManager.get(), _log);
+      HelixUtils.sendUserDefinedMessage(ServiceConfigKeys.HELIX_FLOWSPEC_REMOVE,
+          deletedSpecURI.toString() + ":" + deletedSpecVersion, UUID.randomUUID().toString(), InstanceType.CONTROLLER,
+          helixManager.get(), _log);
       return;
     }
 
@@ -235,12 +262,12 @@ public class GobblinServiceJobScheduler extends JobScheduler implements SpecCata
         this.scheduledFlowSpecs.remove(deletedSpecURI.toString());
         unscheduleJob(deletedSpecURI.toString());
       } else {
-        _log.warn(String.format("Spec with URI: %s was not found in cache. May be it was cleaned, if not please "
-                + "clean it manually", deletedSpecURI));
+        _log.warn(String.format(
+            "Spec with URI: %s was not found in cache. May be it was cleaned, if not please " + "clean it manually",
+            deletedSpecURI));
       }
     } catch (JobException e) {
-      _log.warn(String.format("Spec with URI: %s was not unscheduled cleaning",
-          deletedSpecURI), e);
+      _log.warn(String.format("Spec with URI: %s was not unscheduled cleaning", deletedSpecURI), e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/94bcc169/gobblin-service/src/test/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobSchedulerTest.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/test/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobSchedulerTest.java b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobSchedulerTest.java
new file mode 100644
index 0000000..a6e1bc5
--- /dev/null
+++ b/gobblin-service/src/test/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobSchedulerTest.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gobblin.service.modules.scheduler;
+
+import java.net.URI;
+import java.util.Properties;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.runtime.api.FlowSpec;
+
+
+public class GobblinServiceJobSchedulerTest {
+  private static final String TEST_GROUP_NAME = "testGroup";
+  private static final String TEST_FLOW_NAME = "testFlow";
+  private static final String TEST_SCHEDULE = "0 1/0 * ? * *";
+  private static final String TEST_TEMPLATE_URI = "FS:///templates/test.template";
+
+  @Test
+  public void testDisableFlowRunImmediatelyOnStart()
+      throws Exception {
+    Properties properties = new Properties();
+    properties.setProperty(ConfigurationKeys.FLOW_RUN_IMMEDIATELY, "true");
+    properties.setProperty(ConfigurationKeys.JOB_SCHEDULE_KEY, TEST_SCHEDULE);
+    properties.setProperty(ConfigurationKeys.JOB_GROUP_KEY, TEST_GROUP_NAME);
+    properties.setProperty(ConfigurationKeys.JOB_NAME_KEY, TEST_FLOW_NAME);
+    Config config = ConfigFactory.parseProperties(properties);
+    FlowSpec spec = FlowSpec.builder().withTemplate(new URI(TEST_TEMPLATE_URI)).withVersion("version")
+        .withConfigAsProperties(properties).withConfig(config).build();
+    FlowSpec modifiedSpec = (FlowSpec) GobblinServiceJobScheduler.disableFlowRunImmediatelyOnStart(spec);
+    for (URI templateURI : modifiedSpec.getTemplateURIs().get()) {
+      Assert.assertEquals(templateURI.toString(), TEST_TEMPLATE_URI);
+    }
+    Assert.assertEquals(modifiedSpec.getVersion(), "version");
+    Config modifiedConfig = modifiedSpec.getConfig();
+    Assert.assertFalse(modifiedConfig.getBoolean(ConfigurationKeys.FLOW_RUN_IMMEDIATELY));
+    Assert.assertEquals(modifiedConfig.getString(ConfigurationKeys.JOB_GROUP_KEY), TEST_GROUP_NAME);
+    Assert.assertEquals(modifiedConfig.getString(ConfigurationKeys.JOB_NAME_KEY), TEST_FLOW_NAME);
+  }
+}
\ No newline at end of file


[39/50] incubator-gobblin git commit: [GOBBLIN-416] Add java option for task child process

Posted by ab...@apache.org.
[GOBBLIN-416] Add java option for task child process

Closes #2292 from yukuai518/javaopt


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/0d89640c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/0d89640c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/0d89640c

Branch: refs/heads/0.12.0
Commit: 0d89640cfcfbfb14da6ec80f484eae16b5b98eaf
Parents: 97e29f4
Author: Kuai Yu <ku...@linkedin.com>
Authored: Mon Feb 26 11:49:35 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Mon Feb 26 11:49:35 2018 -0800

----------------------------------------------------------------------
 .../gobblin/cluster/GobblinClusterConfigurationKeys.java     | 2 ++
 .../java/org/apache/gobblin/cluster/SingleTaskLauncher.java  | 8 ++++++++
 2 files changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/0d89640c/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
index 68a8208..5719b23 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
@@ -37,6 +37,8 @@ public class GobblinClusterConfigurationKeys {
       GOBBLIN_CLUSTER_PREFIX + "task.classpath";
   public static final String TASK_LOG_CONFIG =
       GOBBLIN_CLUSTER_PREFIX + "task.log.config";
+  public static final String TASK_JAVA_OPTIONS =
+      GOBBLIN_CLUSTER_PREFIX + "task.java.options";
 
   // General Gobblin Cluster application configuration properties.
   public static final String APPLICATION_NAME_OPTION_NAME = "app_name";

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/0d89640c/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
index e6fac31..078f122 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
@@ -79,6 +79,7 @@ class SingleTaskLauncher {
 
     List<String> build() {
       addJavaBin();
+      addJavaOptions();
       addClassPath();
       addLogConfig();
       addClassName();
@@ -86,6 +87,13 @@ class SingleTaskLauncher {
       return this.cmd;
     }
 
+    private void addJavaOptions() {
+      if (sysConfig.hasPath(GobblinClusterConfigurationKeys.TASK_JAVA_OPTIONS)) {
+        final String javaOptions = sysConfig.getString(GobblinClusterConfigurationKeys.TASK_JAVA_OPTIONS);
+        this.cmd.add(javaOptions);
+      }
+    }
+
     private void addClassName() {
       final String runnerClassName = SingleTaskRunnerMain.class.getCanonicalName();
       this.cmd.add(runnerClassName);


[42/50] incubator-gobblin git commit: [GOBBLIN-414] Added lineage event for convertible hive datasets

Posted by ab...@apache.org.
[GOBBLIN-414] Added lineage event for convertible hive datasets

Closes #2290 from aditya1105/metadata


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/faa27f41
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/faa27f41
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/faa27f41

Branch: refs/heads/0.12.0
Commit: faa27f41f00f1d142c128e13a3da0f8c388d83b9
Parents: 5e6bfb0
Author: aditya1105 <ad...@linkedin.com>
Authored: Thu Mar 1 07:34:58 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Thu Mar 1 07:34:58 2018 -0800

----------------------------------------------------------------------
 .../conversion/hive/source/HiveSource.java      | 54 +++++++++++++++++-
 .../dataset/ConvertibleHiveDatasetTest.java     | 60 ++++++++++++++++++++
 2 files changed, 111 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/faa27f41/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/conversion/hive/source/HiveSource.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/conversion/hive/source/HiveSource.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/conversion/hive/source/HiveSource.java
index 3ad99fd..4cee48f 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/conversion/hive/source/HiveSource.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/conversion/hive/source/HiveSource.java
@@ -27,6 +27,13 @@ import java.util.concurrent.TimeUnit;
 import lombok.extern.slf4j.Slf4j;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.gobblin.broker.gobblin_scopes.GobblinScopeTypes;
+import org.apache.gobblin.broker.iface.SharedResourcesBroker;
+import org.apache.gobblin.data.management.conversion.hive.dataset.ConvertibleHiveDataset;
+import org.apache.gobblin.dataset.DatasetConstants;
+import org.apache.gobblin.dataset.DatasetDescriptor;
+import org.apache.gobblin.metrics.event.lineage.LineageInfo;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
@@ -148,6 +155,7 @@ public class HiveSource implements Source {
   protected long maxLookBackTime;
   protected long beginGetWorkunitsTime;
   protected List<String> ignoreDataPathIdentifierList;
+  protected SharedResourcesBroker<GobblinScopeTypes> sharedJobBroker;
 
   protected final ClassAliasResolver<HiveBaseExtractorFactory> classAliasResolver =
       new ClassAliasResolver<>(HiveBaseExtractorFactory.class);
@@ -214,6 +222,7 @@ public class HiveSource implements Source {
     this.maxLookBackTime = new DateTime().minusDays(maxLookBackDays).getMillis();
     this.ignoreDataPathIdentifierList = COMMA_BASED_SPLITTER.splitToList(state.getProp(HIVE_SOURCE_IGNORE_DATA_PATH_IDENTIFIER_KEY,
         DEFAULT_HIVE_SOURCE_IGNORE_DATA_PATH_IDENTIFIER));
+    this.sharedJobBroker = state.getBroker();
 
     silenceHiveLoggers();
   }
@@ -252,7 +261,10 @@ public class HiveSource implements Source {
 
         EventWorkunitUtils.setTableSlaEventMetadata(hiveWorkUnit, hiveDataset.getTable(), updateTime, lowWatermark.getValue(),
             this.beginGetWorkunitsTime);
-
+        if (hiveDataset instanceof ConvertibleHiveDataset) {
+          setLineageInfo((ConvertibleHiveDataset) hiveDataset, hiveWorkUnit, this.sharedJobBroker);
+          log.info("Added lineage event for dataset " + hiveDataset.getUrn());
+        }
         this.workunits.add(hiveWorkUnit);
         log.debug(String.format("Workunit added for table: %s", hiveWorkUnit));
 
@@ -281,7 +293,7 @@ public class HiveSource implements Source {
   }
 
   protected void createWorkunitsForPartitionedTable(HiveDataset hiveDataset, AutoReturnableObject<IMetaStoreClient> client) throws IOException {
-
+    boolean setLineageInfo = false;
     long tableProcessTime = new DateTime().getMillis();
     this.watermarker.onTableProcessBegin(hiveDataset.getTable(), tableProcessTime);
 
@@ -329,7 +341,12 @@ public class HiveSource implements Source {
 
           EventWorkunitUtils.setPartitionSlaEventMetadata(hiveWorkUnit, hiveDataset.getTable(), sourcePartition, updateTime,
               lowWatermark.getValue(), this.beginGetWorkunitsTime);
-
+          if (hiveDataset instanceof ConvertibleHiveDataset && !setLineageInfo) {
+            setLineageInfo((ConvertibleHiveDataset) hiveDataset, hiveWorkUnit, this.sharedJobBroker);
+            log.info("Added lineage event for dataset " + hiveDataset.getUrn());
+            // Add lineage information only once per hive table
+            setLineageInfo = true;
+          }
           workunits.add(hiveWorkUnit);
           log.info(String.format("Creating workunit for partition %s as updateTime %s is greater than low watermark %s",
               sourcePartition.getCompleteName(), updateTime, lowWatermark.getValue()));
@@ -474,4 +491,35 @@ public class HiveSource implements Source {
   private boolean isAvro(Table table) {
     return AvroSerDe.class.getName().equals(table.getSd().getSerdeInfo().getSerializationLib());
   }
+
+  public static void setLineageInfo(ConvertibleHiveDataset convertibleHiveDataset, WorkUnit workUnit,
+      SharedResourcesBroker<GobblinScopeTypes> sharedJobBroker)
+      throws IOException {
+    String sourceTable =
+        convertibleHiveDataset.getTable().getDbName() + "." + convertibleHiveDataset.getTable().getTableName();
+    DatasetDescriptor source = new DatasetDescriptor(DatasetConstants.PLATFORM_HIVE, sourceTable);
+    source.addMetadata(DatasetConstants.FS_URI,
+        convertibleHiveDataset.getTable().getDataLocation().getFileSystem(new Configuration()).getUri().toString());
+
+    int virtualBranch = 0;
+    for (String format : convertibleHiveDataset.getDestFormats()) {
+      ++virtualBranch;
+      Optional<ConvertibleHiveDataset.ConversionConfig> conversionConfigForFormat =
+          convertibleHiveDataset.getConversionConfigForFormat(format);
+      Optional<LineageInfo> lineageInfo = LineageInfo.getLineageInfo(sharedJobBroker);
+      if (!lineageInfo.isPresent()) {
+        continue;
+      } else if (!conversionConfigForFormat.isPresent()) {
+        continue;
+      }
+      String destTable = conversionConfigForFormat.get().getDestinationDbName() + "." + conversionConfigForFormat.get()
+          .getDestinationTableName();
+      DatasetDescriptor dest = new DatasetDescriptor(DatasetConstants.PLATFORM_HIVE, destTable);
+      Path destPath = new Path(conversionConfigForFormat.get().getDestinationDataPath());
+      dest.addMetadata(DatasetConstants.FS_URI, destPath.getFileSystem(new Configuration()).getUri().toString());
+
+      lineageInfo.get().setSource(source, workUnit);
+      lineageInfo.get().putDestination(dest, virtualBranch, workUnit);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/faa27f41/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/conversion/hive/dataset/ConvertibleHiveDatasetTest.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/conversion/hive/dataset/ConvertibleHiveDatasetTest.java b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/conversion/hive/dataset/ConvertibleHiveDatasetTest.java
index 5021d4d..c399264 100644
--- a/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/conversion/hive/dataset/ConvertibleHiveDatasetTest.java
+++ b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/conversion/hive/dataset/ConvertibleHiveDatasetTest.java
@@ -19,8 +19,20 @@ package org.apache.gobblin.data.management.conversion.hive.dataset;
 import java.io.InputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Properties;
 
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.broker.gobblin_scopes.GobblinScopeTypes;
+import org.apache.gobblin.broker.gobblin_scopes.JobScopeInstance;
+import org.apache.gobblin.broker.iface.SharedResourcesBroker;
+import org.apache.gobblin.data.management.conversion.hive.source.HiveSource;
+import org.apache.gobblin.dataset.DatasetConstants;
+import org.apache.gobblin.dataset.DatasetDescriptor;
+import org.apache.gobblin.metrics.event.lineage.LineageEventBuilder;
+import org.apache.gobblin.metrics.event.lineage.LineageInfo;
+import org.apache.gobblin.source.workunit.WorkUnit;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -31,6 +43,7 @@ import org.testng.annotations.Test;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
+import com.google.gson.Gson;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
 
@@ -45,6 +58,44 @@ import static org.mockito.Mockito.when;
 public class ConvertibleHiveDatasetTest {
 
   @Test
+  public void testLineageInfo()
+      throws Exception {
+    String testConfFilePath = "convertibleHiveDatasetTest/flattenedAndNestedOrc.conf";
+    Config config = ConfigFactory.parseResources(testConfFilePath).getConfig("hive.conversion.avro");
+    WorkUnit workUnit = WorkUnit.createEmpty();
+    Gson GSON = new Gson();
+    HiveSource.setLineageInfo(createTestConvertibleDataset(config), workUnit, getSharedJobBroker());
+    Properties props = workUnit.getSpecProperties();
+    // Asset that lineage name is correct
+    Assert.assertEquals(props.getProperty("gobblin.event.lineage.name"), "db1.tb1");
+
+    // Assert that source is correct for lineage event
+    Assert.assertTrue(props.containsKey("gobblin.event.lineage.source"));
+    DatasetDescriptor sourceDD =
+        GSON.fromJson(props.getProperty("gobblin.event.lineage.source"), DatasetDescriptor.class);
+    Assert.assertEquals(sourceDD.getPlatform(), DatasetConstants.PLATFORM_HIVE);
+    Assert.assertEquals(sourceDD.getName(), "db1.tb1");
+
+    // Assert that first dest is correct for lineage event
+    Assert.assertTrue(props.containsKey("gobblin.event.lineage.branch.1.destination"));
+    DatasetDescriptor destDD1 =
+        GSON.fromJson(props.getProperty("gobblin.event.lineage.branch.1.destination"), DatasetDescriptor.class);
+    Assert.assertEquals(destDD1.getPlatform(), DatasetConstants.PLATFORM_HIVE);
+    Assert.assertEquals(destDD1.getName(), "db1_nestedOrcDb.tb1_nestedOrc");
+
+    // Assert that second dest is correct for lineage event
+    Assert.assertTrue(props.containsKey("gobblin.event.lineage.branch.2.destination"));
+    DatasetDescriptor destDD2 =
+        GSON.fromJson(props.getProperty("gobblin.event.lineage.branch.2.destination"), DatasetDescriptor.class);
+    Assert.assertEquals(destDD2.getPlatform(), DatasetConstants.PLATFORM_HIVE);
+    Assert.assertEquals(destDD2.getName(), "db1_flattenedOrcDb.tb1_flattenedOrc");
+
+    // Assert that there are two eventBuilders for nestedOrc and flattenedOrc
+    Collection<LineageEventBuilder> lineageEventBuilders = LineageInfo.load(Collections.singleton(workUnit));
+    Assert.assertEquals(lineageEventBuilders.size(), 2);
+  }
+
+  @Test
   public void testFlattenedOrcConfig() throws Exception {
     String testConfFilePath = "convertibleHiveDatasetTest/flattenedOrc.conf";
     Config config = ConfigFactory.parseResources(testConfFilePath).getConfig("hive.conversion.avro");
@@ -181,4 +232,13 @@ public class ConvertibleHiveDatasetTest {
     table.setSd(sd);
     return table;
   }
+
+  public static SharedResourcesBroker<GobblinScopeTypes> getSharedJobBroker() {
+    SharedResourcesBroker<GobblinScopeTypes> instanceBroker = SharedResourcesBrokerFactory
+        .createDefaultTopLevelBroker(ConfigFactory.empty(), GobblinScopeTypes.GLOBAL.defaultScopeInstance());
+    SharedResourcesBroker<GobblinScopeTypes> jobBroker = instanceBroker
+        .newSubscopedBuilder(new JobScopeInstance("ConvertibleHiveDatasetLineageEventTest", String.valueOf(System.currentTimeMillis())))
+        .build();
+    return jobBroker;
+  }
 }


[33/50] incubator-gobblin git commit: [GOBBLIN-410] Support REPLACE_TABLE_AND_PARTITIONS for Hive copies.

Posted by ab...@apache.org.
[GOBBLIN-410] Support REPLACE_TABLE_AND_PARTITIONS for Hive copies.

Closes #2287 from ibuenros/replace-table-and-
partitions


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/11182dcf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/11182dcf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/11182dcf

Branch: refs/heads/0.12.0
Commit: 11182dcf3ed771a38fbbaa350cb115cf5e0fe3e3
Parents: 4f52230
Author: ibuenros <is...@gmail.com>
Authored: Tue Feb 20 10:35:59 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Tue Feb 20 10:35:59 2018 -0800

----------------------------------------------------------------------
 .../gobblin/data/management/copy/hive/HiveCopyEntityHelper.java   | 2 ++
 .../gobblin/data/management/copy/hive/HivePartitionFileSet.java   | 3 ++-
 .../data/management/copy/hive/UnpartitionedTableFileSet.java      | 1 +
 3 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11182dcf/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/HiveCopyEntityHelper.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/HiveCopyEntityHelper.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/HiveCopyEntityHelper.java
index 2580775..2b1a142 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/HiveCopyEntityHelper.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/HiveCopyEntityHelper.java
@@ -200,6 +200,8 @@ public class HiveCopyEntityHelper {
     REPLACE_PARTITIONS,
     /** Deregister target table, do NOT delete its files, and create a new table with correct values. */
     REPLACE_TABLE,
+    /** A combination of {@link #REPLACE_TABLE} and {@link #REPLACE_PARTITIONS}*/
+    REPLACE_TABLE_AND_PARTITIONS,
     /** Keep the target table as registered while updating the file location */
     UPDATE_TABLE,
     /** Abort copying of conflict table. */

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11182dcf/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/HivePartitionFileSet.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/HivePartitionFileSet.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/HivePartitionFileSet.java
index 34b6933..2c3817e 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/HivePartitionFileSet.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/HivePartitionFileSet.java
@@ -96,7 +96,8 @@ public class HivePartitionFileSet extends HiveFileSet {
         try {
           checkPartitionCompatibility(targetPartition, this.existingTargetPartition.get());
         } catch (IOException ioe) {
-          if (hiveCopyEntityHelper.getExistingEntityPolicy() != HiveCopyEntityHelper.ExistingEntityPolicy.REPLACE_PARTITIONS) {
+          if (hiveCopyEntityHelper.getExistingEntityPolicy() != HiveCopyEntityHelper.ExistingEntityPolicy.REPLACE_PARTITIONS &&
+              hiveCopyEntityHelper.getExistingEntityPolicy() != HiveCopyEntityHelper.ExistingEntityPolicy.REPLACE_TABLE_AND_PARTITIONS) {
             log.error("Source and target partitions are not compatible. Aborting copy of partition " + this.partition,
                 ioe);
             return Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11182dcf/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/UnpartitionedTableFileSet.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/UnpartitionedTableFileSet.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/UnpartitionedTableFileSet.java
index 21813fb..756b4dd 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/UnpartitionedTableFileSet.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/hive/UnpartitionedTableFileSet.java
@@ -73,6 +73,7 @@ public class UnpartitionedTableFileSet extends HiveFileSet {
             existingTargetTable = Optional.absent();
             break ;
           case REPLACE_TABLE:
+          case REPLACE_TABLE_AND_PARTITIONS:
             // Required to de-register the original table.
             log.warn("Source and target table are not compatible. Will override target table " + existingTargetTable.get()
                 .getDataLocation());


[31/50] incubator-gobblin git commit: [Gobblin-371][GOBBLIN-371] eagerly test for jira-python package

Posted by ab...@apache.org.
[Gobblin-371][GOBBLIN-371] eagerly test for jira-python package

Merge remote-tracking branch 'apache/master'

Eagerly load jira-python library so that things
don't break because of a missing dependency while
running

Merge branch 'master' into GOBBLIN-
371__eagerly_test_for_jira_package

Closes #2246 from kadaan/GOBBLIN-
371__eagerly_test_for_jira_package


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/4997fa9b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/4997fa9b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/4997fa9b

Branch: refs/heads/0.12.0
Commit: 4997fa9bb286dbe451456e48cf6d07f0f0aa48f8
Parents: c43cf26
Author: Joel Baranick <jb...@apache.org>
Authored: Wed Feb 14 00:06:25 2018 -0800
Committer: Joel Baranick <jo...@ensighten.com>
Committed: Wed Feb 14 00:06:25 2018 -0800

----------------------------------------------------------------------
 dev/gobblin-jira-version | 12 ++++++------
 dev/gobblin-pr           | 12 ++++++------
 2 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/4997fa9b/dev/gobblin-jira-version
----------------------------------------------------------------------
diff --git a/dev/gobblin-jira-version b/dev/gobblin-jira-version
index 5796c54..c8bb65d 100755
--- a/dev/gobblin-jira-version
+++ b/dev/gobblin-jira-version
@@ -49,6 +49,12 @@ try:
 except ImportError:
     print("Could not find the keyring library. Run 'sudo pip install keyring' to install.")
     sys.exit(-1)
+
+try:
+    import jira.client
+except ImportError:
+    print("Could not find jira-python library; exiting. Run 'sudo pip install jira' to install.")
+    sys.exit(-1)
     
 JIRA_BASE = "https://issues.apache.org/jira/browse"
 JIRA_API_BASE = "https://issues.apache.org/jira"
@@ -87,12 +93,6 @@ def update_jira_issue(fix_version):
 
     fix_version: the version to assign to the Gobblin JIRAs.
     """
-    try:
-        import jira.client
-    except ImportError:
-        print("Could not find jira-python library; exiting. Run "
-            "'sudo pip install jira' to install.")
-        sys.exit(-1)
 
     # ASF JIRA username
     JIRA_USERNAME = os.environ.get("JIRA_USERNAME", '')

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/4997fa9b/dev/gobblin-pr
----------------------------------------------------------------------
diff --git a/dev/gobblin-pr b/dev/gobblin-pr
index a7b3209..8b34ad9 100755
--- a/dev/gobblin-pr
+++ b/dev/gobblin-pr
@@ -61,6 +61,12 @@ except ImportError:
     print("Could not find the keyring library. Run 'sudo pip3 install keyring' to install.")
     sys.exit(-1)
 
+try:
+    import jira.client
+except ImportError:
+    print("Could not find jira-python library; exiting. Run 'sudo pip3 install jira' to install.")
+    sys.exit(-1)
+
 # Location of your Gobblin git development area
 GOBBLIN_GIT_LOCATION = os.environ.get(
     "GOBBLIN_GIT",
@@ -531,12 +537,6 @@ def resolve_jira_issue(comment=None, jira_id=None, merge_branches=None):
     jira_id: an Gobblin JIRA id, either an integer or a string with the form
         GOBBLIN-X. If not provided, the user will be prompted to provide one.
     """
-    try:
-        import jira.client
-    except ImportError:
-        raise PRToolError(
-            "Could not find jira-python library; exiting. Run "
-            "'sudo pip3 install jira' to install.")
 
     if merge_branches is None:
         merge_branches = []


[09/50] incubator-gobblin git commit: [GOBBLIN-395] Add lineage for copying config based dataset

Posted by ab...@apache.org.
[GOBBLIN-395] Add lineage for copying config based dataset

Closes #2269 from zxcware/c2


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/161bef09
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/161bef09
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/161bef09

Branch: refs/heads/0.12.0
Commit: 161bef09dd5cbbbb65f9f6965008c57b632fb075
Parents: c35f76e
Author: zhchen <zh...@linkedin.com>
Authored: Tue Jan 30 17:26:03 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Tue Jan 30 17:26:03 2018 -0800

----------------------------------------------------------------------
 .../data/management/copy/CopyableFile.java      | 25 +++++++++
 .../copy/RecursiveCopyableDataset.java          | 21 +-------
 .../copy/replication/ConfigBasedDataset.java    | 11 ++--
 .../data/management/copy/CopyableFileTest.java  | 53 +++++++++++++++++++-
 .../gobblin/metrics/reporter/EventReporter.java |  2 +-
 5 files changed, 84 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/161bef09/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopyableFile.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopyableFile.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopyableFile.java
index 843a7e3..d2547b4 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopyableFile.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopyableFile.java
@@ -19,6 +19,7 @@ package org.apache.gobblin.data.management.copy;
 
 import org.apache.gobblin.data.management.partition.File;
 import org.apache.gobblin.data.management.copy.PreserveAttributes.Option;
+import org.apache.gobblin.dataset.DatasetConstants;
 import org.apache.gobblin.dataset.DatasetDescriptor;
 import org.apache.gobblin.util.PathUtils;
 import org.apache.gobblin.util.guid.Guid;
@@ -116,6 +117,30 @@ public class CopyableFile extends CopyEntity implements File {
   }
 
   /**
+   * Set file system based source and destination dataset for this {@link CopyableFile}
+   *
+   * @param originFs {@link FileSystem} where this {@link CopyableFile} origins
+   * @param targetFs {@link FileSystem} where this {@link CopyableFile} is copied to
+   */
+  public void setFsDatasets(FileSystem originFs, FileSystem targetFs) {
+    /*
+     * By default, the raw Gobblin dataset for CopyableFile lineage is its parent folder
+     * if itself is not a folder
+     */
+    boolean isDir = origin.isDirectory();
+
+    Path fullSourcePath = Path.getPathWithoutSchemeAndAuthority(origin.getPath());
+    String sourceDatasetName = isDir ? fullSourcePath.toString() : fullSourcePath.getParent().toString();
+    sourceDataset = new DatasetDescriptor(originFs.getScheme(), sourceDatasetName);
+    sourceDataset.addMetadata(DatasetConstants.FS_URI, originFs.getUri().toString());
+
+    Path fullDestinationPath = Path.getPathWithoutSchemeAndAuthority(destination);
+    String destinationDatasetName = isDir ? fullDestinationPath.toString() : fullDestinationPath.getParent().toString();
+    destinationDataset = new DatasetDescriptor(targetFs.getScheme(), destinationDatasetName);
+    destinationDataset.addMetadata(DatasetConstants.FS_URI, targetFs.getUri().toString());
+  }
+
+  /**
    * Get a {@link CopyableFile.Builder}.
    *
    * @param originFs {@link FileSystem} where original file exists.

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/161bef09/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/RecursiveCopyableDataset.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/RecursiveCopyableDataset.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/RecursiveCopyableDataset.java
index 252dafa..2d1f740 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/RecursiveCopyableDataset.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/RecursiveCopyableDataset.java
@@ -20,9 +20,7 @@ package org.apache.gobblin.data.management.copy;
 import org.apache.gobblin.commit.CommitStep;
 import org.apache.gobblin.data.management.copy.entities.PrePublishStep;
 import org.apache.gobblin.data.management.dataset.DatasetUtils;
-import org.apache.gobblin.dataset.DatasetConstants;
 import org.apache.gobblin.dataset.FileSystemDataset;
-import org.apache.gobblin.dataset.DatasetDescriptor;
 import org.apache.gobblin.util.PathUtils;
 import org.apache.gobblin.util.FileListUtils;
 import org.apache.gobblin.util.commit.DeleteFileCommitStep;
@@ -148,24 +146,7 @@ public class RecursiveCopyableDataset implements CopyableDataset, FileSystemData
               .datasetOutputPath(thisTargetPath.toString()).ancestorsOwnerAndPermission(CopyableFile
               .resolveReplicatedOwnerAndPermissionsRecursively(this.fs, file.getPath().getParent(), nonGlobSearchPath,
                   configuration)).build();
-
-      /*
-       * By default, the raw Gobblin dataset for CopyableFile lineage is its parent folder
-       * if itself is not a folder
-       */
-      boolean isDir = file.isDirectory();
-
-      Path fullSourcePath = Path.getPathWithoutSchemeAndAuthority(file.getPath());
-      String sourceDataset = isDir ? fullSourcePath.toString() : fullSourcePath.getParent().toString();
-      DatasetDescriptor source = new DatasetDescriptor(this.fs.getScheme(), sourceDataset);
-      source.addMetadata(DatasetConstants.FS_URI, this.fs.getUri().toString());
-      copyableFile.setSourceDataset(source);
-
-      String destinationDataset = isDir ? thisTargetPath.toString() : thisTargetPath.getParent().toString();
-      DatasetDescriptor destination = new DatasetDescriptor(targetFs.getScheme(), destinationDataset);
-      destination.addMetadata(DatasetConstants.FS_URI, targetFs.getUri().toString());
-      copyableFile.setDestinationDataset(destination);
-
+      copyableFile.setFsDatasets(this.fs, targetFs);
       copyableFiles.add(copyableFile);
     }
     copyEntities.addAll(this.copyableFileFilter.filter(this.fs, targetFs, copyableFiles));

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/161bef09/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDataset.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDataset.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDataset.java
index 293034b..cc893a6 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDataset.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDataset.java
@@ -187,12 +187,11 @@ public class ConfigBasedDataset implements CopyableDataset {
         if (copyToFileMap.containsKey(newPath)) {
           deletedPaths.add(newPath);
         }
-
-        copyableFiles.add(
-            CopyableFile.fromOriginAndDestination(copyFromFs, originFileStatus, copyToFs.makeQualified(newPath),
-                copyConfiguration)
-                .fileSet(PathUtils.getPathWithoutSchemeAndAuthority(copyTo.getDatasetPath()).toString())
-                .build());
+        CopyableFile copyableFile = CopyableFile
+            .fromOriginAndDestination(copyFromFs, originFileStatus, copyToFs.makeQualified(newPath), copyConfiguration)
+            .fileSet(PathUtils.getPathWithoutSchemeAndAuthority(copyTo.getDatasetPath()).toString()).build();
+        copyableFile.setFsDatasets(copyFromFs, copyToFs);
+        copyableFiles.add(copyableFile);
       }
 
       // clean up already checked paths

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/161bef09/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/CopyableFileTest.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/CopyableFileTest.java b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/CopyableFileTest.java
index 30ba0af..986efeb 100644
--- a/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/CopyableFileTest.java
+++ b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/CopyableFileTest.java
@@ -18,6 +18,7 @@ package org.apache.gobblin.data.management.copy;
 
 import java.io.IOException;
 import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.List;
 import java.util.Properties;
 
@@ -36,8 +37,13 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.dataset.DatasetDescriptor;
 import org.apache.gobblin.util.PathUtils;
 
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
 public class CopyableFileTest {
 
   @Test
@@ -86,6 +92,51 @@ public class CopyableFileTest {
 
   }
 
+  @Test
+  public void testSetFsDatasets() throws URISyntaxException {
+    FileSystem originFs = mock(FileSystem.class);
+    String originFsUri = "hdfs://source.company.biz:2000";
+    String originPath = "/data/databases/source/profile";
+    when(originFs.getUri()).thenReturn(new URI(originFsUri));
+    when(originFs.getScheme()).thenReturn("hdfs");
+
+    FileSystem targetFs = mock(FileSystem.class);
+    String targetFsUri = "file:///";
+    String destinationPath = "/data/databases/destination/profile";
+    when(targetFs.getUri()).thenReturn(new URI(targetFsUri));
+    when(targetFs.getScheme()).thenReturn("file");
+
+    // Test when source file is not a directory
+    FileStatus origin = new FileStatus(0l, false, 0, 0l, 0l, new Path(originPath));
+    CopyableFile copyableFile = new CopyableFile(origin, new Path(destinationPath), null, null, null,
+        PreserveAttributes.fromMnemonicString(""), "", 0, 0, Maps.<String, String>newHashMap(), "");
+    copyableFile.setFsDatasets(originFs, targetFs);
+    DatasetDescriptor source = copyableFile.getSourceDataset();
+    Assert.assertEquals(source.getName(), "/data/databases/source");
+    Assert.assertEquals(source.getPlatform(), "hdfs");
+    Assert.assertEquals(source.getMetadata().get("fsUri"), originFsUri);
+    DatasetDescriptor destination = copyableFile.getDestinationDataset();
+    Assert.assertEquals(destination.getName(), "/data/databases/destination");
+    Assert.assertEquals(destination.getPlatform(), "file");
+    Assert.assertEquals(destination.getMetadata().get("fsUri"), targetFsUri);
+
+    // Test when source file is a directory
+    originPath = originFsUri + originPath;
+    destinationPath = targetFsUri + destinationPath;
+    origin = new FileStatus(0l, true, 0, 0l, 0l, new Path(originPath));
+    copyableFile = new CopyableFile(origin, new Path(destinationPath), null, null, null,
+        PreserveAttributes.fromMnemonicString(""), "", 0, 0, Maps.<String, String>newHashMap(), "");
+    copyableFile.setFsDatasets(originFs, targetFs);
+    source = copyableFile.getSourceDataset();
+    Assert.assertEquals(source.getName(), "/data/databases/source/profile");
+    Assert.assertEquals(source.getPlatform(), "hdfs");
+    Assert.assertEquals(source.getMetadata().get("fsUri"), originFsUri);
+    destination = copyableFile.getDestinationDataset();
+    Assert.assertEquals(destination.getName(), "/data/databases/destination/profile");
+    Assert.assertEquals(destination.getPlatform(), "file");
+    Assert.assertEquals(destination.getMetadata().get("fsUri"), targetFsUri);
+  }
+
 
   @Test
   public void testCopyableFileBuilderMinimumConfiguration()
@@ -200,7 +251,7 @@ public class CopyableFileTest {
 
     FileStatus fileStatus = new FileStatus(1, false, 0, 0, 0, 0, FsPermission.getDefault(), "owner", "group", path);
 
-    FileSystem fs = Mockito.mock(FileSystem.class);
+    FileSystem fs = mock(FileSystem.class);
     Mockito.doReturn(fileStatus).when(fs).getFileStatus(path);
     Mockito.doReturn(path).when(fs).makeQualified(path);
     Mockito.doReturn(new URI("hdfs://uri")).when(fs).getUri();

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/161bef09/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/reporter/EventReporter.java
----------------------------------------------------------------------
diff --git a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/reporter/EventReporter.java b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/reporter/EventReporter.java
index 498ad58..a733d6a 100644
--- a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/reporter/EventReporter.java
+++ b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/reporter/EventReporter.java
@@ -125,7 +125,7 @@ public abstract class EventReporter extends ScheduledReporter implements Closeab
     }
     try {
       if (!this.reportingQueue.offer(sanitizeEvent(event), 10, TimeUnit.SECONDS)) {
-        log.error("Enqueuing of event %s at reporter with class %s timed out. Sending of events is probably stuck.",
+        log.error("Enqueuing of event {} at reporter with class {} timed out. Sending of events is probably stuck.",
             event, this.getClass().getCanonicalName());
       }
     } catch (InterruptedException ie) {


[04/50] incubator-gobblin git commit: [GOBBLIN-381][GOBBLIN-368] Add ability to filter hidden directories for ConfigBasedDatasets

Posted by ab...@apache.org.
[GOBBLIN-381][GOBBLIN-368] Add ability to filter hidden directories for ConfigBasedDatasets

Closes #2260 from sv2000/gobblin-381


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/11abf9f5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/11abf9f5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/11abf9f5

Branch: refs/heads/0.12.0
Commit: 11abf9f5f604a90376f73c01ed5e4b73b14c35cf
Parents: f2f6e46
Author: suvasude <su...@linkedin.biz>
Authored: Wed Jan 24 14:43:56 2018 -0800
Committer: Issac Buenrostro <ib...@apache.org>
Committed: Wed Jan 24 14:43:56 2018 -0800

----------------------------------------------------------------------
 .../copy/replication/ConfigBasedDataset.java    |  64 ++++----
 .../copy/replication/HadoopFsEndPoint.java      |  12 +-
 .../replication/ReplicaHadoopFsEndPoint.java    |  53 +++---
 .../replication/SourceHadoopFsEndPoint.java     |  37 +++--
 .../replication/ConfigBasedDatasetTest.java     | 162 +++++++++++++++++++
 .../replication/ConfigBasedDatasetsTest.java    | 135 ----------------
 .../configBasedDatasetTest/src/_dir1/file1      |   1 +
 .../configBasedDatasetTest/src/_dir1/file2      |   1 +
 8 files changed, 263 insertions(+), 202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11abf9f5/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDataset.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDataset.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDataset.java
index 3881323..293034b 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDataset.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDataset.java
@@ -29,8 +29,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 
-import com.google.common.base.Predicate;
-import com.google.common.collect.Collections2;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
@@ -71,14 +69,21 @@ public class ConfigBasedDataset implements CopyableDataset {
   private final ReplicationConfiguration rc;
   private String datasetURN;
   private boolean watermarkEnabled;
+  private final PathFilter pathFilter;
+
+  //Apply filter to directories
+  private final boolean applyFilterToDirectories;
 
   public ConfigBasedDataset(ReplicationConfiguration rc, Properties props, CopyRoute copyRoute) {
     this.props = props;
     this.copyRoute = copyRoute;
     this.rc = rc;
     calculateDatasetURN();
-    this.watermarkEnabled = Boolean.parseBoolean
-        (this.props.getProperty(ConfigBasedDatasetsFinder.WATERMARK_ENABLE, "true"));
+    this.watermarkEnabled =
+        Boolean.parseBoolean(this.props.getProperty(ConfigBasedDatasetsFinder.WATERMARK_ENABLE, "true"));
+    this.pathFilter = DatasetUtils.instantiatePathFilter(this.props);
+    this.applyFilterToDirectories =
+        Boolean.parseBoolean(this.props.getProperty(CopyConfiguration.APPLY_FILTER_TO_DIRECTORIES, "false"));
   }
 
   public ConfigBasedDataset(ReplicationConfiguration rc, Properties props, CopyRoute copyRoute, String datasetURN) {
@@ -86,9 +91,12 @@ public class ConfigBasedDataset implements CopyableDataset {
     this.copyRoute = copyRoute;
     this.rc = rc;
     this.datasetURN = datasetURN;
+    this.pathFilter = DatasetUtils.instantiatePathFilter(this.props);
+    this.applyFilterToDirectories =
+        Boolean.parseBoolean(this.props.getProperty(CopyConfiguration.APPLY_FILTER_TO_DIRECTORIES, "false"));
   }
 
-  private void calculateDatasetURN(){
+  private void calculateDatasetURN() {
     EndPoint e = this.copyRoute.getCopyTo();
     if (e instanceof HadoopFsEndPoint) {
       HadoopFsEndPoint copyTo = (HadoopFsEndPoint) e;
@@ -120,6 +128,14 @@ public class ConfigBasedDataset implements CopyableDataset {
       return copyableFiles;
     }
 
+    //For {@link HadoopFsEndPoint}s, set pathfilter and applyFilterToDirectories
+    HadoopFsEndPoint copyFrom = (HadoopFsEndPoint) copyFromRaw;
+    HadoopFsEndPoint copyTo = (HadoopFsEndPoint) copyToRaw;
+    copyFrom.setPathFilter(pathFilter);
+    copyFrom.setApplyFilterToDirectories(applyFilterToDirectories);
+    copyTo.setPathFilter(pathFilter);
+    copyTo.setApplyFilterToDirectories(applyFilterToDirectories);
+
     if (this.watermarkEnabled) {
       if ((!copyFromRaw.getWatermark().isPresent() && copyToRaw.getWatermark().isPresent()) || (
           copyFromRaw.getWatermark().isPresent() && copyToRaw.getWatermark().isPresent()
@@ -132,8 +148,6 @@ public class ConfigBasedDataset implements CopyableDataset {
       }
     }
 
-    HadoopFsEndPoint copyFrom = (HadoopFsEndPoint) copyFromRaw;
-    HadoopFsEndPoint copyTo = (HadoopFsEndPoint) copyToRaw;
     Configuration conf = HadoopUtils.newConfiguration();
     FileSystem copyFromFs = FileSystem.get(copyFrom.getFsURI(), conf);
     FileSystem copyToFs = FileSystem.get(copyTo.getFsURI(), conf);
@@ -141,20 +155,10 @@ public class ConfigBasedDataset implements CopyableDataset {
     Collection<FileStatus> allFilesInSource = copyFrom.getFiles();
     Collection<FileStatus> allFilesInTarget = copyTo.getFiles();
 
-    final PathFilter pathFilter = DatasetUtils.instantiatePathFilter(this.props);
-    Predicate<FileStatus> predicate = new Predicate<FileStatus>() {
-      @Override
-      public boolean apply(FileStatus input) {
-        return pathFilter.accept(input.getPath());
-      }
-    };
-
-    Set<FileStatus> copyFromFileStatuses = Sets.newHashSet(Collections2.filter(allFilesInSource, predicate));
+    Set<FileStatus> copyFromFileStatuses = Sets.newHashSet(allFilesInSource);
     Map<Path, FileStatus> copyToFileMap = Maps.newHashMap();
-    for(FileStatus f: allFilesInTarget){
-      if(pathFilter.accept(f.getPath())){
-        copyToFileMap.put(PathUtils.getPathWithoutSchemeAndAuthority(f.getPath()), f);
-      }
+    for (FileStatus f : allFilesInTarget) {
+      copyToFileMap.put(PathUtils.getPathWithoutSchemeAndAuthority(f.getPath()), f);
     }
 
     Collection<Path> deletedPaths = Lists.newArrayList();
@@ -184,10 +188,11 @@ public class ConfigBasedDataset implements CopyableDataset {
           deletedPaths.add(newPath);
         }
 
-        copyableFiles
-            .add(CopyableFile.fromOriginAndDestination(copyFromFs, originFileStatus, copyToFs.makeQualified(newPath), copyConfiguration)
-                .fileSet(PathUtils.getPathWithoutSchemeAndAuthority(copyTo.getDatasetPath()).toString()).build());
-
+        copyableFiles.add(
+            CopyableFile.fromOriginAndDestination(copyFromFs, originFileStatus, copyToFs.makeQualified(newPath),
+                copyConfiguration)
+                .fileSet(PathUtils.getPathWithoutSchemeAndAuthority(copyTo.getDatasetPath()).toString())
+                .build());
       }
 
       // clean up already checked paths
@@ -202,18 +207,17 @@ public class ConfigBasedDataset implements CopyableDataset {
     // delete old files first
     if (!deletedPaths.isEmpty()) {
       DeleteFileCommitStep deleteCommitStep = DeleteFileCommitStep.fromPaths(copyToFs, deletedPaths, this.props);
-      copyableFiles.add(new PrePublishStep(copyTo.getDatasetPath().toString(), Maps.<String, String> newHashMap(),
-          deleteCommitStep, 0));
+      copyableFiles.add(
+          new PrePublishStep(copyTo.getDatasetPath().toString(), Maps.<String, String>newHashMap(), deleteCommitStep,
+              0));
     }
 
     // generate the watermark file even if watermark checking is disabled. Make sure it can come into functional once disired.
     if ((!watermarkMetadataCopied) && copyFrom.getWatermark().isPresent()) {
-      copyableFiles.add(new PostPublishStep(copyTo.getDatasetPath().toString(), Maps.<String, String> newHashMap(),
+      copyableFiles.add(new PostPublishStep(copyTo.getDatasetPath().toString(), Maps.<String, String>newHashMap(),
           new WatermarkMetadataGenerationCommitStep(copyTo.getFsURI().toString(), copyTo.getDatasetPath(),
-              copyFrom.getWatermark().get()),
-          1));
+              copyFrom.getWatermark().get()), 1));
     }
     return copyableFiles;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11abf9f5/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/HadoopFsEndPoint.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/HadoopFsEndPoint.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/HadoopFsEndPoint.java
index 97fd20e..ea93ea8 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/HadoopFsEndPoint.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/HadoopFsEndPoint.java
@@ -20,6 +20,8 @@ package org.apache.gobblin.data.management.copy.replication;
 import java.io.IOException;
 import java.net.URI;
 
+import lombok.Getter;
+import lombok.Setter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -28,9 +30,15 @@ import com.typesafe.config.Config;
 
 import org.apache.gobblin.util.HadoopUtils;
 import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.fs.PathFilter;
+
 
 @Slf4j
-public abstract class HadoopFsEndPoint implements EndPoint{
+@Getter
+@Setter
+public abstract class HadoopFsEndPoint implements EndPoint {
+  private PathFilter pathFilter;
+  private boolean applyFilterToDirectories;
 
   /**
    *
@@ -56,7 +64,7 @@ public abstract class HadoopFsEndPoint implements EndPoint{
    * @param path The path to be checked. For fs availability checking, just use "/"
    * @return If the filesystem/path exists or not.
    */
-  public boolean isPathAvailable(Path path){
+  public boolean isPathAvailable(Path path) {
     try {
       Configuration conf = HadoopUtils.newConfiguration();
       FileSystem fs = FileSystem.get(this.getFsURI(), conf);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11abf9f5/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ReplicaHadoopFsEndPoint.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ReplicaHadoopFsEndPoint.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ReplicaHadoopFsEndPoint.java
index 8ab4e7e..024b239 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ReplicaHadoopFsEndPoint.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/ReplicaHadoopFsEndPoint.java
@@ -72,30 +72,31 @@ public class ReplicaHadoopFsEndPoint extends HadoopFsEndPoint {
   }
 
   @Override
-  public synchronized Collection<FileStatus> getFiles() throws IOException{
-    if(filesInitialized){
+  public synchronized Collection<FileStatus> getFiles() throws IOException {
+    if (filesInitialized) {
       return this.allFileStatus;
     }
 
     this.filesInitialized = true;
     FileSystem fs = FileSystem.get(rc.getFsURI(), new Configuration());
 
-    if(!fs.exists(this.rc.getPath())){
+    if (!fs.exists(this.rc.getPath())) {
       return Collections.emptyList();
     }
 
     Collection<Path> validPaths = ReplicationDataValidPathPicker.getValidPaths(this);
-        //ReplicationDataValidPathPicker.getValidPaths(fs, this.rc.getPath(), this.rdc);
+    //ReplicationDataValidPathPicker.getValidPaths(fs, this.rc.getPath(), this.rdc);
 
-    for(Path p: validPaths){
-      this.allFileStatus.addAll(FileListUtils.listFilesRecursively(fs, p));
+    for (Path p : validPaths) {
+      this.allFileStatus.addAll(
+          FileListUtils.listFilesRecursively(fs, p, super.getPathFilter(), super.isApplyFilterToDirectories()));
     }
     return this.allFileStatus;
   }
 
   @Override
   public synchronized Optional<ComparableWatermark> getWatermark() {
-    if(this.watermarkInitialized) {
+    if (this.watermarkInitialized) {
       return this.cachedWatermark;
     }
 
@@ -104,12 +105,12 @@ public class ReplicaHadoopFsEndPoint extends HadoopFsEndPoint {
       Path metaData = new Path(rc.getPath(), WATERMARK_FILE);
       FileSystem fs = FileSystem.get(rc.getFsURI(), new Configuration());
       if (fs.exists(metaData)) {
-        try(FSDataInputStream fin = fs.open(metaData)){
+        try (FSDataInputStream fin = fs.open(metaData)) {
           InputStreamReader reader = new InputStreamReader(fin, Charsets.UTF_8);
           String content = CharStreams.toString(reader);
           Watermark w = WatermarkMetadataUtil.deserialize(content);
-          if(w instanceof ComparableWatermark){
-            this.cachedWatermark = Optional.of((ComparableWatermark)w);
+          if (w instanceof ComparableWatermark) {
+            this.cachedWatermark = Optional.of((ComparableWatermark) w);
           }
         }
         return this.cachedWatermark;
@@ -120,7 +121,7 @@ public class ReplicaHadoopFsEndPoint extends HadoopFsEndPoint {
     } catch (IOException e) {
       log.warn("Can not find " + WATERMARK_FILE + " for replica " + this);
       return this.cachedWatermark;
-    } catch (WatermarkMetadataUtil.WatermarkMetadataMulFormatException e){
+    } catch (WatermarkMetadataUtil.WatermarkMetadataMulFormatException e) {
       log.warn("Can not create watermark from " + WATERMARK_FILE + " for replica " + this);
       return this.cachedWatermark;
     }
@@ -143,8 +144,11 @@ public class ReplicaHadoopFsEndPoint extends HadoopFsEndPoint {
 
   @Override
   public String toString() {
-    return Objects.toStringHelper(this.getClass()).add("is source", this.isSource()).add("end point name", this.getEndPointName())
-        .add("hadoopfs config", this.rc).toString();
+    return Objects.toStringHelper(this.getClass())
+        .add("is source", this.isSource())
+        .add("end point name", this.getEndPointName())
+        .add("hadoopfs config", this.rc)
+        .toString();
   }
 
   @Override
@@ -153,7 +157,7 @@ public class ReplicaHadoopFsEndPoint extends HadoopFsEndPoint {
   }
 
   @Override
-  public Path getDatasetPath(){
+  public Path getDatasetPath() {
     return this.rc.getPath();
   }
 
@@ -168,23 +172,30 @@ public class ReplicaHadoopFsEndPoint extends HadoopFsEndPoint {
 
   @Override
   public boolean equals(Object obj) {
-    if (this == obj)
+    if (this == obj) {
       return true;
-    if (obj == null)
+    }
+    if (obj == null) {
       return false;
-    if (getClass() != obj.getClass())
+    }
+    if (getClass() != obj.getClass()) {
       return false;
+    }
     ReplicaHadoopFsEndPoint other = (ReplicaHadoopFsEndPoint) obj;
     if (rc == null) {
-      if (other.rc != null)
+      if (other.rc != null) {
         return false;
-    } else if (!rc.equals(other.rc))
+      }
+    } else if (!rc.equals(other.rc)) {
       return false;
+    }
     if (replicaName == null) {
-      if (other.replicaName != null)
+      if (other.replicaName != null) {
         return false;
-    } else if (!replicaName.equals(other.replicaName))
+      }
+    } else if (!replicaName.equals(other.replicaName)) {
       return false;
+    }
     return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11abf9f5/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/SourceHadoopFsEndPoint.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/SourceHadoopFsEndPoint.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/SourceHadoopFsEndPoint.java
index 0769c5c..2a56f2e 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/SourceHadoopFsEndPoint.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/replication/SourceHadoopFsEndPoint.java
@@ -39,7 +39,7 @@ import lombok.extern.slf4j.Slf4j;
 
 
 @Slf4j
-public class SourceHadoopFsEndPoint extends HadoopFsEndPoint{
+public class SourceHadoopFsEndPoint extends HadoopFsEndPoint {
 
   @Getter
   private final HadoopFsReplicaConfig rc;
@@ -57,8 +57,8 @@ public class SourceHadoopFsEndPoint extends HadoopFsEndPoint{
   }
 
   @Override
-  public synchronized Collection<FileStatus> getFiles() throws IOException{
-    if(!this.initialized){
+  public synchronized Collection<FileStatus> getFiles() throws IOException {
+    if (!this.initialized) {
       this.getWatermark();
     }
     return this.allFileStatus;
@@ -66,7 +66,7 @@ public class SourceHadoopFsEndPoint extends HadoopFsEndPoint{
 
   @Override
   public synchronized Optional<ComparableWatermark> getWatermark() {
-    if(this.initialized) {
+    if (this.initialized) {
       return this.cachedWatermark;
     }
     try {
@@ -74,8 +74,9 @@ public class SourceHadoopFsEndPoint extends HadoopFsEndPoint{
       FileSystem fs = FileSystem.get(rc.getFsURI(), new Configuration());
 
       Collection<Path> validPaths = ReplicationDataValidPathPicker.getValidPaths(this);
-      for(Path p: validPaths){
-        this.allFileStatus.addAll(FileListUtils.listFilesRecursively(fs, p));
+      for (Path p : validPaths) {
+        this.allFileStatus.addAll(
+            FileListUtils.listFilesRecursively(fs, p, super.getPathFilter(), super.isApplyFilterToDirectories()));
       }
 
       for (FileStatus f : this.allFileStatus) {
@@ -115,8 +116,11 @@ public class SourceHadoopFsEndPoint extends HadoopFsEndPoint{
 
   @Override
   public String toString() {
-    return Objects.toStringHelper(this.getClass()).add("is source", this.isSource()).add("end point name", this.getEndPointName())
-        .add("hadoopfs config", this.rc).toString();
+    return Objects.toStringHelper(this.getClass())
+        .add("is source", this.isSource())
+        .add("end point name", this.getEndPointName())
+        .add("hadoopfs config", this.rc)
+        .toString();
   }
 
   @Override
@@ -125,7 +129,7 @@ public class SourceHadoopFsEndPoint extends HadoopFsEndPoint{
   }
 
   @Override
-  public Path getDatasetPath(){
+  public Path getDatasetPath() {
     return this.rc.getPath();
   }
 
@@ -139,18 +143,23 @@ public class SourceHadoopFsEndPoint extends HadoopFsEndPoint{
 
   @Override
   public boolean equals(Object obj) {
-    if (this == obj)
+    if (this == obj) {
       return true;
-    if (obj == null)
+    }
+    if (obj == null) {
       return false;
-    if (getClass() != obj.getClass())
+    }
+    if (getClass() != obj.getClass()) {
       return false;
+    }
     SourceHadoopFsEndPoint other = (SourceHadoopFsEndPoint) obj;
     if (rc == null) {
-      if (other.rc != null)
+      if (other.rc != null) {
         return false;
-    } else if (!rc.equals(other.rc))
+      }
+    } else if (!rc.equals(other.rc)) {
       return false;
+    }
     return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11abf9f5/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDatasetTest.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDatasetTest.java b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDatasetTest.java
new file mode 100644
index 0000000..f925243
--- /dev/null
+++ b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDatasetTest.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.data.management.copy.replication;
+
+import java.net.URI;
+import java.util.Collection;
+import java.util.Properties;
+import java.util.Set;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.gobblin.data.management.dataset.DatasetUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Sets;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.data.management.copy.CopyConfiguration;
+import org.apache.gobblin.data.management.copy.CopyEntity;
+import org.apache.gobblin.data.management.copy.CopyableFile;
+import org.apache.gobblin.data.management.copy.PreserveAttributes;
+import org.apache.gobblin.data.management.copy.entities.PostPublishStep;
+import org.apache.gobblin.data.management.copy.entities.PrePublishStep;
+import org.apache.gobblin.source.extractor.ComparableWatermark;
+import org.apache.gobblin.source.extractor.extract.LongWatermark;
+import org.apache.gobblin.util.FileListUtils;
+import org.apache.gobblin.util.PathUtils;
+import org.apache.gobblin.util.commit.DeleteFileCommitStep;
+
+
+/**
+ * Unit test for {@link ConfigBasedDataset}
+ * @author mitu
+ *
+ */
+@Test(groups = {"gobblin.data.management.copy.replication"})
+@Slf4j
+public class ConfigBasedDatasetTest {
+
+  public Collection<? extends CopyEntity> testGetCopyableFilesHelper(String sourceDir, String destinationDir,
+      long sourceWatermark, boolean isFilterEnabled) throws Exception {
+    FileSystem localFs = FileSystem.getLocal(new Configuration());
+    URI local = localFs.getUri();
+
+    Properties properties = new Properties();
+    properties.setProperty(ConfigurationKeys.DATA_PUBLISHER_FINAL_DIR, "/publisher");
+    PathFilter pathFilter = DatasetUtils.instantiatePathFilter(properties);
+    boolean applyFilterToDirectories = false;
+    if (isFilterEnabled) {
+      properties.setProperty(DatasetUtils.CONFIGURATION_KEY_PREFIX + "path.filter.class",
+          "org.apache.gobblin.util.filters.HiddenFilter");
+      properties.setProperty(CopyConfiguration.APPLY_FILTER_TO_DIRECTORIES, "true");
+
+      pathFilter = DatasetUtils.instantiatePathFilter(properties);
+      applyFilterToDirectories =
+          Boolean.parseBoolean(properties.getProperty(CopyConfiguration.APPLY_FILTER_TO_DIRECTORIES, "false"));
+    }
+
+    CopyConfiguration copyConfiguration =
+        CopyConfiguration.builder(FileSystem.getLocal(new Configuration()), properties)
+            .publishDir(new Path(destinationDir))
+            .preserve(PreserveAttributes.fromMnemonicString("ugp"))
+            .build();
+
+    ReplicationMetaData mockMetaData = Mockito.mock(ReplicationMetaData.class);
+    Mockito.when(mockMetaData.toString()).thenReturn("Mock Meta Data");
+
+    ReplicationConfiguration mockRC = Mockito.mock(ReplicationConfiguration.class);
+    Mockito.when(mockRC.getCopyMode()).thenReturn(ReplicationCopyMode.PULL);
+    Mockito.when(mockRC.getMetaData()).thenReturn(mockMetaData);
+
+    HadoopFsEndPoint copyFrom = Mockito.mock(HadoopFsEndPoint.class);
+    Mockito.when(copyFrom.getDatasetPath()).thenReturn(new Path(sourceDir));
+    Mockito.when(copyFrom.getFsURI()).thenReturn(local);
+    ComparableWatermark sw = new LongWatermark(sourceWatermark);
+    Mockito.when(copyFrom.getWatermark()).thenReturn(Optional.of(sw));
+    Mockito.when(copyFrom.getFiles())
+        .thenReturn(
+            FileListUtils.listFilesRecursively(localFs, new Path(sourceDir), pathFilter, applyFilterToDirectories));
+
+    HadoopFsEndPoint copyTo = Mockito.mock(HadoopFsEndPoint.class);
+    Mockito.when(copyTo.getDatasetPath()).thenReturn(new Path(destinationDir));
+    Mockito.when(copyTo.getFsURI()).thenReturn(local);
+    Optional<ComparableWatermark> tmp = Optional.absent();
+    Mockito.when(copyTo.getWatermark()).thenReturn(tmp);
+    Mockito.when(copyTo.getFiles())
+        .thenReturn(FileListUtils.listFilesRecursively(localFs, new Path(destinationDir), pathFilter,
+            applyFilterToDirectories));
+
+    CopyRoute route = Mockito.mock(CopyRoute.class);
+    Mockito.when(route.getCopyFrom()).thenReturn(copyFrom);
+    Mockito.when(route.getCopyTo()).thenReturn(copyTo);
+
+    ConfigBasedDataset dataset = new ConfigBasedDataset(mockRC, properties, route);
+    Collection<? extends CopyEntity> copyableFiles = dataset.getCopyableFiles(localFs, copyConfiguration);
+    return copyableFiles;
+  }
+
+  @Test
+  public void testGetCopyableFiles() throws Exception {
+    String sourceDir = getClass().getClassLoader().getResource("configBasedDatasetTest/src").getFile();
+    String destinationDir = getClass().getClassLoader().getResource("configBasedDatasetTest/dest").getFile();
+    long sourceWatermark = 100L;
+
+    Collection<? extends CopyEntity> copyableFiles =
+        testGetCopyableFilesHelper(sourceDir, destinationDir, sourceWatermark, false);
+    Assert.assertEquals(copyableFiles.size(), 8);
+    copyableFiles = testGetCopyableFilesHelper(sourceDir, destinationDir, sourceWatermark, true);
+    Assert.assertEquals(copyableFiles.size(), 6);
+
+    Set<Path> paths =
+        Sets.newHashSet(new Path("dir1/file2"), new Path("dir1/file1"), new Path("dir2/file1"), new Path("dir2/file3"));
+    for (CopyEntity copyEntity : copyableFiles) {
+      if (copyEntity instanceof CopyableFile) {
+        CopyableFile file = (CopyableFile) copyEntity;
+        Path originRelativePath =
+            PathUtils.relativizePath(PathUtils.getPathWithoutSchemeAndAuthority(file.getOrigin().getPath()),
+                PathUtils.getPathWithoutSchemeAndAuthority(new Path(sourceDir)));
+        Path targetRelativePath =
+            PathUtils.relativizePath(PathUtils.getPathWithoutSchemeAndAuthority(file.getDestination()),
+                PathUtils.getPathWithoutSchemeAndAuthority(new Path(destinationDir)));
+
+        Assert.assertTrue(paths.contains(originRelativePath));
+        Assert.assertTrue(paths.contains(targetRelativePath));
+        Assert.assertEquals(originRelativePath, targetRelativePath);
+      } else if (copyEntity instanceof PrePublishStep) {
+        PrePublishStep pre = (PrePublishStep) copyEntity;
+        Assert.assertTrue(pre.getStep() instanceof DeleteFileCommitStep);
+        // need to delete this file
+        Assert.assertTrue(pre.explain().indexOf("configBasedDatasetTest/dest/dir1/file1") > 0);
+      } else if (copyEntity instanceof PostPublishStep) {
+        PostPublishStep post = (PostPublishStep) copyEntity;
+        Assert.assertTrue(post.getStep() instanceof WatermarkMetadataGenerationCommitStep);
+        Assert.assertTrue(
+            post.explain().indexOf("dest/_metadata") > 0 && post.explain().indexOf("" + sourceWatermark) > 0);
+      } else {
+        throw new Exception("Wrong type");
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11abf9f5/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDatasetsTest.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDatasetsTest.java b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDatasetsTest.java
deleted file mode 100644
index 1965a41..0000000
--- a/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/replication/ConfigBasedDatasetsTest.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.gobblin.data.management.copy.replication;
-
-import java.net.URI;
-import java.util.Collection;
-import java.util.Properties;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.mockito.Mockito;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Sets;
-
-import org.apache.gobblin.configuration.ConfigurationKeys;
-import org.apache.gobblin.data.management.copy.CopyConfiguration;
-import org.apache.gobblin.data.management.copy.CopyEntity;
-import org.apache.gobblin.data.management.copy.CopyableFile;
-import org.apache.gobblin.data.management.copy.PreserveAttributes;
-import org.apache.gobblin.data.management.copy.entities.PostPublishStep;
-import org.apache.gobblin.data.management.copy.entities.PrePublishStep;
-import org.apache.gobblin.source.extractor.ComparableWatermark;
-import org.apache.gobblin.source.extractor.extract.LongWatermark;
-import org.apache.gobblin.util.FileListUtils;
-import org.apache.gobblin.util.PathUtils;
-import org.apache.gobblin.util.commit.DeleteFileCommitStep;
-
-
-/**
- * Unit test for {@link ConfigBasedDatasets}
- * @author mitu
- *
- */
-@Test(groups = {"gobblin.data.management.copy.replication"})
-
-public class ConfigBasedDatasetsTest {
-
-  @Test
-  public void testGetCopyableFiles() throws Exception {
-    String sourceDir = getClass().getClassLoader().getResource("configBasedDatasetTest/src").getFile();
-    String destinationDir = getClass().getClassLoader().getResource("configBasedDatasetTest/dest").getFile();
-    FileSystem localFs = FileSystem.getLocal(new Configuration());
-    URI local = localFs.getUri();
-    long sourceWatermark = 100L;
-
-    Properties properties = new Properties();
-    properties.setProperty(ConfigurationKeys.DATA_PUBLISHER_FINAL_DIR, "/publisher");
-
-    CopyConfiguration copyConfiguration =
-        CopyConfiguration.builder(FileSystem.getLocal(new Configuration()), properties).publishDir(new Path(destinationDir))
-        .preserve(PreserveAttributes.fromMnemonicString("ugp")).build();
-
-    ReplicationMetaData mockMetaData = Mockito.mock(ReplicationMetaData.class);
-    Mockito.when(mockMetaData.toString()).thenReturn("Mock Meta Data");
-
-    ReplicationConfiguration mockRC = Mockito.mock(ReplicationConfiguration.class);
-    Mockito.when(mockRC.getCopyMode()).thenReturn(ReplicationCopyMode.PULL);
-    Mockito.when(mockRC.getMetaData()).thenReturn(mockMetaData);
-
-    HadoopFsEndPoint copyFrom = Mockito.mock(HadoopFsEndPoint.class);
-    Mockito.when(copyFrom.getDatasetPath()).thenReturn(new Path(sourceDir));
-    Mockito.when(copyFrom.getFsURI()).thenReturn(local);
-    ComparableWatermark sw = new LongWatermark(sourceWatermark);
-    Mockito.when(copyFrom.getWatermark()).thenReturn(Optional.of(sw));
-    Mockito.when(copyFrom.getFiles()).thenReturn(FileListUtils.listFilesRecursively(localFs, new Path(sourceDir)));
-
-    HadoopFsEndPoint copyTo = Mockito.mock(HadoopFsEndPoint.class);
-    Mockito.when(copyTo.getDatasetPath()).thenReturn(new Path(destinationDir));
-    Mockito.when(copyTo.getFsURI()).thenReturn(local);
-    Optional<ComparableWatermark>tmp = Optional.absent();
-    Mockito.when(copyTo.getWatermark()).thenReturn(tmp);
-    Mockito.when(copyTo.getFiles()).thenReturn(FileListUtils.listFilesRecursively(localFs, new Path(destinationDir)));
-
-    CopyRoute route = Mockito.mock(CopyRoute.class);
-    Mockito.when(route.getCopyFrom()).thenReturn(copyFrom);
-    Mockito.when(route.getCopyTo()).thenReturn(copyTo);
-
-    ConfigBasedDataset dataset = new ConfigBasedDataset(mockRC, properties, route);
-
-    Collection<? extends CopyEntity> copyableFiles = dataset.getCopyableFiles(localFs, copyConfiguration);
-    Assert.assertEquals(copyableFiles.size(), 6);
-
-    Set<Path> paths = Sets.newHashSet(new Path("dir1/file2"), new Path("dir1/file1"), new Path("dir2/file1"), new Path("dir2/file3"));
-    for (CopyEntity copyEntity : copyableFiles) {
-      if(copyEntity instanceof CopyableFile) {
-        CopyableFile file = (CopyableFile) copyEntity;
-        Path originRelativePath =
-            PathUtils.relativizePath(PathUtils.getPathWithoutSchemeAndAuthority(file.getOrigin().getPath()),
-                PathUtils.getPathWithoutSchemeAndAuthority(new Path(sourceDir)));
-        Path targetRelativePath =
-            PathUtils.relativizePath(PathUtils.getPathWithoutSchemeAndAuthority(file.getDestination()),
-                PathUtils.getPathWithoutSchemeAndAuthority(new Path(destinationDir)));
-
-        Assert.assertTrue(paths.contains(originRelativePath));
-        Assert.assertTrue(paths.contains(targetRelativePath));
-        Assert.assertEquals(originRelativePath, targetRelativePath);
-      }
-      else if(copyEntity instanceof PrePublishStep){
-        PrePublishStep pre = (PrePublishStep)copyEntity;
-        Assert.assertTrue(pre.getStep() instanceof DeleteFileCommitStep);
-        // need to delete this file
-        Assert.assertTrue(pre.explain().indexOf("configBasedDatasetTest/dest/dir1/file1") > 0);
-      }
-      else if(copyEntity instanceof PostPublishStep){
-        PostPublishStep post = (PostPublishStep)copyEntity;
-        Assert.assertTrue(post.getStep() instanceof WatermarkMetadataGenerationCommitStep);
-        Assert.assertTrue(post.explain().indexOf("dest/_metadata") > 0 && post.explain().indexOf(""+sourceWatermark)>0);
-      }
-      else{
-        throw new Exception("Wrong type");
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11abf9f5/gobblin-data-management/src/test/resources/configBasedDatasetTest/src/_dir1/file1
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/test/resources/configBasedDatasetTest/src/_dir1/file1 b/gobblin-data-management/src/test/resources/configBasedDatasetTest/src/_dir1/file1
new file mode 100644
index 0000000..d87e628
--- /dev/null
+++ b/gobblin-data-management/src/test/resources/configBasedDatasetTest/src/_dir1/file1
@@ -0,0 +1 @@
+_dir1:file1content

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/11abf9f5/gobblin-data-management/src/test/resources/configBasedDatasetTest/src/_dir1/file2
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/test/resources/configBasedDatasetTest/src/_dir1/file2 b/gobblin-data-management/src/test/resources/configBasedDatasetTest/src/_dir1/file2
new file mode 100644
index 0000000..248eb41
--- /dev/null
+++ b/gobblin-data-management/src/test/resources/configBasedDatasetTest/src/_dir1/file2
@@ -0,0 +1 @@
+_dir1:file2content


[08/50] incubator-gobblin git commit: [GOBBLIN-392] Load all dataset states when getLatestDatasetStatesByUrns() is called

Posted by ab...@apache.org.
[GOBBLIN-392] Load all dataset states when getLatestDatasetStatesByUrns() is called

Closes #2268 from htran1/fix_dataset_state_fetch


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/c35f76e4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/c35f76e4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/c35f76e4

Branch: refs/heads/0.12.0
Commit: c35f76e4e2a0f9f25580924d46cef1b732af7d63
Parents: 378ccaa
Author: Hung Tran <hu...@linkedin.com>
Authored: Mon Jan 29 17:08:46 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Mon Jan 29 17:08:46 2018 -0800

----------------------------------------------------------------------
 .../gobblin/runtime/ZkDatasetStateStore.java     |  4 ++--
 .../gobblin/runtime/ZkDatasetStateStoreTest.java | 19 ++++++++++++++++++-
 .../gobblin/runtime/MysqlDatasetStateStore.java  |  4 ++--
 .../runtime/MysqlDatasetStateStoreTest.java      | 19 ++++++++++++++++++-
 4 files changed, 40 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c35f76e4/gobblin-modules/gobblin-helix/src/main/java/org/apache/gobblin/runtime/ZkDatasetStateStore.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-helix/src/main/java/org/apache/gobblin/runtime/ZkDatasetStateStore.java b/gobblin-modules/gobblin-helix/src/main/java/org/apache/gobblin/runtime/ZkDatasetStateStore.java
index dbde3fc..e9ecf35 100644
--- a/gobblin-modules/gobblin-helix/src/main/java/org/apache/gobblin/runtime/ZkDatasetStateStore.java
+++ b/gobblin-modules/gobblin-helix/src/main/java/org/apache/gobblin/runtime/ZkDatasetStateStore.java
@@ -65,8 +65,8 @@ public class ZkDatasetStateStore extends ZkStateStore<JobState.DatasetState>
       }});
 
     Map<String, JobState.DatasetState> datasetStatesByUrns = Maps.newHashMap();
-    if (!previousDatasetStates.isEmpty()) {
-      JobState.DatasetState previousDatasetState = previousDatasetStates.get(0);
+
+    for (JobState.DatasetState previousDatasetState : previousDatasetStates) {
       datasetStatesByUrns.put(previousDatasetState.getDatasetUrn(), previousDatasetState);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c35f76e4/gobblin-modules/gobblin-helix/src/test/java/org/apache/gobblin/runtime/ZkDatasetStateStoreTest.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-helix/src/test/java/org/apache/gobblin/runtime/ZkDatasetStateStoreTest.java b/gobblin-modules/gobblin-helix/src/test/java/org/apache/gobblin/runtime/ZkDatasetStateStoreTest.java
index 742aa98..1091cf7 100644
--- a/gobblin-modules/gobblin-helix/src/test/java/org/apache/gobblin/runtime/ZkDatasetStateStoreTest.java
+++ b/gobblin-modules/gobblin-helix/src/test/java/org/apache/gobblin/runtime/ZkDatasetStateStoreTest.java
@@ -44,6 +44,7 @@ public class ZkDatasetStateStoreTest {
   private static final String TEST_JOB_ID = "TestJob1";
   private static final String TEST_TASK_ID_PREFIX = "TestTask-";
   private static final String TEST_DATASET_URN = "TestDataset";
+  private static final String TEST_DATASET_URN2 = "TestDataset2";
 
   private TestingServer testingServer;
   private StateStore<JobState> zkJobStateStore;
@@ -142,6 +143,13 @@ public class ZkDatasetStateStoreTest {
     }
 
     zkDatasetStateStore.persistDatasetState(TEST_DATASET_URN, datasetState);
+
+    // persist a second dataset state to test that retrieval of multiple dataset states works
+    datasetState.setDatasetUrn(TEST_DATASET_URN2);
+    datasetState.setId(TEST_DATASET_URN2);
+    datasetState.setDuration(2000);
+
+    zkDatasetStateStore.persistDatasetState(TEST_DATASET_URN2, datasetState);
   }
 
   @Test(dependsOnMethods = "testPersistDatasetState")
@@ -171,7 +179,7 @@ public class ZkDatasetStateStoreTest {
   public void testGetPreviousDatasetStatesByUrns() throws IOException {
     Map<String, JobState.DatasetState> datasetStatesByUrns =
         zkDatasetStateStore.getLatestDatasetStatesByUrns(TEST_JOB_NAME);
-    Assert.assertEquals(datasetStatesByUrns.size(), 1);
+    Assert.assertEquals(datasetStatesByUrns.size(), 2);
 
     JobState.DatasetState datasetState = datasetStatesByUrns.get(TEST_DATASET_URN);
     Assert.assertEquals(datasetState.getDatasetUrn(), TEST_DATASET_URN);
@@ -181,6 +189,15 @@ public class ZkDatasetStateStoreTest {
     Assert.assertEquals(datasetState.getStartTime(), this.startTime);
     Assert.assertEquals(datasetState.getEndTime(), this.startTime + 1000);
     Assert.assertEquals(datasetState.getDuration(), 1000);
+
+    datasetState = datasetStatesByUrns.get(TEST_DATASET_URN2);
+    Assert.assertEquals(datasetState.getDatasetUrn(), TEST_DATASET_URN2);
+    Assert.assertEquals(datasetState.getJobName(), TEST_JOB_NAME);
+    Assert.assertEquals(datasetState.getJobId(), TEST_JOB_ID);
+    Assert.assertEquals(datasetState.getState(), JobState.RunningState.COMMITTED);
+    Assert.assertEquals(datasetState.getStartTime(), this.startTime);
+    Assert.assertEquals(datasetState.getEndTime(), this.startTime + 1000);
+    Assert.assertEquals(datasetState.getDuration(), 2000);
   }
 
   @Test(dependsOnMethods = "testGetPreviousDatasetStatesByUrns")

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c35f76e4/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/MysqlDatasetStateStore.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/MysqlDatasetStateStore.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/MysqlDatasetStateStore.java
index 400e52c..741ac07 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/MysqlDatasetStateStore.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/MysqlDatasetStateStore.java
@@ -69,8 +69,8 @@ public class MysqlDatasetStateStore extends MysqlStateStore<JobState.DatasetStat
         getAll(jobName, "%" + CURRENT_DATASET_STATE_FILE_SUFFIX + DATASET_STATE_STORE_TABLE_SUFFIX, true);
 
     Map<String, JobState.DatasetState> datasetStatesByUrns = Maps.newHashMap();
-    if (!previousDatasetStates.isEmpty()) {
-      JobState.DatasetState previousDatasetState = previousDatasetStates.get(0);
+
+    for (JobState.DatasetState previousDatasetState : previousDatasetStates) {
       datasetStatesByUrns.put(previousDatasetState.getDatasetUrn(), previousDatasetState);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c35f76e4/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/MysqlDatasetStateStoreTest.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/MysqlDatasetStateStoreTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/MysqlDatasetStateStoreTest.java
index 9c35610..86ba8ba 100644
--- a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/MysqlDatasetStateStoreTest.java
+++ b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/MysqlDatasetStateStoreTest.java
@@ -46,6 +46,7 @@ public class MysqlDatasetStateStoreTest {
   private static final String TEST_JOB_ID = "TestJob1";
   private static final String TEST_TASK_ID_PREFIX = "TestTask-";
   private static final String TEST_DATASET_URN = "TestDataset";
+  private static final String TEST_DATASET_URN2 = "TestDataset2";
 
   private StateStore<JobState> dbJobStateStore;
   private DatasetStateStore<JobState.DatasetState> dbDatasetStateStore;
@@ -154,6 +155,13 @@ public class MysqlDatasetStateStoreTest {
     }
 
     dbDatasetStateStore.persistDatasetState(TEST_DATASET_URN, datasetState);
+
+    // persist a second dataset state to test that retrieval of multiple dataset states works
+    datasetState.setDatasetUrn(TEST_DATASET_URN2);
+    datasetState.setId(TEST_DATASET_URN2);
+    datasetState.setDuration(2000);
+
+    dbDatasetStateStore.persistDatasetState(TEST_DATASET_URN2, datasetState);
   }
 
   @Test(dependsOnMethods = "testPersistDatasetState")
@@ -183,7 +191,7 @@ public class MysqlDatasetStateStoreTest {
   public void testGetPreviousDatasetStatesByUrns() throws IOException {
     Map<String, JobState.DatasetState> datasetStatesByUrns =
         dbDatasetStateStore.getLatestDatasetStatesByUrns(TEST_JOB_NAME);
-    Assert.assertEquals(datasetStatesByUrns.size(), 1);
+    Assert.assertEquals(datasetStatesByUrns.size(), 2);
 
     JobState.DatasetState datasetState = datasetStatesByUrns.get(TEST_DATASET_URN);
     Assert.assertEquals(datasetState.getDatasetUrn(), TEST_DATASET_URN);
@@ -193,6 +201,15 @@ public class MysqlDatasetStateStoreTest {
     Assert.assertEquals(datasetState.getStartTime(), this.startTime);
     Assert.assertEquals(datasetState.getEndTime(), this.startTime + 1000);
     Assert.assertEquals(datasetState.getDuration(), 1000);
+
+    datasetState = datasetStatesByUrns.get(TEST_DATASET_URN2);
+    Assert.assertEquals(datasetState.getDatasetUrn(), TEST_DATASET_URN2);
+    Assert.assertEquals(datasetState.getJobName(), TEST_JOB_NAME);
+    Assert.assertEquals(datasetState.getJobId(), TEST_JOB_ID);
+    Assert.assertEquals(datasetState.getState(), JobState.RunningState.COMMITTED);
+    Assert.assertEquals(datasetState.getStartTime(), this.startTime);
+    Assert.assertEquals(datasetState.getEndTime(), this.startTime + 1000);
+    Assert.assertEquals(datasetState.getDuration(), 2000);
   }
 
   @Test(dependsOnMethods = "testGetPreviousDatasetStatesByUrns")


[34/50] incubator-gobblin git commit: [GOBBLIN-408] Add more info to the KafkaExtractorTopicMetadata event for tracking execution times and rates

Posted by ab...@apache.org.
[GOBBLIN-408] Add more info to the KafkaExtractorTopicMetadata event for tracking execution times and rates

Closes #2285 from htran1/kafka_load_factor


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/a3189d73
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/a3189d73
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/a3189d73

Branch: refs/heads/0.12.0
Commit: a3189d73360c13412d91d42bea05f6ded1e4006a
Parents: 11182dc
Author: Hung Tran <hu...@linkedin.com>
Authored: Tue Feb 20 11:16:32 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Tue Feb 20 11:16:32 2018 -0800

----------------------------------------------------------------------
 .../extractor/extract/kafka/KafkaExtractor.java | 81 ++++++++++++++++--
 .../extractor/extract/kafka/KafkaSource.java    | 87 ++++++++++++++++++--
 .../workunit/packer/KafkaWorkUnitPacker.java    | 18 ++++
 3 files changed, 173 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/a3189d73/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaExtractor.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaExtractor.java b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaExtractor.java
index 1ff0159..0ec3caf 100644
--- a/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaExtractor.java
+++ b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaExtractor.java
@@ -69,7 +69,12 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
   public static final String LOW_WATERMARK = "lowWatermark";
   public static final String ACTUAL_HIGH_WATERMARK = "actualHighWatermark";
   public static final String EXPECTED_HIGH_WATERMARK = "expectedHighWatermark";
+  public static final String ELAPSED_TIME = "elapsedTime";
+  public static final String PROCESSED_RECORD_COUNT = "processedRecordCount";
   public static final String AVG_RECORD_PULL_TIME = "avgRecordPullTime";
+  public static final String READ_RECORD_TIME = "readRecordTime";
+  public static final String DECODE_RECORD_TIME = "decodeRecordTime";
+  public static final String FETCH_MESSAGE_BUFFER_TIME = "fetchMessageBufferTime";
   public static final String GOBBLIN_KAFKA_NAMESPACE = "gobblin.kafka";
   public static final String KAFKA_EXTRACTOR_TOPIC_METADATA_EVENT_NAME = "KafkaExtractorTopicMetadata";
 
@@ -87,6 +92,11 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
   protected final Map<KafkaPartition, Integer> decodingErrorCount;
   private final Map<KafkaPartition, Double> avgMillisPerRecord;
   private final Map<KafkaPartition, Long> avgRecordSizes;
+  private final Map<KafkaPartition, Long> elapsedTime;
+  private final Map<KafkaPartition, Long> processedRecordCount;
+  private final Map<KafkaPartition, Long> decodeRecordTime;
+  private final Map<KafkaPartition, Long> fetchMessageBufferTime;
+  private final Map<KafkaPartition, Long> readRecordTime;
 
   private final Set<Integer> errorPartitions;
   private int undecodableMessageCount = 0;
@@ -95,6 +105,10 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
   private int currentPartitionIdx = INITIAL_PARTITION_IDX;
   private long currentPartitionRecordCount = 0;
   private long currentPartitionTotalSize = 0;
+  private long currentPartitionFetchDuration = 0;
+  private long currentPartitionDecodeRecordTime = 0;
+  private long currentPartitionFetchMessageBufferTime = 0;
+  private long currentPartitionReadRecordTime = 0;
 
   public KafkaExtractor(WorkUnitState state) {
     super(state);
@@ -121,6 +135,11 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
     this.decodingErrorCount = Maps.newHashMap();
     this.avgMillisPerRecord = Maps.newHashMapWithExpectedSize(this.partitions.size());
     this.avgRecordSizes = Maps.newHashMapWithExpectedSize(this.partitions.size());
+    this.elapsedTime = Maps.newHashMapWithExpectedSize(this.partitions.size());
+    this.processedRecordCount = Maps.newHashMapWithExpectedSize(this.partitions.size());
+    this.decodeRecordTime = Maps.newHashMapWithExpectedSize(this.partitions.size());
+    this.fetchMessageBufferTime = Maps.newHashMapWithExpectedSize(this.partitions.size());
+    this.readRecordTime = Maps.newHashMapWithExpectedSize(this.partitions.size());
 
     this.errorPartitions = Sets.newHashSet();
 
@@ -142,6 +161,8 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
   @SuppressWarnings("unchecked")
   @Override
   public D readRecordImpl(D reuse) throws DataRecordException, IOException {
+    long readStartTime = System.nanoTime();
+
     while (!allPartitionsFinished()) {
       if (currentPartitionFinished()) {
         moveToNextPartition();
@@ -149,7 +170,9 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
       }
       if (this.messageIterator == null || !this.messageIterator.hasNext()) {
         try {
+          long fetchStartTime = System.nanoTime();
           this.messageIterator = fetchNextMessageBuffer();
+          this.currentPartitionFetchMessageBufferTime += System.nanoTime() - fetchStartTime;
         } catch (Exception e) {
           LOG.error(String.format("Failed to fetch next message buffer for partition %s. Will skip this partition.",
               getCurrentPartition()), e);
@@ -178,6 +201,9 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
         this.nextWatermark.set(this.currentPartitionIdx, nextValidMessage.getNextOffset());
         try {
           D record = null;
+          // track time for decode/convert depending on the record type
+          long decodeStartTime = System.nanoTime();
+
           if (nextValidMessage instanceof ByteArrayBasedKafkaRecord) {
             record = decodeRecord((ByteArrayBasedKafkaRecord)nextValidMessage);
           } else if (nextValidMessage instanceof DecodeableKafkaRecord){
@@ -194,8 +220,10 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
                     + " or DecodeableKafkaRecord");
           }
 
+          this.currentPartitionDecodeRecordTime += System.nanoTime() - decodeStartTime;
           this.currentPartitionRecordCount++;
           this.currentPartitionTotalSize += nextValidMessage.getValueSizeInBytes();
+          this.currentPartitionReadRecordTime += System.nanoTime() - readStartTime;
           return record;
         } catch (Throwable t) {
           this.errorPartitions.add(this.currentPartitionIdx);
@@ -208,6 +236,8 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
       }
     }
     LOG.info("Finished pulling topic " + this.topicName);
+
+    this.currentPartitionReadRecordTime += System.nanoTime() - readStartTime;
     return null;
   }
 
@@ -235,10 +265,13 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
       LOG.info("Pulling topic " + this.topicName);
       this.currentPartitionIdx = 0;
     } else {
-      computeAvgMillisPerRecordForCurrentPartition();
+      updateStatisticsForCurrentPartition();
       this.currentPartitionIdx++;
       this.currentPartitionRecordCount = 0;
-      this.currentPartitionTotalSize = 0;
+      this.currentPartitionFetchDuration = 0;
+      this.currentPartitionDecodeRecordTime = 0;
+      this.currentPartitionFetchMessageBufferTime = 0;
+      this.currentPartitionReadRecordTime = 0;
     }
 
     this.messageIterator = null;
@@ -251,16 +284,25 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
     this.stopwatch.start();
   }
 
-  private void computeAvgMillisPerRecordForCurrentPartition() {
+  private void updateStatisticsForCurrentPartition() {
     this.stopwatch.stop();
+
     if (this.currentPartitionRecordCount != 0) {
+      this.currentPartitionFetchDuration = this.stopwatch.elapsed(TimeUnit.MILLISECONDS);
       double avgMillisForCurrentPartition =
-          (double) this.stopwatch.elapsed(TimeUnit.MILLISECONDS) / (double) this.currentPartitionRecordCount;
+          (double) this.currentPartitionFetchDuration / (double) this.currentPartitionRecordCount;
       this.avgMillisPerRecord.put(this.getCurrentPartition(), avgMillisForCurrentPartition);
 
       long avgRecordSize = this.currentPartitionTotalSize / this.currentPartitionRecordCount;
       this.avgRecordSizes.put(this.getCurrentPartition(), avgRecordSize);
+
+      this.elapsedTime.put(this.getCurrentPartition(), this.currentPartitionFetchDuration);
+      this.processedRecordCount.put(this.getCurrentPartition(), this.currentPartitionRecordCount);
+      this.decodeRecordTime.put(this.getCurrentPartition(), this.currentPartitionDecodeRecordTime);
+      this.fetchMessageBufferTime.put(this.getCurrentPartition(), this.currentPartitionFetchMessageBufferTime);
+      this.readRecordTime.put(this.getCurrentPartition(), this.currentPartitionReadRecordTime);
     }
+
     this.stopwatch.reset();
   }
 
@@ -317,7 +359,7 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
   @Override
   public void close() throws IOException {
 
-    computeAvgMillisPerRecordForCurrentPartition();
+    updateStatisticsForCurrentPartition();
 
     Map<KafkaPartition, Map<String, String>> tagsForPartitionsMap = Maps.newHashMap();
 
@@ -336,7 +378,36 @@ public abstract class KafkaExtractor<S, D> extends EventBasedExtractor<S, D> {
       tagsForPartition.put(PARTITION, Integer.toString(partition.getId()));
       tagsForPartition.put(LOW_WATERMARK, Long.toString(this.lowWatermark.get(i)));
       tagsForPartition.put(ACTUAL_HIGH_WATERMARK, Long.toString(this.nextWatermark.get(i)));
+      // These are used to compute the load factor,
+      // gobblin consumption rate relative to the kafka production rate.
+      // The gobblin rate is computed as (processed record count/elapsed time)
+      // The kafka rate is computed as (expected high watermark - previous latest offset) /
+      // (current offset fetch epoch time - previous offset fetch epoch time).
       tagsForPartition.put(EXPECTED_HIGH_WATERMARK, Long.toString(this.highWatermark.get(i)));
+      tagsForPartition.put(KafkaSource.PREVIOUS_OFFSET_FETCH_EPOCH_TIME,
+          this.workUnitState.getProp(KafkaUtils.getPartitionPropName(KafkaSource.PREVIOUS_OFFSET_FETCH_EPOCH_TIME,
+              i)));
+      tagsForPartition.put(KafkaSource.OFFSET_FETCH_EPOCH_TIME,
+          this.workUnitState.getProp(KafkaUtils.getPartitionPropName(KafkaSource.OFFSET_FETCH_EPOCH_TIME, i)));
+      tagsForPartition.put(KafkaSource.PREVIOUS_LATEST_OFFSET,
+          this.workUnitState.getProp(KafkaUtils.getPartitionPropName(KafkaSource.PREVIOUS_LATEST_OFFSET, i)));
+
+      if (this.processedRecordCount.containsKey(partition)) {
+        tagsForPartition.put(PROCESSED_RECORD_COUNT, Long.toString(this.processedRecordCount.get(partition)));
+        tagsForPartition.put(ELAPSED_TIME, Long.toString(this.elapsedTime.get(partition)));
+        tagsForPartition.put(DECODE_RECORD_TIME, Long.toString(TimeUnit.NANOSECONDS.toMillis(
+            this.decodeRecordTime.get(partition))));
+        tagsForPartition.put(FETCH_MESSAGE_BUFFER_TIME, Long.toString(TimeUnit.NANOSECONDS.toMillis(
+            this.fetchMessageBufferTime.get(partition))));
+        tagsForPartition.put(READ_RECORD_TIME, Long.toString(TimeUnit.NANOSECONDS.toMillis(
+            this.readRecordTime.get(partition))));
+      } else {
+        tagsForPartition.put(PROCESSED_RECORD_COUNT, "0");
+        tagsForPartition.put(ELAPSED_TIME, "0");
+        tagsForPartition.put(DECODE_RECORD_TIME, "0");
+        tagsForPartition.put(FETCH_MESSAGE_BUFFER_TIME, "0");
+        tagsForPartition.put(READ_RECORD_TIME, "0");
+      }
 
       tagsForPartitionsMap.put(partition, tagsForPartition);
     }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/a3189d73/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaSource.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaSource.java b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaSource.java
index 69ebea6..b96412c 100644
--- a/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaSource.java
+++ b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaSource.java
@@ -104,6 +104,9 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
   public static final String ALL_TOPICS = "all";
   public static final String AVG_RECORD_SIZE = "avg.record.size";
   public static final String AVG_RECORD_MILLIS = "avg.record.millis";
+  public static final String PREVIOUS_LATEST_OFFSET = "previousLatestOffset";
+  public static final String OFFSET_FETCH_EPOCH_TIME = "offsetFetchEpochTime";
+  public static final String PREVIOUS_OFFSET_FETCH_EPOCH_TIME = "previousOffsetFetchEpochTime";
   public static final String GOBBLIN_KAFKA_CONSUMER_CLIENT_FACTORY_CLASS = "gobblin.kafka.consumerClient.class";
   public static final String GOBBLIN_KAFKA_EXTRACT_ALLOW_TABLE_TYPE_NAMESPACE_CUSTOMIZATION =
       "gobblin.kafka.extract.allowTableTypeAndNamspaceCustomization";
@@ -116,6 +119,8 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
 
   private final Set<String> moveToLatestTopics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
   private final Map<KafkaPartition, Long> previousOffsets = Maps.newConcurrentMap();
+  private final Map<KafkaPartition, Long> previousExpectedHighWatermarks = Maps.newConcurrentMap();
+  private final Map<KafkaPartition, Long> previousOffsetFetchEpochTimes = Maps.newConcurrentMap();
 
   private final Set<KafkaPartition> partitionsToBeProcessed = Sets.newConcurrentHashSet();
 
@@ -298,7 +303,7 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
       Map<String, State> topicSpecificStateMap, SourceState state) {
 
     // in case the previous offset not been set
-    getAllPreviousOffsets(state);
+    getAllPreviousOffsetState(state);
 
     // For each partition that has a previous offset, create an empty WorkUnit for it if
     // it is not in this.partitionsToBeProcessed.
@@ -309,6 +314,7 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
         if (!this.isDatasetStateEnabled.get() || this.topicsToProcess.contains(topicName)) {
           long previousOffset = entry.getValue();
           WorkUnit emptyWorkUnit = createEmptyWorkUnit(partition, previousOffset,
+              this.previousOffsetFetchEpochTimes.get(partition),
               Optional.fromNullable(topicSpecificStateMap.get(partition.getTopicName())));
 
           if (workUnits.containsKey(topicName)) {
@@ -368,6 +374,7 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
     boolean failedToGetKafkaOffsets = false;
 
     try (Timer.Context context = this.metricContext.timer(OFFSET_FETCH_TIMER).time()) {
+      offsets.setOffsetFetchEpochTime(System.currentTimeMillis());
       offsets.setEarliestOffset(this.kafkaConsumerClient.get().getEarliestOffset(partition));
       offsets.setLatestOffset(this.kafkaConsumerClient.get().getLatestOffset(partition));
     } catch (KafkaOffsetRetrievalFailureException e) {
@@ -375,9 +382,13 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
     }
 
     long previousOffset = 0;
+    long previousOffsetFetchEpochTime = 0;
     boolean previousOffsetNotFound = false;
     try {
       previousOffset = getPreviousOffsetForPartition(partition, state);
+      offsets.setPreviousLatestOffset(getPreviousExpectedHighWatermark(partition, state));
+      previousOffsetFetchEpochTime = getPreviousOffsetFetchEpochTimeForPartition(partition, state);
+      offsets.setPreviousOffsetFetchEpochTime(previousOffsetFetchEpochTime);
     } catch (PreviousOffsetNotFoundException e) {
       previousOffsetNotFound = true;
     }
@@ -392,7 +403,8 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
       LOG.warn(String
           .format("Failed to retrieve earliest and/or latest offset for partition %s. This partition will be skipped.",
               partition));
-      return previousOffsetNotFound ? null : createEmptyWorkUnit(partition, previousOffset, topicSpecificState);
+      return previousOffsetNotFound ? null : createEmptyWorkUnit(partition, previousOffset, previousOffsetFetchEpochTime,
+          topicSpecificState);
     }
 
     if (shouldMoveToLatestOffset(partition, state)) {
@@ -444,7 +456,7 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
           offsets.startAtEarliestOffset();
         } else {
           LOG.warn(offsetOutOfRangeMsg + "This partition will be skipped.");
-          return createEmptyWorkUnit(partition, previousOffset, topicSpecificState);
+          return createEmptyWorkUnit(partition, previousOffset, previousOffsetFetchEpochTime, topicSpecificState);
         }
       }
     }
@@ -452,10 +464,24 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
     return getWorkUnitForTopicPartition(partition, offsets, topicSpecificState);
   }
 
+  private long getPreviousOffsetFetchEpochTimeForPartition(KafkaPartition partition, SourceState state)
+      throws PreviousOffsetNotFoundException {
+
+    getAllPreviousOffsetState(state);
+
+    if (this.previousOffsetFetchEpochTimes.containsKey(partition)) {
+      return this.previousOffsetFetchEpochTimes.get(partition);
+    }
+
+    throw new PreviousOffsetNotFoundException(String
+        .format("Previous offset fetch epoch time for topic %s, partition %s not found.", partition.getTopicName(),
+            partition.getId()));
+  }
+
   private long getPreviousOffsetForPartition(KafkaPartition partition, SourceState state)
       throws PreviousOffsetNotFoundException {
 
-    getAllPreviousOffsets(state);
+    getAllPreviousOffsetState(state);
 
     if (this.previousOffsets.containsKey(partition)) {
       return this.previousOffsets.get(partition);
@@ -464,12 +490,28 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
         .format("Previous offset for topic %s, partition %s not found.", partition.getTopicName(), partition.getId()));
   }
 
-  // need to be synchronized as this.previousOffsets need to be initialized once
-  private synchronized void getAllPreviousOffsets(SourceState state) {
+  private long getPreviousExpectedHighWatermark(KafkaPartition partition, SourceState state)
+      throws PreviousOffsetNotFoundException {
+
+    getAllPreviousOffsetState(state);
+
+    if (this.previousExpectedHighWatermarks.containsKey(partition)) {
+      return this.previousExpectedHighWatermarks.get(partition);
+    }
+    throw new PreviousOffsetNotFoundException(String
+        .format("Previous expected high watermark for topic %s, partition %s not found.", partition.getTopicName(),
+            partition.getId()));
+  }
+
+  // need to be synchronized as this.previousOffsets, this.previousExpectedHighWatermarks, and
+  // this.previousOffsetFetchEpochTimes need to be initialized once
+  private synchronized void getAllPreviousOffsetState(SourceState state) {
     if (this.doneGettingAllPreviousOffsets) {
       return;
     }
     this.previousOffsets.clear();
+    this.previousExpectedHighWatermarks.clear();
+    this.previousOffsetFetchEpochTimes.clear();
     Map<String, Iterable<WorkUnitState>> workUnitStatesByDatasetUrns = state.getPreviousWorkUnitStatesByDatasetUrns();
 
     if (!workUnitStatesByDatasetUrns.isEmpty() &&
@@ -481,13 +523,26 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
     for (WorkUnitState workUnitState : state.getPreviousWorkUnitStates()) {
       List<KafkaPartition> partitions = KafkaUtils.getPartitions(workUnitState);
       MultiLongWatermark watermark = workUnitState.getActualHighWatermark(MultiLongWatermark.class);
+      MultiLongWatermark previousExpectedHighWatermark =
+          workUnitState.getWorkunit().getExpectedHighWatermark(MultiLongWatermark.class);
       Preconditions.checkArgument(partitions.size() == watermark.size(), String
           .format("Num of partitions doesn't match number of watermarks: partitions=%s, watermarks=%s", partitions,
               watermark));
+
       for (int i = 0; i < partitions.size(); i++) {
+        KafkaPartition partition = partitions.get(i);
+
         if (watermark.get(i) != ConfigurationKeys.DEFAULT_WATERMARK_VALUE) {
-          this.previousOffsets.put(partitions.get(i), watermark.get(i));
+          this.previousOffsets.put(partition, watermark.get(i));
+        }
+
+        if (previousExpectedHighWatermark.get(i) != ConfigurationKeys.DEFAULT_WATERMARK_VALUE) {
+          this.previousExpectedHighWatermarks.put(partition, previousExpectedHighWatermark.get(i));
         }
+
+        this.previousOffsetFetchEpochTimes.put(partition,
+          Long.valueOf(workUnitState.getProp(KafkaUtils.getPartitionPropName(KafkaSource.OFFSET_FETCH_EPOCH_TIME, i),
+              "0")));
       }
     }
 
@@ -511,12 +566,13 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
   }
 
   // thread safe
-  private WorkUnit createEmptyWorkUnit(KafkaPartition partition, long previousOffset,
+  private WorkUnit createEmptyWorkUnit(KafkaPartition partition, long previousOffset, long previousFetchEpochTime,
       Optional<State> topicSpecificState) {
     Offsets offsets = new Offsets();
     offsets.setEarliestOffset(previousOffset);
     offsets.setLatestOffset(previousOffset);
     offsets.startAtEarliestOffset();
+    offsets.setOffsetFetchEpochTime(previousFetchEpochTime);
     return getWorkUnitForTopicPartition(partition, offsets, topicSpecificState);
   }
 
@@ -552,6 +608,9 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
     workUnit.setProp(LEADER_HOSTANDPORT, partition.getLeader().getHostAndPort().toString());
     workUnit.setProp(ConfigurationKeys.WORK_UNIT_LOW_WATER_MARK_KEY, offsets.getStartOffset());
     workUnit.setProp(ConfigurationKeys.WORK_UNIT_HIGH_WATER_MARK_KEY, offsets.getLatestOffset());
+    workUnit.setProp(PREVIOUS_OFFSET_FETCH_EPOCH_TIME, offsets.getPreviousOffsetFetchEpochTime());
+    workUnit.setProp(OFFSET_FETCH_EPOCH_TIME, offsets.getOffsetFetchEpochTime());
+    workUnit.setProp(PREVIOUS_LATEST_OFFSET, offsets.getPreviousLatestOffset());
 
     // Add lineage info
     DatasetDescriptor source = new DatasetDescriptor(DatasetConstants.PLATFORM_KAFKA, partition.getTopicName());
@@ -608,6 +667,18 @@ public abstract class KafkaSource<S, D> extends EventBasedSource<S, D> {
     @Setter
     private long latestOffset = 0;
 
+    @Getter
+    @Setter
+    private long offsetFetchEpochTime = 0;
+
+    @Getter
+    @Setter
+    private long previousOffsetFetchEpochTime = 0;
+
+    @Getter
+    @Setter
+    private long previousLatestOffset = 0;
+
     private void startAt(long offset)
         throws StartOffsetOutOfRangeException {
       if (offset < this.earliestOffset || offset > this.latestOffset) {

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/a3189d73/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/workunit/packer/KafkaWorkUnitPacker.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/workunit/packer/KafkaWorkUnitPacker.java b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/workunit/packer/KafkaWorkUnitPacker.java
index 8d03f4f..0d93796 100644
--- a/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/workunit/packer/KafkaWorkUnitPacker.java
+++ b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/workunit/packer/KafkaWorkUnitPacker.java
@@ -219,6 +219,24 @@ public abstract class KafkaWorkUnitPacker {
     workUnit.removeProp(ConfigurationKeys.WORK_UNIT_LOW_WATER_MARK_KEY);
     workUnit.removeProp(ConfigurationKeys.WORK_UNIT_HIGH_WATER_MARK_KEY);
     workUnit.setWatermarkInterval(interval);
+
+    // Update offset fetch epoch time and previous latest offset. These are used to compute the load factor,
+    // gobblin consumption rate relative to the kafka production rate. The kafka rate is computed as
+    // (current latest offset - previous latest offset)/(current epoch time - previous epoch time).
+    int index = 0;
+    for (WorkUnit wu : multiWorkUnit.getWorkUnits()) {
+      workUnit.setProp(KafkaUtils.getPartitionPropName(KafkaSource.PREVIOUS_OFFSET_FETCH_EPOCH_TIME, index),
+          wu.getProp(KafkaSource.PREVIOUS_OFFSET_FETCH_EPOCH_TIME));
+      workUnit.setProp(KafkaUtils.getPartitionPropName(KafkaSource.OFFSET_FETCH_EPOCH_TIME, index),
+          wu.getProp(KafkaSource.OFFSET_FETCH_EPOCH_TIME));
+      workUnit.setProp(KafkaUtils.getPartitionPropName(KafkaSource.PREVIOUS_LATEST_OFFSET, index),
+          wu.getProp(KafkaSource.PREVIOUS_LATEST_OFFSET));
+      index++;
+    }
+    workUnit.removeProp(KafkaSource.PREVIOUS_OFFSET_FETCH_EPOCH_TIME);
+    workUnit.removeProp(KafkaSource.OFFSET_FETCH_EPOCH_TIME);
+    workUnit.removeProp(KafkaSource.PREVIOUS_LATEST_OFFSET);
+
     // Remove the original partition information
     workUnit.removeProp(KafkaSource.PARTITION_ID);
     workUnit.removeProp(KafkaSource.LEADER_ID);


[43/50] incubator-gobblin git commit: [GOBBLIN-418] Change Gobblin Service behavior to not call addSpec for pre-existing specs on FlowCatalog start up.[]

Posted by ab...@apache.org.
[GOBBLIN-418] Change Gobblin Service behavior to not call addSpec for pre-existing specs on FlowCatalog start up.[]

Closes #2297 from sv2000/gobblin-418


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/ca5835b1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/ca5835b1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/ca5835b1

Branch: refs/heads/0.12.0
Commit: ca5835b164055815efe5968979b6261c2c9b9fdc
Parents: faa27f4
Author: suvasude <su...@linkedin.biz>
Authored: Fri Mar 2 10:46:02 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Fri Mar 2 10:46:02 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/ca5835b1/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java
index a91baed..f78be47 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java
@@ -121,7 +121,7 @@ public class FlowCatalog extends AbstractIdleService implements SpecCatalog, Mut
 
   @Override
   protected void startUp() throws Exception {
-    notifyAllListeners();
+    //Do nothing
   }
 
   @Override


[12/50] incubator-gobblin git commit: [GOBBLIN-399] Refactor HiveSource#shouldCreateWorkunit() to accept table as parameter

Posted by ab...@apache.org.
[GOBBLIN-399] Refactor HiveSource#shouldCreateWorkunit() to accept table as parameter

Closes #2273 from aditya1105/metadata


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/af68d7ed
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/af68d7ed
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/af68d7ed

Branch: refs/heads/0.12.0
Commit: af68d7edd788685279e6e0cf2ad0b6cbf8e7f2d0
Parents: d29b72f
Author: aditya1105 <ad...@linkedin.com>
Authored: Thu Feb 1 11:38:08 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Thu Feb 1 11:38:08 2018 -0800

----------------------------------------------------------------------
 .../data/management/conversion/hive/source/HiveSource.java  | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/af68d7ed/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/conversion/hive/source/HiveSource.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/conversion/hive/source/HiveSource.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/conversion/hive/source/HiveSource.java
index 94d5ad9..3ad99fd 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/conversion/hive/source/HiveSource.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/conversion/hive/source/HiveSource.java
@@ -238,7 +238,7 @@ public class HiveSource implements Source {
         return;
       }
 
-      if (shouldCreateWorkunit(getCreateTime(hiveDataset.getTable()), updateTime, lowWatermark)) {
+      if (shouldCreateWorkunit(hiveDataset.getTable(), lowWatermark)) {
 
         log.info(String.format(
             "Creating workunit for table %s as updateTime %s or createTime %s is greater than low watermark %s",
@@ -384,6 +384,13 @@ public class HiveSource implements Source {
     return shouldCreateWorkunit(createTime, updateTime, lowWatermark);
   }
 
+  protected boolean shouldCreateWorkunit(Table table, LongWatermark lowWatermark)
+      throws UpdateNotFoundException {
+    long updateTime = this.updateProvider.getUpdateTime(table);
+    long createTime = getCreateTime(table);
+    return shouldCreateWorkunit(createTime, updateTime, lowWatermark);
+  }
+
   /**
    * Check if workunit needs to be created. Returns <code>true</code> If the
    * <code>updateTime</code> is greater than the <code>lowWatermark</code> and <code>maxLookBackTime</code>


[25/50] incubator-gobblin git commit: [GOBBLIN-405] Fix race condition with access to immediately invalidated resources

Posted by ab...@apache.org.
[GOBBLIN-405] Fix race condition with access to immediately invalidated resources

Closes #2280 from htran1/broker_cache_race


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/bde5bb1f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/bde5bb1f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/bde5bb1f

Branch: refs/heads/0.12.0
Commit: bde5bb1f9d2eb310b6a16d52bad383eefaf0d75c
Parents: 19b2d81
Author: Hung Tran <hu...@linkedin.com>
Authored: Wed Feb 7 10:19:45 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Wed Feb 7 10:19:45 2018 -0800

----------------------------------------------------------------------
 .../apache/gobblin/broker/ResourceEntry.java    | 17 ++++
 .../publisher/DataPublisherFactoryTest.java     | 48 +++++++++++
 .../gobblin/broker/DefaultBrokerCache.java      | 87 ++++++++++++++------
 .../broker/ImmediatelyInvalidResourceEntry.java | 15 +++-
 4 files changed, 141 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/bde5bb1f/gobblin-api/src/main/java/org/apache/gobblin/broker/ResourceEntry.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/broker/ResourceEntry.java b/gobblin-api/src/main/java/org/apache/gobblin/broker/ResourceEntry.java
index 6402391..cbdebe7 100644
--- a/gobblin-api/src/main/java/org/apache/gobblin/broker/ResourceEntry.java
+++ b/gobblin-api/src/main/java/org/apache/gobblin/broker/ResourceEntry.java
@@ -44,4 +44,21 @@ public interface ResourceEntry<T> extends SharedResourceFactoryResponse<T> {
    * key, blocking all requests for that key. As suck, this method should be reasonably fast.
    */
   void onInvalidate();
+
+  /**
+   * This method should guarantee that if all callers accessing the resource using this method then the object is
+   * returned atomically with respect to any validity state change.
+   *
+   * This is to avoid race conditions in cases where the state is changed when getting the resource. Some examples are
+   * resources that can only be used a certain number of times.
+   *
+   * @return null if the object is not valid, otherwise the valid object
+   */
+  default T getResourceIfValid() {
+    if (isValid()) {
+      return getResource();
+    } else {
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/bde5bb1f/gobblin-core/src/test/java/org/apache/gobblin/publisher/DataPublisherFactoryTest.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/java/org/apache/gobblin/publisher/DataPublisherFactoryTest.java b/gobblin-core/src/test/java/org/apache/gobblin/publisher/DataPublisherFactoryTest.java
index b2cd739..6f58a50 100644
--- a/gobblin-core/src/test/java/org/apache/gobblin/publisher/DataPublisherFactoryTest.java
+++ b/gobblin-core/src/test/java/org/apache/gobblin/publisher/DataPublisherFactoryTest.java
@@ -17,9 +17,16 @@
 package org.apache.gobblin.publisher;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -94,6 +101,47 @@ public class DataPublisherFactoryTest {
     Assert.assertTrue(publisher1.supportsCapability(Capability.THREADSAFE, Collections.EMPTY_MAP));
   }
 
+  @Test()
+  public void testMultiThreadedGetNonThreadSafePublisher()
+      throws InterruptedException, ExecutionException, IOException {
+    SharedResourcesBroker broker =
+        SharedResourcesBrokerFactory.<SimpleScopeType>createDefaultTopLevelBroker(ConfigFactory.empty(),
+            SimpleScopeType.GLOBAL.defaultScopeInstance());
+
+    ExecutorService service = Executors.newFixedThreadPool(40);
+    List<Future<?>> futures = new ArrayList<>();
+
+    for (int i = 0; i < 100000; i++) {
+      futures.add(service.submit(new GetNonThreadSafePublisher(broker)));
+    }
+
+    for (Future f: futures) {
+      f.get();
+    }
+    service.shutdown();
+    service.awaitTermination(100, TimeUnit.SECONDS);
+  }
+
+  private static class GetNonThreadSafePublisher implements Runnable {
+    private final SharedResourcesBroker broker;
+    private static long count = 0;
+
+    GetNonThreadSafePublisher(SharedResourcesBroker broker) {
+      this.broker = broker;
+    }
+
+    @Override
+    public void run() {
+      try {
+        DataPublisher publisher1 = DataPublisherFactory.get(TestNonThreadsafeDataPublisher.class.getName(), null, this.broker);
+        Assert.assertNotNull(publisher1);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+
   private static class TestNonThreadsafeDataPublisher extends DataPublisher {
     public TestNonThreadsafeDataPublisher(State state) {
       super(state);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/bde5bb1f/gobblin-utility/src/main/java/org/apache/gobblin/broker/DefaultBrokerCache.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/broker/DefaultBrokerCache.java b/gobblin-utility/src/main/java/org/apache/gobblin/broker/DefaultBrokerCache.java
index 6425bab..0c001f1 100644
--- a/gobblin-utility/src/main/java/org/apache/gobblin/broker/DefaultBrokerCache.java
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/broker/DefaultBrokerCache.java
@@ -37,6 +37,7 @@ import com.google.common.util.concurrent.Striped;
 
 import org.apache.gobblin.broker.iface.ScopeType;
 import org.apache.gobblin.broker.iface.SharedResourceFactory;
+import org.apache.gobblin.broker.iface.SharedResourceFactoryResponse;
 import org.apache.gobblin.broker.iface.SharedResourceKey;
 import org.apache.gobblin.broker.iface.NoSuchScopeException;
 
@@ -98,42 +99,78 @@ class DefaultBrokerCache<S extends ScopeType<S>> {
   }
 
   /**
-   * Get an object for the specified factory, key, scope, and broker. {@link DefaultBrokerCache}
-   * guarantees that calling this method for the same factory, key, and scope will return the same object.
+   * Get a scoped object from the cache.
    */
   @SuppressWarnings(value = "unchecked")
-  <T, K extends SharedResourceKey> T getScoped(final SharedResourceFactory<T, K, S> factory, @Nonnull final K key,
+  <T, K extends SharedResourceKey> SharedResourceFactoryResponse<T> getScopedFromCache(
+      final SharedResourceFactory<T, K, S> factory, @Nonnull final K key,
       @Nonnull final ScopeWrapper<S> scope, final SharedResourcesBrokerImpl<S> broker)
       throws ExecutionException {
-
     RawJobBrokerKey fullKey = new RawJobBrokerKey(scope, factory.getName(), key);
     Object obj = this.sharedResourceCache.get(fullKey, new Callable<Object>() {
       @Override
       public Object call() throws Exception {
-        return factory.createResource(broker.getScopedView(scope.getType()), broker.getConfigView(scope.getType(), key, factory.getName()));
+        return factory.createResource(broker.getScopedView(scope.getType()), broker.getConfigView(scope.getType(), key,
+            factory.getName()));
       }
     });
-    if (obj instanceof ResourceCoordinate) {
-      ResourceCoordinate<T, K, S> resourceCoordinate = (ResourceCoordinate<T, K, S>) obj;
-      if (!SharedResourcesBrokerUtils.isScopeTypeAncestor((ScopeType) scope.getType(), ((ResourceCoordinate) obj).getScope())) {
-        throw new RuntimeException(String.format("%s returned an invalid coordinate: scope %s is not an ancestor of %s.",
-            factory.getName(), ((ResourceCoordinate) obj).getScope(), scope.getType()));
-      }
-      try {
-        return getScoped(resourceCoordinate.getFactory(), resourceCoordinate.getKey(),
-            broker.getWrappedScope(resourceCoordinate.getScope()), broker);
-      } catch (NoSuchScopeException nsse) {
-        throw new RuntimeException(String.format("%s returned an invalid coordinate: scope %s is not available.",
-            factory.getName(), resourceCoordinate.getScope().name()), nsse);
-      }
-    } else if (obj instanceof ResourceEntry) {
-      if (!((ResourceEntry) obj).isValid()) {
-        safeInvalidate(fullKey);
-        return getScoped(factory, key, scope, broker);
+
+    return (SharedResourceFactoryResponse<T>)obj;
+  }
+
+  /**
+   * Get an object for the specified factory, key, scope, and broker. {@link DefaultBrokerCache}
+   * guarantees that calling this method for the same factory, key, and scope will return the same object.
+   */
+  @SuppressWarnings(value = "unchecked")
+  <T, K extends SharedResourceKey> T getScoped(final SharedResourceFactory<T, K, S> factory, @Nonnull final K key,
+      @Nonnull final ScopeWrapper<S> scope, final SharedResourcesBrokerImpl<S> broker)
+      throws ExecutionException {
+    SharedResourceFactory<T, K, S> currentFactory = factory;
+    K currentKey = key;
+    ScopeWrapper<S> currentScope = scope;
+
+    Object obj = getScopedFromCache(currentFactory, currentKey, currentScope, broker);
+
+    // this loop is to continue looking up objects through redirection or reloading until a valid resource is found
+    while (true) {
+      if (obj instanceof ResourceCoordinate) {
+        ResourceCoordinate<T, K, S> resourceCoordinate = (ResourceCoordinate<T, K, S>) obj;
+        if (!SharedResourcesBrokerUtils.isScopeTypeAncestor((ScopeType) currentScope.getType(), ((ResourceCoordinate) obj).getScope())) {
+          throw new RuntimeException(String
+              .format("%s returned an invalid coordinate: scope %s is not an ancestor of %s.", currentFactory.getName(),
+                  ((ResourceCoordinate) obj).getScope(), currentScope.getType()));
+        }
+        try {
+          obj = getScopedFromCache(resourceCoordinate.getFactory(), resourceCoordinate.getKey(),
+              broker.getWrappedScope(resourceCoordinate.getScope()), broker);
+        } catch (NoSuchScopeException nsse) {
+          throw new RuntimeException(String
+              .format("%s returned an invalid coordinate: scope %s is not available.", factory.getName(),
+                  resourceCoordinate.getScope().name()), nsse);
+        }
+      } else if (obj instanceof ResourceEntry) {
+        T resource = ((ResourceEntry<T>) obj).getResourceIfValid();
+
+        // valid resource found
+        if (resource != null) {
+          return resource;
+        }
+
+        // resource is invalid. The lock in this block is to reduce the chance of starvation where a thread keeps
+        // getting objects that are invalidated by another thread.
+        Lock lock = this.invalidationLock.get(key);
+        try {
+          lock.lock();
+          RawJobBrokerKey fullKey = new RawJobBrokerKey(currentScope, currentFactory.getName(), currentKey);
+          safeInvalidate(fullKey);
+          obj = getScopedFromCache(currentFactory, currentKey, currentScope, broker);
+        } finally {
+          lock.unlock();
+        }
+      } else {
+        throw new RuntimeException(String.format("Invalid response from %s: %s.", factory.getName(), obj.getClass()));
       }
-      return ((ResourceEntry<T>) obj).getResource();
-    } else {
-      throw new RuntimeException(String.format("Invalid response from %s: %s.", factory.getName(), obj.getClass()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/bde5bb1f/gobblin-utility/src/main/java/org/apache/gobblin/broker/ImmediatelyInvalidResourceEntry.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/broker/ImmediatelyInvalidResourceEntry.java b/gobblin-utility/src/main/java/org/apache/gobblin/broker/ImmediatelyInvalidResourceEntry.java
index b3f8502..ccb569c 100644
--- a/gobblin-utility/src/main/java/org/apache/gobblin/broker/ImmediatelyInvalidResourceEntry.java
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/broker/ImmediatelyInvalidResourceEntry.java
@@ -36,7 +36,7 @@ public class ImmediatelyInvalidResourceEntry<T> extends ResourceInstance<T> {
   }
 
   @Override
-  public T getResource() {
+  public synchronized T getResource() {
     // mark the object as invalid before returning so that a new one will be created on the next
     // request from the factory
     this.valid = false;
@@ -53,4 +53,17 @@ public class ImmediatelyInvalidResourceEntry<T> extends ResourceInstance<T> {
   public void onInvalidate() {
     // these type of resource cannot be closed on invalidation since the lifetime can't be determined
   }
+
+  /**
+   * This method is synchronized so that the validity check and validity change is atomic for callers of this method.
+   * @return
+   */
+  @Override
+  public synchronized T getResourceIfValid() {
+    if (this.valid) {
+      return getResource();
+    } else {
+      return null;
+    }
+  }
 }


[19/50] incubator-gobblin git commit: Don't let Helix retry when a child process fails

Posted by ab...@apache.org.
Don't let Helix retry when a child process fails

Closes #2258 from HappyRay/do-not-retry-when-
child-process-fails


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/6cbc0cdd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/6cbc0cdd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/6cbc0cdd

Branch: refs/heads/0.12.0
Commit: 6cbc0cdd4e6e90f04a77f87ec77f454a7b5ac3ff
Parents: 9e78835
Author: Ray Yang <ru...@gmail.com>
Authored: Mon Feb 5 11:51:41 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Mon Feb 5 11:51:41 2018 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/gobblin/cluster/SingleHelixTask.java  | 2 +-
 .../test/java/org/apache/gobblin/cluster/SingleHelixTaskTest.java  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/6cbc0cdd/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleHelixTask.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleHelixTask.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleHelixTask.java
index 8a49f76..d10792d 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleHelixTask.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleHelixTask.java
@@ -66,7 +66,7 @@ public class SingleHelixTask implements Task {
       } else {
         logger.warn("Task process failed with exitcode ({}). job name: {}. job id: {}", exitCode,
             this.jobName, this.jobId);
-        return new TaskResult(TaskResult.Status.FAILED, "Exit code: " + exitCode);
+        return new TaskResult(TaskResult.Status.FATAL_FAILED, "Exit code: " + exitCode);
       }
     } catch (final Throwable t) {
       logger.error("SingleHelixTask failed due to " + t.getMessage(), t);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/6cbc0cdd/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/SingleHelixTaskTest.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/SingleHelixTaskTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/SingleHelixTaskTest.java
index 2f07452..256ef22 100644
--- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/SingleHelixTaskTest.java
+++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/SingleHelixTaskTest.java
@@ -66,7 +66,7 @@ public class SingleHelixTaskTest {
 
     final TaskResult result = createAndRunTask();
 
-    assertThat(result.getStatus()).isEqualTo(TaskResult.Status.FAILED);
+    assertThat(result.getStatus()).isEqualTo(TaskResult.Status.FATAL_FAILED);
   }
 
   @Test


[26/50] incubator-gobblin git commit: [GOBBLIN-379] Submit an event when DistCp job resource requirements exceed a hard bound

Posted by ab...@apache.org.
[GOBBLIN-379] Submit an event when DistCp job resource requirements exceed a hard bound

Closes #2257 from sv2000/gobblin-379


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/457ede26
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/457ede26
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/457ede26

Branch: refs/heads/0.12.0
Commit: 457ede26da2c7f693d4fe321d4c50b0e25e0d22d
Parents: bde5bb1
Author: suvasude <su...@linkedin.biz>
Authored: Thu Feb 8 11:37:21 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Thu Feb 8 11:37:21 2018 -0800

----------------------------------------------------------------------
 .../data/management/copy/CopyConfiguration.java |  17 ++-
 .../data/management/copy/CopySource.java        | 105 ++++++++++++++-----
 .../data/management/copy/CopySourceTest.java    |  84 +++++++++++++++
 .../ConcurrentBoundedPriorityIterable.java      |  80 +++++++++-----
 .../PriorityIterableBasedRequestAllocator.java  |  71 +++++++++----
 .../RequestAllocatorConfig.java                 |  21 +++-
 .../ConcurrentBoundedPriorityIterableTest.java  |  21 ++--
 7 files changed, 309 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/457ede26/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopyConfiguration.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopyConfiguration.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopyConfiguration.java
index 211ad13..c4d07e2 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopyConfiguration.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopyConfiguration.java
@@ -23,6 +23,8 @@ import lombok.AllArgsConstructor;
 import lombok.Builder;
 import lombok.Data;
 
+import org.apache.gobblin.util.request_allocation.ConcurrentBoundedPriorityIterable;
+import org.apache.gobblin.util.request_allocation.RequestAllocatorConfig;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
@@ -65,6 +67,13 @@ public class CopyConfiguration {
 
   public static final String ABORT_ON_SINGLE_DATASET_FAILURE = COPY_PREFIX + ".abortOnSingleDatasetFailure";
 
+  /*
+   * Config to store different classes of rejected requests. Possible values are "all","none", or "min" (default).
+   */
+  public static final String STORE_REJECTED_REQUESTS_KEY = COPY_PREFIX + ".store.rejected.requests";
+  public static final String DEFAULT_STORE_REJECTED_REQUESTS =
+      RequestAllocatorConfig.StoreRejectedRequestsConfig.MIN.name();
+
   /**
    * User supplied directory where files should be published. This value is identical for all datasets in the distcp job.
    */
@@ -81,6 +90,7 @@ public class CopyConfiguration {
   private final FileSystem targetFs;
   private final Optional<FileSetComparator> prioritizer;
   private final ResourcePool maxToCopy;
+  private final String storeRejectedRequestsSetting;
 
   private final Config config;
 
@@ -114,8 +124,8 @@ public class CopyConfiguration {
       if (properties.containsKey(PRIORITIZER_ALIAS_KEY)) {
         try {
           this.prioritizer = Optional.of(GobblinConstructorUtils.<FileSetComparator>invokeLongestConstructor(
-              new ClassAliasResolver(FileSetComparator.class).resolveClass(
-                  properties.getProperty(PRIORITIZER_ALIAS_KEY)), properties));
+              new ClassAliasResolver(FileSetComparator.class)
+                  .resolveClass(properties.getProperty(PRIORITIZER_ALIAS_KEY)), properties));
         } catch (ReflectiveOperationException roe) {
           throw new RuntimeException("Could not build prioritizer.", roe);
         }
@@ -124,6 +134,9 @@ public class CopyConfiguration {
       }
       this.maxToCopy = CopyResourcePool.fromConfig(ConfigUtils.getConfigOrEmpty(this.config, MAX_COPY_PREFIX));
 
+      this.storeRejectedRequestsSetting =
+          properties.getProperty(CopyConfiguration.STORE_REJECTED_REQUESTS_KEY, DEFAULT_STORE_REJECTED_REQUESTS);
+
       this.abortOnSingleDatasetFailure = false;
       if (this.config.hasPath(ABORT_ON_SINGLE_DATASET_FAILURE)) {
         this.abortOnSingleDatasetFailure = this.config.getBoolean(ABORT_ON_SINGLE_DATASET_FAILURE);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/457ede26/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopySource.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopySource.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopySource.java
index 615d6ad..3355f3d 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopySource.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/CopySource.java
@@ -29,6 +29,9 @@ import java.util.concurrent.Future;
 
 import javax.annotation.Nullable;
 
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 
@@ -37,6 +40,7 @@ import com.google.common.base.Optional;
 import com.google.common.base.Predicates;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Multimaps;
@@ -62,6 +66,7 @@ import org.apache.gobblin.dataset.IterableDatasetFinder;
 import org.apache.gobblin.dataset.IterableDatasetFinderImpl;
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.metrics.GobblinMetrics;
+import org.apache.gobblin.metrics.GobblinTrackingEvent;
 import org.apache.gobblin.metrics.MetricContext;
 import org.apache.gobblin.metrics.Tag;
 import org.apache.gobblin.metrics.event.EventSubmitter;
@@ -84,13 +89,11 @@ import org.apache.gobblin.util.guid.Guid;
 import org.apache.gobblin.util.request_allocation.GreedyAllocator;
 import org.apache.gobblin.util.request_allocation.HierarchicalAllocator;
 import org.apache.gobblin.util.request_allocation.HierarchicalPrioritizer;
+import org.apache.gobblin.util.request_allocation.PriorityIterableBasedRequestAllocator;
 import org.apache.gobblin.util.request_allocation.RequestAllocator;
 import org.apache.gobblin.util.request_allocation.RequestAllocatorConfig;
 import org.apache.gobblin.util.request_allocation.RequestAllocatorUtils;
 
-import lombok.AllArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
-
 
 /**
  * {@link org.apache.gobblin.source.Source} that generates work units from {@link org.apache.gobblin.data.management.copy.CopyableDataset}s.
@@ -112,11 +115,21 @@ public class CopySource extends AbstractSource<String, FileAwareInputStream> {
   public static final String SIMULATE = CopyConfiguration.COPY_PREFIX + ".simulate";
   public static final String MAX_SIZE_MULTI_WORKUNITS = CopyConfiguration.COPY_PREFIX + ".binPacking.maxSizePerBin";
   public static final String MAX_WORK_UNITS_PER_BIN = CopyConfiguration.COPY_PREFIX + ".binPacking.maxWorkUnitsPerBin";
+  public static final String REQUESTS_EXCEEDING_AVAILABLE_RESOURCE_POOL_EVENT_NAME =
+      "RequestsExceedingAvailableResourcePoolEvent";
+  public static final String REQUESTS_DROPPED_EVENT_NAME = "RequestsDroppedEvent";
+  public static final String REQUESTS_REJECTED_DUE_TO_INSUFFICIENT_EVICTION_EVENT_NAME =
+      "RequestsRejectedDueToInsufficientEvictionEvent";
+  public static final String REQUESTS_REJECTED_WITH_LOW_PRIORITY_EVENT_NAME = "RequestsRejectedWithLowPriorityEvent";
+  public static final String FILESET_NAME = "fileset.name";
+  public static final String FILESET_TOTAL_ENTITIES = "fileset.total.entities";
+  public static final String FILESET_TOTAL_SIZE_IN_BYTES = "fileset.total.size";
 
   private static final String WORK_UNIT_WEIGHT = CopyConfiguration.COPY_PREFIX + ".workUnitWeight";
   private final WorkUnitWeighter weighter = new FieldWeighter(WORK_UNIT_WEIGHT);
 
   public MetricContext metricContext;
+  public EventSubmitter eventSubmitter;
 
   protected Optional<LineageInfo> lineageInfo;
 
@@ -145,16 +158,17 @@ public class CopySource extends AbstractSource<String, FileAwareInputStream> {
 
     try {
 
-      DeprecationUtils.renameDeprecatedKeys(state, CopyConfiguration.MAX_COPY_PREFIX + "." + CopyResourcePool.ENTITIES_KEY,
-          Lists.newArrayList(MAX_FILES_COPIED_KEY));
+      DeprecationUtils
+          .renameDeprecatedKeys(state, CopyConfiguration.MAX_COPY_PREFIX + "." + CopyResourcePool.ENTITIES_KEY,
+              Lists.newArrayList(MAX_FILES_COPIED_KEY));
 
       final FileSystem sourceFs = HadoopUtils.getSourceFileSystem(state);
       final FileSystem targetFs = HadoopUtils.getWriterFileSystem(state, 1, 0);
       state.setProp(SlaEventKeys.SOURCE_URI, sourceFs.getUri());
       state.setProp(SlaEventKeys.DESTINATION_URI, targetFs.getUri());
 
-      log.info("Identified source file system at {} and target file system at {}.",
-          sourceFs.getUri(), targetFs.getUri());
+      log.info("Identified source file system at {} and target file system at {}.", sourceFs.getUri(),
+          targetFs.getUri());
 
       long maxSizePerBin = state.getPropAsLong(MAX_SIZE_MULTI_WORKUNITS, 0);
       long maxWorkUnitsPerMultiWorkUnit = state.getPropAsLong(MAX_WORK_UNITS_PER_BIN, 50);
@@ -165,26 +179,31 @@ public class CopySource extends AbstractSource<String, FileAwareInputStream> {
 
       final CopyConfiguration copyConfiguration = CopyConfiguration.builder(targetFs, state.getProperties()).build();
 
+      this.eventSubmitter = new EventSubmitter.Builder(this.metricContext, CopyConfiguration.COPY_PREFIX).build();
       DatasetsFinder<CopyableDatasetBase> datasetFinder = DatasetUtils
           .instantiateDatasetFinder(state.getProperties(), sourceFs, DEFAULT_DATASET_PROFILE_CLASS_KEY,
-              new EventSubmitter.Builder(this.metricContext, CopyConfiguration.COPY_PREFIX).build(), state);
+              this.eventSubmitter, state);
 
       IterableDatasetFinder<CopyableDatasetBase> iterableDatasetFinder =
           datasetFinder instanceof IterableDatasetFinder ? (IterableDatasetFinder<CopyableDatasetBase>) datasetFinder
               : new IterableDatasetFinderImpl<>(datasetFinder);
 
-      Iterator<CopyableDatasetRequestor> requestorIteratorWithNulls =
-          Iterators.transform(iterableDatasetFinder.getDatasetsIterator(),
+      Iterator<CopyableDatasetRequestor> requestorIteratorWithNulls = Iterators
+          .transform(iterableDatasetFinder.getDatasetsIterator(),
               new CopyableDatasetRequestor.Factory(targetFs, copyConfiguration, log));
-      Iterator<CopyableDatasetRequestor> requestorIterator = Iterators.filter(requestorIteratorWithNulls,
-          Predicates.<CopyableDatasetRequestor>notNull());
+      Iterator<CopyableDatasetRequestor> requestorIterator =
+          Iterators.filter(requestorIteratorWithNulls, Predicates.<CopyableDatasetRequestor>notNull());
 
       final SetMultimap<FileSet<CopyEntity>, WorkUnit> workUnitsMap =
           Multimaps.<FileSet<CopyEntity>, WorkUnit>synchronizedSetMultimap(
               HashMultimap.<FileSet<CopyEntity>, WorkUnit>create());
 
       RequestAllocator<FileSet<CopyEntity>> allocator = createRequestAllocator(copyConfiguration, maxThreads);
-      Iterator<FileSet<CopyEntity>> prioritizedFileSets = allocator.allocateRequests(requestorIterator, copyConfiguration.getMaxToCopy());
+      Iterator<FileSet<CopyEntity>> prioritizedFileSets =
+          allocator.allocateRequests(requestorIterator, copyConfiguration.getMaxToCopy());
+
+      //Submit alertable events for unfulfilled requests
+      submitUnfulfilledRequestEvents(allocator);
 
       Iterator<Callable<Void>> callableIterator =
           Iterators.transform(prioritizedFileSets, new Function<FileSet<CopyEntity>, Callable<Void>>() {
@@ -197,8 +216,7 @@ public class CopySource extends AbstractSource<String, FileAwareInputStream> {
           });
 
       try {
-        List<Future<Void>> futures = new IteratorExecutor<>(callableIterator,
-            maxThreads,
+        List<Future<Void>> futures = new IteratorExecutor<>(callableIterator, maxThreads,
             ExecutorsUtils.newDaemonThreadFactory(Optional.of(log), Optional.of("Copy-file-listing-pool-%d")))
             .execute();
 
@@ -231,8 +249,8 @@ public class CopySource extends AbstractSource<String, FileAwareInputStream> {
         return Lists.newArrayList();
       }
 
-      List<? extends WorkUnit> workUnits =
-          new WorstFitDecreasingBinPacking(maxSizePerBin).pack(Lists.newArrayList(workUnitsMap.values()), this.weighter);
+      List<? extends WorkUnit> workUnits = new WorstFitDecreasingBinPacking(maxSizePerBin)
+          .pack(Lists.newArrayList(workUnitsMap.values()), this.weighter);
       log.info(String.format(
           "Bin packed work units. Initial work units: %d, packed work units: %d, max weight per bin: %d, "
               + "max work units per bin: %d.", workUnitsMap.size(), workUnits.size(), maxSizePerBin,
@@ -243,11 +261,42 @@ public class CopySource extends AbstractSource<String, FileAwareInputStream> {
     }
   }
 
-  private RequestAllocator<FileSet<CopyEntity>> createRequestAllocator(CopyConfiguration copyConfiguration, int maxThreads) {
-    Optional<FileSetComparator> prioritizer = copyConfiguration.getPrioritizer();
+  private void submitUnfulfilledRequestEventsHelper(List<FileSet<CopyEntity>> fileSetList, String eventName) {
+    for (FileSet<CopyEntity> fileSet : fileSetList) {
+      GobblinTrackingEvent event =
+          GobblinTrackingEvent.newBuilder().setName(eventName).setNamespace(CopySource.class.getName()).setMetadata(
+              ImmutableMap.<String, String>builder()
+                  .put(ConfigurationKeys.DATASET_URN_KEY, fileSet.getDataset().getUrn())
+                  .put(FILESET_TOTAL_ENTITIES, Integer.toString(fileSet.getTotalEntities()))
+                  .put(FILESET_TOTAL_SIZE_IN_BYTES, Long.toString(fileSet.getTotalSizeInBytes()))
+                  .put(FILESET_NAME, fileSet.getName()).build()).build();
+      this.metricContext.submitEvent(event);
+    }
+  }
 
+  private void submitUnfulfilledRequestEvents(RequestAllocator<FileSet<CopyEntity>> allocator) {
+    if (PriorityIterableBasedRequestAllocator.class.isAssignableFrom(allocator.getClass())) {
+      PriorityIterableBasedRequestAllocator<FileSet<CopyEntity>> priorityIterableBasedRequestAllocator =
+          (PriorityIterableBasedRequestAllocator<FileSet<CopyEntity>>) allocator;
+      submitUnfulfilledRequestEventsHelper(
+          priorityIterableBasedRequestAllocator.getRequestsExceedingAvailableResourcePool(),
+          REQUESTS_EXCEEDING_AVAILABLE_RESOURCE_POOL_EVENT_NAME);
+      submitUnfulfilledRequestEventsHelper(
+          priorityIterableBasedRequestAllocator.getRequestsRejectedDueToInsufficientEviction(),
+          REQUESTS_REJECTED_DUE_TO_INSUFFICIENT_EVICTION_EVENT_NAME);
+      submitUnfulfilledRequestEventsHelper(priorityIterableBasedRequestAllocator.getRequestsRejectedWithLowPriority(),
+          REQUESTS_REJECTED_WITH_LOW_PRIORITY_EVENT_NAME);
+      submitUnfulfilledRequestEventsHelper(priorityIterableBasedRequestAllocator.getRequestsDropped(),
+          REQUESTS_DROPPED_EVENT_NAME);
+    }
+  }
+
+  private RequestAllocator<FileSet<CopyEntity>> createRequestAllocator(CopyConfiguration copyConfiguration,
+      int maxThreads) {
+    Optional<FileSetComparator> prioritizer = copyConfiguration.getPrioritizer();
     RequestAllocatorConfig.Builder<FileSet<CopyEntity>> configBuilder =
         RequestAllocatorConfig.builder(new FileSetResourceEstimator()).allowParallelization(maxThreads)
+            .storeRejectedRequests(copyConfiguration.getStoreRejectedRequestsSetting())
             .withLimitedScopeConfig(copyConfiguration.getPrioritizationConfig());
 
     if (!prioritizer.isPresent()) {
@@ -323,9 +372,8 @@ public class CopySource extends AbstractSource<String, FileAwareInputStream> {
        * a DatasetFinder. Consequently, the source and destination dataset for the CopyableFile lineage are expected
        * to be set by the same logic
        */
-      if (lineageInfo.isPresent() &&
-          copyableFile.getSourceDataset() != null &&
-          copyableFile.getDestinationDataset() != null) {
+      if (lineageInfo.isPresent() && copyableFile.getSourceDataset() != null
+          && copyableFile.getDestinationDataset() != null) {
         lineageInfo.get().setSource(copyableFile.getSourceDataset(), workUnit);
       }
     }
@@ -350,7 +398,8 @@ public class CopySource extends AbstractSource<String, FileAwareInputStream> {
     return new EmptyExtractor<>("empty");
   }
 
-  protected Extractor<String, FileAwareInputStream> extractorForCopyableFile(FileSystem fs, CopyableFile cf, WorkUnitState state)
+  protected Extractor<String, FileAwareInputStream> extractorForCopyableFile(FileSystem fs, CopyableFile cf,
+      WorkUnitState state)
       throws IOException {
     return new FileAwareInputStreamExtractor(fs, cf, state);
   }
@@ -365,7 +414,8 @@ public class CopySource extends AbstractSource<String, FileAwareInputStream> {
   @Deprecated
   protected FileSystem getSourceFileSystem(State state)
       throws IOException {
-    Configuration conf = HadoopUtils.getConfFromState(state, Optional.of(ConfigurationKeys.SOURCE_FILEBASED_ENCRYPTED_CONFIG_PATH));
+    Configuration conf =
+        HadoopUtils.getConfFromState(state, Optional.of(ConfigurationKeys.SOURCE_FILEBASED_ENCRYPTED_CONFIG_PATH));
     String uri = state.getProp(ConfigurationKeys.SOURCE_FILEBASED_FS_URI, ConfigurationKeys.LOCAL_FS_URI);
     return HadoopUtils.getOptionallyThrottledFileSystem(FileSystem.get(URI.create(uri), conf), state);
   }
@@ -456,11 +506,12 @@ public class CopySource extends AbstractSource<String, FileAwareInputStream> {
     return CopyableDatasetMetadata.deserialize(state.getProp(SERIALIZED_COPYABLE_DATASET));
   }
 
-  private void setWorkUnitWatermark(WorkUnit workUnit, Optional<CopyableFileWatermarkGenerator> watermarkGenerator, CopyEntity copyEntity)
+  private void setWorkUnitWatermark(WorkUnit workUnit, Optional<CopyableFileWatermarkGenerator> watermarkGenerator,
+      CopyEntity copyEntity)
       throws IOException {
-    if (copyEntity instanceof  CopyableFile) {
+    if (copyEntity instanceof CopyableFile) {
       Optional<WatermarkInterval> watermarkIntervalOptional =
-          CopyableFileWatermarkHelper.getCopyableFileWatermark((CopyableFile)copyEntity, watermarkGenerator);
+          CopyableFileWatermarkHelper.getCopyableFileWatermark((CopyableFile) copyEntity, watermarkGenerator);
       if (watermarkIntervalOptional.isPresent()) {
         workUnit.setWatermarkInterval(watermarkIntervalOptional.get());
       }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/457ede26/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/CopySourceTest.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/CopySourceTest.java b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/CopySourceTest.java
index d9e5368..377b2cf 100644
--- a/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/CopySourceTest.java
+++ b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/copy/CopySourceTest.java
@@ -17,19 +17,41 @@
 
 package org.apache.gobblin.data.management.copy;
 
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Iterator;
 import java.util.List;
 
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.util.request_allocation.RequestAllocatorConfig;
+import org.apache.hadoop.fs.FileSystem;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import com.google.common.base.Predicates;
+import com.google.common.collect.Iterators;
+
 import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.SourceState;
 import org.apache.gobblin.data.management.dataset.DatasetUtils;
+import org.apache.gobblin.data.management.partition.CopyableDatasetRequestor;
+import org.apache.gobblin.data.management.partition.FileSet;
+import org.apache.gobblin.dataset.DatasetsFinder;
+import org.apache.gobblin.dataset.IterableDatasetFinder;
+import org.apache.gobblin.dataset.IterableDatasetFinderImpl;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.event.EventSubmitter;
 import org.apache.gobblin.source.workunit.Extract;
 import org.apache.gobblin.source.workunit.WorkUnit;
+import org.apache.gobblin.util.HadoopUtils;
 import org.apache.gobblin.util.JobLauncherUtils;
+import org.apache.gobblin.util.request_allocation.PriorityIterableBasedRequestAllocator;
 
 
+@Slf4j
 public class CopySourceTest {
 
   @Test
@@ -106,4 +128,66 @@ public class CopySourceTest {
     Assert.assertNotNull(extractBelow);
   }
 
+  @Test
+  public void testSubmitUnfulfilledRequestEvents()
+      throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException {
+    SourceState state = new SourceState();
+
+    state.setProp(ConfigurationKeys.SOURCE_FILEBASED_FS_URI, "file:///");
+    state.setProp(ConfigurationKeys.WRITER_FILE_SYSTEM_URI, "file:///");
+    state.setProp(ConfigurationKeys.DATA_PUBLISHER_FINAL_DIR, "/target/dir");
+    state.setProp(DatasetUtils.DATASET_PROFILE_CLASS_KEY,
+        TestCopyablePartitionableDatasedFinder.class.getCanonicalName());
+    state.setProp(CopySource.MAX_CONCURRENT_LISTING_SERVICES, 2);
+    state.setProp(CopyConfiguration.MAX_COPY_PREFIX + ".size", "50");
+    state.setProp(CopyConfiguration.MAX_COPY_PREFIX + ".copyEntities", 2);
+    state.setProp(CopyConfiguration.STORE_REJECTED_REQUESTS_KEY,
+        RequestAllocatorConfig.StoreRejectedRequestsConfig.ALL.name().toLowerCase());
+    state.setProp(ConfigurationKeys.METRICS_CUSTOM_BUILDERS, "org.apache.gobblin.metrics.ConsoleEventReporterFactory");
+
+    CopySource source = new CopySource();
+
+    final FileSystem sourceFs = HadoopUtils.getSourceFileSystem(state);
+    final FileSystem targetFs = HadoopUtils.getWriterFileSystem(state, 1, 0);
+
+    int maxThreads = state
+        .getPropAsInt(CopySource.MAX_CONCURRENT_LISTING_SERVICES, CopySource.DEFAULT_MAX_CONCURRENT_LISTING_SERVICES);
+
+    final CopyConfiguration copyConfiguration = CopyConfiguration.builder(targetFs, state.getProperties()).build();
+
+    MetricContext metricContext = Instrumented.getMetricContext(state, CopySource.class);
+    EventSubmitter eventSubmitter = new EventSubmitter.Builder(metricContext, CopyConfiguration.COPY_PREFIX).build();
+    DatasetsFinder<CopyableDatasetBase> datasetFinder = DatasetUtils
+        .instantiateDatasetFinder(state.getProperties(), sourceFs, CopySource.DEFAULT_DATASET_PROFILE_CLASS_KEY,
+            eventSubmitter, state);
+
+    IterableDatasetFinder<CopyableDatasetBase> iterableDatasetFinder =
+        datasetFinder instanceof IterableDatasetFinder ? (IterableDatasetFinder<CopyableDatasetBase>) datasetFinder
+            : new IterableDatasetFinderImpl<>(datasetFinder);
+
+    Iterator<CopyableDatasetRequestor> requestorIteratorWithNulls = Iterators
+        .transform(iterableDatasetFinder.getDatasetsIterator(),
+            new CopyableDatasetRequestor.Factory(targetFs, copyConfiguration, log));
+    Iterator<CopyableDatasetRequestor> requestorIterator =
+        Iterators.filter(requestorIteratorWithNulls, Predicates.<CopyableDatasetRequestor>notNull());
+
+    Method m = CopySource.class.getDeclaredMethod("createRequestAllocator", CopyConfiguration.class, int.class);
+    m.setAccessible(true);
+    PriorityIterableBasedRequestAllocator<FileSet<CopyEntity>> allocator =
+        (PriorityIterableBasedRequestAllocator<FileSet<CopyEntity>>) m.invoke(source, copyConfiguration, maxThreads);
+    Iterator<FileSet<CopyEntity>> prioritizedFileSets =
+        allocator.allocateRequests(requestorIterator, copyConfiguration.getMaxToCopy());
+    List<FileSet<CopyEntity>> fileSetList = allocator.getRequestsExceedingAvailableResourcePool();
+    Assert.assertEquals(fileSetList.size(), 2);
+
+    FileSet<CopyEntity> fileSet = fileSetList.get(0);
+    Assert.assertEquals(fileSet.getDataset().getUrn(), "/test");
+    Assert.assertEquals(fileSet.getTotalEntities(), 5);
+    Assert.assertEquals(fileSet.getTotalSizeInBytes(), 50);
+
+    fileSet = fileSetList.get(1);
+    Assert.assertEquals(fileSet.getDataset().getUrn(), "/test");
+    Assert.assertEquals(fileSet.getTotalEntities(), 5);
+    Assert.assertEquals(fileSet.getTotalSizeInBytes(), 50);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/457ede26/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/ConcurrentBoundedPriorityIterable.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/ConcurrentBoundedPriorityIterable.java b/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/ConcurrentBoundedPriorityIterable.java
index c1a4505..bcfcf29 100644
--- a/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/ConcurrentBoundedPriorityIterable.java
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/ConcurrentBoundedPriorityIterable.java
@@ -22,15 +22,15 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.TreeSet;
 
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
 import org.slf4j.Logger;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
 
-import lombok.Getter;
-import lombok.extern.slf4j.Slf4j;
-
 
 /**
  * A concurrent bounded priority {@link Iterable}. Given a {@link ResourcePool}, a {@link ResourceEstimator}, and a
@@ -65,12 +65,24 @@ public class ConcurrentBoundedPriorityIterable<T> implements Iterable<AllocatedR
   private int requestsRefused = 0;
   private int requestsEvicted = 0;
 
+  //These are for submitting alertable events
+  private String storeRejectedRequestsSetting;
+  @Getter
+  private List<T> requestsExceedingAvailableResourcePool = Lists.newArrayList();
+  @Getter
+  private List<T> requestsRejectedWithLowPriority = Lists.newArrayList();
+  @Getter
+  private List<T> requestsRejectedDueToInsufficientEviction = Lists.newArrayList();
+  @Getter
+  private List<T> requestsDropped = Lists.newArrayList();
+
   // These are ResourceRequirements for temporary use to avoid instantiation costs
   private final ResourceRequirement candidateRequirement;
   private final ResourceRequirement tmpRequirement;
   private final ResourceRequirement reuse;
 
-  public ConcurrentBoundedPriorityIterable(final Comparator<? super T> prioritizer, ResourceEstimator<T> resourceEstimator, ResourcePool pool) {
+  public ConcurrentBoundedPriorityIterable(final Comparator<? super T> prioritizer,
+      ResourceEstimator<T> resourceEstimator, String storeRejectedRequestsSetting, ResourcePool pool) {
 
     this.estimator = resourceEstimator;
     this.resourcePool = pool;
@@ -79,6 +91,8 @@ public class ConcurrentBoundedPriorityIterable<T> implements Iterable<AllocatedR
     this.allDifferentComparator = new AllDifferentComparator();
     this.elements = new TreeSet<>(this.allDifferentComparator);
 
+    this.storeRejectedRequestsSetting = storeRejectedRequestsSetting;
+
     this.currentRequirement = this.resourcePool.getResourceRequirementBuilder().zero().build();
     this.maxResourceRequirement = new ResourceRequirement(this.currentRequirement);
 
@@ -94,7 +108,8 @@ public class ConcurrentBoundedPriorityIterable<T> implements Iterable<AllocatedR
    */
   private class AllDifferentComparator implements Comparator<AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T>> {
     @Override
-    public int compare(AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T> t1, AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T> t2) {
+    public int compare(AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T> t1,
+        AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T> t2) {
       int providedComparison = ConcurrentBoundedPriorityIterable.this.comparator.compare(t1.getT(), t2.getT());
       if (providedComparison != 0) {
         return providedComparison;
@@ -111,12 +126,13 @@ public class ConcurrentBoundedPriorityIterable<T> implements Iterable<AllocatedR
    */
   public boolean add(T t) {
     if (this.closed) {
-      throw new RuntimeException(ConcurrentBoundedPriorityIterable.class.getSimpleName() + " is no longer accepting requests!");
+      throw new RuntimeException(
+          ConcurrentBoundedPriorityIterable.class.getSimpleName() + " is no longer accepting requests!");
     }
 
-    AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T>
-        newElement = new AllocatedRequestsIteratorBase.RequestWithResourceRequirement<>(t,
-        this.estimator.estimateRequirement(t, this.resourcePool));
+    AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T> newElement =
+        new AllocatedRequestsIteratorBase.RequestWithResourceRequirement<>(t,
+            this.estimator.estimateRequirement(t, this.resourcePool));
     boolean addedWorkunits = addImpl(newElement);
     if (!addedWorkunits) {
       this.rejectedElement = true;
@@ -132,21 +148,30 @@ public class ConcurrentBoundedPriorityIterable<T> implements Iterable<AllocatedR
     if (this.resourcePool.exceedsHardBound(newElement.getResourceRequirement(), false)) {
       // item does not fit even in empty pool
       log.warn(String.format("Request %s is larger than the available resource pool. If the pool is not expanded, "
-          + "it will never be selected. Request: %s.", newElement.getT(),
+              + "it will never be selected. Request: %s.", newElement.getT(),
           this.resourcePool.stringifyRequirement(newElement.getResourceRequirement())));
+      if (!this.storeRejectedRequestsSetting
+          .equalsIgnoreCase(RequestAllocatorConfig.StoreRejectedRequestsConfig.NONE.name())) {
+        this.requestsExceedingAvailableResourcePool.add(newElement.getT());
+      }
       this.requestsRefused++;
       return false;
     }
 
-    ResourceRequirement candidateRequirement =
-        ResourceRequirement.add(this.currentRequirement, newElement.getResourceRequirement(), this.candidateRequirement);
+    ResourceRequirement candidateRequirement = ResourceRequirement
+        .add(this.currentRequirement, newElement.getResourceRequirement(), this.candidateRequirement);
 
     if (this.resourcePool.exceedsHardBound(candidateRequirement, false)) {
 
       if (this.comparator.compare(this.elements.last().getT(), newElement.getT()) <= 0) {
-        log.debug("Request {} does not fit in resource pool and is lower priority than current lowest priority request. "
-            + "Rejecting", newElement.getT());
+        log.debug(
+            "Request {} does not fit in resource pool and is lower priority than current lowest priority request. "
+                + "Rejecting", newElement.getT());
         this.requestsRefused++;
+        if (this.storeRejectedRequestsSetting
+            .equalsIgnoreCase(RequestAllocatorConfig.StoreRejectedRequestsConfig.ALL.name())) {
+          this.requestsRejectedWithLowPriority.add(newElement.getT());
+        }
         return false;
       }
 
@@ -154,11 +179,15 @@ public class ConcurrentBoundedPriorityIterable<T> implements Iterable<AllocatedR
 
       this.currentRequirement.copyInto(this.tmpRequirement);
 
-      for (AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T> dropCandidate : this.elements.descendingSet()) {
+      for (AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T> dropCandidate : this.elements
+          .descendingSet()) {
         if (this.comparator.compare(dropCandidate.getT(), newElement.getT()) <= 0) {
-          log.debug("Cannot evict enough requests to fit request {}. "
-              + "Rejecting", newElement.getT());
+          log.debug("Cannot evict enough requests to fit request {}. " + "Rejecting", newElement.getT());
           this.requestsRefused++;
+          if (this.storeRejectedRequestsSetting
+              .equalsIgnoreCase(RequestAllocatorConfig.StoreRejectedRequestsConfig.ALL.name())) {
+            this.requestsRejectedDueToInsufficientEviction.add(newElement.getT());
+          }
           return false;
         }
         this.tmpRequirement.subtract(dropCandidate.getResourceRequirement());
@@ -172,6 +201,10 @@ public class ConcurrentBoundedPriorityIterable<T> implements Iterable<AllocatedR
       for (AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T> drop : toDrop) {
         log.debug("Evicting request {}.", drop.getT());
         this.requestsEvicted++;
+        if (this.storeRejectedRequestsSetting
+            .equalsIgnoreCase(RequestAllocatorConfig.StoreRejectedRequestsConfig.ALL.name())) {
+          this.requestsDropped.add(drop.getT());
+        }
         this.elements.remove(drop);
         this.currentRequirement.subtract(drop.getResourceRequirement());
       }
@@ -205,13 +238,13 @@ public class ConcurrentBoundedPriorityIterable<T> implements Iterable<AllocatedR
     StringBuilder messageBuilder = new StringBuilder("Statistics for ").
         append(ConcurrentBoundedPriorityIterable.class.getSimpleName()).append(": {");
     messageBuilder.append(this.resourcePool).append(", ");
-    messageBuilder.append("totalResourcesUsed: ").append(this.resourcePool.stringifyRequirement(this.currentRequirement))
-        .append(", ");
-    messageBuilder.append("maxRequirementPerDimension: ").append(this.resourcePool.stringifyRequirement(this.maxResourceRequirement))
-        .append(", ");
+    messageBuilder.append("totalResourcesUsed: ")
+        .append(this.resourcePool.stringifyRequirement(this.currentRequirement)).append(", ");
+    messageBuilder.append("maxRequirementPerDimension: ")
+        .append(this.resourcePool.stringifyRequirement(this.maxResourceRequirement)).append(", ");
     messageBuilder.append("requestsOffered: ").append(this.requestsOffered).append(", ");
-    messageBuilder.append("requestsAccepted: ").append(this.requestsOffered - this.requestsEvicted - this.requestsRefused)
-        .append(", ");
+    messageBuilder.append("requestsAccepted: ")
+        .append(this.requestsOffered - this.requestsEvicted - this.requestsRefused).append(", ");
     messageBuilder.append("requestsRefused: ").append(this.requestsRefused).append(", ");
     messageBuilder.append("requestsEvicted: ").append(this.requestsEvicted);
     messageBuilder.append("}");
@@ -228,5 +261,4 @@ public class ConcurrentBoundedPriorityIterable<T> implements Iterable<AllocatedR
     this.closed = true;
     return this.elements.iterator();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/457ede26/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/PriorityIterableBasedRequestAllocator.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/PriorityIterableBasedRequestAllocator.java b/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/PriorityIterableBasedRequestAllocator.java
index 876a4eb..f5be73e 100644
--- a/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/PriorityIterableBasedRequestAllocator.java
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/PriorityIterableBasedRequestAllocator.java
@@ -22,6 +22,9 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 
+import lombok.AccessLevel;
+import lombok.Getter;
+
 import org.slf4j.Logger;
 
 import com.google.common.base.Function;
@@ -32,22 +35,35 @@ import org.apache.gobblin.util.Either;
 import org.apache.gobblin.util.ExecutorsUtils;
 import org.apache.gobblin.util.executors.IteratorExecutor;
 
-import lombok.AccessLevel;
-import lombok.AllArgsConstructor;
-import lombok.Getter;
-
 
-@AllArgsConstructor
 public abstract class PriorityIterableBasedRequestAllocator<T extends Request<T>> implements RequestAllocator<T> {
 
   private final Logger log;
   @Getter(value = AccessLevel.PROTECTED)
   private final RequestAllocatorConfig<T> configuration;
 
+  //These are for submitting alertable events
+  @Getter
+  private List<T> requestsExceedingAvailableResourcePool;
+  @Getter
+  private List<T> requestsRejectedWithLowPriority;
+  @Getter
+  private List<T> requestsRejectedDueToInsufficientEviction;
+  @Getter
+  private List<T> requestsDropped;
+
+  public PriorityIterableBasedRequestAllocator(Logger log, RequestAllocatorConfig<T> configuration) {
+    this.log = log;
+    this.configuration = configuration;
+  }
+
   @Override
-  public AllocatedRequestsIterator<T> allocateRequests(Iterator<? extends Requestor<T>> requestors, ResourcePool resourcePool) {
+  public AllocatedRequestsIterator<T> allocateRequests(Iterator<? extends Requestor<T>> requestors,
+      ResourcePool resourcePool) {
     final ConcurrentBoundedPriorityIterable<T> iterable =
-        new ConcurrentBoundedPriorityIterable<>(this.configuration.getPrioritizer(), this.configuration.getResourceEstimator(), resourcePool);
+        new ConcurrentBoundedPriorityIterable<>(this.configuration.getPrioritizer(),
+            this.configuration.getResourceEstimator(), this.configuration.getStoreRejectedRequestsSetting(),
+            resourcePool);
 
     final Iterator<T> joinIterator = getJoinIterator(requestors, iterable);
 
@@ -57,36 +73,47 @@ public abstract class PriorityIterableBasedRequestAllocator<T extends Request<T>
       }
     } else {
 
-      IteratorExecutor<Void> executor = new IteratorExecutor<>(Iterators.transform(joinIterator, new Function<T, Callable<Void>>() {
-        @Override
-        public Callable<Void> apply(final T input) {
-          return new Callable<Void>() {
+      IteratorExecutor<Void> executor =
+          new IteratorExecutor<>(Iterators.transform(joinIterator, new Function<T, Callable<Void>>() {
             @Override
-            public Void call()
-                throws Exception {
-              iterable.add(input);
-              return null;
+            public Callable<Void> apply(final T input) {
+              return new Callable<Void>() {
+                @Override
+                public Void call()
+                    throws Exception {
+                  iterable.add(input);
+                  return null;
+                }
+              };
             }
-          };
-        }
-      }), this.configuration.getAllowedThreads(),
-          ExecutorsUtils.newThreadFactory(Optional.of(log), Optional.of("request-allocator-%d")));
+          }), this.configuration.getAllowedThreads(),
+              ExecutorsUtils.newThreadFactory(Optional.of(log), Optional.of("request-allocator-%d")));
 
       try {
         List<Either<Void, ExecutionException>> results = executor.executeAndGetResults();
         IteratorExecutor.logFailures(results, log, 10);
       } catch (InterruptedException ie) {
         log.error("Request allocation was interrupted.");
-        return new AllocatedRequestsIteratorBase<>(Iterators.<AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T>>emptyIterator(),
-            resourcePool);
+        return new AllocatedRequestsIteratorBase<>(
+            Iterators.<AllocatedRequestsIteratorBase.RequestWithResourceRequirement<T>>emptyIterator(), resourcePool);
       }
     }
 
     iterable.logStatistics(Optional.of(this.log));
+
+    //Get all requests rejected/dropped
+    getRejectedAndDroppedRequests(iterable);
+
     return new AllocatedRequestsIteratorBase<>(iterable.iterator(), resourcePool);
   }
 
+  public void getRejectedAndDroppedRequests(ConcurrentBoundedPriorityIterable<T> iterable) {
+    requestsExceedingAvailableResourcePool = iterable.getRequestsExceedingAvailableResourcePool();
+    requestsRejectedWithLowPriority = iterable.getRequestsRejectedWithLowPriority();
+    requestsRejectedDueToInsufficientEviction = iterable.getRequestsRejectedDueToInsufficientEviction();
+    requestsDropped = iterable.getRequestsDropped();
+  }
+
   protected abstract Iterator<T> getJoinIterator(Iterator<? extends Requestor<T>> requestors,
       ConcurrentBoundedPriorityIterable<T> requestIterable);
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/457ede26/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/RequestAllocatorConfig.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/RequestAllocatorConfig.java b/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/RequestAllocatorConfig.java
index b33070b..5a3051a 100644
--- a/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/RequestAllocatorConfig.java
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/util/request_allocation/RequestAllocatorConfig.java
@@ -20,12 +20,12 @@ package org.apache.gobblin.util.request_allocation;
 import java.io.Serializable;
 import java.util.Comparator;
 
-import com.typesafe.config.Config;
-import com.typesafe.config.ConfigFactory;
-
 import lombok.AllArgsConstructor;
 import lombok.Getter;
 
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
 
 @AllArgsConstructor
 @Getter
@@ -34,6 +34,11 @@ public class RequestAllocatorConfig<T extends Request<T>> {
   private final ResourceEstimator<T> resourceEstimator;
   private final int allowedThreads;
   private Config limitedScopeConfig;
+  private String storeRejectedRequestsSetting;
+
+  public enum StoreRejectedRequestsConfig {
+    ALL, MIN, NONE
+  }
 
   public static <T extends Request<T>> Builder<T> builder(ResourceEstimator<T> resourceEstimator) {
     return new Builder<>(resourceEstimator);
@@ -44,6 +49,7 @@ public class RequestAllocatorConfig<T extends Request<T>> {
     private final ResourceEstimator<T> resourceEstimator;
     private int allowedThreads = 1;
     private Config limitedScopeConfig;
+    private String storeRejectedRequestsSetting = StoreRejectedRequestsConfig.MIN.name();
 
     public Builder(ResourceEstimator<T> resourceEstimator) {
       this.resourceEstimator = resourceEstimator;
@@ -68,11 +74,17 @@ public class RequestAllocatorConfig<T extends Request<T>> {
       return this;
     }
 
+    public Builder<T> storeRejectedRequests(String storeRejectedRequestsSetting) {
+      this.storeRejectedRequestsSetting = storeRejectedRequestsSetting;
+      return this;
+    }
+
     public RequestAllocatorConfig<T> build() {
       if (this.limitedScopeConfig == null) {
         this.limitedScopeConfig = ConfigFactory.empty();
       }
-      return new RequestAllocatorConfig<>(this.prioritizer, this.resourceEstimator, this.allowedThreads, this.limitedScopeConfig);
+      return new RequestAllocatorConfig<>(this.prioritizer, this.resourceEstimator, this.allowedThreads,
+          this.limitedScopeConfig, this.storeRejectedRequestsSetting);
     }
   }
 
@@ -82,5 +94,4 @@ public class RequestAllocatorConfig<T extends Request<T>> {
       return 0;
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/457ede26/gobblin-utility/src/test/java/org/apache/gobblin/util/request_allocation/ConcurrentBoundedPriorityIterableTest.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/test/java/org/apache/gobblin/util/request_allocation/ConcurrentBoundedPriorityIterableTest.java b/gobblin-utility/src/test/java/org/apache/gobblin/util/request_allocation/ConcurrentBoundedPriorityIterableTest.java
index 58a56ea..2145bc0 100644
--- a/gobblin-utility/src/test/java/org/apache/gobblin/util/request_allocation/ConcurrentBoundedPriorityIterableTest.java
+++ b/gobblin-utility/src/test/java/org/apache/gobblin/util/request_allocation/ConcurrentBoundedPriorityIterableTest.java
@@ -32,10 +32,12 @@ public class ConcurrentBoundedPriorityIterableTest {
   public static final String MEMORY = "memory";
 
   @Test
-  public void test() throws Exception {
+  public void test()
+      throws Exception {
 
-    ConcurrentBoundedPriorityIterable<String> iterable = new ConcurrentBoundedPriorityIterable<>(new MyComparator(),
-        new MyEstimator(), ResourcePool.builder().maxResource(MEMORY, 100.).build());
+    ConcurrentBoundedPriorityIterable<String> iterable =
+        new ConcurrentBoundedPriorityIterable<>(new MyComparator(), new MyEstimator(), "min",
+            ResourcePool.builder().maxResource(MEMORY, 100.).build());
 
     // doesn't fit
     Assert.assertFalse(iterable.add("a-500"));
@@ -57,8 +59,8 @@ public class ConcurrentBoundedPriorityIterableTest {
     Assert.assertTrue(iterable.add("b-50"));
 
     // Check items
-    List<String> items = Lists.newArrayList(Iterators.transform(iterable.iterator(),
-        new AllocatedRequestsIteratorBase.TExtractor<String>()));
+    List<String> items = Lists
+        .newArrayList(Iterators.transform(iterable.iterator(), new AllocatedRequestsIteratorBase.TExtractor<String>()));
     Assert.assertEquals(items.size(), 2);
     Assert.assertEquals(items.get(0), "b-50");
     Assert.assertEquals(items.get(1), "d-50");
@@ -66,15 +68,15 @@ public class ConcurrentBoundedPriorityIterableTest {
     iterable.reopen();
     // a high priority that won't fit even with evictions should not evict anything
     Assert.assertFalse(iterable.add("c-500"));
-    items = Lists.newArrayList(Iterators.transform(iterable.iterator(),
-        new AllocatedRequestsIteratorBase.TExtractor<String>()));
+    items = Lists
+        .newArrayList(Iterators.transform(iterable.iterator(), new AllocatedRequestsIteratorBase.TExtractor<String>()));
     Assert.assertEquals(items.size(), 2);
 
     iterable.reopen();
     // even if it is higher priority than everything else
     Assert.assertFalse(iterable.add("a-500"));
-    items = Lists.newArrayList(Iterators.transform(iterable.iterator(),
-        new AllocatedRequestsIteratorBase.TExtractor<String>()));
+    items = Lists
+        .newArrayList(Iterators.transform(iterable.iterator(), new AllocatedRequestsIteratorBase.TExtractor<String>()));
     Assert.assertEquals(items.size(), 2);
   }
 
@@ -94,5 +96,4 @@ public class ConcurrentBoundedPriorityIterableTest {
       return resourcePool.getResourceRequirementBuilder().setRequirement(MEMORY, memory).build();
     }
   }
-
 }
\ No newline at end of file


[30/50] incubator-gobblin git commit: [GOBBLIN-409] Set collation to latin1_bin for the MySql state store backing table

Posted by ab...@apache.org.
[GOBBLIN-409] Set collation to latin1_bin for the MySql state store backing table

Closes #2286 from
htran1/mysql_state_store_collation


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/c43cf268
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/c43cf268
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/c43cf268

Branch: refs/heads/0.12.0
Commit: c43cf268940bed4d98c0ccba6389d15bacdfaa26
Parents: 0791ee9
Author: Hung Tran <hu...@linkedin.com>
Authored: Tue Feb 13 15:48:12 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Tue Feb 13 15:48:12 2018 -0800

----------------------------------------------------------------------
 .../gobblin/metastore/MysqlStateStore.java      |  4 +--
 .../runtime/MysqlDatasetStateStoreTest.java     | 38 +++++++++++++++++++-
 2 files changed, 39 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c43cf268/gobblin-metastore/src/main/java/org/apache/gobblin/metastore/MysqlStateStore.java
----------------------------------------------------------------------
diff --git a/gobblin-metastore/src/main/java/org/apache/gobblin/metastore/MysqlStateStore.java b/gobblin-metastore/src/main/java/org/apache/gobblin/metastore/MysqlStateStore.java
index 9756507..b276702 100644
--- a/gobblin-metastore/src/main/java/org/apache/gobblin/metastore/MysqlStateStore.java
+++ b/gobblin-metastore/src/main/java/org/apache/gobblin/metastore/MysqlStateStore.java
@@ -101,8 +101,8 @@ public class MysqlStateStore<T extends State> implements StateStore<T> {
 
   // MySQL key length limit is 767 bytes
   private static final String CREATE_JOB_STATE_TABLE_TEMPLATE =
-      "CREATE TABLE IF NOT EXISTS $TABLE$ (store_name varchar(100) CHARACTER SET latin1 not null,"
-          + "table_name varchar(667) CHARACTER SET latin1 not null,"
+      "CREATE TABLE IF NOT EXISTS $TABLE$ (store_name varchar(100) CHARACTER SET latin1 COLLATE latin1_bin not null,"
+          + "table_name varchar(667) CHARACTER SET latin1 COLLATE latin1_bin not null,"
           + " modified_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,"
           + " state longblob, primary key(store_name, table_name))";
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/c43cf268/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/MysqlDatasetStateStoreTest.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/MysqlDatasetStateStoreTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/MysqlDatasetStateStoreTest.java
index 86ba8ba..45bb44d 100644
--- a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/MysqlDatasetStateStoreTest.java
+++ b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/MysqlDatasetStateStoreTest.java
@@ -43,9 +43,11 @@ public class MysqlDatasetStateStoreTest {
 
   private static final String TEST_STATE_STORE = "TestStateStore";
   private static final String TEST_JOB_NAME = "TestJob";
+  private static final String TEST_JOB_NAME_LOWER = "testjob";
   private static final String TEST_JOB_ID = "TestJob1";
   private static final String TEST_TASK_ID_PREFIX = "TestTask-";
   private static final String TEST_DATASET_URN = "TestDataset";
+  private static final String TEST_DATASET_URN_LOWER = "testdataset";
   private static final String TEST_DATASET_URN2 = "TestDataset2";
 
   private StateStore<JobState> dbJobStateStore;
@@ -109,6 +111,13 @@ public class MysqlDatasetStateStoreTest {
     dbJobStateStore.put(TEST_JOB_NAME,
         MysqlDatasetStateStore.CURRENT_DATASET_STATE_FILE_SUFFIX + MysqlDatasetStateStore.DATASET_STATE_STORE_TABLE_SUFFIX,
         jobState);
+
+    // for testing a colliding lowercase job name
+    jobState.setJobName(TEST_JOB_NAME_LOWER);
+    jobState.setProp("lower", "case");
+    dbJobStateStore.put(TEST_JOB_NAME_LOWER,
+        MysqlDatasetStateStore.CURRENT_DATASET_STATE_FILE_SUFFIX + MysqlDatasetStateStore.DATASET_STATE_STORE_TABLE_SUFFIX,
+        jobState);
   }
 
   @Test(dependsOnMethods = "testPersistJobState")
@@ -119,6 +128,8 @@ public class MysqlDatasetStateStoreTest {
 
     Assert.assertEquals(jobState.getJobName(), TEST_JOB_NAME);
     Assert.assertEquals(jobState.getJobId(), TEST_JOB_ID);
+    Assert.assertEquals(jobState.getProp("foo"), "bar");
+    Assert.assertNotEquals(jobState.getProp("lower"), "case");
     Assert.assertEquals(jobState.getState(), JobState.RunningState.COMMITTED);
     Assert.assertEquals(jobState.getStartTime(), this.startTime);
     Assert.assertEquals(jobState.getEndTime(), this.startTime + 1000);
@@ -132,6 +143,15 @@ public class MysqlDatasetStateStoreTest {
       Assert.assertEquals(taskState.getId(), TEST_TASK_ID_PREFIX + i);
       Assert.assertEquals(taskState.getWorkingState(), WorkUnitState.WorkingState.COMMITTED);
     }
+
+    jobState = dbJobStateStore.get(TEST_JOB_NAME_LOWER,
+        dbDatasetStateStore.CURRENT_DATASET_STATE_FILE_SUFFIX + dbDatasetStateStore.DATASET_STATE_STORE_TABLE_SUFFIX,
+        TEST_JOB_ID);
+
+    Assert.assertEquals(jobState.getJobName(), TEST_JOB_NAME_LOWER);
+    Assert.assertEquals(jobState.getJobId(), TEST_JOB_ID);
+    Assert.assertEquals(jobState.getProp("foo"), "bar");
+    Assert.assertEquals(jobState.getProp("lower"), "case");
   }
 
   @Test(dependsOnMethods = "testGetJobState")
@@ -162,6 +182,13 @@ public class MysqlDatasetStateStoreTest {
     datasetState.setDuration(2000);
 
     dbDatasetStateStore.persistDatasetState(TEST_DATASET_URN2, datasetState);
+
+    // persist a colliding lowercase dataset state to test that retrieval is case sensitive
+    datasetState.setDatasetUrn(TEST_DATASET_URN_LOWER);
+    datasetState.setId(TEST_DATASET_URN_LOWER );
+    datasetState.setDuration(3000);
+
+    dbDatasetStateStore.persistDatasetState(TEST_DATASET_URN_LOWER, datasetState);
   }
 
   @Test(dependsOnMethods = "testPersistDatasetState")
@@ -191,7 +218,7 @@ public class MysqlDatasetStateStoreTest {
   public void testGetPreviousDatasetStatesByUrns() throws IOException {
     Map<String, JobState.DatasetState> datasetStatesByUrns =
         dbDatasetStateStore.getLatestDatasetStatesByUrns(TEST_JOB_NAME);
-    Assert.assertEquals(datasetStatesByUrns.size(), 2);
+    Assert.assertEquals(datasetStatesByUrns.size(), 3);
 
     JobState.DatasetState datasetState = datasetStatesByUrns.get(TEST_DATASET_URN);
     Assert.assertEquals(datasetState.getDatasetUrn(), TEST_DATASET_URN);
@@ -210,6 +237,15 @@ public class MysqlDatasetStateStoreTest {
     Assert.assertEquals(datasetState.getStartTime(), this.startTime);
     Assert.assertEquals(datasetState.getEndTime(), this.startTime + 1000);
     Assert.assertEquals(datasetState.getDuration(), 2000);
+
+    datasetState = datasetStatesByUrns.get(TEST_DATASET_URN_LOWER);
+    Assert.assertEquals(datasetState.getDatasetUrn(), TEST_DATASET_URN_LOWER);
+    Assert.assertEquals(datasetState.getJobName(), TEST_JOB_NAME);
+    Assert.assertEquals(datasetState.getJobId(), TEST_JOB_ID);
+    Assert.assertEquals(datasetState.getState(), JobState.RunningState.COMMITTED);
+    Assert.assertEquals(datasetState.getStartTime(), this.startTime);
+    Assert.assertEquals(datasetState.getEndTime(), this.startTime + 1000);
+    Assert.assertEquals(datasetState.getDuration(), 3000);
   }
 
   @Test(dependsOnMethods = "testGetPreviousDatasetStatesByUrns")


[46/50] incubator-gobblin git commit: [GOBBLIN-424] Ensure job context is closed even if job fails.

Posted by ab...@apache.org.
[GOBBLIN-424] Ensure job context is closed even if job fails.

Closes #2301 from ibuenros/broker-close-fix


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/1ddaffa4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/1ddaffa4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/1ddaffa4

Branch: refs/heads/0.12.0
Commit: 1ddaffa4d2b93ea27c407ea3c8e33cd874649a31
Parents: 979ad2a
Author: ibuenros <is...@gmail.com>
Authored: Mon Mar 12 13:28:22 2018 -0700
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Mon Mar 12 13:28:22 2018 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java   | 1 +
 .../src/main/java/org/apache/gobblin/runtime/JobContext.java        | 1 -
 2 files changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/1ddaffa4/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java
index 06782a9..da770af 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java
@@ -340,6 +340,7 @@ public abstract class AbstractJobLauncher implements JobLauncher {
       TimingEvent launchJobTimer = this.eventSubmitter.getTimingEvent(TimingEvent.LauncherTimings.FULL_JOB_EXECUTION);
 
       try (Closer closer = Closer.create()) {
+        closer.register(this.jobContext);
         notifyListeners(this.jobContext, jobListener, TimingEvent.LauncherTimings.JOB_PREPARE, new JobListenerAction() {
           @Override
           public void apply(JobListener jobListener, JobContext jobContext)

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/1ddaffa4/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/JobContext.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/JobContext.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/JobContext.java
index 73d613c..d8fbe4e 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/JobContext.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/JobContext.java
@@ -481,7 +481,6 @@ public class JobContext implements Closeable {
       throw new IOException(exc);
     }
     this.jobState.setState(JobState.RunningState.COMMITTED);
-    close();
   }
 
   @Override


[10/50] incubator-gobblin git commit: [GOBBLIN-397] Create a new dataset version selection policy for filtering dataset versions that have "hidden" paths.

Posted by ab...@apache.org.
[GOBBLIN-397] Create a new dataset version selection policy for filtering dataset versions that have "hidden" paths.

Closes #2271 from sv2000/gobblin-397


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/ff13dde1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/ff13dde1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/ff13dde1

Branch: refs/heads/0.12.0
Commit: ff13dde1c88d21048494cf79fdf2319c488b81c5
Parents: 161bef0
Author: suvasude <su...@linkedin.biz>
Authored: Wed Jan 31 14:00:49 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Wed Jan 31 14:00:49 2018 -0800

----------------------------------------------------------------------
 .../policy/HiddenFilterSelectionPolicy.java     | 90 ++++++++++++++++++++
 .../policy/HiddenFilterSelectionPolicyTest.java | 77 +++++++++++++++++
 2 files changed, 167 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/ff13dde1/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/policy/HiddenFilterSelectionPolicy.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/policy/HiddenFilterSelectionPolicy.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/policy/HiddenFilterSelectionPolicy.java
new file mode 100644
index 0000000..1c515ae
--- /dev/null
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/policy/HiddenFilterSelectionPolicy.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.data.management.policy;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.fs.Path;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Lists;
+import com.typesafe.config.Config;
+
+import org.apache.gobblin.data.management.version.FileSystemDatasetVersion;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/*
+ * Select dataset versions that pass the hidden path filter i.e. accept paths that do not have sub-dirs whose names start with "." or "_".
+ */
+public class HiddenFilterSelectionPolicy implements VersionSelectionPolicy<FileSystemDatasetVersion> {
+  public static final String HIDDEN_FILTER_HIDDEN_FILE_PREFIX_KEY = "selection.hiddenFilter.hiddenFilePrefix";
+  private static final String[] DEFAULT_HIDDEN_FILE_PREFIXES = {".", "_"};
+  private List<String> hiddenFilePrefixes;
+
+  public HiddenFilterSelectionPolicy(Config config) {
+    if (config.hasPath(HIDDEN_FILTER_HIDDEN_FILE_PREFIX_KEY)) {
+      this.hiddenFilePrefixes = ConfigUtils.getStringList(config, HIDDEN_FILTER_HIDDEN_FILE_PREFIX_KEY);
+    } else {
+      this.hiddenFilePrefixes = Arrays.asList(DEFAULT_HIDDEN_FILE_PREFIXES);
+    }
+  }
+
+  @Override
+  public Class<? extends FileSystemDatasetVersion> versionClass() {
+    return FileSystemDatasetVersion.class;
+  }
+
+  private boolean isPathHidden(Path path) {
+    while (path != null) {
+      String name = path.getName();
+      for (String prefix : this.hiddenFilePrefixes) {
+        if (name.startsWith(prefix)) {
+          return true;
+        }
+      }
+      path = path.getParent();
+    }
+    return false;
+  }
+
+  private Predicate<FileSystemDatasetVersion> getSelectionPredicate() {
+    return new Predicate<FileSystemDatasetVersion>() {
+      @Override
+      public boolean apply(FileSystemDatasetVersion version) {
+        Set<Path> paths = version.getPaths();
+        for (Path path : paths) {
+          Path p = path.getPathWithoutSchemeAndAuthority(path);
+          if (isPathHidden(p)) {
+            return false;
+          }
+        }
+        return true;
+      }
+    };
+  }
+
+  @Override
+  public Collection<FileSystemDatasetVersion> listSelectedVersions(List<FileSystemDatasetVersion> allVersions) {
+    return Lists.newArrayList(Collections2.filter(allVersions, getSelectionPredicate()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/ff13dde1/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/policy/HiddenFilterSelectionPolicyTest.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/policy/HiddenFilterSelectionPolicyTest.java b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/policy/HiddenFilterSelectionPolicyTest.java
new file mode 100644
index 0000000..5c08b7c
--- /dev/null
+++ b/gobblin-data-management/src/test/java/org/apache/gobblin/data/management/policy/HiddenFilterSelectionPolicyTest.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.data.management.policy;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.fs.Path;
+import org.joda.time.DateTime;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableMap;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+
+import org.apache.gobblin.data.management.version.FileSystemDatasetVersion;
+import org.apache.gobblin.data.management.version.TimestampedDatasetVersion;
+
+
+public class HiddenFilterSelectionPolicyTest {
+  @Test
+  public void testListSelectedVersions() throws Exception {
+    List<FileSystemDatasetVersion> versionList = new ArrayList<>();
+    Set<String> pathSet = new HashSet<>();
+    Path path1 = new Path("/data/dataset/versions/version1");
+    pathSet.add(path1.toString());
+    Path path2 = new Path("/data/dataset/versions/version2");
+    pathSet.add(path2.toString());
+    Path path3 = new Path("/data/dataset/.temp/tmpPath");
+    Path path4 = new Path("/data/dataset/_temp/tmpPath");
+
+    versionList.add(new TimestampedDatasetVersion(new DateTime(), path1));
+    versionList.add(new TimestampedDatasetVersion(new DateTime(), path2));
+    versionList.add(new TimestampedDatasetVersion(new DateTime(), path3));
+    versionList.add(new TimestampedDatasetVersion(new DateTime(), path4));
+
+    List<String> hiddenFilePrefixes = Arrays.asList("_", ".");
+    List<Config> configList = new ArrayList<>();
+    Config config1 = ConfigFactory.parseMap(
+        ImmutableMap.of(HiddenFilterSelectionPolicy.HIDDEN_FILTER_HIDDEN_FILE_PREFIX_KEY, hiddenFilePrefixes));
+    configList.add(config1);
+    Config config2 = ConfigFactory.parseMap(
+        ImmutableMap.of(HiddenFilterSelectionPolicy.HIDDEN_FILTER_HIDDEN_FILE_PREFIX_KEY, "_,."));
+    configList.add(config2);
+    for (Config config : configList) {
+      HiddenFilterSelectionPolicy policy = new HiddenFilterSelectionPolicy(config);
+      Collection<FileSystemDatasetVersion> selectedVersions = policy.listSelectedVersions(versionList);
+      Assert.assertEquals(selectedVersions.size(), 2);
+      for (FileSystemDatasetVersion version : selectedVersions) {
+        Set<Path> paths = version.getPaths();
+        for (Path path : paths) {
+          Assert.assertTrue(pathSet.contains(path.toString()));
+        }
+      }
+    }
+  }
+}
\ No newline at end of file


[41/50] incubator-gobblin git commit: [GOBBLIN-419] Add more metrics for cluster job monitoring

Posted by ab...@apache.org.
[GOBBLIN-419] Add more metrics for cluster job monitoring

Closes #2296 from yukuai518/metrics


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/5e6bfb07
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/5e6bfb07
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/5e6bfb07

Branch: refs/heads/0.12.0
Commit: 5e6bfb079cfdd3d2026e7cd674dd6673933437d3
Parents: 4c15fde
Author: Kuai Yu <ku...@linkedin.com>
Authored: Wed Feb 28 09:04:36 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Wed Feb 28 09:04:36 2018 -0800

----------------------------------------------------------------------
 .../configuration/ConfigurationKeys.java        |   5 +
 .../gobblin/cluster/GobblinClusterManager.java  |  31 ++---
 .../cluster/GobblinHelixJobLauncher.java        |   3 +
 .../cluster/GobblinHelixJobScheduler.java       | 125 +++++++------------
 .../gobblin/cluster/GobblinTaskRunner.java      |  82 +++++++++---
 .../cluster/GobblinTaskRunnerMetrics.java       |  78 ++++++++++++
 .../instrumented/StandardMetricsBridge.java     |  52 ++++----
 .../service/StreamingKafkaSpecConsumer.java     |  44 ++-----
 .../apache/gobblin/runtime/TaskExecutor.java    |  33 ++++-
 .../apache/gobblin/runtime/api/JobCatalog.java  |  39 +++---
 .../runtime/api/JobExecutionLauncher.java       |   6 +-
 .../gobblin/runtime/api/MutableJobCatalog.java  |  23 ++--
 .../gobblin/runtime/api/MutableSpecCatalog.java |  23 ++--
 .../apache/gobblin/runtime/api/SpecCatalog.java |  39 +++---
 .../runtime/job_catalog/FSJobCatalog.java       |   4 +-
 .../job_catalog/ImmutableFSJobCatalog.java      |   2 +-
 .../runtime/job_catalog/JobCatalogBase.java     |  12 +-
 .../runtime/spec_catalog/FlowCatalog.java       |   2 +-
 .../runtime/spec_catalog/TopologyCatalog.java   |   2 +-
 .../modules/core/GobblinServiceManager.java     |  28 +----
 20 files changed, 359 insertions(+), 274 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
index c80ceaf..612fd8b 100644
--- a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
+++ b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
@@ -169,6 +169,9 @@ public class ConfigurationKeys {
 
   public static final String WORK_UNIT_RETRY_POLICY_KEY = "workunit.retry.policy";
   public static final String WORK_UNIT_RETRY_ENABLED_KEY = "workunit.retry.enabled";
+  public static final String WORK_UNIT_CREATION_TIME_IN_MILLIS = "workunit.creation.time.in.millis";
+  public static final String WORK_UNIT_CREATION_AND_RUN_INTERVAL = "workunit.creation.and.run.interval";
+
   public static final String JOB_RUN_ONCE_KEY = "job.runonce";
   public static final String JOB_DISABLED_KEY = "job.disabled";
   public static final String JOB_JAR_FILES_KEY = "job.jars";
@@ -631,6 +634,8 @@ public class ConfigurationKeys {
   public static final String METRICS_REPORT_INTERVAL_KEY = METRICS_CONFIGURATIONS_PREFIX + "report.interval";
   public static final String DEFAULT_METRICS_REPORT_INTERVAL = Long.toString(TimeUnit.SECONDS.toMillis(30));
   public static final String METRIC_CONTEXT_NAME_KEY = "metrics.context.name";
+  public static final String METRIC_TIMER_WINDOW_SIZE_IN_MINUTES = METRICS_CONFIGURATIONS_PREFIX + "timer.window.size.in.minutes";
+  public static final int DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES = 15;
 
   // File-based reporting
   public static final String METRICS_REPORTING_FILE_ENABLED_KEY =

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
index 3393df6..d57c61e 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
@@ -40,6 +40,7 @@ import org.apache.gobblin.configuration.State;
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.instrumented.StandardMetricsBridge;
 import org.apache.gobblin.metrics.ContextAwareHistogram;
+import org.apache.gobblin.metrics.ContextAwareMetric;
 import org.apache.gobblin.metrics.GobblinMetrics;
 import org.apache.gobblin.metrics.MetricContext;
 import org.apache.hadoop.conf.Configuration;
@@ -155,7 +156,7 @@ public class GobblinClusterManager implements ApplicationLauncher, StandardMetri
   private GobblinHelixJobScheduler jobScheduler;
   @Getter
   private JobConfigurationManager jobConfigurationManager;
-  
+
   private final String clusterName;
   private final Config config;
   private final MetricContext metricContext;
@@ -165,7 +166,7 @@ public class GobblinClusterManager implements ApplicationLauncher, StandardMetri
     this.clusterName = clusterName;
     this.config = config;
     this.metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(config), this.getClass());
-    this.metrics = new Metrics(this.metricContext);
+    this.metrics = new Metrics(this.metricContext, this.config);
     this.isStandaloneMode = ConfigUtils.getBoolean(config, GobblinClusterConfigurationKeys.STANDALONE_CLUSTER_MODE_KEY,
         GobblinClusterConfigurationKeys.DEFAULT_STANDALONE_CLUSTER_MODE);
 
@@ -557,21 +558,6 @@ public class GobblinClusterManager implements ApplicationLauncher, StandardMetri
     return GobblinMetrics.isEnabled(ConfigUtils.configToProperties(this.config));
   }
 
-  @Override
-  public List<Tag<?>> generateTags(State state) {
-    return ImmutableList.of();
-  }
-
-  @Override
-  public void switchMetricContext(List<Tag<?>> tags) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void switchMetricContext(MetricContext context) {
-    throw new UnsupportedOperationException();
-  }
-
   /**
    * A custom implementation of {@link LiveInstanceChangeListener}.
    */
@@ -588,19 +574,16 @@ public class GobblinClusterManager implements ApplicationLauncher, StandardMetri
   private class Metrics extends StandardMetrics {
     public static final String CLUSTER_LEADERSHIP_CHANGE = "clusterLeadershipChange";
     private ContextAwareHistogram clusterLeadershipChange;
-    public Metrics(final MetricContext metricContext) {
-      clusterLeadershipChange = metricContext.contextAwareHistogram(CLUSTER_LEADERSHIP_CHANGE, 1, TimeUnit.MINUTES);
+    public Metrics(final MetricContext metricContext, final Config config) {
+      int timeWindowSizeInMinutes = ConfigUtils.getInt(config, ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES);
+      this.clusterLeadershipChange = metricContext.contextAwareHistogram(CLUSTER_LEADERSHIP_CHANGE, timeWindowSizeInMinutes, TimeUnit.MINUTES);
+      this.contextAwareMetrics.add(clusterLeadershipChange);
     }
 
     @Override
     public String getName() {
       return GobblinClusterManager.class.getName();
     }
-
-    @Override
-    public Collection<ContextAwareHistogram> getHistograms() {
-      return ImmutableList.of(this.clusterLeadershipChange);
-    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
index 62c9b3f..d502462 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
@@ -183,6 +183,9 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
   @Override
   protected void runWorkUnits(List<WorkUnit> workUnits) throws Exception {
     try {
+      long workUnitStartTime = System.currentTimeMillis();
+      workUnits.forEach((k) -> k.setProp(ConfigurationKeys.WORK_UNIT_CREATION_TIME_IN_MILLIS, workUnitStartTime));
+
       // Start the output TaskState collector service
       this.taskStateCollectorService.startAsync().awaitRunning();
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java
index 141e3d1..48b12f2 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java
@@ -28,13 +28,13 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.gobblin.util.ConfigUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.helix.HelixManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.eventbus.EventBus;
@@ -47,7 +47,7 @@ import org.apache.gobblin.cluster.event.NewJobConfigArrivalEvent;
 import org.apache.gobblin.cluster.event.UpdateJobConfigArrivalEvent;
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.instrumented.StandardMetricsBridge;
-import org.apache.gobblin.metrics.ContextAwareCounter;
+import org.apache.gobblin.metrics.ContextAwareMetric;
 import org.apache.gobblin.metrics.ContextAwareGauge;
 import org.apache.gobblin.metrics.ContextAwareTimer;
 import org.apache.gobblin.metrics.GobblinMetrics;
@@ -121,21 +121,6 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe
   }
 
   @Override
-  public List<Tag<?>> generateTags(org.apache.gobblin.configuration.State state) {
-    return null;
-  }
-
-  @Override
-  public void switchMetricContext(List<Tag<?>> tags) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void switchMetricContext(MetricContext context) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public StandardMetrics getStandardMetrics() {
     return metrics;
   }
@@ -147,29 +132,25 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe
     private final AtomicLong totalJobsCommitted;
     private final AtomicLong totalJobsFailed;
     private final AtomicLong totalJobsCancelled;
-    private final ContextAwareGauge<Long> numJobsLaunched;
-    private final ContextAwareGauge<Long> numJobsCompleted;
-    private final ContextAwareGauge<Long> numJobsCommitted;
-    private final ContextAwareGauge<Long> numJobsFailed;
-    private final ContextAwareGauge<Long> numJobsCancelled;
-    private final ContextAwareGauge<Integer> numJobsRunning;
-
-    private final ContextAwareTimer timeForJobCompletion;
-    private final ContextAwareTimer timeForJobFailure;
+
+    private final ContextAwareTimer timeForCompletedJobs;
+    private final ContextAwareTimer timeForFailedJobs;
+    private final ContextAwareTimer timeForCommittedJobs;
     private final ContextAwareTimer timeBeforeJobScheduling;
     private final ContextAwareTimer timeBeforeJobLaunching;
+    private final ContextAwareTimer timeBetwenJobSchedulingAndLaunching;
 
     private final ThreadPoolExecutor threadPoolExecutor;
-    private final ContextAwareGauge<Integer> executorActiveCount;
-    private final ContextAwareGauge<Integer> executorMaximumPoolSize;
-    private final ContextAwareGauge<Integer> executorPoolSize;
-    private final ContextAwareGauge<Integer> executorCorePoolSize;
-    private final ContextAwareGauge<Integer> executorQueueSize;
 
     public Metrics(final MetricContext metricContext) {
       // Thread executor reference from job scheduler
       this.threadPoolExecutor = (ThreadPoolExecutor)GobblinHelixJobScheduler.this.jobExecutor;
 
+      // timer duration setup
+      int windowSize = ConfigUtils.getInt(ConfigUtils.propertiesToConfig(GobblinHelixJobScheduler.this.properties),
+          ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES,
+          ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES);
+
       // All historical counters
       this.totalJobsLaunched = new AtomicLong(0);
       this.totalJobsCompleted = new AtomicLong(0);
@@ -177,25 +158,34 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe
       this.totalJobsFailed = new AtomicLong(0);
       this.totalJobsCancelled = new AtomicLong(0);
 
-      this.numJobsLaunched = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_LAUNCHED, ()->this.totalJobsLaunched.get());
-      this.numJobsCompleted = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_COMPLETED, ()->this.totalJobsCompleted.get());
-      this.numJobsCommitted = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_COMMITTED, ()->this.totalJobsCommitted.get());
-      this.numJobsFailed = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_FAILED, ()->this.totalJobsFailed.get());
-      this.numJobsCancelled = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_CANCELLED, ()->this.totalJobsCancelled.get());
-      this.numJobsRunning = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_RUNNING,
-          ()->(int)(Metrics.this.totalJobsLaunched.get() - Metrics.this.totalJobsCompleted.get()));
-
-      this.timeForJobCompletion = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_FOR_JOB_COMPLETION, 1, TimeUnit.MINUTES);
-      this.timeForJobFailure = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_FOR_JOB_FAILURE,1, TimeUnit.MINUTES);
-      this.timeBeforeJobScheduling = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_BEFORE_JOB_SCHEDULING, 1, TimeUnit.MINUTES);
-      this.timeBeforeJobLaunching = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_BEFORE_JOB_LAUNCHING, 1, TimeUnit.MINUTES);
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_LAUNCHED, ()->this.totalJobsLaunched.get()));
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_COMPLETED, ()->this.totalJobsCompleted.get()));
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_COMMITTED, ()->this.totalJobsCommitted.get()));
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_FAILED, ()->this.totalJobsFailed.get()));
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_CANCELLED, ()->this.totalJobsCancelled.get()));
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.NUM_JOBS_RUNNING,
+          ()->(int)(Metrics.this.totalJobsLaunched.get() - Metrics.this.totalJobsCompleted.get())));
+
+      this.timeForCompletedJobs = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_FOR_COMPLETED_JOBS, windowSize, TimeUnit.MINUTES);
+      this.timeForFailedJobs = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_FOR_FAILED_JOBS, windowSize, TimeUnit.MINUTES);
+      this.timeForCommittedJobs = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_FOR_COMMITTED_JOBS, windowSize, TimeUnit.MINUTES);
+      this.timeBeforeJobScheduling = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_BEFORE_JOB_SCHEDULING, windowSize, TimeUnit.MINUTES);
+      this.timeBeforeJobLaunching = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_BEFORE_JOB_LAUNCHING, windowSize, TimeUnit.MINUTES);
+      this.timeBetwenJobSchedulingAndLaunching = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_BETWEEN_JOB_SCHEDULING_AND_LAUNCHING, windowSize, TimeUnit.MINUTES);
 
       // executor metrics
-      this.executorActiveCount = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_ACTIVE_COUNT, ()->this.threadPoolExecutor.getActiveCount());
-      this.executorMaximumPoolSize = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_MAX_POOL_SIZE, ()->this.threadPoolExecutor.getMaximumPoolSize());
-      this.executorPoolSize = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_POOL_SIZE, ()->this.threadPoolExecutor.getPoolSize());
-      this.executorCorePoolSize =  metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_CORE_POOL_SIZE, ()->this.threadPoolExecutor.getCorePoolSize());
-      this.executorQueueSize = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_QUEUE_SIZE, ()->this.threadPoolExecutor.getQueue().size());
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_ACTIVE_COUNT, ()->this.threadPoolExecutor.getActiveCount()));
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_MAX_POOL_SIZE, ()->this.threadPoolExecutor.getMaximumPoolSize()));
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_POOL_SIZE, ()->this.threadPoolExecutor.getPoolSize()));
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_CORE_POOL_SIZE, ()->this.threadPoolExecutor.getCorePoolSize()));
+      this.contextAwareMetrics.add(metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_QUEUE_SIZE, ()->this.threadPoolExecutor.getQueue().size()));
+
+      this.contextAwareMetrics.add(timeForCommittedJobs);
+      this.contextAwareMetrics.add(timeForCompletedJobs);
+      this.contextAwareMetrics.add(timeForFailedJobs);
+      this.contextAwareMetrics.add(timeBeforeJobScheduling);
+      this.contextAwareMetrics.add(timeBeforeJobLaunching);
+      this.contextAwareMetrics.add(timeBetwenJobSchedulingAndLaunching);
     }
 
     private void updateTimeBeforeJobScheduling (Properties jobConfig) {
@@ -208,36 +198,13 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe
       Instrumented.updateTimer(Optional.of(timeBeforeJobLaunching), System.currentTimeMillis() - jobCreationTime, TimeUnit.MILLISECONDS);
     }
 
-    @Override
-    public String getName() {
-      return GobblinHelixJobScheduler.class.getName();
-    }
-
-    @Override
-    public Collection<ContextAwareGauge<?>> getGauges() {
-      List<ContextAwareGauge<?>> list = Lists.newArrayList();
-      list.add(numJobsRunning);
-      list.add(numJobsLaunched);
-      list.add(numJobsCompleted);
-      list.add(numJobsCommitted);
-      list.add(numJobsFailed);
-      list.add(numJobsCancelled);
-      list.add(executorActiveCount);
-      list.add(executorMaximumPoolSize);
-      list.add(executorPoolSize);
-      list.add(executorCorePoolSize);
-      list.add(executorQueueSize);
-      return list;
+    private void updateTimeBetweenJobSchedulingAndJobLaunching (long scheduledTime, long launchingTime) {
+      Instrumented.updateTimer(Optional.of(timeBetwenJobSchedulingAndLaunching), launchingTime - scheduledTime, TimeUnit.MILLISECONDS);
     }
 
     @Override
-    public Collection<ContextAwareCounter> getCounters() {
-      return ImmutableList.of();
-    }
-
-    @Override
-    public Collection<ContextAwareTimer> getTimers() {
-      return ImmutableList.of(timeForJobCompletion, timeForJobFailure, timeBeforeJobScheduling, timeBeforeJobLaunching);
+    public String getName() {
+      return GobblinHelixJobScheduler.class.getName();
     }
   }
 
@@ -265,12 +232,13 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe
       long startTime = jobContext.getJobState().getPropAsLong(START_TIME);
       if (GobblinHelixJobScheduler.this.isInstrumentationEnabled()) {
         metrics.totalJobsCompleted.incrementAndGet();
-        Instrumented.updateTimer(Optional.of(metrics.timeForJobCompletion), System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
+        Instrumented.updateTimer(Optional.of(metrics.timeForCompletedJobs), System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
         if (jobContext.getJobState().getState() == JobState.RunningState.FAILED) {
             metrics.totalJobsFailed.incrementAndGet();
-            Instrumented.updateTimer(Optional.of(metrics.timeForJobFailure), System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
+            Instrumented.updateTimer(Optional.of(metrics.timeForFailedJobs), System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
         } else {
             metrics.totalJobsCommitted.incrementAndGet();
+            Instrumented.updateTimer(Optional.of(metrics.timeForCommittedJobs), System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
         }
       }
     }
@@ -382,17 +350,20 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe
     private final String jobUri;
     private final Properties jobConfig;
     private final JobListener jobListener;
+    private final Long creationTimeInMillis;
 
     public NonScheduledJobRunner(String jobUri, Properties jobConfig, JobListener jobListener) {
       this.jobUri = jobUri;
       this.jobConfig = jobConfig;
       this.jobListener = jobListener;
+      this.creationTimeInMillis = System.currentTimeMillis();
     }
 
     @Override
     public void run() {
       try {
         ((MetricsTrackingListener)jobListener).metrics.updateTimeBeforeJobLaunching(this.jobConfig);
+        ((MetricsTrackingListener)jobListener).metrics.updateTimeBetweenJobSchedulingAndJobLaunching(this.creationTimeInMillis, System.currentTimeMillis());
         GobblinHelixJobScheduler.this.runJob(this.jobConfig, this.jobListener);
 
         // remove non-scheduled job catalog once done so it won't be re-executed

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
index a3fddab..3ec40dc 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunner.java
@@ -36,6 +36,11 @@ import org.apache.commons.cli.DefaultParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.gobblin.configuration.State;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.instrumented.StandardMetricsBridge;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.Tag;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -81,6 +86,9 @@ import org.apache.gobblin.util.HadoopUtils;
 import org.apache.gobblin.util.JvmUtils;
 import org.apache.gobblin.util.PathUtils;
 
+import javax.annotation.Nonnull;
+import lombok.Getter;
+
 import static org.apache.gobblin.cluster.GobblinClusterConfigurationKeys.CLUSTER_WORK_DIR;
 
 
@@ -109,7 +117,7 @@ import static org.apache.gobblin.cluster.GobblinClusterConfigurationKeys.CLUSTER
  * @author Yinan Li
  */
 @Alpha
-public class GobblinTaskRunner {
+public class GobblinTaskRunner implements StandardMetricsBridge {
 
   private static final Logger logger = LoggerFactory.getLogger(GobblinTaskRunner.class);
   static final java.nio.file.Path CLUSTER_CONF_PATH = Paths.get("generated-gobblin-cluster.conf");
@@ -141,6 +149,8 @@ public class GobblinTaskRunner {
   private final String applicationName;
   private final String applicationId;
   private final Path appWorkPath;
+  private final MetricContext metricContext;
+  private final StandardMetricsBridge.StandardMetrics metrics;
 
   public GobblinTaskRunner(String applicationName, String helixInstanceName, String applicationId,
       String taskRunnerId, Config config, Optional<Path> appWorkDirOptional)
@@ -160,8 +170,10 @@ public class GobblinTaskRunner {
     initHelixManager();
 
     this.containerMetrics = buildContainerMetrics();
-
-    this.taskStateModelFactory = registerHelixTaskFactory();
+    TaskFactoryBuilder builder = new TaskFactoryBuilder(this.config);
+    this.taskStateModelFactory = createTaskStateModelFactory(builder.build());
+    this.metrics = builder.getTaskMetrics();
+    this.metricContext = builder.getMetricContext();
 
     services.addAll(getServices());
     if (services.isEmpty()) {
@@ -174,6 +186,38 @@ public class GobblinTaskRunner {
         applicationName, helixInstanceName, applicationId, taskRunnerId, config, appWorkDirOptional);
   }
 
+  private class TaskFactoryBuilder {
+    private final boolean isRunTaskInSeparateProcessEnabled;
+    private final TaskFactory taskFactory;
+    @Getter
+    private final MetricContext metricContext;
+    @Getter
+    private StandardMetricsBridge.StandardMetrics taskMetrics;
+
+    public TaskFactoryBuilder(Config config) {
+      isRunTaskInSeparateProcessEnabled = getIsRunTaskInSeparateProcessEnabled(config);
+      metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(config), this.getClass());
+      if (isRunTaskInSeparateProcessEnabled) {
+        logger.info("Running a task in a separate process is enabled.");
+        taskFactory = new HelixTaskFactory(GobblinTaskRunner.this.containerMetrics, CLUSTER_CONF_PATH, config);
+        taskMetrics = new GobblinTaskRunnerMetrics.JvmTaskRunnerMetrics();
+      } else {
+        Properties properties = ConfigUtils.configToProperties(config);
+        TaskExecutor taskExecutor = new TaskExecutor(properties);
+        taskFactory = getInProcessTaskFactory(taskExecutor);
+        taskMetrics = new GobblinTaskRunnerMetrics.InProcessTaskRunnerMetrics(taskExecutor, metricContext);
+      }
+    }
+
+    public TaskFactory build(){
+       return taskFactory;
+    }
+
+    private Boolean getIsRunTaskInSeparateProcessEnabled(Config config) {
+      return ConfigUtils.getBoolean(config, GobblinClusterConfigurationKeys.ENABLE_TASK_IN_SEPARATE_PROCESS, false);
+    }
+  }
+
   private Path initAppWorkDir(Config config, Optional<Path> appWorkDirOptional) {
     return appWorkDirOptional.isPresent() ? appWorkDirOptional.get() : GobblinClusterUtils
         .getAppWorkDirPathFromConfig(config, this.fs, this.applicationName, this.applicationId);
@@ -189,19 +233,10 @@ public class GobblinTaskRunner {
         this.helixInstanceName, InstanceType.PARTICIPANT, zkConnectionString);
   }
 
-  private TaskStateModelFactory registerHelixTaskFactory() {
+  private TaskStateModelFactory createTaskStateModelFactory(TaskFactory factory) {
     Map<String, TaskFactory> taskFactoryMap = Maps.newHashMap();
 
-    boolean isRunTaskInSeparateProcessEnabled = getIsRunTaskInSeparateProcessEnabled();
-    TaskFactory taskFactory;
-    if (isRunTaskInSeparateProcessEnabled) {
-      logger.info("Running a task in a separate process is enabled.");
-      taskFactory = new HelixTaskFactory(this.containerMetrics, CLUSTER_CONF_PATH, config);
-    } else {
-      taskFactory = getInProcessTaskFactory();
-    }
-
-    taskFactoryMap.put(GOBBLIN_TASK_FACTORY_NAME, taskFactory);
+    taskFactoryMap.put(GOBBLIN_TASK_FACTORY_NAME, factory);
     TaskStateModelFactory taskStateModelFactory =
         new TaskStateModelFactory(this.helixManager, taskFactoryMap);
     this.helixManager.getStateMachineEngine()
@@ -209,14 +244,13 @@ public class GobblinTaskRunner {
     return taskStateModelFactory;
   }
 
-  private TaskFactory getInProcessTaskFactory() {
+  private TaskFactory getInProcessTaskFactory(TaskExecutor taskExecutor) {
     Properties properties = ConfigUtils.configToProperties(this.config);
     URI rootPathUri = PathUtils.getRootPath(this.appWorkPath).toUri();
     Config stateStoreJobConfig = ConfigUtils.propertiesToConfig(properties)
         .withValue(ConfigurationKeys.STATE_STORE_FS_URI_KEY,
             ConfigValueFactory.fromAnyRef(rootPathUri.toString()));
 
-    TaskExecutor taskExecutor = new TaskExecutor(properties);
     TaskStateTracker taskStateTracker = new GobblinHelixTaskStateTracker(properties);
 
     services.add(taskExecutor);
@@ -385,6 +419,22 @@ public class GobblinTaskRunner {
     }
   }
 
+  @Override
+  public StandardMetrics getStandardMetrics() {
+    return this.metrics;
+  }
+
+  @Nonnull
+  @Override
+  public MetricContext getMetricContext() {
+    return this.metricContext;
+  }
+
+  @Override
+  public boolean isInstrumentationEnabled() {
+    return GobblinMetrics.isEnabled(this.config);
+  }
+
   /**
    * A custom {@link MessageHandlerFactory} for {@link ParticipantShutdownMessageHandler}s that handle messages
    * of type "SHUTDOWN" for shutting down the participants.

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunnerMetrics.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunnerMetrics.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunnerMetrics.java
new file mode 100644
index 0000000..51e8b36
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTaskRunnerMetrics.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.cluster;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import com.codahale.metrics.Metric;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.StandardMetricsBridge;
+import org.apache.gobblin.metrics.ContextAwareMetric;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.runtime.TaskExecutor;
+
+
+public class GobblinTaskRunnerMetrics {
+
+  static class InProcessTaskRunnerMetrics extends StandardMetricsBridge.StandardMetrics {
+    private TaskExecutor taskExecutor;
+    private static String CURRENT_QUEUED_TASK_COUNT = "currentQueuedTaskCount";
+    private static String HISTORICAL_QUEUED_TASK_COUNT = "historicalQueuedTaskCount";
+    private static String QUEUED_TASK_COUNT = "queuedTaskCount";
+    private static String CURRENT_QUEUED_TASK_TOTAL_TIME = "currentQueuedTaskTotalTime";
+    private static String HISTORICAL_QUEUED_TASK_TOTAL_TIME = "historicalQueuedTaskTotalTime";
+    private static String QUEUED_TASK_TOTAL_TIME = "queuedTaskTotalTime";
+    private static String FAILED_TASK_COUNT = "failedTaskCount";
+    private static String SUCCESSFUL_TASK_COUNT = "successfulTaskCount";
+    private static String RUNNING_TASK_COUNT = "runningTaskCount";
+
+    public InProcessTaskRunnerMetrics (TaskExecutor executor, MetricContext context) {
+      taskExecutor = executor;
+      contextAwareMetrics.add(context.newContextAwareGauge(CURRENT_QUEUED_TASK_COUNT, ()->this.taskExecutor.getCurrentQueuedTaskCount().longValue()));
+      contextAwareMetrics.add(context.newContextAwareGauge(CURRENT_QUEUED_TASK_TOTAL_TIME, ()->this.taskExecutor.getCurrentQueuedTaskTotalTime().longValue()));
+      contextAwareMetrics.add(context.newContextAwareGauge(HISTORICAL_QUEUED_TASK_COUNT, ()->this.taskExecutor.getHistoricalQueuedTaskCount().longValue()));
+      contextAwareMetrics.add(context.newContextAwareGauge(HISTORICAL_QUEUED_TASK_TOTAL_TIME, ()->this.taskExecutor.getHistoricalQueuedTaskTotalTime().longValue()));
+      contextAwareMetrics.add(context.newContextAwareGauge(QUEUED_TASK_COUNT, ()->this.taskExecutor.getQueuedTaskCount().longValue()));
+      contextAwareMetrics.add(context.newContextAwareGauge(QUEUED_TASK_TOTAL_TIME, ()->this.taskExecutor.getQueuedTaskTotalTime().longValue()));
+      contextAwareMetrics.add(context.newContextAwareGauge(FAILED_TASK_COUNT, ()->this.taskExecutor.getFailedTaskCount().getCount()));
+      contextAwareMetrics.add(context.newContextAwareGauge(SUCCESSFUL_TASK_COUNT, ()->this.taskExecutor.getSuccessfulTaskCount().getCount()));
+      contextAwareMetrics.add(context.newContextAwareGauge(RUNNING_TASK_COUNT, ()->this.taskExecutor.getRunningTaskCount().getCount()));
+
+      this.rawMetrics.put(ConfigurationKeys.WORK_UNIT_CREATION_AND_RUN_INTERVAL, this.taskExecutor.getTaskCreateAndRunTimer());
+    }
+
+    @Override
+    public String getName() {
+      return InProcessTaskRunnerMetrics.class.getName();
+    }
+  }
+
+  static class JvmTaskRunnerMetrics extends StandardMetricsBridge.StandardMetrics {
+    //TODO: add metrics to monitor the process execution status (will be revisited after process isolation work is done)
+    @Override
+    public String getName() {
+      return JvmTaskRunnerMetrics.class.getName();
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-core-base/src/main/java/org/apache/gobblin/instrumented/StandardMetricsBridge.java
----------------------------------------------------------------------
diff --git a/gobblin-core-base/src/main/java/org/apache/gobblin/instrumented/StandardMetricsBridge.java b/gobblin-core-base/src/main/java/org/apache/gobblin/instrumented/StandardMetricsBridge.java
index 3993dce..355139b 100644
--- a/gobblin-core-base/src/main/java/org/apache/gobblin/instrumented/StandardMetricsBridge.java
+++ b/gobblin-core-base/src/main/java/org/apache/gobblin/instrumented/StandardMetricsBridge.java
@@ -18,15 +18,18 @@
 package org.apache.gobblin.instrumented;
 
 import java.util.Collection;
+import java.util.List;
+import java.util.Map;
 
-import org.apache.gobblin.metrics.ContextAwareCounter;
-import org.apache.gobblin.metrics.ContextAwareGauge;
-import org.apache.gobblin.metrics.ContextAwareHistogram;
-import org.apache.gobblin.metrics.ContextAwareMeter;
-import org.apache.gobblin.metrics.ContextAwareTimer;
-
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricSet;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
+import org.apache.gobblin.metrics.ContextAwareMetric;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.Tag;
 
 /**
  * This interface indicates a class will expose its metrics to some external systems.
@@ -35,30 +38,37 @@ public interface StandardMetricsBridge extends Instrumentable {
 
   StandardMetrics getStandardMetrics();
 
-  public class StandardMetrics {
+  default void switchMetricContext(MetricContext context) {
+    throw new UnsupportedOperationException();
+  }
 
-    public String getName() {
-      return this.getClass().getName();
-    }
+  default void switchMetricContext(List<Tag<?>> tags) {
+    throw new UnsupportedOperationException();
+  }
 
-    public Collection<ContextAwareGauge<?>> getGauges() {
-      return ImmutableList.of();
-    }
+  default List<Tag<?>> generateTags(org.apache.gobblin.configuration.State state) {
+    return ImmutableList.of();
+  }
+
+  public class StandardMetrics implements MetricSet {
+    protected final List<ContextAwareMetric> contextAwareMetrics;
+    protected final Map<String, Metric> rawMetrics;
 
-    public Collection<ContextAwareCounter> getCounters() {
-      return ImmutableList.of();
+    public StandardMetrics() {
+      this.contextAwareMetrics = Lists.newArrayList();
+      this.rawMetrics = Maps.newHashMap();
     }
 
-    public Collection<ContextAwareMeter> getMeters() {
-      return ImmutableList.of();
+    public String getName() {
+      return this.getClass().getName();
     }
 
-    public Collection<ContextAwareTimer> getTimers() {
-      return ImmutableList.of();
+    public Collection<ContextAwareMetric> getContextAwareMetrics() {
+      return contextAwareMetrics;
     }
 
-    public Collection<ContextAwareHistogram> getHistograms() {
-      return ImmutableList.of();
+    public Map<String, Metric> getMetrics() {
+      return rawMetrics;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java b/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
index 5fd5413..6d8de39 100644
--- a/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
+++ b/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
@@ -34,6 +34,7 @@ import org.apache.commons.lang3.tuple.Pair;
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.instrumented.StandardMetricsBridge;
 import org.apache.gobblin.metrics.ContextAwareGauge;
+import org.apache.gobblin.metrics.ContextAwareMetric;
 import org.apache.gobblin.metrics.GobblinMetrics;
 import org.apache.gobblin.metrics.MetricContext;
 import org.apache.gobblin.metrics.Tag;
@@ -199,12 +200,6 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
   }
 
   private class Metrics extends StandardMetricsBridge.StandardMetrics {
-    private ContextAwareGauge<Integer> jobSpecQueueSize;
-    private ContextAwareGauge<Long> jobSpecEnq;
-    private ContextAwareGauge<Long> jobSpecDeq;
-    private ContextAwareGauge<Long> jobSpecConsumed;
-    private ContextAwareGauge<Long> jobSpecParseFailures;
-
     private AtomicLong jobSpecEnqCount = new AtomicLong(0);
     private AtomicLong jobSpecDeqCount = new AtomicLong(0);
 
@@ -215,12 +210,12 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
     public static final String SPEC_CONSUMER_JOB_SPEC_PARSE_FAILURES = "specConsumerJobSpecParseFailures";
 
     public Metrics(MetricContext context) {
-      this.jobSpecQueueSize = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_QUEUE_SIZE, ()->StreamingKafkaSpecConsumer.this._jobSpecQueue.size());
-      this.jobSpecEnq = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_ENQ, ()->jobSpecEnqCount.get());
-      this.jobSpecDeq = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_DEQ, ()->jobSpecDeqCount.get());
-      this.jobSpecConsumed = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_CONSUMED,
-          ()->getNewSpecs() + getRemovedSpecs() + getMessageParseFailures());
-      this.jobSpecParseFailures = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_PARSE_FAILURES, ()->getMessageParseFailures());
+      this.contextAwareMetrics.add(context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_QUEUE_SIZE, ()->StreamingKafkaSpecConsumer.this._jobSpecQueue.size()));
+      this.contextAwareMetrics.add(context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_ENQ, ()->jobSpecEnqCount.get()));
+      this.contextAwareMetrics.add(context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_DEQ, ()->jobSpecDeqCount.get()));
+      this.contextAwareMetrics.add(context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_CONSUMED,
+          ()->getNewSpecs() + getRemovedSpecs() + getMessageParseFailures()));
+      this.contextAwareMetrics.add(context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_PARSE_FAILURES, ()->getMessageParseFailures()));
     }
 
     private long getNewSpecs() {
@@ -237,16 +232,6 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
       return StreamingKafkaSpecConsumer.this._jobMonitor.getMessageParseFailures() != null?
           StreamingKafkaSpecConsumer.this._jobMonitor.getMessageParseFailures().getCount():0;
     }
-
-    public Collection<ContextAwareGauge<?>> getGauges() {
-      List list = Lists.newArrayList();
-      list.add(jobSpecQueueSize);
-      list.add(jobSpecEnq);
-      list.add(jobSpecDeq);
-      list.add(jobSpecConsumed);
-      list.add(jobSpecParseFailures);
-      return list;
-    }
   }
 
   @Override
@@ -264,19 +249,4 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
   public boolean isInstrumentationEnabled() {
     return _isInstrumentedEnabled;
   }
-
-  @Override
-  public List<Tag<?>> generateTags(org.apache.gobblin.configuration.State state) {
-    return ImmutableList.of();
-  }
-
-  @Override
-  public void switchMetricContext(List<Tag<?>> tags) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void switchMetricContext(MetricContext context) {
-    throw new UnsupportedOperationException();
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/TaskExecutor.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/TaskExecutor.java
index be78275..b868893 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/TaskExecutor.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/TaskExecutor.java
@@ -39,6 +39,8 @@ import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Meter;
 import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricSet;
+import com.codahale.metrics.SlidingTimeWindowReservoir;
+import com.codahale.metrics.Timer;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
@@ -75,12 +77,15 @@ public class TaskExecutor extends AbstractIdleService {
   private final ExecutorService forkExecutor;
 
   // Task retry interval
+  @Getter
   private final long retryIntervalInSeconds;
 
   // The maximum number of items in the queued task time map.
+  @Getter
   private final int queuedTaskTimeMaxSize;
 
   // The maximum age of the items in the queued task time map.
+  @Getter
   private final long queuedTaskTimeMaxAge ;
 
   // Map of queued task ids to queue times.  The key is the task id, the value is the time the task was queued.  If the
@@ -96,32 +101,44 @@ public class TaskExecutor extends AbstractIdleService {
   private long lastCalculationTime = 0;
 
   // The total number of tasks currently queued and queued over the historical lookback period.
+  @Getter
   private AtomicInteger queuedTaskCount = new AtomicInteger();
 
   // The total number of tasks currently queued.
+  @Getter
   private AtomicInteger currentQueuedTaskCount = new AtomicInteger();
 
   // The total number of tasks queued over the historical lookback period.
+  @Getter
   private AtomicInteger historicalQueuedTaskCount = new AtomicInteger();
 
   // The total time tasks have currently been in the queue and were in the queue during the historical lookback period.
+  @Getter
   private AtomicLong queuedTaskTotalTime = new AtomicLong();
 
   // The total time tasks have currently been in the queue.
+  @Getter
   private AtomicLong currentQueuedTaskTotalTime = new AtomicLong();
 
   // The total time tasks have been in the queue during the historical lookback period.
+  @Getter
   private AtomicLong historicalQueuedTaskTotalTime = new AtomicLong();
 
   // Count of running tasks.
+  @Getter
   private final Counter runningTaskCount = new Counter();
 
   // Count of failed tasks.
+  @Getter
   private final Meter successfulTaskCount = new Meter();
 
   // Count of failed tasks.
+  @Getter
   private final Meter failedTaskCount = new Meter();
 
+  @Getter
+  private final Timer taskCreateAndRunTimer;
+
   // The metric set exposed from the task executor.
   private final TaskExecutorQueueMetricSet metricSet = new TaskExecutorQueueMetricSet();
 
@@ -129,7 +146,7 @@ public class TaskExecutor extends AbstractIdleService {
    * Constructor used internally.
    */
   private TaskExecutor(int taskExecutorThreadPoolSize, int coreRetryThreadPoolSize, long retryIntervalInSeconds,
-                       int queuedTaskTimeMaxSize, long queuedTaskTimeMaxAge) {
+                       int queuedTaskTimeMaxSize, long queuedTaskTimeMaxAge, int timerWindowSize) {
     Preconditions.checkArgument(taskExecutorThreadPoolSize > 0, "Task executor thread pool size should be positive");
     Preconditions.checkArgument(retryIntervalInSeconds > 0, "Task retry interval should be positive");
     Preconditions.checkArgument(queuedTaskTimeMaxSize > 0, "Queued task time max size should be positive");
@@ -143,6 +160,7 @@ public class TaskExecutor extends AbstractIdleService {
     this.retryIntervalInSeconds = retryIntervalInSeconds;
     this.queuedTaskTimeMaxSize = queuedTaskTimeMaxSize;
     this.queuedTaskTimeMaxAge = queuedTaskTimeMaxAge;
+    this.taskCreateAndRunTimer = new Timer(new SlidingTimeWindowReservoir(timerWindowSize, TimeUnit.MINUTES));
 
     this.forkExecutor = ExecutorsUtils.loggingDecorator(
         new ThreadPoolExecutor(
@@ -175,7 +193,9 @@ public class TaskExecutor extends AbstractIdleService {
         Integer.parseInt(properties.getProperty(ConfigurationKeys.QUEUED_TASK_TIME_MAX_SIZE,
             Integer.toString(ConfigurationKeys.DEFAULT_QUEUED_TASK_TIME_MAX_SIZE))),
         Long.parseLong(properties.getProperty(ConfigurationKeys.QUEUED_TASK_TIME_MAX_AGE,
-            Long.toString(ConfigurationKeys.DEFAULT_QUEUED_TASK_TIME_MAX_AGE))));
+            Long.toString(ConfigurationKeys.DEFAULT_QUEUED_TASK_TIME_MAX_AGE))),
+        Integer.parseInt(properties.getProperty(ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES,
+            Integer.toString(ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES))));
   }
 
   /**
@@ -191,7 +211,9 @@ public class TaskExecutor extends AbstractIdleService {
         conf.getInt(ConfigurationKeys.QUEUED_TASK_TIME_MAX_SIZE,
             ConfigurationKeys.DEFAULT_QUEUED_TASK_TIME_MAX_SIZE),
         conf.getLong(ConfigurationKeys.QUEUED_TASK_TIME_MAX_AGE,
-            ConfigurationKeys.DEFAULT_QUEUED_TASK_TIME_MAX_AGE));
+            ConfigurationKeys.DEFAULT_QUEUED_TASK_TIME_MAX_AGE),
+        conf.getInt(ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES,
+            ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES));
   }
 
   @Override
@@ -430,7 +452,12 @@ public class TaskExecutor extends AbstractIdleService {
 
     private void onStart(long startTime) {
       Long queueTime = queuedTasks.remove(this.underlyingTask.getTaskId());
+      long workUnitCreationTime = this.underlyingTask.getTaskContext().getTaskState().getPropAsLong(ConfigurationKeys.WORK_UNIT_CREATION_TIME_IN_MILLIS, 0);
       long timeInQueue = startTime - queueTime;
+      long timeSinceWorkUnitCreation = startTime - workUnitCreationTime;
+
+      taskCreateAndRunTimer.update(timeSinceWorkUnitCreation, TimeUnit.MILLISECONDS);
+
       LOG.debug(String.format("Task %s started. Saving queued time of %d ms to history.", underlyingTask.getTaskId(), timeInQueue));
       queuedTaskTimeHistorical.putIfAbsent(System.currentTimeMillis(), timeInQueue);
       runningTaskCount.inc();

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobCatalog.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobCatalog.java
index 42ecef3..6afa94a 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobCatalog.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobCatalog.java
@@ -18,23 +18,27 @@ package org.apache.gobblin.runtime.api;
 
 import java.net.URI;
 import java.util.Collection;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.typesafe.config.Config;
 
 import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.instrumented.GobblinMetricsKeys;
 import org.apache.gobblin.instrumented.Instrumentable;
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.instrumented.StandardMetricsBridge;
-import org.apache.gobblin.metrics.ContextAwareCounter;
 import org.apache.gobblin.metrics.ContextAwareGauge;
+import org.apache.gobblin.metrics.ContextAwareMetric;
 import org.apache.gobblin.metrics.ContextAwareTimer;
 import org.apache.gobblin.metrics.GobblinTrackingEvent;
 import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.util.ConfigUtils;
 
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
@@ -76,6 +80,7 @@ public interface JobCatalog extends JobCatalogListenersContainer, Instrumentable
     public static final String JOB_UPDATED_OPERATION_TYPE = "JobUpdated";
 
     private final MetricContext metricsContext;
+    protected final int timeWindowSizeInMinutes;
     @Getter private final AtomicLong totalAddedJobs;
     @Getter private final AtomicLong totalDeletedJobs;
     @Getter private final AtomicLong totalUpdatedJobs;
@@ -85,17 +90,28 @@ public interface JobCatalog extends JobCatalogListenersContainer, Instrumentable
     @Getter private final ContextAwareGauge<Long> totalUpdateCalls;
     @Getter private final ContextAwareGauge<Integer> numActiveJobs;
 
-    public StandardMetrics(final JobCatalog jobCatalog) {
+    public StandardMetrics(final JobCatalog jobCatalog, Optional<Config> sysConfig) {
+      // timer window size
+      this.timeWindowSizeInMinutes = sysConfig.isPresent()?
+          ConfigUtils.getInt(sysConfig.get(), ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES) :
+          ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES;
+
       this.metricsContext = jobCatalog.getMetricContext();
       this.totalAddedJobs = new AtomicLong(0);
       this.totalDeletedJobs = new AtomicLong(0);
       this.totalUpdatedJobs = new AtomicLong(0);
 
-      this.timeForJobCatalogGet = metricsContext.contextAwareTimer(TIME_FOR_JOB_CATALOG_GET, 1, TimeUnit.MINUTES);
+      this.timeForJobCatalogGet = metricsContext.contextAwareTimer(TIME_FOR_JOB_CATALOG_GET, timeWindowSizeInMinutes, TimeUnit.MINUTES);
       this.totalAddCalls = metricsContext.newContextAwareGauge(TOTAL_ADD_CALLS, ()->this.totalAddedJobs.get());
       this.totalUpdateCalls = metricsContext.newContextAwareGauge(TOTAL_UPDATE_CALLS, ()->this.totalUpdatedJobs.get());
       this.totalDeleteCalls = metricsContext.newContextAwareGauge(TOTAL_DELETE_CALLS, ()->this.totalDeletedJobs.get());
       this.numActiveJobs = metricsContext.newContextAwareGauge(NUM_ACTIVE_JOBS_NAME, ()->(int)(totalAddedJobs.get() - totalDeletedJobs.get()));
+
+      this.contextAwareMetrics.add(timeForJobCatalogGet);
+      this.contextAwareMetrics.add(totalAddCalls);
+      this.contextAwareMetrics.add(totalDeleteCalls);
+      this.contextAwareMetrics.add(totalUpdateCalls);
+      this.contextAwareMetrics.add(numActiveJobs);
     }
 
     public void updateGetJobTime(long startTime) {
@@ -136,20 +152,5 @@ public interface JobCatalog extends JobCatalogListenersContainer, Instrumentable
       this.totalUpdatedJobs.incrementAndGet();
       submitTrackingEvent(updatedJob, JOB_UPDATED_OPERATION_TYPE);
     }
-
-    @Override
-    public Collection<ContextAwareGauge<?>> getGauges() {
-      return ImmutableList.of(totalAddCalls, totalDeleteCalls, totalUpdateCalls, numActiveJobs);
-    }
-
-    @Override
-    public Collection<ContextAwareCounter> getCounters() {
-      return ImmutableList.of();
-    }
-
-    @Override
-    public Collection<ContextAwareTimer> getTimers() {
-      return ImmutableList.of(timeForJobCatalogGet);
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobExecutionLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobExecutionLauncher.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobExecutionLauncher.java
index 3f50ee7..7bc9cc0 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobExecutionLauncher.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobExecutionLauncher.java
@@ -44,10 +44,12 @@ public interface JobExecutionLauncher extends Instrumentable {
     public static final String NUM_JOBS_CANCELLED = "numJobsCancelled";
     public static final String NUM_JOBS_RUNNING = "numJobsRunning";
 
-    public static final String TIMER_FOR_JOB_COMPLETION = "timerForJobCompletion";
-    public static final String TIMER_FOR_JOB_FAILURE = "timerForJobFailure";
+    public static final String TIMER_FOR_COMPLETED_JOBS = "timeForCompletedJobs";
+    public static final String TIMER_FOR_FAILED_JOBS = "timeForFailedJobs";
+    public static final String TIMER_FOR_COMMITTED_JOBS = "timerForCommittedJobs";
     public static final String TIMER_BEFORE_JOB_SCHEDULING = "timerBeforeJobScheduling";
     public static final String TIMER_BEFORE_JOB_LAUNCHING = "timerBeforeJobLaunching";
+    public static final String TIMER_BETWEEN_JOB_SCHEDULING_AND_LAUNCHING = "timerBetwenJobSchedulingAndLaunching";
 
     public static final String EXECUTOR_ACTIVE_COUNT = "executorActiveCount";
     public static final String EXECUTOR_MAX_POOL_SIZE = "executorMaximumPoolSize";

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MutableJobCatalog.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MutableJobCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MutableJobCatalog.java
index 8b6e98c..57dfce5 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MutableJobCatalog.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MutableJobCatalog.java
@@ -22,10 +22,14 @@ import java.util.Collection;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMetric;
 import org.apache.gobblin.metrics.ContextAwareTimer;
+import org.apache.gobblin.util.ConfigUtils;
 
 import com.google.common.base.Optional;
+import com.typesafe.config.Config;
 
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
@@ -55,10 +59,12 @@ public interface MutableJobCatalog extends JobCatalog {
     public static final String TIME_FOR_JOB_CATALOG_PUT = "timeForJobCatalogPut";
     @Getter private final ContextAwareTimer timeForJobCatalogPut;
     @Getter private final ContextAwareTimer timeForJobCatalogRemove;
-    public MutableStandardMetrics(JobCatalog catalog) {
-      super(catalog);
-      timeForJobCatalogPut = catalog.getMetricContext().contextAwareTimer(TIME_FOR_JOB_CATALOG_PUT, 1, TimeUnit.MINUTES);
-      timeForJobCatalogRemove =  catalog.getMetricContext().contextAwareTimer(TIME_FOR_JOB_CATALOG_REMOVE, 1, TimeUnit.MINUTES);
+    public MutableStandardMetrics(JobCatalog catalog, Optional<Config> sysConfig) {
+      super(catalog, sysConfig);
+      timeForJobCatalogPut = catalog.getMetricContext().contextAwareTimer(TIME_FOR_JOB_CATALOG_PUT, timeWindowSizeInMinutes, TimeUnit.MINUTES);
+      timeForJobCatalogRemove =  catalog.getMetricContext().contextAwareTimer(TIME_FOR_JOB_CATALOG_REMOVE, this.timeWindowSizeInMinutes, TimeUnit.MINUTES);
+      this.contextAwareMetrics.add(timeForJobCatalogPut);
+      this.contextAwareMetrics.add(timeForJobCatalogRemove);
     }
 
     public void updatePutJobTime(long startTime) {
@@ -70,14 +76,5 @@ public interface MutableJobCatalog extends JobCatalog {
       log.info("updateRemoveJobTime...");
       Instrumented.updateTimer(Optional.of(this.timeForJobCatalogRemove), System.currentTimeMillis() - startTime, TimeUnit.MILLISECONDS);
     }
-
-    @Override
-    public Collection<ContextAwareTimer> getTimers() {
-      Collection<ContextAwareTimer> all = new ArrayList<>();
-      all.addAll(super.getTimers());
-      all.add(this.timeForJobCatalogPut);
-      all.add(this.timeForJobCatalogRemove);
-      return all;
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MutableSpecCatalog.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MutableSpecCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MutableSpecCatalog.java
index 3aa16be..108a324 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MutableSpecCatalog.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/MutableSpecCatalog.java
@@ -22,10 +22,14 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.metrics.ContextAwareMetric;
 import org.apache.gobblin.metrics.ContextAwareTimer;
+import org.apache.gobblin.util.ConfigUtils;
 
 import com.google.common.base.Optional;
+import com.typesafe.config.Config;
 
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
@@ -55,10 +59,12 @@ public interface MutableSpecCatalog extends SpecCatalog {
     public static final String TIME_FOR_SPEC_CATALOG_PUT = "timeForSpecCatalogPut";
     @Getter private final ContextAwareTimer timeForSpecCatalogPut;
     @Getter private final ContextAwareTimer timeForSpecCatalogRemove;
-    public MutableStandardMetrics(SpecCatalog catalog) {
-      super(catalog);
-      timeForSpecCatalogPut = catalog.getMetricContext().contextAwareTimer(TIME_FOR_SPEC_CATALOG_PUT, 1, TimeUnit.MINUTES);
-      timeForSpecCatalogRemove =  catalog.getMetricContext().contextAwareTimer(TIME_FOR_SPEC_CATALOG_REMOVE, 1, TimeUnit.MINUTES);
+    public MutableStandardMetrics(SpecCatalog catalog, Optional<Config> sysConfig) {
+      super(catalog, sysConfig);
+      timeForSpecCatalogPut = catalog.getMetricContext().contextAwareTimer(TIME_FOR_SPEC_CATALOG_PUT, this.timeWindowSizeInMinutes, TimeUnit.MINUTES);
+      timeForSpecCatalogRemove =  catalog.getMetricContext().contextAwareTimer(TIME_FOR_SPEC_CATALOG_REMOVE, this.timeWindowSizeInMinutes, TimeUnit.MINUTES);
+      this.contextAwareMetrics.add(timeForSpecCatalogPut);
+      this.contextAwareMetrics.add(timeForSpecCatalogRemove);
     }
 
     public void updatePutSpecTime(long startTime) {
@@ -70,14 +76,5 @@ public interface MutableSpecCatalog extends SpecCatalog {
       log.info("updateRemoveSpecTime...");
       Instrumented.updateTimer(Optional.of(this.timeForSpecCatalogRemove), System.currentTimeMillis() - startTime, TimeUnit.MILLISECONDS);
     }
-
-    @Override
-    public Collection<ContextAwareTimer> getTimers() {
-      Collection<ContextAwareTimer> all = new ArrayList<>();
-      all.addAll(super.getTimers());
-      all.add(this.timeForSpecCatalogPut);
-      all.add(this.timeForSpecCatalogRemove);
-      return all;
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SpecCatalog.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SpecCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SpecCatalog.java
index 6e8510f..457be9a 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SpecCatalog.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/SpecCatalog.java
@@ -19,21 +19,25 @@ package org.apache.gobblin.runtime.api;
 
 import java.net.URI;
 import java.util.Collection;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.typesafe.config.Config;
 
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.instrumented.GobblinMetricsKeys;
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.instrumented.StandardMetricsBridge;
-import org.apache.gobblin.metrics.ContextAwareCounter;
 import org.apache.gobblin.metrics.ContextAwareGauge;
+import org.apache.gobblin.metrics.ContextAwareMetric;
 import org.apache.gobblin.metrics.ContextAwareTimer;
 import org.apache.gobblin.metrics.GobblinTrackingEvent;
 import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.util.ConfigUtils;
 
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
@@ -70,6 +74,8 @@ public interface SpecCatalog extends SpecCatalogListenersContainer, StandardMetr
     public static final String TIME_FOR_SPEC_CATALOG_GET = "timeForSpecCatalogGet";
 
     private final MetricContext metricsContext;
+    protected final int timeWindowSizeInMinutes;
+
     @Getter private final AtomicLong totalAddedSpecs;
     @Getter private final AtomicLong totalDeletedSpecs;
     @Getter private final AtomicLong totalUpdatedSpecs;
@@ -80,9 +86,13 @@ public interface SpecCatalog extends SpecCatalogListenersContainer, StandardMetr
 
     @Getter private final ContextAwareTimer timeForSpecCatalogGet;
 
-    public StandardMetrics(final SpecCatalog specCatalog) {
+    public StandardMetrics(final SpecCatalog specCatalog, Optional<Config> sysConfig) {
+      this.timeWindowSizeInMinutes = sysConfig.isPresent()?
+          ConfigUtils.getInt(sysConfig.get(), ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES) :
+          ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES;
+
       this.metricsContext = specCatalog.getMetricContext();
-      this.timeForSpecCatalogGet = metricsContext.contextAwareTimer(TIME_FOR_SPEC_CATALOG_GET, 1, TimeUnit.MINUTES);
+      this.timeForSpecCatalogGet = metricsContext.contextAwareTimer(TIME_FOR_SPEC_CATALOG_GET, timeWindowSizeInMinutes, TimeUnit.MINUTES);
       this.totalAddedSpecs = new AtomicLong(0);
       this.totalDeletedSpecs = new AtomicLong(0);
       this.totalUpdatedSpecs = new AtomicLong(0);
@@ -95,6 +105,12 @@ public interface SpecCatalog extends SpecCatalogListenersContainer, StandardMetr
       this.totalAddCalls = metricsContext.newContextAwareGauge(TOTAL_ADD_CALLS, ()->this.totalAddedSpecs.get());
       this.totalUpdateCalls = metricsContext.newContextAwareGauge(TOTAL_UPDATE_CALLS, ()->this.totalUpdatedSpecs.get());
       this.totalDeleteCalls = metricsContext.newContextAwareGauge(TOTAL_DELETE_CALLS, ()->this.totalDeletedSpecs.get());
+
+      this.contextAwareMetrics.add(numActiveSpecs);
+      this.contextAwareMetrics.add(totalAddCalls);
+      this.contextAwareMetrics.add(totalUpdateCalls);
+      this.contextAwareMetrics.add(totalDeleteCalls);
+      this.contextAwareMetrics.add(timeForSpecCatalogGet);
     }
 
     public void updateGetSpecTime(long startTime) {
@@ -102,21 +118,6 @@ public interface SpecCatalog extends SpecCatalogListenersContainer, StandardMetr
       Instrumented.updateTimer(Optional.of(this.timeForSpecCatalogGet), System.currentTimeMillis() - startTime, TimeUnit.MILLISECONDS);
     }
 
-    @Override
-    public Collection<ContextAwareGauge<?>> getGauges() {
-      return ImmutableList.of(numActiveSpecs, totalAddCalls, totalUpdateCalls, totalDeleteCalls);
-    }
-
-    @Override
-    public Collection<ContextAwareCounter> getCounters() {
-      return ImmutableList.of();
-    }
-
-    @Override
-    public Collection<ContextAwareTimer> getTimers() {
-      return ImmutableList.of(this.timeForSpecCatalogGet);
-    }
-
     @Override public void onAddSpec(Spec addedSpec) {
       this.totalAddedSpecs.incrementAndGet();
       submitTrackingEvent(addedSpec, SPEC_ADDED_OPERATION_TYPE);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/FSJobCatalog.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/FSJobCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/FSJobCatalog.java
index 0f99235..f48b42c 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/FSJobCatalog.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/FSJobCatalog.java
@@ -87,9 +87,9 @@ public class FSJobCatalog extends ImmutableFSJobCatalog implements MutableJobCat
   }
 
   @Override
-  protected JobCatalog.StandardMetrics createStandardMetrics() {
+  protected JobCatalog.StandardMetrics createStandardMetrics(Optional<Config> sysConfig) {
     log.info("create standard metrics {} for {}", MutableStandardMetrics.class.getName(), this.getClass().getName());
-    return new MutableStandardMetrics(this);
+    return new MutableStandardMetrics(this, sysConfig);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/ImmutableFSJobCatalog.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/ImmutableFSJobCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/ImmutableFSJobCatalog.java
index 7162c81..626cac2 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/ImmutableFSJobCatalog.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/ImmutableFSJobCatalog.java
@@ -102,7 +102,7 @@ public class ImmutableFSJobCatalog extends JobCatalogBase implements JobCatalog
   public ImmutableFSJobCatalog(Config sysConfig, PathAlterationObserver observer, Optional<MetricContext> parentMetricContext,
       boolean instrumentationEnabled)
       throws IOException {
-    super(Optional.of(LOGGER), parentMetricContext, instrumentationEnabled);
+    super(Optional.of(LOGGER), parentMetricContext, instrumentationEnabled, Optional.of(sysConfig));
     this.sysConfig = sysConfig;
     ConfigAccessor cfgAccessor = new ConfigAccessor(this.sysConfig);
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/JobCatalogBase.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/JobCatalogBase.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/JobCatalogBase.java
index d40c962..dcb0723 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/JobCatalogBase.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_catalog/JobCatalogBase.java
@@ -28,6 +28,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.AbstractIdleService;
+import com.typesafe.config.Config;
 
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.metrics.MetricContext;
@@ -63,13 +64,18 @@ public abstract class JobCatalogBase extends AbstractIdleService implements JobC
 
   public JobCatalogBase(Optional<Logger> log, Optional<MetricContext> parentMetricContext,
       boolean instrumentationEnabled) {
+    this(log, parentMetricContext, instrumentationEnabled, Optional.absent());
+  }
+
+  public JobCatalogBase(Optional<Logger> log, Optional<MetricContext> parentMetricContext,
+      boolean instrumentationEnabled, Optional<Config> sysConfig) {
     this.log = log.isPresent() ? log.get() : LoggerFactory.getLogger(getClass());
     this.listeners = new JobCatalogListenersList(log);
     if (instrumentationEnabled) {
       MetricContext realParentCtx =
           parentMetricContext.or(Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(), getClass()));
       this.metricContext = realParentCtx.childBuilder(JobCatalog.class.getSimpleName()).build();
-      this.metrics = createStandardMetrics();
+      this.metrics = createStandardMetrics(sysConfig);
       this.addListener(this.metrics);
     }
     else {
@@ -78,8 +84,8 @@ public abstract class JobCatalogBase extends AbstractIdleService implements JobC
     }
   }
 
-  protected StandardMetrics createStandardMetrics() {
-    return new StandardMetrics(this);
+  protected StandardMetrics createStandardMetrics(Optional<Config> sysConfig) {
+    return new StandardMetrics(this, sysConfig);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java
index 482825f..a91baed 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/FlowCatalog.java
@@ -87,7 +87,7 @@ public class FlowCatalog extends AbstractIdleService implements SpecCatalog, Mut
       MetricContext realParentCtx =
           parentMetricContext.or(Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(), getClass()));
       this.metricContext = realParentCtx.childBuilder(FlowCatalog.class.getSimpleName()).build();
-      this.metrics = new MutableStandardMetrics(this);
+      this.metrics = new MutableStandardMetrics(this, Optional.of(config));
       this.addListener(this.metrics);
     }
     else {

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/TopologyCatalog.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/TopologyCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/TopologyCatalog.java
index c334d2b..5c25a67 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/TopologyCatalog.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/spec_catalog/TopologyCatalog.java
@@ -94,7 +94,7 @@ public class TopologyCatalog extends AbstractIdleService implements SpecCatalog,
       MetricContext realParentCtx =
           parentMetricContext.or(Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(), getClass()));
       this.metricContext = realParentCtx.childBuilder(TopologyCatalog.class.getSimpleName()).build();
-      this.metrics = new SpecCatalog.StandardMetrics(this);
+      this.metrics = new SpecCatalog.StandardMetrics(this, Optional.of(config));
       this.addListener(this.metrics);
     }
     else {

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5e6bfb07/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java
index 2cbb113..3137c21 100644
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java
@@ -21,6 +21,7 @@ import org.apache.gobblin.configuration.State;
 import org.apache.gobblin.instrumented.Instrumented;
 import org.apache.gobblin.instrumented.StandardMetricsBridge;
 import org.apache.gobblin.metrics.ContextAwareHistogram;
+import org.apache.gobblin.metrics.ContextAwareMetric;
 import org.apache.gobblin.metrics.MetricContext;
 import org.apache.gobblin.metrics.Tag;
 import org.apache.gobblin.service.FlowId;
@@ -155,7 +156,7 @@ public class GobblinServiceManager implements ApplicationLauncher, StandardMetri
     }
     this.config = config;
     this.metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(config), this.getClass());
-    this.metrics = new Metrics(this.metricContext);
+    this.metrics = new Metrics(this.metricContext, this.config);
     this.serviceId = serviceId;
     this.serviceLauncher = new ServiceBasedAppLauncher(properties, serviceName);
 
@@ -463,31 +464,14 @@ public class GobblinServiceManager implements ApplicationLauncher, StandardMetri
     return false;
   }
 
-  @Override
-  public List<Tag<?>> generateTags(State state) {
-    return null;
-  }
-
-  @Override
-  public void switchMetricContext(List<Tag<?>> tags) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void switchMetricContext(MetricContext context) {
-    throw new UnsupportedOperationException();
-  }
-
   private class Metrics extends StandardMetrics {
     public static final String SERVICE_LEADERSHIP_CHANGE = "serviceLeadershipChange";
     private ContextAwareHistogram serviceLeadershipChange;
-    public Metrics(final MetricContext metricContext) {
-      serviceLeadershipChange = metricContext.contextAwareHistogram(SERVICE_LEADERSHIP_CHANGE, 1, TimeUnit.MINUTES);
-    }
 
-    @Override
-    public Collection<ContextAwareHistogram> getHistograms() {
-      return ImmutableList.of(this.serviceLeadershipChange);
+    public Metrics(final MetricContext metricContext, Config config) {
+      int timeWindowSizeInMinutes = ConfigUtils.getInt(config, ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES, ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES);
+      this.serviceLeadershipChange = metricContext.contextAwareHistogram(SERVICE_LEADERSHIP_CHANGE, timeWindowSizeInMinutes, TimeUnit.MINUTES);
+      this.contextAwareMetrics.add(this.serviceLeadershipChange);
     }
   }
 


[49/50] incubator-gobblin git commit: [GOBBLIN-429] Pass jvm options as a list to child process

Posted by ab...@apache.org.
[GOBBLIN-429] Pass jvm options as a list to child process

Closes #2306 from yukuai518/jvm


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/0795fa7a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/0795fa7a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/0795fa7a

Branch: refs/heads/0.12.0
Commit: 0795fa7a07d17957288a3c9cde75dde91e7649be
Parents: 7e1d843
Author: Kuai Yu <ku...@linkedin.com>
Authored: Fri Mar 16 13:21:13 2018 -0700
Committer: Hung Tran <hu...@linkedin.com>
Committed: Fri Mar 16 13:21:13 2018 -0700

----------------------------------------------------------------------
 .../cluster/GobblinClusterConfigurationKeys.java        |  4 ++--
 .../org/apache/gobblin/cluster/SingleTaskLauncher.java  | 12 ++++++++----
 gobblin-cluster/src/test/resources/BasicCluster.conf    |  2 ++
 3 files changed, 12 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/0795fa7a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
index 5719b23..76382e7 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
@@ -37,8 +37,8 @@ public class GobblinClusterConfigurationKeys {
       GOBBLIN_CLUSTER_PREFIX + "task.classpath";
   public static final String TASK_LOG_CONFIG =
       GOBBLIN_CLUSTER_PREFIX + "task.log.config";
-  public static final String TASK_JAVA_OPTIONS =
-      GOBBLIN_CLUSTER_PREFIX + "task.java.options";
+  public static final String TASK_JVM_OPTIONS =
+      GOBBLIN_CLUSTER_PREFIX + "task.jvm.options";
 
   // General Gobblin Cluster application configuration properties.
   public static final String APPLICATION_NAME_OPTION_NAME = "app_name";

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/0795fa7a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
index 078f122..8cbbc00 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
@@ -23,6 +23,7 @@ import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.lang3.text.StrTokenizer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,7 +61,7 @@ class SingleTaskLauncher {
 
     // The -cp parameter list can be very long.
     final String completeCmdLine = String.join(" ", command);
-    logger.debug("cmd line:\n{}", completeCmdLine);
+    logger.info("cmd line:\n{}", completeCmdLine);
 
     final Process taskProcess = this.processBuilder.start(command);
 
@@ -88,9 +89,12 @@ class SingleTaskLauncher {
     }
 
     private void addJavaOptions() {
-      if (sysConfig.hasPath(GobblinClusterConfigurationKeys.TASK_JAVA_OPTIONS)) {
-        final String javaOptions = sysConfig.getString(GobblinClusterConfigurationKeys.TASK_JAVA_OPTIONS);
-        this.cmd.add(javaOptions);
+      if (sysConfig.hasPath(GobblinClusterConfigurationKeys.TASK_JVM_OPTIONS)) {
+        final String taskJvmOptions = sysConfig.getString(GobblinClusterConfigurationKeys.TASK_JVM_OPTIONS);
+        StrTokenizer tokenizer = new StrTokenizer(taskJvmOptions, ' ', '"');
+        while(tokenizer.hasNext()) {
+          this.cmd.add(tokenizer.next());
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/0795fa7a/gobblin-cluster/src/test/resources/BasicCluster.conf
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/test/resources/BasicCluster.conf b/gobblin-cluster/src/test/resources/BasicCluster.conf
index a42b41d..92016f8 100644
--- a/gobblin-cluster/src/test/resources/BasicCluster.conf
+++ b/gobblin-cluster/src/test/resources/BasicCluster.conf
@@ -21,4 +21,6 @@ gobblin.cluster.workDir=/tmp/gobblinClusterBasicTest/
 gobblin.cluster.job.conf.path=${gobblin.cluster.workDir}/jobs
 gobblin.cluster.standaloneMode=true
 gobblin.cluster.job.executeInSchedulingThread=false
+gobblin.cluster.enableTaskInSeparateProcess=false
+gobblin.cluster.task.jvm.options="-Xms10m -Xmx1g -XX:MinHeapFreeRatio=10 -XX:MaxHeapFreeRatio=20"
 jobexecutor.threadpool.size=20


[37/50] incubator-gobblin git commit: [GOBBLIN-415] Check for the value of configuration key flow.runImmediately in Job config.

Posted by ab...@apache.org.
[GOBBLIN-415] Check for the value of configuration key flow.runImmediately in Job config.

Closes #2291 from sv2000/gobblin-415


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/52bf10de
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/52bf10de
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/52bf10de

Branch: refs/heads/0.12.0
Commit: 52bf10de714e14eddb7eb6ab2f54fb0cc3a1e341
Parents: b4597e9
Author: suvasude <su...@linkedin.biz>
Authored: Thu Feb 22 08:22:45 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Thu Feb 22 08:22:45 2018 -0800

----------------------------------------------------------------------
 .../service/modules/scheduler/GobblinServiceJobScheduler.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/52bf10de/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
index 9cb39fb..ae18fc2 100644
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/scheduler/GobblinServiceJobScheduler.java
@@ -61,6 +61,7 @@ import org.apache.gobblin.service.modules.utils.HelixUtils;
 import org.apache.gobblin.service.ServiceConfigKeys;
 import org.apache.gobblin.service.modules.orchestration.Orchestrator;
 import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.PropertiesUtils;
 
 
 /**
@@ -222,7 +223,7 @@ public class GobblinServiceJobScheduler extends JobScheduler implements SpecCata
         if (jobConfig.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) {
           _log.info("Scheduling flow spec: " + addedSpec);
           scheduleJob(jobConfig, null);
-          if (jobConfig.containsKey(ConfigurationKeys.FLOW_RUN_IMMEDIATELY)) {
+          if (PropertiesUtils.getPropAsBoolean(jobConfig, ConfigurationKeys.FLOW_RUN_IMMEDIATELY, "false")) {
             _log.info("RunImmediately requested, hence executing FlowSpec: " + addedSpec);
             this.jobExecutor.execute(new NonScheduledJobRunner(jobConfig, null));
           }


[29/50] incubator-gobblin git commit: [GOBBLIN-407] fix job output path for full snapshot

Posted by ab...@apache.org.
[GOBBLIN-407] fix job output path for full snapshot

Closes #2284 from arjun4084346/fixOutputDirPath


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/0791ee99
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/0791ee99
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/0791ee99

Branch: refs/heads/0.12.0
Commit: 0791ee992e533957654ee7dedb33358b312acf26
Parents: f895057
Author: Arjun <ab...@linkedin.com>
Authored: Mon Feb 12 18:20:33 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Mon Feb 12 18:20:33 2018 -0800

----------------------------------------------------------------------
 .../gobblin/configuration/WorkUnitState.java    |  4 ++++
 .../apache/gobblin/source/workunit/Extract.java |  2 ++
 .../gobblin/source/workunit/WorkUnit.java       | 21 +++++++++++++++++++-
 .../org/apache/gobblin/util/WriterUtils.java    |  5 ++---
 4 files changed, 28 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/0791ee99/gobblin-api/src/main/java/org/apache/gobblin/configuration/WorkUnitState.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/configuration/WorkUnitState.java b/gobblin-api/src/main/java/org/apache/gobblin/configuration/WorkUnitState.java
index 0b40399..45bf807 100644
--- a/gobblin-api/src/main/java/org/apache/gobblin/configuration/WorkUnitState.java
+++ b/gobblin-api/src/main/java/org/apache/gobblin/configuration/WorkUnitState.java
@@ -62,6 +62,10 @@ public class WorkUnitState extends State {
 
   private static final Gson GSON = new Gson();
 
+  public String getOutputFilePath() {
+    return this.workUnit.getOutputFilePath();
+  }
+
   /**
    * Runtime state of the {@link WorkUnit}.
    *

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/0791ee99/gobblin-api/src/main/java/org/apache/gobblin/source/workunit/Extract.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/source/workunit/Extract.java b/gobblin-api/src/main/java/org/apache/gobblin/source/workunit/Extract.java
index 5161427..7a3735b 100644
--- a/gobblin-api/src/main/java/org/apache/gobblin/source/workunit/Extract.java
+++ b/gobblin-api/src/main/java/org/apache/gobblin/source/workunit/Extract.java
@@ -137,7 +137,9 @@ public class Extract extends State {
    * Get the writer output file path corresponding to this {@link Extract}.
    *
    * @return writer output file path corresponding to this {@link Extract}
+   * @deprecated As {@code this.getIsFull} is deprecated.
    */
+  @Deprecated
   public String getOutputFilePath() {
     return this.getNamespace().replaceAll("\\.", "/") + "/" + this.getTable() + "/" + this.getExtractId() + "_"
         + (this.getIsFull() ? "full" : "append");

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/0791ee99/gobblin-api/src/main/java/org/apache/gobblin/source/workunit/WorkUnit.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/source/workunit/WorkUnit.java b/gobblin-api/src/main/java/org/apache/gobblin/source/workunit/WorkUnit.java
index 38aabcb..7d3f5d3 100644
--- a/gobblin-api/src/main/java/org/apache/gobblin/source/workunit/WorkUnit.java
+++ b/gobblin-api/src/main/java/org/apache/gobblin/source/workunit/WorkUnit.java
@@ -311,9 +311,14 @@ public class WorkUnit extends State {
 
   @Override
   public String getProp(String key) {
+    return getProp(key, null);
+  }
+
+  @Override
+  public String getProp(String key, String def) {
     String value = super.getProp(key);
     if (value == null) {
-      value = this.extract.getProp(key);
+      value = this.extract.getProp(key, def);
     }
     return value;
   }
@@ -359,4 +364,18 @@ public class WorkUnit extends State {
     result = prime * result + ((this.extract == null) ? 0 : this.extract.hashCode());
     return result;
   }
+
+  public String getOutputFilePath() {
+    // Search for the properties in the workunit.
+    // This search for the property first in State and then in the Extract of this workunit.
+    String namespace = getProp(ConfigurationKeys.EXTRACT_NAMESPACE_NAME_KEY, "");
+    String table = getProp(ConfigurationKeys.EXTRACT_TABLE_NAME_KEY, "");
+    String extractId = getProp(ConfigurationKeys.EXTRACT_EXTRACT_ID_KEY, "");
+    // getPropAsBoolean and other similar methods are not overridden in WorkUnit class
+    // Thus, to enable searching in WorkUnit's Extract, we use getProp, and not getPropAsBoolean
+    boolean isFull =  Boolean.parseBoolean(getProp(ConfigurationKeys.EXTRACT_IS_FULL_KEY));
+
+    return namespace.replaceAll("\\.", "/") + "/" + table + "/" + extractId + "_"
+        + (isFull ? "full" : "append");
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/0791ee99/gobblin-utility/src/main/java/org/apache/gobblin/util/WriterUtils.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/util/WriterUtils.java b/gobblin-utility/src/main/java/org/apache/gobblin/util/WriterUtils.java
index f5658ea..9a628d0 100644
--- a/gobblin-utility/src/main/java/org/apache/gobblin/util/WriterUtils.java
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/util/WriterUtils.java
@@ -209,12 +209,11 @@ public class WriterUtils {
   public static Path getDefaultWriterFilePath(State state, int numBranches, int branchId) {
     if (state instanceof WorkUnitState) {
       WorkUnitState workUnitState = (WorkUnitState) state;
-      return new Path(ForkOperatorUtils.getPathForBranch(workUnitState, workUnitState.getExtract().getOutputFilePath(),
+      return new Path(ForkOperatorUtils.getPathForBranch(workUnitState, workUnitState.getOutputFilePath(),
           numBranches, branchId));
-
     } else if (state instanceof WorkUnit) {
       WorkUnit workUnit = (WorkUnit) state;
-      return new Path(ForkOperatorUtils.getPathForBranch(workUnit, workUnit.getExtract().getOutputFilePath(),
+      return new Path(ForkOperatorUtils.getPathForBranch(workUnit, workUnit.getOutputFilePath(),
           numBranches, branchId));
     }
 


[28/50] incubator-gobblin git commit: [GOBBLIN-406] delete job state store on spec delete request

Posted by ab...@apache.org.
[GOBBLIN-406] delete job state store on spec delete request

Closes #2281 from arjun4084346/deleteInCluster


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/f8950570
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/f8950570
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/f8950570

Branch: refs/heads/0.12.0
Commit: f89505702a3a130a7e73c617b33a285ea228bd8c
Parents: 5c678d9
Author: Arjun <ab...@linkedin.com>
Authored: Fri Feb 9 16:04:10 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Fri Feb 9 16:04:10 2018 -0800

----------------------------------------------------------------------
 .../runtime/job_monitor/KafkaJobMonitor.java    | 25 ++++++++++++++++-
 .../job_monitor/KafkaJobMonitorTest.java        | 24 +++++++++++++++--
 .../runtime/kafka/HighLevelConsumerTest.java    |  7 +++++
 .../modules/flow/BaseFlowToJobSpecCompiler.java | 28 +++++++++++++++++++-
 .../flow/MultiHopsFlowToJobSpecCompiler.java    |  8 ++++--
 5 files changed, 86 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f8950570/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
index ba79305..0bb4f14 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
@@ -26,6 +26,8 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
 import com.typesafe.config.Config;
 
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.DatasetStateStore;
 import org.apache.gobblin.runtime.api.JobSpec;
 import org.apache.gobblin.runtime.api.JobSpecMonitor;
 import org.apache.gobblin.runtime.api.MutableJobCatalog;
@@ -50,8 +52,9 @@ public abstract class KafkaJobMonitor extends HighLevelConsumer<byte[], byte[]>
   public static final String KAFKA_AUTO_OFFSET_RESET_KEY = KAFKA_JOB_MONITOR_PREFIX + ".auto.offset.reset";
   public static final String KAFKA_AUTO_OFFSET_RESET_SMALLEST = "smallest";
   public static final String KAFKA_AUTO_OFFSET_RESET_LARGEST = "largest";
-
+  private DatasetStateStore datasetStateStore;
   private final MutableJobCatalog jobCatalog;
+
   @Getter
   private Counter newSpecs;
   @Getter
@@ -67,6 +70,15 @@ public abstract class KafkaJobMonitor extends HighLevelConsumer<byte[], byte[]>
   public KafkaJobMonitor(String topic, MutableJobCatalog catalog, Config config) {
     super(topic, ConfigUtils.getConfigOrEmpty(config, KAFKA_JOB_MONITOR_PREFIX), 1);
     this.jobCatalog = catalog;
+    try {
+      if (config.hasPath(ConfigurationKeys.STATE_STORE_ENABLED) &&
+          config.getBoolean(ConfigurationKeys.STATE_STORE_ENABLED) &&
+          config.hasPath(ConfigurationKeys.STATE_STORE_ROOT_DIR_KEY)) {
+        this.datasetStateStore = DatasetStateStore.buildDatasetStateStore(config);
+      }
+    } catch (IOException e) {
+      log.warn("DatasetStateStore could not be created.");
+    }
   }
 
   @Override
@@ -100,6 +112,17 @@ public abstract class KafkaJobMonitor extends HighLevelConsumer<byte[], byte[]>
         } else if (parsedMessage instanceof Either.Right) {
           this.remmovedSpecs.inc();
           this.jobCatalog.remove(((Either.Right<JobSpec, URI>) parsedMessage).getRight());
+
+          // Refer FlowConfigsResources:delete to understand the pattern of flow URI
+          // FlowToJobSpec Compilers use the flowSpecURI to derive jobSpecURI
+          String[] uriTokens = ((URI)(((Either.Right) parsedMessage).getRight())).getPath().split("/");
+          if (uriTokens.length == 3) {
+            String jobName = uriTokens[2];
+            // Delete the job state if it is a delete spec request
+            if (this.datasetStateStore != null) {
+              this.datasetStateStore.delete(jobName);
+            }
+          }
         }
       }
     } catch (IOException ioe) {

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f8950570/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitorTest.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitorTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitorTest.java
index 1dd90b1..57f99a9 100644
--- a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitorTest.java
+++ b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitorTest.java
@@ -19,11 +19,16 @@ package org.apache.gobblin.runtime.job_monitor;
 
 import java.net.URI;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 import com.google.common.base.Optional;
+import com.typesafe.config.Config;
 
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.runtime.kafka.HighLevelConsumerTest;
 
 
@@ -32,8 +37,11 @@ public class KafkaJobMonitorTest {
   @Test
   public void test() throws Exception {
 
-    MockedKafkaJobMonitor monitor =
-        MockedKafkaJobMonitor.create(HighLevelConsumerTest.getSimpleConfig(Optional.of(KafkaJobMonitor.KAFKA_JOB_MONITOR_PREFIX)));
+    Config config = HighLevelConsumerTest.getSimpleConfig(Optional.of(KafkaJobMonitor.KAFKA_JOB_MONITOR_PREFIX));
+    String stateStoreRootDir = config.getString(ConfigurationKeys.STATE_STORE_ROOT_DIR_KEY);
+    FileSystem fs = FileSystem.getLocal(new Configuration());
+
+    MockedKafkaJobMonitor monitor = MockedKafkaJobMonitor.create(config);
     monitor.startAsync();
 
     monitor.getMockKafkaStream().pushToStream("job1:1");
@@ -58,6 +66,18 @@ public class KafkaJobMonitorTest {
     Assert.assertTrue(monitor.getJobSpecs().containsKey(new URI("job2")));
     Assert.assertEquals(monitor.getJobSpecs().get(new URI("job2")).getVersion(), "2");
 
+    monitor.getMockKafkaStream().pushToStream("/flow3/job3:1");
+    monitor.awaitExactlyNSpecs(3);
+    Assert.assertTrue(monitor.getJobSpecs().containsKey(new URI("/flow3/job3")));
+
+    // TODO: Currently, state stores are not categorized by flow name.
+    //       This can lead to one job overwriting other jobs' job state.
+    fs.create(new Path(stateStoreRootDir, "job3"));
+    Assert.assertTrue(fs.exists(new Path(stateStoreRootDir, "job3")));
+    monitor.getMockKafkaStream().pushToStream(MockedKafkaJobMonitor.REMOVE + ":/flow3/job3");
+    monitor.awaitExactlyNSpecs(2);
+    Assert.assertFalse(fs.exists(new Path(stateStoreRootDir, "job3")));
+
     monitor.shutDown();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f8950570/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/kafka/HighLevelConsumerTest.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/kafka/HighLevelConsumerTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/kafka/HighLevelConsumerTest.java
index 6e8d7a2..e8d4e6c 100644
--- a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/kafka/HighLevelConsumerTest.java
+++ b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/kafka/HighLevelConsumerTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.gobblin.runtime.kafka;
 
+import java.io.File;
 import java.util.Properties;
 import java.util.concurrent.TimeoutException;
 
@@ -24,9 +25,11 @@ import org.testng.Assert;
 import org.testng.annotations.Test;
 
 import com.google.common.base.Optional;
+import com.google.common.io.Files;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
 
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.runtime.job_monitor.MockKafkaStream;
 
 
@@ -35,6 +38,10 @@ public class HighLevelConsumerTest {
   public static Config getSimpleConfig(Optional<String> prefix) {
     Properties properties = new Properties();
     properties.put(getConfigKey(prefix, "zookeeper.connect"), "zookeeper");
+    properties.put(ConfigurationKeys.STATE_STORE_ENABLED, "true");
+    File tmpDir = Files.createTempDir();
+    tmpDir.deleteOnExit();
+    properties.put(ConfigurationKeys.STATE_STORE_ROOT_DIR_KEY, tmpDir.toString());
 
     return ConfigFactory.parseProperties(properties);
   }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f8950570/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/BaseFlowToJobSpecCompiler.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/BaseFlowToJobSpecCompiler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/BaseFlowToJobSpecCompiler.java
index db92ef9..855d692 100644
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/BaseFlowToJobSpecCompiler.java
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/BaseFlowToJobSpecCompiler.java
@@ -211,7 +211,7 @@ public abstract class BaseFlowToJobSpecCompiler implements SpecCompiler {
    */
   protected JobSpec jobSpecGenerator(FlowSpec flowSpec) {
     JobSpec jobSpec;
-    JobSpec.Builder jobSpecBuilder = JobSpec.builder(flowSpec.getUri())
+    JobSpec.Builder jobSpecBuilder = JobSpec.builder(jobSpecURIGenerator(flowSpec))
         .withConfig(flowSpec.getConfig())
         .withDescription(flowSpec.getDescription())
         .withVersion(flowSpec.getVersion());
@@ -254,6 +254,32 @@ public abstract class BaseFlowToJobSpecCompiler implements SpecCompiler {
   }
 
   /**
+   * It can receive multiple number of parameters, needed to generate a unique URI.
+   * Implementation is flowSpecCompiler dependent.
+   * This method should return URI which has job name at third place, when split by "/"
+   * e.g. /flowGroup/flowName
+   *      /flowGroup/flowName/sourceNode-targetNode
+   * SafeDatasetCommit creates state store using this name and
+   * {@link org.apache.gobblin.runtime.job_monitor.KafkaJobMonitor} extract job name to find the state store path.
+   * @param objects
+   * @return
+   */
+  public  URI jobSpecURIGenerator(Object... objects) {
+    return ((FlowSpec)objects[0]).getUri();
+  }
+
+  /**
+   * It returns the template uri for job.
+   * This method can be overridden by derived FlowToJobSpecCompiler classes.
+   * @param flowSpec
+   * @return template URI
+   */
+  protected URI jobSpecTemplateURIGenerator(FlowSpec flowSpec) {
+    // For now only first template uri will be honored for Identity
+    return flowSpec.getTemplateURIs().get().iterator().next();
+  }
+
+  /**
    * Ideally each edge has its own eligible template repository(Based on {@link SpecExecutor})
    * to pick templates from.
    *

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f8950570/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/MultiHopsFlowToJobSpecCompiler.java
----------------------------------------------------------------------
diff --git a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/MultiHopsFlowToJobSpecCompiler.java b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/MultiHopsFlowToJobSpecCompiler.java
index ba5c203..544ca42 100644
--- a/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/MultiHopsFlowToJobSpecCompiler.java
+++ b/gobblin-service/src/main/java/org/apache/gobblin/service/modules/flow/MultiHopsFlowToJobSpecCompiler.java
@@ -323,14 +323,18 @@ public class MultiHopsFlowToJobSpecCompiler extends BaseFlowToJobSpecCompiler {
   private URI getTemplateURI (ServiceNode sourceNode, ServiceNode targetNode, FlowSpec flowSpec, FlowEdge flowEdge) {
     URI firstTemplateURI =
         (edgeTemplateMap != null && edgeTemplateMap.containsKey(flowEdge.getEdgeIdentity())) ? edgeTemplateMap.get(
-            flowEdge.getEdgeIdentity()).get(0) : jobSpecGenerator(flowSpec).getTemplateURI().orNull();
+            flowEdge.getEdgeIdentity()).get(0) : jobSpecTemplateURIGenerator(flowSpec);
     return firstTemplateURI;
   }
 
   /**
    * A naive implementation of generating a jobSpec's URI within a multi-hop logical Flow.
    */
-  public static URI jobSpecURIGenerator(FlowSpec flowSpec, ServiceNode sourceNode, ServiceNode targetNode) {
+  @Override
+  public URI jobSpecURIGenerator(Object... objects) {
+    FlowSpec flowSpec = (FlowSpec) objects[0];
+    ServiceNode sourceNode = (ServiceNode) objects[1];
+    ServiceNode targetNode = (ServiceNode) objects[2];
     try {
       return new URI(JobSpec.Builder.DEFAULT_JOB_CATALOG_SCHEME, flowSpec.getUri().getAuthority(),
           StringUtils.appendIfMissing(StringUtils.prependIfMissing(flowSpec.getUri().getPath(), "/"),"/")


[22/50] incubator-gobblin git commit: [GOBBLIN-351] Add ParquetHdfsDataWriter docs

Posted by ab...@apache.org.
[GOBBLIN-351] Add ParquetHdfsDataWriter docs

[GOBBLIN-351] Add ParquetHdfsDataWriter docs

[GOBBLIN-351] Add more info about builder and
dictionary encoding

Closes #2220 from tilakpatidar/parquet_hdfs_docs


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/3598d10e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/3598d10e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/3598d10e

Branch: refs/heads/0.12.0
Commit: 3598d10eb0ea0d01244a93ff1506a563afeca9ed
Parents: 3094fe5
Author: tilakpatidar <ti...@gmail.com>
Authored: Mon Feb 5 12:03:31 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Mon Feb 5 12:03:31 2018 -0800

----------------------------------------------------------------------
 gobblin-docs/sinks/ParquetHdfsDataWriter.md | 25 ++++++++++++++++++++++++
 mkdocs.yml                                  |  1 +
 2 files changed, 26 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/3598d10e/gobblin-docs/sinks/ParquetHdfsDataWriter.md
----------------------------------------------------------------------
diff --git a/gobblin-docs/sinks/ParquetHdfsDataWriter.md b/gobblin-docs/sinks/ParquetHdfsDataWriter.md
new file mode 100644
index 0000000..f3ad0da
--- /dev/null
+++ b/gobblin-docs/sinks/ParquetHdfsDataWriter.md
@@ -0,0 +1,25 @@
+# Description
+
+An extension to [`FsDataWriter`](https://github.com/apache/incubator-gobblin/blob/master/gobblin-core/src/main/java/org/apache/gobblin/writer/FsDataWriter.java) that writes in Parquet format in the form of [`Group.java`](https://github.com/apache/parquet-mr/blob/master/parquet-column/src/main/java/org/apache/parquet/example/data/Group.java). This implementation allows users to specify the CodecFactory to use through the configuration property [`writer.codec.type`](https://gobblin.readthedocs.io/en/latest/user-guide/Configuration-Properties-Glossary/#writercodectype). By default, the deflate codec is used.
+
+# Usage
+```
+writer.builder.class=org.apache.gobblin.writer.ParquetDataWriterBuilder
+writer.destination.type=HDFS
+writer.output.format=PARQUET
+```
+For more info, see 
+[`ParquetHdfsDataWriter`](https://github.com/apache/incubator-gobblin/blob/master/gobblin-modules/gobblin-parquet/src/main/java/org/apache/gobblin/writer/ParquetHdfsDataWriter.java)
+and
+[`ParquetDataWriterBuilder`](https://github.com/apache/incubator-gobblin/blob/master/gobblin-modules/gobblin-parquet/src/main/java/org/apache/gobblin/writer/ParquetDataWriterBuilder.java)
+
+
+# Configuration
+
+| Key                    | Description | Default Value | Required |
+|------------------------|-------------|---------------|----------|
+| writer.parquet.page.size | The page size threshold. | 1048576 | No |
+| writer.parquet.dictionary.page.size | The block size threshold for the dictionary pages. | 134217728 | No |
+| writer.parquet.dictionary | To turn dictionary encoding on. Parquet has a dictionary encoding for data with a small number of unique values ( < 10^5 ) that aids in significant compression and boosts processing speed. | true | No |
+| writer.parquet.validate | To turn on validation using the schema. This validation is done by [`ParquetWriter`](https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java) not by Gobblin. | false | No |
+| writer.parquet.version | Version of parquet writer to use. Available versions are v1 and v2. | v1 | No |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/3598d10e/mkdocs.yml
----------------------------------------------------------------------
diff --git a/mkdocs.yml b/mkdocs.yml
index f3486b4..7152bd0 100644
--- a/mkdocs.yml
+++ b/mkdocs.yml
@@ -64,6 +64,7 @@ pages:
         - Wikipedia: sources/WikipediaSource.md
     - Record Sinks:
         - Avro HDFS: sinks/AvroHdfsDataWriter.md
+        - Parquet HDFS: sinks/ParquetHdfsDataWriter.md
         - HDFS Byte array: sinks/SimpleBytesWriter.md
         - Console: sinks/ConsoleWriter.md
         - Couchbase: sinks/Couchbase-Writer.md


[38/50] incubator-gobblin git commit: [GOBBLIN-302] Handle stuck Helix workflow

Posted by ab...@apache.org.
[GOBBLIN-302] Handle stuck Helix workflow

Closes #2157 from arjun4084346/stuckHelixJob


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/97e29f43
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/97e29f43
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/97e29f43

Branch: refs/heads/0.12.0
Commit: 97e29f436db8a5949c9d7b593c80097a8952ab68
Parents: 52bf10d
Author: Arjun <ab...@linkedin.com>
Authored: Thu Feb 22 09:40:51 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Thu Feb 22 09:40:51 2018 -0800

----------------------------------------------------------------------
 .../configuration/ConfigurationKeys.java        |  4 ++
 .../cluster/GobblinHelixJobLauncher.java        | 41 ++++++++++++++++++--
 .../runtime/GobblinMultiTaskAttempt.java        | 12 +++++-
 .../java/org/apache/gobblin/runtime/Task.java   | 30 +++++++++++++-
 4 files changed, 80 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/97e29f43/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
index d07d740..c80ceaf 100644
--- a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
+++ b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
@@ -108,6 +108,10 @@ public class ConfigurationKeys {
   public static final String SCHEDULER_WAIT_FOR_JOB_COMPLETION_KEY = "scheduler.wait.for.job.completion";
   public static final String DEFAULT_SCHEDULER_WAIT_FOR_JOB_COMPLETION = Boolean.TRUE.toString();
 
+  public static final String HELIX_JOB_TIMEOUT_ENABLED_KEY = "job.timeout.enabled";
+  public static final String DEFAULT_HELIX_JOB_TIMEOUT_ENABLED = "false";
+  public static final String HELIX_JOB_TIMEOUT_SECONDS = "job.timeout.seconds";
+  public static final String DEFAULT_HELIX_JOB_TIMEOUT_SECONDS = "10800";
 
   /**
    * Task executor and state tracker configuration properties.

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/97e29f43/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
index af15469..62c9b3f 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
@@ -107,6 +107,7 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
   private final TaskDriver helixTaskDriver;
   private final String helixQueueName;
   private final String jobResourceName;
+  private JobListener jobListener;
 
   private final FileSystem fs;
   private final Path appWorkDir;
@@ -167,6 +168,7 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
 
     this.taskStateCollectorService = new TaskStateCollectorService(jobProps, this.jobContext.getJobState(),
         this.eventBus, this.stateStores.getTaskStateStore(), outputTaskStateDir);
+    startCancellationExecutor();
   }
 
   @Override
@@ -290,6 +292,7 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
 
   public void launchJob(@Nullable JobListener jobListener)
       throws JobException {
+    this.jobListener = jobListener;
     boolean isLaunched = false;
     this.runningMap.putIfAbsent(this.jobContext.getJobName(), false);
     try {
@@ -359,8 +362,14 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
     return workUnitFile.toString();
   }
 
-  private void waitForJobCompletion() throws InterruptedException {
-    while (true) {
+  private void waitForJobCompletion()  throws InterruptedException {
+    LOGGER.info("Waiting for job to complete...");
+    boolean timeoutEnabled = Boolean.parseBoolean(this.jobProps.getProperty(ConfigurationKeys.HELIX_JOB_TIMEOUT_ENABLED_KEY,
+        ConfigurationKeys.DEFAULT_HELIX_JOB_TIMEOUT_ENABLED));
+    long timeoutInSeconds = Long.parseLong(this.jobProps.getProperty(ConfigurationKeys.HELIX_JOB_TIMEOUT_SECONDS,
+        ConfigurationKeys.DEFAULT_HELIX_JOB_TIMEOUT_SECONDS));
+    long endTime = System.currentTimeMillis() + timeoutInSeconds*1000;
+    while (!timeoutEnabled || System.currentTimeMillis() <= endTime) {
       WorkflowContext workflowContext = TaskDriver.getWorkflowContext(this.helixManager, this.helixQueueName);
       if (workflowContext != null) {
         org.apache.helix.task.TaskState helixJobState = workflowContext.getJobState(this.jobResourceName);
@@ -370,9 +379,35 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
           return;
         }
       }
-
       Thread.sleep(1000);
     }
+    helixTaskDriverWaitToStop(this.helixQueueName, 10L);
+    try {
+      cancelJob(this.jobListener);
+    } catch (JobException e) {
+      throw new RuntimeException("Unable to cancel job " + jobContext.getJobName() + ": ", e);
+    }
+    this.helixTaskDriver.resume(this.helixQueueName);
+    LOGGER.info("stopped the queue, deleted the job");
+  }
+
+  /**
+   * Because fix https://github.com/apache/helix/commit/ae8e8e2ef37f48d782fc12f85ca97728cf2b70c4
+   * is not available in currently used version 0.6.9
+   */
+  private void helixTaskDriverWaitToStop(String workflow, long timeoutInSeconds) throws InterruptedException {
+    this.helixTaskDriver.stop(workflow);
+    long endTime = System.currentTimeMillis() + timeoutInSeconds*1000;
+    while (System.currentTimeMillis() <= endTime) {
+      WorkflowContext workflowContext = TaskDriver.getWorkflowContext(this.helixManager, this.helixQueueName);
+      if (workflowContext == null || workflowContext.getWorkflowState()
+          .equals(org.apache.helix.task.TaskState.IN_PROGRESS)) {
+        Thread.sleep(1000);
+      } else {
+        LOGGER.info("Successfully stopped the queue");
+        return;
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/97e29f43/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/GobblinMultiTaskAttempt.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/GobblinMultiTaskAttempt.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/GobblinMultiTaskAttempt.java
index aa42121..e5643c0 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/GobblinMultiTaskAttempt.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/GobblinMultiTaskAttempt.java
@@ -202,9 +202,17 @@ public class GobblinMultiTaskAttempt {
       task.shutdown();
     }
 
-    for (Task task: this.tasks) {
+    for (Task task : this.tasks) {
       task.awaitShutdown(1000);
     }
+
+    for (Task task : this.tasks) {
+      if (task.cancel()) {
+        log.info("Task {} cancelled.", task.getTaskId());
+      } else {
+        log.info("Task {} could not be cancelled.", task.getTaskId());
+      }
+    }
   }
 
   private void persistTaskStateStore()
@@ -343,8 +351,8 @@ public class GobblinMultiTaskAttempt {
       // Create a new task from the work unit and submit the task to run
       Task task = createTaskRunnable(workUnitState, countDownLatch);
       this.taskStateTracker.registerNewTask(task);
+      task.setTaskFuture(this.taskExecutor.submit(task));
       tasks.add(task);
-      this.taskExecutor.execute(task);
     }
 
     new EventSubmitter.Builder(JobMetrics.get(this.jobId).getMetricContext(), "gobblin.runtime").build()

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/97e29f43/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/Task.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/Task.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/Task.java
index c3c1b99..52b0960 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/Task.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/Task.java
@@ -150,6 +150,7 @@ public class Task implements TaskIFace {
   private final AtomicBoolean shutdownRequested;
   private volatile long shutdownRequestedTime = Long.MAX_VALUE;
   private final CountDownLatch shutdownLatch;
+  private Future<?> taskFuture;
 
   /**
    * Instantiate a new {@link Task}.
@@ -364,8 +365,15 @@ public class Task implements TaskIFace {
     } catch (Throwable t) {
       failTask(t);
     } finally {
-      this.taskStateTracker.onTaskRunCompletion(this);
-      completeShutdown();
+      synchronized (this) {
+        if (this.taskFuture == null || !this.taskFuture.isCancelled()) {
+          this.taskStateTracker.onTaskRunCompletion(this);
+          completeShutdown();
+          this.taskFuture = null;
+        } else {
+          LOG.info("will not decrease count down latch as this task is cancelled");
+        }
+      }
     }
   }
 
@@ -952,4 +960,22 @@ public class Task implements TaskIFace {
     }
     return true;
   }
+
+  public synchronized void setTaskFuture(Future<?> taskFuture) {
+    this.taskFuture = taskFuture;
+  }
+
+  /**
+   * return true if the task is successfully cancelled.
+   * @return
+   */
+  public synchronized boolean cancel() {
+    if (this.taskFuture != null && this.taskFuture.cancel(true)) {
+      this.taskStateTracker.onTaskRunCompletion(this);
+      this.completeShutdown();
+      return true;
+    } else {
+      return false;
+    }
+  }
 }


[36/50] incubator-gobblin git commit: [GOBBLIN-378] Publish tasks in successful state only

Posted by ab...@apache.org.
[GOBBLIN-378] Publish tasks in successful state only

Closes #2253 from yukuai518/zero


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/b4597e98
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/b4597e98
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/b4597e98

Branch: refs/heads/0.12.0
Commit: b4597e988c9beae18f1b4896bf0b7f36d2ea5c1f
Parents: a7a85e1
Author: Kuai Yu <ku...@linkedin.com>
Authored: Wed Feb 21 17:04:54 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Wed Feb 21 17:04:54 2018 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/gobblin/runtime/Task.java    | 10 +++++++---
 1 file changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b4597e98/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/Task.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/Task.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/Task.java
index 3265ab8..c3c1b99 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/Task.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/Task.java
@@ -869,7 +869,9 @@ public class Task implements TaskIFace {
       if (failedForkIds.size() == 0) {
         // Set the task state to SUCCESSFUL. The state is not set to COMMITTED
         // as the data publisher will do that upon successful data publishing.
-        this.taskState.setWorkingState(WorkUnitState.WorkingState.SUCCESSFUL);
+        if (this.taskState.getWorkingState() != WorkUnitState.WorkingState.FAILED) {
+          this.taskState.setWorkingState(WorkUnitState.WorkingState.SUCCESSFUL);
+        }
       } else {
         failTask(new ForkException("Fork branches " + failedForkIds + " failed for task " + this.taskId));
       }
@@ -903,8 +905,10 @@ public class Task implements TaskIFace {
         if (shouldPublishDataInTask()) {
           // If data should be published by the task, publish the data and set the task state to COMMITTED.
           // Task data can only be published after all forks have been closed by closer.close().
-          publishTaskData();
-          this.taskState.setWorkingState(WorkUnitState.WorkingState.COMMITTED);
+          if (this.taskState.getWorkingState() == WorkUnitState.WorkingState.SUCCESSFUL) {
+            publishTaskData();
+            this.taskState.setWorkingState(WorkUnitState.WorkingState.COMMITTED);
+          }
         }
       } catch (IOException ioe) {
         failTask(ioe);


[03/50] incubator-gobblin git commit: [GOBBLIN-382] Support storing job.state file in mysql state store for standalone cluster

Posted by ab...@apache.org.
[GOBBLIN-382] Support storing job.state file in mysql state store for standalone cluster

Closes #2262 from htran1/cluster_job_state_store


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/f2f6e468
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/f2f6e468
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/f2f6e468

Branch: refs/heads/0.12.0
Commit: f2f6e468536bf3f0d79a3c126f620ac0741df65d
Parents: fd0c30a
Author: Hung Tran <hu...@linkedin.com>
Authored: Tue Jan 23 20:23:20 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Tue Jan 23 20:23:20 2018 -0800

----------------------------------------------------------------------
 .../configuration/ConfigurationKeys.java        |  3 ++
 .../GobblinClusterConfigurationKeys.java        |  3 ++
 .../gobblin/cluster/GobblinClusterUtils.java    | 34 +++++++++++++++-
 .../cluster/GobblinHelixJobLauncher.java        | 35 +++++++++++-----
 .../gobblin/cluster/GobblinHelixTask.java       |  7 +---
 .../cluster/GobblinHelixTaskFactory.java        |  5 ++-
 .../org/apache/gobblin/cluster/SingleTask.java  | 18 ++++++--
 .../gobblin/cluster/SingleTaskRunner.java       | 15 +++----
 .../cluster/GobblinHelixJobLauncherTest.java    |  6 +++
 .../gobblin/runtime/util/StateStores.java       | 43 +++++++++++++++++---
 .../util/test/TestStressTestingSource.java      | 11 +++--
 11 files changed, 138 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
index ed360d9..267a17e 100644
--- a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
+++ b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
@@ -54,6 +54,9 @@ public class ConfigurationKeys {
   public static final String INTERMEDIATE_STATE_STORE_TYPE_KEY = INTERMEDIATE_STATE_STORE_PREFIX + ".state.store.type";
   public static final String DEFAULT_STATE_STORE_TYPE = "fs";
   public static final String STATE_STORE_TYPE_NOOP = "noop";
+  // are the job.state files stored using the state store?
+  public static final String JOB_STATE_IN_STATE_STORE = "state.store.jobStateInStateStore";
+  public static final boolean DEFAULT_JOB_STATE_IN_STATE_STORE = false;
 
   public static final String CONFIG_RUNTIME_PREFIX = "gobblin.config.runtime.";
   // Root directory where task state files are stored

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
index 5e25194..4e78078 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
@@ -63,6 +63,9 @@ public class GobblinClusterConfigurationKeys {
   public static final String JOB_CONF_PATH_KEY = GOBBLIN_CLUSTER_PREFIX + "job.conf.path";
   public static final String INPUT_WORK_UNIT_DIR_NAME = "_workunits";
   public static final String OUTPUT_TASK_STATE_DIR_NAME = "_taskstates";
+  // This is the directory to store job.state files when a state store is used.
+  // Note that a .job.state file is not the same thing as a .jst file.
+  public static final String JOB_STATE_DIR_NAME = "_jobstates";
   public static final String TAR_GZ_FILE_SUFFIX = ".tar.gz";
 
   // Other misc configuration properties.

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterUtils.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterUtils.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterUtils.java
index 3f53443..6b6ead8 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterUtils.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterUtils.java
@@ -19,14 +19,20 @@ package org.apache.gobblin.cluster;
 
 import static org.apache.gobblin.cluster.GobblinClusterConfigurationKeys.CLUSTER_WORK_DIR;
 
-import com.typesafe.config.Config;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+
+import com.typesafe.config.Config;
+
 import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.runtime.AbstractJobLauncher;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
+import lombok.extern.slf4j.Slf4j;
+
 @Alpha
+@Slf4j
 public class GobblinClusterUtils {
 
   /**
@@ -70,4 +76,30 @@ public class GobblinClusterUtils {
   public static String getAppWorkDirPath(String applicationName, String applicationId) {
     return applicationName + Path.SEPARATOR + applicationId;
   }
+
+  /**
+   * Generate the path to the job.state file
+   * @param usingStateStore is a state store being used to store the job.state content
+   * @param appWorkPath work directory
+   * @param jobId job id
+   * @return a {@link Path} referring to the job.state
+   */
+  public static Path getJobStateFilePath(boolean usingStateStore, Path appWorkPath, String jobId) {
+    final Path jobStateFilePath;
+
+    // the state store uses a path of the form workdir/_jobstate/job_id/job_id.job.state while old method stores the file
+    // in the app work dir.
+    if (usingStateStore) {
+      jobStateFilePath = new Path(appWorkPath, GobblinClusterConfigurationKeys.JOB_STATE_DIR_NAME
+          + Path.SEPARATOR + jobId + Path.SEPARATOR + jobId + "."
+          + AbstractJobLauncher.JOB_STATE_FILE_NAME);
+
+    } else {
+      jobStateFilePath = new Path(appWorkPath, jobId + "." + AbstractJobLauncher.JOB_STATE_FILE_NAME);
+    }
+
+    log.info("job state file path: " + jobStateFilePath);
+
+    return jobStateFilePath;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
index fc78053..1a39dfb 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
@@ -162,13 +162,14 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
 
     this.stateStores = new StateStores(stateStoreJobConfig, appWorkDir,
         GobblinClusterConfigurationKeys.OUTPUT_TASK_STATE_DIR_NAME, appWorkDir,
-        GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME);
+        GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME, appWorkDir,
+        GobblinClusterConfigurationKeys.JOB_STATE_DIR_NAME);
 
     URI fsUri = URI.create(jobProps.getProperty(ConfigurationKeys.FS_URI_KEY, ConfigurationKeys.LOCAL_FS_URI));
     this.fs = FileSystem.get(fsUri, new Configuration());
 
     this.taskStateCollectorService = new TaskStateCollectorService(jobProps, this.jobContext.getJobState(),
-        this.eventBus, this.stateStores.taskStateStore, outputTaskStateDir);
+        this.eventBus, this.stateStores.getTaskStateStore(), outputTaskStateDir);
 
     if (Task.getExecutionModel(ConfigUtils.configToState(jobConfig)).equals(ExecutionModel.STREAMING)) {
       // Fix-up Ideal State with a custom rebalancer that will re-balance long-running jobs
@@ -268,8 +269,17 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
         addWorkUnit(workUnit, stateSerDeRunner, taskConfigMap);
       }
 
-      Path jobStateFilePath = new Path(this.appWorkDir, this.jobContext.getJobId() + "." + JOB_STATE_FILE_NAME);
-      SerializationUtils.serializeState(this.fs, jobStateFilePath, this.jobContext.getJobState());
+      Path jobStateFilePath;
+
+      // write the job.state using the state store if present, otherwise serialize directly to the file
+      if (this.stateStores.haveJobStateStore()) {
+        jobStateFilePath = GobblinClusterUtils.getJobStateFilePath(true, this.appWorkDir, this.jobContext.getJobId());
+        this.stateStores.getJobStateStore().put(jobStateFilePath.getParent().getName(), jobStateFilePath.getName(),
+            this.jobContext.getJobState());
+      } else {
+        jobStateFilePath = GobblinClusterUtils.getJobStateFilePath(false, this.appWorkDir, this.jobContext.getJobId());
+        SerializationUtils.serializeState(this.fs, jobStateFilePath, this.jobContext.getJobState());
+      }
 
       LOGGER.debug("GobblinHelixJobLauncher.createJob: jobStateFilePath {}, jobState {} jobProperties {}",
           jobStateFilePath, this.jobContext.getJobState().toString(), this.jobContext.getJobState().getProperties());
@@ -355,10 +365,10 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
 
     if (workUnit instanceof MultiWorkUnit) {
       workUnitFileName += MULTI_WORK_UNIT_FILE_EXTENSION;
-      stateStore = stateStores.mwuStateStore;
+      stateStore = stateStores.getMwuStateStore();
     } else {
       workUnitFileName += WORK_UNIT_FILE_EXTENSION;
-      stateStore = stateStores.wuStateStore;
+      stateStore = stateStores.getWuStateStore();
     }
 
     Path workUnitFile = new Path(workUnitFileDir, workUnitFileName);
@@ -396,14 +406,19 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
    */
   private void cleanupWorkingDirectory() throws IOException {
     LOGGER.info("Deleting persisted work units for job " + this.jobContext.getJobId());
-    stateStores.wuStateStore.delete(this.jobContext.getJobId());
+    stateStores.getWuStateStore().delete(this.jobContext.getJobId());
 
     // delete the directory that stores the task state files
-    stateStores.taskStateStore.delete(outputTaskStateDir.getName());
+    stateStores.getTaskStateStore().delete(outputTaskStateDir.getName());
 
     LOGGER.info("Deleting job state file for job " + this.jobContext.getJobId());
-    Path jobStateFilePath = new Path(this.appWorkDir, this.jobContext.getJobId() + "." + JOB_STATE_FILE_NAME);
-    this.fs.delete(jobStateFilePath, false);
+
+    if (this.stateStores.haveJobStateStore()) {
+      this.stateStores.getJobStateStore().delete(this.jobContext.getJobId());
+    } else {
+      Path jobStateFilePath = GobblinClusterUtils.getJobStateFilePath(false, this.appWorkDir, this.jobContext.getJobId());
+      this.fs.delete(jobStateFilePath, false);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTask.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTask.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTask.java
index 6a6e60d..c6b9514 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTask.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTask.java
@@ -78,17 +78,14 @@ public class GobblinHelixTask implements Task {
     this.taskConfig = taskCallbackContext.getTaskConfig();
     getInfoFromTaskConfig();
 
-    Path jobStateFilePath = constructJobStateFilePath(appWorkDir);
+    Path jobStateFilePath =
+        GobblinClusterUtils.getJobStateFilePath(stateStores.haveJobStateStore(), appWorkDir, this.jobId);
 
     this.task =
         new SingleTask(this.jobId, workUnitFilePath, jobStateFilePath, fs, taskAttemptBuilder,
             stateStores);
   }
 
-  private Path constructJobStateFilePath(Path appWorkDir) {
-    return new Path(appWorkDir, this.jobId + "." + AbstractJobLauncher.JOB_STATE_FILE_NAME);
-  }
-
   private void getInfoFromTaskConfig() {
     Map<String, String> configMap = this.taskConfig.getConfigMap();
     this.jobName = configMap.get(ConfigurationKeys.JOB_NAME_KEY);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTaskFactory.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTaskFactory.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTaskFactory.java
index b8e55d8..2fa845c 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTaskFactory.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTaskFactory.java
@@ -79,14 +79,15 @@ public class GobblinHelixTaskFactory implements TaskFactory {
     this.fs = fs;
     this.appWorkDir = appWorkDir;
     this.stateStores = new StateStores(config, appWorkDir, GobblinClusterConfigurationKeys.OUTPUT_TASK_STATE_DIR_NAME,
-        appWorkDir, GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME);
+        appWorkDir, GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME, appWorkDir,
+        GobblinClusterConfigurationKeys.JOB_STATE_DIR_NAME);
     this.taskAttemptBuilder = createTaskAttemptBuilder();
   }
 
   private TaskAttemptBuilder createTaskAttemptBuilder() {
     TaskAttemptBuilder builder = new TaskAttemptBuilder(this.taskStateTracker, this.taskExecutor);
     builder.setContainerId(this.helixManager.getInstanceName());
-    builder.setTaskStateStore(this.stateStores.taskStateStore);
+    builder.setTaskStateStore(this.stateStores.getTaskStateStore());
 
     return builder;
   }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java
index 3b69e0c..89f2bfa 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java
@@ -97,8 +97,18 @@ public class SingleTask {
 
   private JobState getJobState()
       throws java.io.IOException {
-    JobState jobState = new JobState();
-    SerializationUtils.deserializeState(_fs, _jobStateFilePath, jobState);
+    JobState jobState;
+
+    // read the state from the state store if present, otherwise deserialize directly from the file
+    if (_stateStores.haveJobStateStore()) {
+      jobState = _stateStores.getJobStateStore().get(_jobStateFilePath.getParent().getName(),
+          _jobStateFilePath.getName(),
+          _jobStateFilePath.getParent().getName());
+    } else {
+      jobState = new JobState();
+      SerializationUtils.deserializeState(_fs, _jobStateFilePath, jobState);
+    }
+
     return jobState;
   }
 
@@ -109,9 +119,9 @@ public class SingleTask {
     WorkUnit workUnit;
 
     if (_workUnitFilePath.getName().endsWith(AbstractJobLauncher.MULTI_WORK_UNIT_FILE_EXTENSION)) {
-      workUnit = _stateStores.mwuStateStore.getAll(storeName, fileName).get(0);
+      workUnit = _stateStores.getMwuStateStore().getAll(storeName, fileName).get(0);
     } else {
-      workUnit = _stateStores.wuStateStore.getAll(storeName, fileName).get(0);
+      workUnit = _stateStores.getWuStateStore().getAll(storeName, fileName).get(0);
     }
 
     // The list of individual WorkUnits (flattened) to run

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskRunner.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskRunner.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskRunner.java
index 9cc4733..6226cf1 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskRunner.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskRunner.java
@@ -106,11 +106,13 @@ class SingleTaskRunner {
 
   private void getSingleHelixTask()
       throws IOException {
-    final Path jobStateFilePath = getJobStateFilePath();
     final FileSystem fs = getFileSystem();
     final StateStores stateStores = new StateStores(this.clusterConfig, this.appWorkPath,
         GobblinClusterConfigurationKeys.OUTPUT_TASK_STATE_DIR_NAME, this.appWorkPath,
-        GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME);
+        GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME, this.appWorkPath,
+        GobblinClusterConfigurationKeys.JOB_STATE_DIR_NAME);
+    final Path jobStateFilePath =
+        GobblinClusterUtils.getJobStateFilePath(stateStores.haveJobStateStore(), this.appWorkPath, this.jobId);
 
     final TaskAttemptBuilder taskAttemptBuilder = getTaskAttemptBuilder(stateStores);
 
@@ -122,7 +124,7 @@ class SingleTaskRunner {
     final TaskAttemptBuilder taskAttemptBuilder =
         new TaskAttemptBuilder(this.taskStateTracker, this.taskExecutor);
     // No container id is set. Use the default.
-    taskAttemptBuilder.setTaskStateStore(stateStores.taskStateStore);
+    taskAttemptBuilder.setTaskStateStore(stateStores.getTaskStateStore());
     return taskAttemptBuilder;
   }
 
@@ -135,13 +137,6 @@ class SingleTaskRunner {
     this.serviceManager = new ServiceManager(services);
   }
 
-  private Path getJobStateFilePath() {
-    final String jobStateFileName = this.jobId + "." + AbstractJobLauncher.JOB_STATE_FILE_NAME;
-    final Path jobStateFilePath = new Path(this.appWorkPath, jobStateFileName);
-    logger.info("job state file path: " + jobStateFilePath);
-    return jobStateFilePath;
-  }
-
   private FileSystem getFileSystem()
       throws IOException {
     final Configuration conf = HadoopUtils.newConfiguration();

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobLauncherTest.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobLauncherTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobLauncherTest.java
index 77a33af..aaf5f05 100644
--- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobLauncherTest.java
+++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobLauncherTest.java
@@ -121,6 +121,7 @@ public class GobblinHelixJobLauncherTest {
                    ConfigValueFactory.fromAnyRef(testingZKServer.getConnectString()))
         .withValue(ConfigurationKeys.SOURCE_FILEBASED_FILES_TO_PULL,
             ConfigValueFactory.fromAnyRef(sourceJsonFile.getAbsolutePath()))
+        .withValue(ConfigurationKeys.JOB_STATE_IN_STATE_STORE, ConfigValueFactory.fromAnyRef("true"))
         .resolve();
 
     String zkConnectingString = baseConfig.getString(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY);
@@ -339,6 +340,11 @@ public class GobblinHelixJobLauncherTest {
 
     Assert.assertFalse(workunitsDir.exists());
     Assert.assertFalse(taskstatesDir.exists());
+
+    // check that job.state file is cleaned up
+    final File jobStateFile = new File(GobblinClusterUtils.getJobStateFilePath(true, this.appWorkDir, jobIdKey).toString());
+
+    Assert.assertFalse(jobStateFile.exists());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/StateStores.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/StateStores.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/StateStores.java
index 8d1c51f..cc892f8 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/StateStores.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/StateStores.java
@@ -16,40 +16,52 @@
  */
 package org.apache.gobblin.runtime.util;
 
+import java.util.Map;
+
+import org.apache.hadoop.fs.Path;
+
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
 import com.typesafe.config.ConfigValue;
 import com.typesafe.config.ConfigValueFactory;
+
 import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.metastore.StateStore;
+import org.apache.gobblin.runtime.JobState;
 import org.apache.gobblin.runtime.TaskState;
 import org.apache.gobblin.source.workunit.MultiWorkUnit;
 import org.apache.gobblin.source.workunit.WorkUnit;
 import org.apache.gobblin.util.ClassAliasResolver;
 import org.apache.gobblin.util.ConfigUtils;
-import org.apache.hadoop.fs.Path;
 
-import java.util.Map;
+import lombok.Getter;
 
 /**
  * state stores used for storing work units and task states
  */
 public class StateStores {
-  public final StateStore<TaskState> taskStateStore;
-  public final StateStore<WorkUnit> wuStateStore;
-  public final StateStore<MultiWorkUnit> mwuStateStore;
+  @Getter
+  private final StateStore<TaskState> taskStateStore;
+  @Getter
+  private final StateStore<WorkUnit> wuStateStore;
+  @Getter
+  private final StateStore<MultiWorkUnit> mwuStateStore;
+  // state store for job.state files. This should not be confused with the jst state store
+  @Getter
+  private final StateStore<JobState> jobStateStore;
 
   /**
    * Creates the state stores under storeBase
    * {@link WorkUnit}s will be stored under storeBase/_workunits/subdir/filename.(m)wu
    * {@link TaskState}s will be stored under storeBase/_taskstates/subdir/filename.tst
+   * {@link JobState}s will be stored under StoreBase/_jobStates/subdir/filename.job.state
    * Some state stores such as the MysqlStateStore do not preserve the path prefix of storeRoot.
    * In those cases only the last three components of the path determine the key for the data.
    * @param config config properties
    * @param taskStoreBase the base directory that holds the store root for the task state store
    */
   public StateStores(Config config, Path taskStoreBase, String taskStoreTable, Path workUnitStoreBase,
-      String workUnitStoreTable) {
+      String workUnitStoreTable, Path jobStateStoreBase, String jobStateStoreTable) {
     String stateStoreType = ConfigUtils.getString(config, ConfigurationKeys.INTERMEDIATE_STATE_STORE_TYPE_KEY,
         ConfigUtils.getString(config, ConfigurationKeys.STATE_STORE_TYPE_KEY,
             ConfigurationKeys.DEFAULT_STATE_STORE_TYPE));
@@ -79,6 +91,25 @@ public class StateStores {
     taskStateStore = stateStoreFactory.createStateStore(taskStateStoreConfig, TaskState.class);
     wuStateStore = stateStoreFactory.createStateStore(wuStateStoreConfig, WorkUnit.class);
     mwuStateStore = stateStoreFactory.createStateStore(wuStateStoreConfig, MultiWorkUnit.class);
+
+    // create a state store to store job.state content if configured
+    if (ConfigUtils.getBoolean(config, ConfigurationKeys.JOB_STATE_IN_STATE_STORE,
+        ConfigurationKeys.DEFAULT_JOB_STATE_IN_STATE_STORE)) {
+      // Override properties to place the JobState StateStore at the appropriate location
+      Path jobStateOutputDir = new Path(jobStateStoreBase, jobStateStoreTable);
+      Config jobStateStoreConfig = getStateStoreConfig(config, jobStateOutputDir.toString(), jobStateStoreTable);
+
+      jobStateStore = stateStoreFactory.createStateStore(jobStateStoreConfig, JobState.class);
+    } else {
+      jobStateStore = null;
+    }
+  }
+
+  /**
+   * @return true if a state store is present for storing job.state content
+   */
+  public boolean haveJobStateStore() {
+    return this.jobStateStore != null;
   }
 
   private static Config getStateStoreConfig(Config config, String rootDir, String dbTableKey) {

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/f2f6e468/gobblin-utility/src/test/java/org/apache/gobblin/util/test/TestStressTestingSource.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/test/java/org/apache/gobblin/util/test/TestStressTestingSource.java b/gobblin-utility/src/test/java/org/apache/gobblin/util/test/TestStressTestingSource.java
index aad8a3a..040d69b 100644
--- a/gobblin-utility/src/test/java/org/apache/gobblin/util/test/TestStressTestingSource.java
+++ b/gobblin-utility/src/test/java/org/apache/gobblin/util/test/TestStressTestingSource.java
@@ -92,8 +92,9 @@ public class TestStressTestingSource {
     long endTimeNano = System.nanoTime();
 
     long timeSpentMicro = (endTimeNano - startTimeNano)/(1000);
-    // check that there is less than 2 second difference between expected and actual time spent
-    Assert.assertTrue(Math.abs(timeSpentMicro - (COMPUTE_TIME_MICRO * NUM_RECORDS)) < (2000000));
+    // check that there is less than 5 second difference between expected and actual time spent
+    Assert.assertTrue(Math.abs(timeSpentMicro - (COMPUTE_TIME_MICRO * NUM_RECORDS)) < (5000000),
+        "Time spent " + timeSpentMicro);
   }
 
   @Test
@@ -127,7 +128,8 @@ public class TestStressTestingSource {
 
     long timeSpentMicro = (endTimeNano - startTimeNano)/(1000);
     // check that there is less than 2 second difference between expected and actual time spent
-    Assert.assertTrue(Math.abs(timeSpentMicro - (SLEEP_TIME_MICRO * NUM_RECORDS)) < (2000000));
+    Assert.assertTrue(Math.abs(timeSpentMicro - (SLEEP_TIME_MICRO * NUM_RECORDS)) < (2000000),
+        "Time spent " + timeSpentMicro);
   }
 
   @Test
@@ -163,6 +165,7 @@ public class TestStressTestingSource {
 
     long timeSpentMicro = (endTimeNano - startTimeNano)/(1000);
     // check that there is less than 1 second difference between expected and actual time spent
-    Assert.assertTrue(Math.abs(timeSpentMicro - (RUN_DURATION_SECS * 1000000)) < (1000000));
+    Assert.assertTrue(Math.abs(timeSpentMicro - (RUN_DURATION_SECS * 1000000)) < (1000000),
+        "Time spent " + timeSpentMicro);
   }
 }


[20/50] incubator-gobblin git commit: [GOBBLIN-370] added some log information

Posted by ab...@apache.org.
[GOBBLIN-370] added some log information

Closes #2245 from arjun4084346/enhanceLogging


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/7d11c650
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/7d11c650
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/7d11c650

Branch: refs/heads/0.12.0
Commit: 7d11c65015acb4b52b4ae5dcf6f0f888ad097a2a
Parents: 6cbc0cd
Author: Arjun <ab...@linkedin.com>
Authored: Mon Feb 5 11:56:20 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Mon Feb 5 11:56:20 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/gobblin/metrics/reporter/EventReporter.java  | 4 ++--
 .../java/org/apache/gobblin/runtime/AbstractJobLauncher.java     | 1 +
 .../main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java  | 3 +++
 3 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/7d11c650/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/reporter/EventReporter.java
----------------------------------------------------------------------
diff --git a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/reporter/EventReporter.java b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/reporter/EventReporter.java
index a733d6a..70e581c 100644
--- a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/reporter/EventReporter.java
+++ b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/reporter/EventReporter.java
@@ -129,8 +129,8 @@ public abstract class EventReporter extends ScheduledReporter implements Closeab
             event, this.getClass().getCanonicalName());
       }
     } catch (InterruptedException ie) {
-      log.warn(String.format("Enqueuing of event %s at reporter with class %s was interrupted.", event,
-          this.getClass().getCanonicalName()), ie);
+      log.warn("Enqueuing of event {} at reporter with class {} was interrupted.", event,
+          this.getClass().getCanonicalName(), ie);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/7d11c650/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java
index d59e097..06782a9 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/AbstractJobLauncher.java
@@ -483,6 +483,7 @@ public abstract class AbstractJobLauncher implements JobLauncher {
         // Set the overall job state to FAILED if the job failed to process any dataset
         if (datasetState.getState() == JobState.RunningState.FAILED) {
           jobState.setState(JobState.RunningState.FAILED);
+          LOG.warn("At least one dataset state is FAILED. Setting job state to FAILED.");
           break;
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/7d11c650/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java
index 7e2e9fa..a80d832 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java
@@ -399,6 +399,9 @@ final class SafeDatasetCommit implements Callable<Void> {
           // 2. Otherwise, the processing of the dataset is considered successful even if some tasks for the
           //    dataset failed to be committed.
           datasetState.setState(JobState.RunningState.FAILED);
+          Optional<String> taskStateException = taskState.getTaskFailureException();
+          log.warn("At least one task did not committed successfully. Setting dataset state to FAILED.",
+              taskStateException.isPresent() ? taskStateException.get() : "Exception not set.");
         }
       }
     }


[32/50] incubator-gobblin git commit: [GOBBLIN-411] Fix bug in FIFO based pull file loader

Posted by ab...@apache.org.
[GOBBLIN-411] Fix bug in FIFO based pull file loader

Closes #2288 from arjun4084346/sortFix


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/4f522302
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/4f522302
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/4f522302

Branch: refs/heads/0.12.0
Commit: 4f5223028ebf3f79349720cca2f80f9dabf8f604
Parents: 4997fa9
Author: Arjun <ab...@linkedin.com>
Authored: Thu Feb 15 20:52:17 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Thu Feb 15 20:52:17 2018 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/gobblin/util/PullFileLoader.java      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/4f522302/gobblin-utility/src/main/java/org/apache/gobblin/util/PullFileLoader.java
----------------------------------------------------------------------
diff --git a/gobblin-utility/src/main/java/org/apache/gobblin/util/PullFileLoader.java b/gobblin-utility/src/main/java/org/apache/gobblin/util/PullFileLoader.java
index 210615c..9270491 100644
--- a/gobblin-utility/src/main/java/org/apache/gobblin/util/PullFileLoader.java
+++ b/gobblin-utility/src/main/java/org/apache/gobblin/util/PullFileLoader.java
@@ -168,7 +168,7 @@ public class PullFileLoader {
 
   private List<Config> getSortedConfigs(List<ConfigWithTimeStamp> configsWithTimeStamps) {
     List<Config> sortedConfigs = Lists.newArrayList();
-    Collections.sort(configsWithTimeStamps, (config1, config2) -> (config1.timeStamp > config2.timeStamp) ? 1 : -1);
+    Collections.sort(configsWithTimeStamps, Comparator.comparingLong(o -> o.timeStamp));
     for (ConfigWithTimeStamp configWithTimeStamp : configsWithTimeStamps) {
       sortedConfigs.add(configWithTimeStamp.config);
     }


[44/50] incubator-gobblin git commit: [GOBBLIN-421] Add parameterized type for Pusher message type

Posted by ab...@apache.org.
[GOBBLIN-421] Add parameterized type for Pusher message type

Closes #2298 from htran1/pusher_message_type


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/8636b0cc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/8636b0cc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/8636b0cc

Branch: refs/heads/0.12.0
Commit: 8636b0ccabb517c9783287a7a902e3881a878141
Parents: ca5835b
Author: Hung Tran <hu...@linkedin.com>
Authored: Sun Mar 4 16:15:50 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Sun Mar 4 16:15:50 2018 -0800

----------------------------------------------------------------------
 .../metrics/kafka/KafkaProducerPusher.java      |  2 +-
 .../gobblin/metrics/kafka/KafkaPusher.java      |  2 +-
 .../metrics/kafka/KafkaProducerPusher.java      |  2 +-
 .../gobblin/metrics/kafka/LoggingPusher.java    | 68 ++++++++++++++++++++
 .../gobblin/metrics/kafka/NoopPusher.java       | 47 ++++++++++++++
 .../apache/gobblin/metrics/kafka/Pusher.java    |  4 +-
 .../metrics/kafka/LoggingPusherTest.java        | 64 ++++++++++++++++++
 .../metrics/reporter/MockKafkaPusher.java       |  2 +-
 8 files changed, 185 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/8636b0cc/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/metrics/kafka/KafkaProducerPusher.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/metrics/kafka/KafkaProducerPusher.java b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/metrics/kafka/KafkaProducerPusher.java
index ff75a92..d83cc36 100644
--- a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/metrics/kafka/KafkaProducerPusher.java
+++ b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/metrics/kafka/KafkaProducerPusher.java
@@ -37,7 +37,7 @@ import org.apache.gobblin.util.ConfigUtils;
 /**
  * Establishes a connection to a Kafka cluster and push byte messages to a specified topic.
  */
-public class KafkaProducerPusher implements Pusher {
+public class KafkaProducerPusher implements Pusher<byte[]> {
 
   private final String topic;
   private final KafkaProducer<String, byte[]> producer;

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/8636b0cc/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/metrics/kafka/KafkaPusher.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/metrics/kafka/KafkaPusher.java b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/metrics/kafka/KafkaPusher.java
index 1c977ff..b32899c 100644
--- a/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/metrics/kafka/KafkaPusher.java
+++ b/gobblin-modules/gobblin-kafka-08/src/main/java/org/apache/gobblin/metrics/kafka/KafkaPusher.java
@@ -33,7 +33,7 @@ import kafka.producer.ProducerConfig;
 /**
  * Establishes a connection to a Kafka cluster and pushed byte messages to a specified topic.
  */
-public class KafkaPusher implements Pusher {
+public class KafkaPusher implements Pusher<byte[]> {
 
   private final String topic;
   private final ProducerCloseable<String, byte[]> producer;

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/8636b0cc/gobblin-modules/gobblin-kafka-09/src/main/java/org/apache/gobblin/metrics/kafka/KafkaProducerPusher.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-09/src/main/java/org/apache/gobblin/metrics/kafka/KafkaProducerPusher.java b/gobblin-modules/gobblin-kafka-09/src/main/java/org/apache/gobblin/metrics/kafka/KafkaProducerPusher.java
index 3d2de9b..52d416b 100644
--- a/gobblin-modules/gobblin-kafka-09/src/main/java/org/apache/gobblin/metrics/kafka/KafkaProducerPusher.java
+++ b/gobblin-modules/gobblin-kafka-09/src/main/java/org/apache/gobblin/metrics/kafka/KafkaProducerPusher.java
@@ -37,7 +37,7 @@ import org.apache.gobblin.util.ConfigUtils;
 /**
  * Establish a connection to a Kafka cluster and push byte messages to a specified topic.
  */
-public class KafkaProducerPusher implements Pusher {
+public class KafkaProducerPusher implements Pusher<byte[]> {
 
   private final String topic;
   private final KafkaProducer<String, byte[]> producer;

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/8636b0cc/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/LoggingPusher.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/LoggingPusher.java b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/LoggingPusher.java
new file mode 100644
index 0000000..b86287e
--- /dev/null
+++ b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/LoggingPusher.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gobblin.metrics.kafka;
+
+import java.io.IOException;
+import java.util.List;
+
+import com.google.common.base.Optional;
+import com.typesafe.config.Config;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * This is a {@Pusher} class that logs the messages
+ * @param <M> message type
+ */
+@Slf4j
+public class LoggingPusher<M> implements Pusher<M> {
+  private final String brokers;
+  private final String topic;
+  private static final String KAFKA_TOPIC = "kafka.topic";
+  private static final String NO_BROKERS = "NoBrokers";
+  private static final String NO_TOPIC = "NoTopic";
+
+  public LoggingPusher() {
+    this(NO_BROKERS, NO_TOPIC, Optional.absent());
+  }
+
+  public LoggingPusher(Config config) {
+    this.brokers = ConfigUtils.getString(config, ConfigurationKeys.KAFKA_BROKERS, NO_BROKERS);
+    this.topic = ConfigUtils.getString(config, KAFKA_TOPIC, NO_TOPIC);
+  }
+
+  /**
+   * Constructor like the one in KafkaProducerPusher for compatibility
+   */
+  public LoggingPusher(String brokers, String topic, Optional<Config> kafkaConfig) {
+    this.brokers = brokers;
+    this.topic = topic;
+  }
+
+  public void pushMessages(List<M> messages) {
+    for (M message: messages) {
+      log.info("Pushing to {}:{}: {}", this.brokers, this.topic, message.toString());
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/8636b0cc/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/NoopPusher.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/NoopPusher.java b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/NoopPusher.java
new file mode 100644
index 0000000..2c1edd5
--- /dev/null
+++ b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/NoopPusher.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gobblin.metrics.kafka;
+
+import java.io.IOException;
+import java.util.List;
+
+import com.google.common.base.Optional;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+
+/**
+ * This is a {@Pusher} class that ignores the messages
+ * @param <M> message type
+ */
+@Slf4j
+public class NoopPusher<M> implements Pusher<M> {
+  public NoopPusher() {}
+
+  public NoopPusher(Config config) {}
+
+  /**
+   * Constructor like the one in KafkaProducerPusher for compatibility
+   */
+  public NoopPusher(String brokers, String topic, Optional<Config> kafkaConfig) {}
+
+  public void pushMessages(List<M> messages) {}
+
+  @Override
+  public void close() throws IOException {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/8636b0cc/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/Pusher.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/Pusher.java b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/Pusher.java
index 5abd503..9024a88 100644
--- a/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/Pusher.java
+++ b/gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/Pusher.java
@@ -24,10 +24,10 @@ import java.util.List;
 /**
  * Establish a connection to a Kafka cluster and push byte messages to a specified topic.
  */
-public interface Pusher extends Closeable {
+public interface Pusher<M> extends Closeable {
   /**
    * Push all byte array messages to the Kafka topic.
    * @param messages List of byte array messages to push to Kakfa.
    */
-  void pushMessages(List<byte[]> messages);
+  void pushMessages(List<M> messages);
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/8636b0cc/gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/metrics/kafka/LoggingPusherTest.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/metrics/kafka/LoggingPusherTest.java b/gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/metrics/kafka/LoggingPusherTest.java
new file mode 100644
index 0000000..3e861de
--- /dev/null
+++ b/gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/metrics/kafka/LoggingPusherTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.gobblin.metrics.kafka;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.log4j.AppenderSkeleton;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.log4j.spi.LoggingEvent;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableList;
+
+
+@Test
+public class LoggingPusherTest {
+
+  @Test
+  public void testKafkaReporter() {
+
+    TestAppender testAppender = new TestAppender();
+    Logger logger = LogManager.getLogger(LoggingPusher.class.getName());
+    logger.addAppender(testAppender);
+
+    LoggingPusher<String> loggingPusher = new LoggingPusher<String>("broker", "topic", Optional.absent());
+
+    loggingPusher.pushMessages(ImmutableList.of("message1", "message2"));
+
+    Assert.assertEquals(testAppender.events.size(), 2);
+    Assert.assertEquals(testAppender.events.get(0).getRenderedMessage(), "Pushing to broker:topic: message1");
+    Assert.assertEquals(testAppender.events.get(1).getRenderedMessage(), "Pushing to broker:topic: message2");
+
+    logger.removeAppender(testAppender);
+  }
+
+
+  private class TestAppender extends AppenderSkeleton {
+    List<LoggingEvent> events = new ArrayList<LoggingEvent>();
+    public void close() {}
+    public boolean requiresLayout() {return false;}
+    @Override
+    protected void append(LoggingEvent event) {
+      events.add(event);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/8636b0cc/gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/metrics/reporter/MockKafkaPusher.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/metrics/reporter/MockKafkaPusher.java b/gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/metrics/reporter/MockKafkaPusher.java
index 71decbb..8bb827f 100644
--- a/gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/metrics/reporter/MockKafkaPusher.java
+++ b/gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/metrics/reporter/MockKafkaPusher.java
@@ -31,7 +31,7 @@ import org.apache.gobblin.metrics.kafka.Pusher;
 /**
  * Mock instance of {@link org.apache.gobblin.metrics.kafka.Pusher} used for testing.
  */
-public class MockKafkaPusher implements Pusher {
+public class MockKafkaPusher implements Pusher<byte[]> {
 
   Queue<byte[]> messages = Queues.newLinkedBlockingQueue();
 


[02/50] incubator-gobblin git commit: [GOBBLIN-384] Update Python version to v3

Posted by ab...@apache.org.
[GOBBLIN-384] Update Python version to v3

Closes #2261 from abti/gobblin-dev


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/fd0c30a6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/fd0c30a6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/fd0c30a6

Branch: refs/heads/0.12.0
Commit: fd0c30a61e87c2d6208a0d2f9c4761dd1b5d37b7
Parents: c6b3824
Author: Abhishek Tiwari <ab...@gmail.com>
Authored: Tue Jan 23 14:05:43 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Tue Jan 23 14:05:43 2018 -0800

----------------------------------------------------------------------
 dev/gobblin-pr | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/fd0c30a6/dev/gobblin-pr
----------------------------------------------------------------------
diff --git a/dev/gobblin-pr b/dev/gobblin-pr
index a189db7..a7b3209 100755
--- a/dev/gobblin-pr
+++ b/dev/gobblin-pr
@@ -1,4 +1,4 @@
-#!/usr/bin/env python
+#!/usr/bin/env python3
 
 #
 # Licensed to the Apache Software Foundation (ASF) under one or more
@@ -37,6 +37,10 @@ import subprocess
 import sys
 import textwrap
 
+# Print current SSL version, commented and left around for debugging
+# import ssl
+# print(ssl.OPENSSL_VERSION)
+
 # Python 3 compatibility
 try:
     import urllib2 as urllib
@@ -48,13 +52,13 @@ if sys.version_info[0] == 3:
 try:
     import click
 except ImportError:
-    print("Could not find the click library. Run 'sudo pip install click' to install.")
+    print("Could not find the click library. Run 'sudo pip3 install click' to install.")
     sys.exit(-1)
 
 try:
     import keyring
 except ImportError:
-    print("Could not find the keyring library. Run 'sudo pip install keyring' to install.")
+    print("Could not find the keyring library. Run 'sudo pip3 install keyring' to install.")
     sys.exit(-1)
 
 # Location of your Gobblin git development area
@@ -532,7 +536,7 @@ def resolve_jira_issue(comment=None, jira_id=None, merge_branches=None):
     except ImportError:
         raise PRToolError(
             "Could not find jira-python library; exiting. Run "
-            "'sudo pip install jira' to install.")
+            "'sudo pip3 install jira' to install.")
 
     if merge_branches is None:
         merge_branches = []


[07/50] incubator-gobblin git commit: [GOBBLIN-391] Use the DataPublisherFactory to allow sharing publisher…

Posted by ab...@apache.org.
[GOBBLIN-391] Use the DataPublisherFactory to allow sharing publisher…

Closes #2267 from htran1/share_publisher


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/378ccaa8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/378ccaa8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/378ccaa8

Branch: refs/heads/0.12.0
Commit: 378ccaa8a253a1eda873ffbe74300c6bf8a755e8
Parents: 41fd2b9
Author: Hung Tran <hu...@linkedin.com>
Authored: Fri Jan 26 11:45:31 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Fri Jan 26 11:45:31 2018 -0800

----------------------------------------------------------------------
 .../gobblin/publisher/DataPublisherFactory.java |  12 ++-
 .../gobblin/runtime/SafeDatasetCommit.java      |  27 +++--
 .../runtime/mapreduce/MRJobLauncherTest.java    | 103 +++++++++++++++++++
 3 files changed, 133 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/378ccaa8/gobblin-core/src/main/java/org/apache/gobblin/publisher/DataPublisherFactory.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/publisher/DataPublisherFactory.java b/gobblin-core/src/main/java/org/apache/gobblin/publisher/DataPublisherFactory.java
index 4e565ad..8d77fd6 100644
--- a/gobblin-core/src/main/java/org/apache/gobblin/publisher/DataPublisherFactory.java
+++ b/gobblin-core/src/main/java/org/apache/gobblin/publisher/DataPublisherFactory.java
@@ -54,6 +54,16 @@ public class DataPublisherFactory<S extends ScopeType<S>>
     }
   }
 
+  /**
+   * Is the publisher cacheable in the SharedResourcesBroker?
+   * @param publisher
+   * @return true if cacheable, else false
+   */
+  public static boolean isPublisherCacheable(DataPublisher publisher) {
+    // only threadsafe publishers are cacheable. non-threadsafe publishers are marked immediately for invalidation
+    return publisher.supportsCapability(Capability.THREADSAFE, Collections.EMPTY_MAP);
+  }
+
   @Override
   public String getName() {
     return FACTORY_NAME;
@@ -75,7 +85,7 @@ public class DataPublisherFactory<S extends ScopeType<S>>
       // by the broker.
       // Otherwise, it is not shareable, so return it as an immediately invalidated resource that will only be returned
       // once from the broker.
-      if (publisher.supportsCapability(Capability.THREADSAFE, Collections.EMPTY_MAP)) {
+      if (isPublisherCacheable(publisher)) {
         return new ResourceInstance<>(publisher);
       } else {
         return new ImmediatelyInvalidResourceEntry<>(publisher);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/378ccaa8/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java
index 43e5c59..7e2e9fa 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/SafeDatasetCommit.java
@@ -44,6 +44,7 @@ import org.apache.gobblin.metrics.event.FailureEventBuilder;
 import org.apache.gobblin.metrics.event.lineage.LineageInfo;
 import org.apache.gobblin.publisher.CommitSequencePublisher;
 import org.apache.gobblin.publisher.DataPublisher;
+import org.apache.gobblin.publisher.DataPublisherFactory;
 import org.apache.gobblin.publisher.UnpublishedHandling;
 import org.apache.gobblin.runtime.commit.DatasetStateCommitStep;
 import org.apache.gobblin.runtime.task.TaskFactory;
@@ -133,9 +134,23 @@ final class SafeDatasetCommit implements Callable<Void> {
             }
             generateCommitSequenceBuilder(this.datasetState, entry.getValue());
           } else {
-            DataPublisher publisher = taskFactory == null ? closer
-                .register(DataPublisher.getInstance(dataPublisherClass, this.jobContext.getJobState()))
-                : taskFactory.createDataPublisher(this.datasetState);
+            DataPublisher publisher;
+
+            if (taskFactory == null) {
+              publisher = DataPublisherFactory.get(dataPublisherClass.getName(), this.jobContext.getJobState(),
+                  this.jobContext.getJobBroker());
+
+              // non-threadsafe publishers are not shareable and are not retained in the broker, so register them with
+              // the closer
+              if (!DataPublisherFactory.isPublisherCacheable(publisher)) {
+                closer.register(publisher);
+              }
+            } else {
+              // NOTE: sharing of publishers is not supported when they are instantiated through the TaskFactory.
+              // This should be revisited if sharing is required.
+              publisher = taskFactory.createDataPublisher(this.datasetState);
+            }
+
             if (this.isJobCancelled) {
               if (publisher.canBeSkipped()) {
                 log.warn(publisher.getClass() + " will be skipped.");
@@ -160,11 +175,7 @@ final class SafeDatasetCommit implements Callable<Void> {
           this.datasetState.setState(JobState.RunningState.COMMITTED);
         }
       }
-    } catch (ReflectiveOperationException roe) {
-      log.error(String.format("Failed to instantiate data publisher for dataset %s of job %s.", this.datasetUrn,
-          this.jobContext.getJobId()), roe);
-      throw new RuntimeException(roe);
-    } catch (Throwable throwable) {
+    }  catch (Throwable throwable) {
       log.error(String.format("Failed to commit dataset state for dataset %s of job %s", this.datasetUrn,
           this.jobContext.getJobId()), throwable);
       throw new RuntimeException(throwable);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/378ccaa8/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/mapreduce/MRJobLauncherTest.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/mapreduce/MRJobLauncherTest.java b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/mapreduce/MRJobLauncherTest.java
index 8d4f308..e8e996e 100644
--- a/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/mapreduce/MRJobLauncherTest.java
+++ b/gobblin-runtime/src/test/java/org/apache/gobblin/runtime/mapreduce/MRJobLauncherTest.java
@@ -20,7 +20,10 @@ package org.apache.gobblin.runtime.mapreduce;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.Collection;
+import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.io.FileUtils;
 import org.jboss.byteman.contrib.bmunit.BMNGRunner;
@@ -37,14 +40,17 @@ import com.google.common.collect.ImmutableMap;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
 
+import org.apache.gobblin.capability.Capability;
 import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.DynamicConfigGenerator;
+import org.apache.gobblin.configuration.State;
 import org.apache.gobblin.configuration.WorkUnitState;
 import org.apache.gobblin.metastore.FsStateStore;
 import org.apache.gobblin.metastore.StateStore;
 import org.apache.gobblin.metastore.testing.ITestMetastoreDatabase;
 import org.apache.gobblin.metastore.testing.TestMetastoreDatabaseFactory;
 import org.apache.gobblin.metrics.GobblinMetrics;
+import org.apache.gobblin.publisher.DataPublisher;
 import org.apache.gobblin.runtime.JobLauncherTestHelper;
 import org.apache.gobblin.runtime.JobState;
 import org.apache.gobblin.util.limiter.BaseLimiterType;
@@ -360,6 +366,55 @@ public class MRJobLauncherTest extends BMNGRunner {
     }
   }
 
+  @Test
+  public void testLaunchJobWithNonThreadsafeDataPublisher() throws Exception {
+    final Logger log = LoggerFactory.getLogger(getClass().getName() + ".testLaunchJobWithNonThreadsafeDataPublisher");
+    log.info("in");
+    Properties jobProps = loadJobProps();
+    jobProps.setProperty(ConfigurationKeys.JOB_NAME_KEY,
+        jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY) + "-testLaunchJobWithNonThreadsafeDataPublisher");
+    jobProps.setProperty(ConfigurationKeys.JOB_DATA_PUBLISHER_TYPE, TestNonThreadsafeDataPublisher.class.getName());
+
+    // make sure the count starts from 0
+    TestNonThreadsafeDataPublisher.instantiatedCount.set(0);
+
+    try {
+      this.jobLauncherTestHelper.runTestWithMultipleDatasets(jobProps);
+    } finally {
+      this.jobLauncherTestHelper.deleteStateStore(jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY));
+    }
+
+    // A different  publisher is used for each dataset
+    Assert.assertEquals(TestNonThreadsafeDataPublisher.instantiatedCount.get(), 4);
+
+    log.info("out");
+  }
+
+  @Test
+  public void testLaunchJobWithThreadsafeDataPublisher() throws Exception {
+    final Logger log = LoggerFactory.getLogger(getClass().getName() + ".testLaunchJobWithThreadsafeDataPublisher");
+    log.info("in");
+    Properties jobProps = loadJobProps();
+    jobProps.setProperty(ConfigurationKeys.JOB_NAME_KEY,
+        jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY) + "-testLaunchJobWithThreadsafeDataPublisher");
+    jobProps.setProperty(ConfigurationKeys.JOB_DATA_PUBLISHER_TYPE, TestThreadsafeDataPublisher.class.getName());
+
+    // make sure the count starts from 0
+    TestThreadsafeDataPublisher.instantiatedCount.set(0);
+
+    try {
+      this.jobLauncherTestHelper.runTestWithMultipleDatasets(jobProps);
+    } finally {
+      this.jobLauncherTestHelper.deleteStateStore(jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY));
+    }
+
+    // The same publisher is used for all the data sets
+    Assert.assertEquals(TestThreadsafeDataPublisher.instantiatedCount.get(), 1);
+
+    log.info("out");
+  }
+
+
   @AfterClass(alwaysRun = true)
   public void tearDown() throws IOException {
     if (testMetastoreDatabase != null) {
@@ -390,4 +445,52 @@ public class MRJobLauncherTest extends BMNGRunner {
           JobLauncherTestHelper.DYNAMIC_VALUE1));
     }
   }
+
+  public static class TestNonThreadsafeDataPublisher extends DataPublisher {
+    // for counting how many times the object is instantiated in the test case
+    static AtomicInteger instantiatedCount = new AtomicInteger(0);
+
+    public TestNonThreadsafeDataPublisher(State state) {
+      super(state);
+      instantiatedCount.incrementAndGet();
+    }
+
+    @Override
+    public void initialize() throws IOException {
+    }
+
+    @Override
+    public void publishData(Collection<? extends WorkUnitState> states) throws IOException {
+      for (WorkUnitState workUnitState : states) {
+        // Upon successfully committing the data to the final output directory, set states
+        // of successful tasks to COMMITTED. leaving states of unsuccessful ones unchanged.
+        // This makes sense to the COMMIT_ON_PARTIAL_SUCCESS policy.
+        workUnitState.setWorkingState(WorkUnitState.WorkingState.COMMITTED);
+      }
+    }
+
+    @Override
+    public void publishMetadata(Collection<? extends WorkUnitState> states) throws IOException {
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    @Override
+    public boolean supportsCapability(Capability c, Map<String, Object> properties) {
+      return c == DataPublisher.REUSABLE;
+    }
+  }
+
+  public static class TestThreadsafeDataPublisher extends TestNonThreadsafeDataPublisher {
+    public TestThreadsafeDataPublisher(State state) {
+      super(state);
+    }
+
+    @Override
+    public boolean supportsCapability(Capability c, Map<String, Object> properties) {
+      return (c == Capability.THREADSAFE || c == DataPublisher.REUSABLE);
+    }
+  }
 }


[47/50] incubator-gobblin git commit: [GOBBLIN-426] Change signature of AzkabanJobLauncher.initJobListener()[]

Posted by ab...@apache.org.
[GOBBLIN-426] Change signature of AzkabanJobLauncher.initJobListener()[]

Closes #2303 from epattuk/master


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/2e61a10a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/2e61a10a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/2e61a10a

Branch: refs/heads/0.12.0
Commit: 2e61a10ac799d2bafe57a2162eafbbed19c91546
Parents: 1ddaffa
Author: Erman Pattuk <ep...@epattuk-ld1.linkedin.biz>
Authored: Tue Mar 13 11:14:41 2018 -0700
Committer: Hung Tran <hu...@linkedin.com>
Committed: Tue Mar 13 11:14:41 2018 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/2e61a10a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java b/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java
index 2a7d311..c78d098 100644
--- a/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java
+++ b/gobblin-modules/gobblin-azkaban/src/main/java/org/apache/gobblin/azkaban/AzkabanJobLauncher.java
@@ -224,7 +224,7 @@ public class AzkabanJobLauncher extends AbstractJob implements ApplicationLaunch
         this.closer.register(new ServiceBasedAppLauncher(jobProps, "Azkaban-" + UUID.randomUUID()));
   }
 
-  private JobListener initJobListener() {
+  protected JobListener initJobListener() {
     CompositeJobListener compositeJobListener = new CompositeJobListener();
     List<String> listeners = new State(props).getPropAsList(GOBBLIN_CUSTOM_JOB_LISTENERS, EmailNotificationJobListener.class.getSimpleName());
     try {


[11/50] incubator-gobblin git commit: [GOBBLIN-398] Upgrade helix to 0.6.9

Posted by ab...@apache.org.
[GOBBLIN-398] Upgrade helix to 0.6.9

Closes #2272 from htran1/helix_069


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/d29b72f4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/d29b72f4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/d29b72f4

Branch: refs/heads/0.12.0
Commit: d29b72f4997b4a435397353bb93a66ac4213d55e
Parents: ff13dde
Author: Hung Tran <hu...@linkedin.com>
Authored: Wed Jan 31 17:29:21 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Wed Jan 31 17:29:21 2018 -0800

----------------------------------------------------------------------
 .../gobblin/cluster/GobblinClusterManager.java  |  15 +-
 .../cluster/GobblinHelixJobLauncher.java        |  72 +-
 .../gobblin/cluster/GobblinHelixTaskDriver.java | 296 +-------
 .../apache/helix/task/GobblinJobRebalancer.java | 713 -------------------
 .../cluster/GobblinHelixJobLauncherTest.java    |  22 +-
 gradle/scripts/computeVersions.gradle           |   2 +-
 gradle/scripts/dependencyDefinitions.gradle     |   2 +-
 7 files changed, 57 insertions(+), 1065 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/d29b72f4/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
index 6b53c6c..77e511e 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
@@ -20,7 +20,6 @@ package org.apache.gobblin.cluster;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.net.URI;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -60,8 +59,8 @@ import org.apache.helix.messaging.handling.MessageHandler;
 import org.apache.helix.messaging.handling.MessageHandlerFactory;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
+import org.apache.helix.task.TargetState;
 import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskUtil;
 import org.apache.helix.task.WorkflowConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -261,21 +260,17 @@ public class GobblinClusterManager implements ApplicationLauncher, StandardMetri
 
         // Clean up existing jobs
         TaskDriver taskDriver = new TaskDriver(this.helixManager);
-        GobblinHelixTaskDriver gobblinHelixTaskDriver = new GobblinHelixTaskDriver(this.helixManager);
         Map<String, WorkflowConfig> workflows = taskDriver.getWorkflows();
 
         for (Map.Entry<String, WorkflowConfig> entry : workflows.entrySet()) {
           String queueName = entry.getKey();
           WorkflowConfig workflowConfig = entry.getValue();
 
-          for (String namespacedJobName : workflowConfig.getJobDag().getAllNodes()) {
-            String jobName = TaskUtil.getDenamespacedJobName(queueName, namespacedJobName);
-            LOGGER.info("job {} found for queue {} ", jobName, queueName);
+          // request delete if not already requested
+          if (workflowConfig.getTargetState() != TargetState.DELETE) {
+            taskDriver.delete(queueName);
 
-            // #HELIX-0.6.7-WORKAROUND
-            // working around 0.6.7 delete job issue for queues with IN_PROGRESS state
-            gobblinHelixTaskDriver.deleteJob(queueName, jobName);
-            LOGGER.info("deleted job {} from queue {}", jobName, queueName);
+            LOGGER.info("Requested delete of queue {}", queueName);
           }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/d29b72f4/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
index 1a39dfb..af15469 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobLauncher.java
@@ -30,17 +30,14 @@ import org.apache.gobblin.runtime.listeners.JobListener;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixManager;
-import org.apache.helix.IdealStateChangeListener;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.task.GobblinJobRebalancer;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobQueue;
+import org.apache.helix.task.TargetState;
 import org.apache.helix.task.TaskConfig;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.task.TaskUtil;
+import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -170,36 +167,6 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
 
     this.taskStateCollectorService = new TaskStateCollectorService(jobProps, this.jobContext.getJobState(),
         this.eventBus, this.stateStores.getTaskStateStore(), outputTaskStateDir);
-
-    if (Task.getExecutionModel(ConfigUtils.configToState(jobConfig)).equals(ExecutionModel.STREAMING)) {
-      // Fix-up Ideal State with a custom rebalancer that will re-balance long-running jobs
-      final String clusterName =
-          ConfigUtils.getString(jobConfig, GobblinClusterConfigurationKeys.HELIX_CLUSTER_NAME_KEY, "");
-      final String rebalancerToReplace = "org.apache.helix.task.JobRebalancer";
-      final String rebalancerClassDesired = GobblinJobRebalancer.class.getName();
-      final String jobResourceName = this.jobResourceName;
-
-      if (!clusterName.isEmpty()) {
-        this.helixManager.addIdealStateChangeListener(new IdealStateChangeListener() {
-          @Override
-          public void onIdealStateChange(List<IdealState> list, NotificationContext notificationContext) {
-            HelixAdmin helixAdmin = helixManager.getClusterManagmentTool();
-            for (String resource : helixAdmin.getResourcesInCluster(clusterName)) {
-              if (resource.equals(jobResourceName)) {
-                IdealState idealState = helixAdmin.getResourceIdealState(clusterName, resource);
-                if (idealState != null) {
-                  String rebalancerClassFound = idealState.getRebalancerClassName();
-                  if (rebalancerToReplace.equals(rebalancerClassFound)) {
-                    idealState.setRebalancerClassName(rebalancerClassDesired);
-                    helixAdmin.setResourceIdealState(clusterName, resource, idealState);
-                  }
-                }
-              }
-            }
-          }
-        });
-      }
-    }
   }
 
   @Override
@@ -240,15 +207,8 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
   protected void executeCancellation() {
     if (this.jobSubmitted) {
       try {
-        // #HELIX-0.6.7-WORKAROUND
-        // working around helix 0.6.7 job delete issue with custom taskDriver
-        LOGGER.info("Cancelling job {} in Helix", this.jobContext.getJobId());
-        GobblinHelixTaskDriver taskDriver = new GobblinHelixTaskDriver(this.helixManager);
-        taskDriver.deleteJob(this.helixQueueName, this.jobContext.getJobId());
-        LOGGER.info("Job {} in cancelled Helix", this.jobContext.getJobId());
-
-        taskDriver.deleteWorkflow(this.helixQueueName, this.jobQueueDeleteTimeoutSeconds);
-      } catch (InterruptedException | IllegalArgumentException e) {
+        this.helixTaskDriver.delete(this.helixQueueName);
+      } catch (IllegalArgumentException e) {
         LOGGER.warn("Failed to cancel job {} in Helix", this.jobContext.getJobId(), e);
       }
     }
@@ -293,6 +253,11 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
     jobConfigBuilder.setNumConcurrentTasksPerInstance(ConfigUtils.getInt(jobConfig,
         GobblinClusterConfigurationKeys.HELIX_CLUSTER_TASK_CONCURRENCY,
         GobblinClusterConfigurationKeys.HELIX_CLUSTER_TASK_CONCURRENCY_DEFAULT));
+
+    if (Task.getExecutionModel(ConfigUtils.configToState(jobConfig)).equals(ExecutionModel.STREAMING)) {
+      jobConfigBuilder.setRebalanceRunningTask(true);
+    }
+
     return jobConfigBuilder;
   }
 
@@ -300,18 +265,27 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher {
    * Submit a job to run.
    */
   private void submitJobToHelix(JobConfig.Builder jobConfigBuilder) throws Exception {
+    WorkflowConfig workflowConfig = this.helixTaskDriver.getWorkflowConfig(this.helixManager, this.helixQueueName);
+
+    // If the queue is present, but in delete state then wait for cleanup before recreating the queue
+    if (workflowConfig != null && workflowConfig.getTargetState() == TargetState.DELETE) {
+      GobblinHelixTaskDriver gobblinHelixTaskDriver = new GobblinHelixTaskDriver(this.helixManager);
+      gobblinHelixTaskDriver.deleteWorkflow(this.helixQueueName, this.jobQueueDeleteTimeoutSeconds);
+      // if we get here then the workflow was successfully deleted
+      workflowConfig = null;
+    }
+
     // Create one queue for each job with the job name being the queue name
-    if (null == this.helixTaskDriver.getWorkflowConfig(this.helixManager, this.helixQueueName)) {
-      JobQueue jobQueue = new JobQueue.Builder(this.helixQueueName).build();
-      this.helixTaskDriver.createQueue(jobQueue);
-      LOGGER.info("Created job queue {}", this.helixQueueName);
+    if (workflowConfig == null) {
+        JobQueue jobQueue = new JobQueue.Builder(this.helixQueueName).build();
+        this.helixTaskDriver.createQueue(jobQueue);
+        LOGGER.info("Created job queue {}", this.helixQueueName);
     } else {
       LOGGER.info("Job queue {} already exists", this.helixQueueName);
     }
 
     // Put the job into the queue
     this.helixTaskDriver.enqueueJob(this.jobContext.getJobName(), this.jobContext.getJobId(), jobConfigBuilder);
-
   }
 
   public void launchJob(@Nullable JobListener jobListener)

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/d29b72f4/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTaskDriver.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTaskDriver.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTaskDriver.java
index cedb111..ebe2b52 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTaskDriver.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixTaskDriver.java
@@ -16,54 +16,23 @@
  */
 package org.apache.gobblin.cluster;
 
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.I0Itec.zkclient.DataUpdater;
-import org.apache.helix.AccessOption;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.PropertyPathConfig;
-import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.manager.zk.ZKHelixAdmin;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.model.IdealState;
 import org.apache.helix.store.HelixPropertyStore;
-import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.helix.task.JobDag;
 import org.apache.helix.task.TargetState;
-import org.apache.helix.task.TaskConstants;
 import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.TaskState;
-import org.apache.helix.task.TaskUtil;
 import org.apache.helix.task.WorkflowConfig;
 import org.apache.helix.task.WorkflowContext;
-import org.apache.log4j.Logger;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.Lists;
 
 /**
  * #HELIX-0.6.7-WORKAROUND
  * Replacement TaskDriver methods to workaround bugs and changes in behavior for the 0.6.7 upgrade
  */
 public class GobblinHelixTaskDriver {
-  /** For logging */
-  private static final Logger LOG = Logger.getLogger(GobblinHelixTaskDriver.class);
-
-  private final HelixDataAccessor _accessor;
-  private final ConfigAccessor _cfgAccessor;
-  private final HelixPropertyStore<ZNRecord> _propertyStore;
-  private final HelixAdmin _admin;
-  private final String _clusterName;
   private final TaskDriver _taskDriver;
 
   public GobblinHelixTaskDriver(HelixManager manager) {
@@ -71,277 +40,24 @@ public class GobblinHelixTaskDriver {
         .getConfigAccessor(), manager.getHelixPropertyStore(), manager.getClusterName());
   }
 
-  public GobblinHelixTaskDriver(ZkClient client, String clusterName) {
-    this(client, new ZkBaseDataAccessor<ZNRecord>(client), clusterName);
-  }
-
-  public GobblinHelixTaskDriver(ZkClient client, ZkBaseDataAccessor<ZNRecord> baseAccessor, String clusterName) {
-    this(new ZKHelixAdmin(client), new ZKHelixDataAccessor(clusterName, baseAccessor),
-        new ConfigAccessor(client), new ZkHelixPropertyStore<ZNRecord>(baseAccessor,
-            PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, clusterName), null), clusterName);
-  }
-
   public GobblinHelixTaskDriver(HelixAdmin admin, HelixDataAccessor accessor, ConfigAccessor cfgAccessor,
       HelixPropertyStore<ZNRecord> propertyStore, String clusterName) {
-    _admin = admin;
-    _accessor = accessor;
-    _cfgAccessor = cfgAccessor;
-    _propertyStore = propertyStore;
-    _clusterName = clusterName;
     _taskDriver = new TaskDriver(admin, accessor, cfgAccessor, propertyStore, clusterName);
   }
 
   /**
-   * Delete a job from an existing named queue,
-   * the queue has to be stopped prior to this call
-   *
-   * @param queueName
-   * @param jobName
-   */
-  public void deleteJob(final String queueName, final String jobName) {
-    WorkflowConfig workflowCfg =
-        _taskDriver.getWorkflowConfig(queueName);
-
-    if (workflowCfg == null) {
-      throw new IllegalArgumentException("Queue " + queueName + " does not yet exist!");
-    }
-    if (workflowCfg.isTerminable()) {
-      throw new IllegalArgumentException(queueName + " is not a queue!");
-    }
-
-    boolean isRecurringWorkflow =
-        (workflowCfg.getScheduleConfig() != null && workflowCfg.getScheduleConfig().isRecurring());
-
-    if (isRecurringWorkflow) {
-      WorkflowContext wCtx = _taskDriver.getWorkflowContext(queueName);
-
-      String lastScheduledQueue = wCtx.getLastScheduledSingleWorkflow();
-
-      // delete the current scheduled one
-      deleteJobFromScheduledQueue(lastScheduledQueue, jobName, true);
-
-      // Remove the job from the original queue template's DAG
-      removeJobFromDag(queueName, jobName);
-
-      // delete the ideal state and resource config for the template job
-      final String namespacedJobName = TaskUtil.getNamespacedJobName(queueName, jobName);
-      _admin.dropResource(_clusterName, namespacedJobName);
-
-      // Delete the job template from property store
-      String jobPropertyPath =
-          Joiner.on("/")
-              .join(TaskConstants.REBALANCER_CONTEXT_ROOT, namespacedJobName);
-      _propertyStore.remove(jobPropertyPath, AccessOption.PERSISTENT);
-    } else {
-      deleteJobFromScheduledQueue(queueName, jobName, false);
-    }
-  }
-
-  /**
-   * delete a job from a scheduled (non-recurrent) queue.
-   *
-   * @param queueName
-   * @param jobName
-   */
-  private void deleteJobFromScheduledQueue(final String queueName, final String jobName,
-      boolean isRecurrent) {
-    WorkflowConfig workflowCfg = _taskDriver.getWorkflowConfig(queueName);
-
-    if (workflowCfg == null) {
-      // When try to delete recurrent job, it could be either not started or finished. So
-      // there may not be a workflow config.
-      if (isRecurrent) {
-        return;
-      } else {
-        throw new IllegalArgumentException("Queue " + queueName + " does not yet exist!");
-      }
-    }
-
-    WorkflowContext wCtx = _taskDriver.getWorkflowContext(queueName);
-    if (wCtx != null && wCtx.getWorkflowState() == null) {
-      throw new IllegalStateException("Queue " + queueName + " does not have a valid work state!");
-    }
-
-    // #HELIX-0.6.7-WORKAROUND
-    // This check is removed to get the same behavior as 0.6.6-SNAPSHOT until new APIs to support delete are provided
-    //String workflowState =
-    //    (wCtx != null) ? wCtx.getWorkflowState().name() : TaskState.NOT_STARTED.name();
-    //if (workflowState.equals(TaskState.IN_PROGRESS.name())) {
-    //  throw new IllegalStateException("Queue " + queueName + " is still in progress!");
-    //}
-
-    removeJob(queueName, jobName);
-  }
-
-  private boolean removeJobContext(HelixPropertyStore<ZNRecord> propertyStore,
-      String jobResource) {
-    return propertyStore.remove(
-        Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, jobResource),
-        AccessOption.PERSISTENT);
-  }
-
-  private void removeJob(String queueName, String jobName) {
-    // Remove the job from the queue in the DAG
-    removeJobFromDag(queueName, jobName);
-
-    // delete the ideal state and resource config for the job
-    final String namespacedJobName = TaskUtil.getNamespacedJobName(queueName, jobName);
-    _admin.dropResource(_clusterName, namespacedJobName);
-
-    // update queue's property to remove job from JOB_STATES if it is already started.
-    removeJobStateFromQueue(queueName, jobName);
-
-    // Delete the job from property store
-    removeJobContext(_propertyStore, namespacedJobName);
-  }
-
-  /** Remove the job name from the DAG from the queue configuration */
-  private void removeJobFromDag(final String queueName, final String jobName) {
-    final String namespacedJobName = TaskUtil.getNamespacedJobName(queueName, jobName);
-
-    DataUpdater<ZNRecord> dagRemover = new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord currentData) {
-        if (currentData == null) {
-          LOG.error("Could not update DAG for queue: " + queueName + " ZNRecord is null.");
-          return null;
-        }
-        // Add the node to the existing DAG
-        JobDag jobDag = JobDag.fromJson(
-            currentData.getSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name()));
-        Set<String> allNodes = jobDag.getAllNodes();
-        if (!allNodes.contains(namespacedJobName)) {
-          LOG.warn(
-              "Could not delete job from queue " + queueName + ", job " + jobName + " not exists");
-          return currentData;
-        }
-        String parent = null;
-        String child = null;
-        // remove the node from the queue
-        for (String node : allNodes) {
-          if (jobDag.getDirectChildren(node).contains(namespacedJobName)) {
-            parent = node;
-            jobDag.removeParentToChild(parent, namespacedJobName);
-          } else if (jobDag.getDirectParents(node).contains(namespacedJobName)) {
-            child = node;
-            jobDag.removeParentToChild(namespacedJobName, child);
-          }
-        }
-        if (parent != null && child != null) {
-          jobDag.addParentToChild(parent, child);
-        }
-        jobDag.removeNode(namespacedJobName);
-
-        // Save the updated DAG
-        try {
-          currentData
-              .setSimpleField(WorkflowConfig.WorkflowConfigProperty.Dag.name(), jobDag.toJson());
-        } catch (Exception e) {
-          throw new IllegalStateException(
-              "Could not remove job " + jobName + " from DAG of queue " + queueName, e);
-        }
-        return currentData;
-      }
-    };
-
-    String path = _accessor.keyBuilder().resourceConfig(queueName).getPath();
-    if (!_accessor.getBaseDataAccessor().update(path, dagRemover, AccessOption.PERSISTENT)) {
-      throw new IllegalArgumentException(
-          "Could not remove job " + jobName + " from DAG of queue " + queueName);
-    }
-  }
-
-  /** update queue's property to remove job from JOB_STATES if it is already started. */
-  private void removeJobStateFromQueue(final String queueName, final String jobName) {
-    final String namespacedJobName = TaskUtil.getNamespacedJobName(queueName, jobName);
-    String queuePropertyPath =
-        Joiner.on("/")
-            .join(TaskConstants.REBALANCER_CONTEXT_ROOT, queueName, TaskUtil.CONTEXT_NODE);
-
-    DataUpdater<ZNRecord> updater = new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord currentData) {
-        if (currentData != null) {
-          Map<String, String> states = currentData.getMapField(WorkflowContext.JOB_STATES);
-          if (states != null && states.containsKey(namespacedJobName)) {
-            states.keySet().remove(namespacedJobName);
-          }
-        }
-        return currentData;
-      }
-    };
-    if (!_propertyStore.update(queuePropertyPath, updater, AccessOption.PERSISTENT)) {
-      LOG.warn("Fail to remove job state for job " + namespacedJobName + " from queue " + queueName);
-    }
-  }
-
-  /**
-   * Trigger a controller pipeline execution for a given resource.
-   *
-   * @param accessor Helix data accessor
-   * @param resource the name of the resource changed to triggering the execution
-   */
-  private void invokeRebalance(HelixDataAccessor accessor, String resource) {
-    // The pipeline is idempotent, so touching an ideal state is enough to trigger a pipeline run
-    LOG.info("invoke rebalance for " + resource);
-    PropertyKey key = accessor.keyBuilder().idealStates(resource);
-    IdealState is = accessor.getProperty(key);
-    if (is != null && is.getStateModelDefRef().equals(TaskConstants.STATE_MODEL_NAME)) {
-      if (!accessor.updateProperty(key, is)) {
-        LOG.warn("Failed to invoke rebalance on resource " + resource);
-      }
-    } else {
-      LOG.warn("Can't find ideal state or ideal state is not for right type for " + resource);
-    }
-  }
-
-  /** Helper function to change target state for a given workflow */
-  private void setSingleWorkflowTargetState(String workflowName, final TargetState state) {
-    LOG.info("Set " + workflowName + " to target state " + state);
-    DataUpdater<ZNRecord> updater = new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord currentData) {
-        if (currentData != null) {
-          // Only update target state for non-completed workflows
-          String finishTime = currentData.getSimpleField(WorkflowContext.FINISH_TIME);
-          if (finishTime == null || finishTime.equals(String.valueOf(WorkflowContext.UNFINISHED))) {
-            currentData.setSimpleField(WorkflowConfig.WorkflowConfigProperty.TargetState.name(),
-                state.name());
-          } else {
-            LOG.info("TargetState DataUpdater: ignore to update target state " + finishTime);
-          }
-        } else {
-          LOG.error("TargetState DataUpdater: Fails to update target state ");
-        }
-        return currentData;
-      }
-    };
-    List<DataUpdater<ZNRecord>> updaters = Lists.newArrayList();
-    List<String> paths = Lists.newArrayList();
-
-    PropertyKey cfgKey = TaskUtil.getWorkflowConfigKey(_accessor, workflowName);
-    if (_accessor.getProperty(cfgKey) != null) {
-      paths.add(_accessor.keyBuilder().resourceConfig(workflowName).getPath());
-      updaters.add(updater);
-      _accessor.updateChildren(paths, updaters, AccessOption.PERSISTENT);
-      invokeRebalance(_accessor, workflowName);
-    } else {
-      LOG.error("Configuration path " + cfgKey + " not found!");
-    }
-  }
-
-  /**
    * Delete the workflow
    *
    * @param workflow  The workflow name
    * @param timeout   The timeout for deleting the workflow/queue in seconds
    */
   public void deleteWorkflow(String workflow, long timeout) throws InterruptedException {
-    // #HELIX-0.6.7-WORKAROUND
-    // Helix 0.6.7 has a bug where TaskDriver.delete(workflow) will delete all resources with a
-    // workflow as the prefix. Work around the bug by pulling in the code from TaskDriver and calling
-    // setSingleWorkflowTargetState directly to bypass the prefix matching code.
-    setSingleWorkflowTargetState(workflow, TargetState.DELETE);
+    WorkflowConfig workflowConfig = _taskDriver.getWorkflowConfig(workflow);
+
+    // set the target state if not already set
+    if (workflowConfig != null && workflowConfig.getTargetState() != TargetState.DELETE) {
+      _taskDriver.delete(workflow);
+    }
 
     long endTime = System.currentTimeMillis() + (timeout * 1000);
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/d29b72f4/gobblin-cluster/src/main/java/org/apache/helix/task/GobblinJobRebalancer.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/helix/task/GobblinJobRebalancer.java b/gobblin-cluster/src/main/java/org/apache/helix/task/GobblinJobRebalancer.java
deleted file mode 100644
index 25241d5..0000000
--- a/gobblin-cluster/src/main/java/org/apache/helix/task/GobblinJobRebalancer.java
+++ /dev/null
@@ -1,713 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.helix.task;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import org.apache.helix.AccessOption;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.stages.ClusterDataCache;
-import org.apache.helix.controller.stages.CurrentStateOutput;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Partition;
-import org.apache.helix.model.Resource;
-import org.apache.helix.model.ResourceAssignment;
-import org.apache.log4j.Logger;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Sets;
-
-/**
- * Custom rebalancer implementation for the {@code Job} in task model.
- */
-public class GobblinJobRebalancer extends TaskRebalancer {
-  private static final Logger LOG = Logger.getLogger(GobblinJobRebalancer.class);
-  private static TaskAssignmentCalculator fixTaskAssignmentCal =
-      new FixedTargetTaskAssignmentCalculator();
-  private static TaskAssignmentCalculator genericTaskAssignmentCal =
-      new GenericTaskAssignmentCalculator();
-
-  private static final String PREV_RA_NODE = "PreviousResourceAssignment";
-
-  @Override
-  public ResourceAssignment computeBestPossiblePartitionState(ClusterDataCache clusterData,
-      IdealState taskIs, Resource resource, CurrentStateOutput currStateOutput) {
-    final String jobName = resource.getResourceName();
-    LOG.debug("Computer Best Partition for job: " + jobName);
-
-    // Fetch job configuration
-    JobConfig jobCfg = TaskUtil.getJobCfg(_manager, jobName);
-    if (jobCfg == null) {
-      LOG.error("Job configuration is NULL for " + jobName);
-      return buildEmptyAssignment(jobName, currStateOutput);
-    }
-    String workflowResource = jobCfg.getWorkflow();
-
-    // Fetch workflow configuration and context
-    WorkflowConfig workflowCfg = TaskUtil.getWorkflowCfg(_manager, workflowResource);
-    if (workflowCfg == null) {
-      LOG.error("Workflow configuration is NULL for " + jobName);
-      return buildEmptyAssignment(jobName, currStateOutput);
-    }
-
-    WorkflowContext workflowCtx = TaskUtil.getWorkflowContext(_manager, workflowResource);
-    if (workflowCtx == null) {
-      LOG.error("Workflow context is NULL for " + jobName);
-      return buildEmptyAssignment(jobName, currStateOutput);
-    }
-
-    TargetState targetState = workflowCfg.getTargetState();
-    if (targetState != TargetState.START && targetState != TargetState.STOP) {
-      LOG.info("Target state is " + targetState.name() + " for workflow " + workflowResource
-          + ".Stop scheduling job " + jobName);
-      return buildEmptyAssignment(jobName, currStateOutput);
-    }
-
-    // Stop current run of the job if workflow or job is already in final state (failed or completed)
-    TaskState workflowState = workflowCtx.getWorkflowState();
-    TaskState jobState = workflowCtx.getJobState(jobName);
-    // The job is already in a final state (completed/failed).
-    if (workflowState == TaskState.FAILED || workflowState == TaskState.COMPLETED ||
-        jobState == TaskState.FAILED || jobState == TaskState.COMPLETED) {
-      LOG.info(String.format(
-          "Workflow %s or job %s is already failed or completed, workflow state (%s), job state (%s), clean up job IS.",
-          workflowResource, jobName, workflowState, jobState));
-      cleanupIdealStateExtView(_manager.getHelixDataAccessor(), jobName);
-      _scheduledRebalancer.removeScheduledRebalance(jobName);
-      return buildEmptyAssignment(jobName, currStateOutput);
-    }
-
-    if (!isWorkflowReadyForSchedule(workflowCfg)) {
-      LOG.info("Job is not ready to be run since workflow is not ready " + jobName);
-      return buildEmptyAssignment(jobName, currStateOutput);
-    }
-
-    if (!isJobStarted(jobName, workflowCtx) && !isJobReadyToSchedule(jobName, workflowCfg,
-        workflowCtx)) {
-      LOG.info("Job is not ready to run " + jobName);
-      return buildEmptyAssignment(jobName, currStateOutput);
-    }
-
-    // Fetch any existing context information from the property store.
-    JobContext jobCtx = TaskUtil.getJobContext(_manager, jobName);
-    if (jobCtx == null) {
-      jobCtx = new JobContext(new ZNRecord("TaskContext"));
-      jobCtx.setStartTime(System.currentTimeMillis());
-    }
-
-    // Grab the old assignment, or an empty one if it doesn't exist
-    ResourceAssignment prevAssignment = getPrevResourceAssignment(jobName);
-    if (prevAssignment == null) {
-      prevAssignment = new ResourceAssignment(jobName);
-    }
-
-    // Will contain the list of partitions that must be explicitly dropped from the ideal state that
-    // is stored in zk.
-    // Fetch the previous resource assignment from the property store. This is required because of
-    // HELIX-230.
-    Set<String> liveInstances = jobCfg.getInstanceGroupTag() == null
-        ? clusterData.getAllEnabledLiveInstances()
-        : clusterData.getAllEnabledLiveInstancesWithTag(jobCfg.getInstanceGroupTag());
-
-    if (liveInstances.isEmpty()) {
-      LOG.error("No available instance found for job!");
-    }
-
-    Set<Integer> partitionsToDrop = new TreeSet<Integer>();
-    ResourceAssignment newAssignment =
-        computeResourceMapping(jobName, workflowCfg, jobCfg, prevAssignment, liveInstances,
-            currStateOutput, workflowCtx, jobCtx, partitionsToDrop, clusterData);
-
-    if (!partitionsToDrop.isEmpty()) {
-      for (Integer pId : partitionsToDrop) {
-        taskIs.getRecord().getMapFields().remove(pName(jobName, pId));
-      }
-      HelixDataAccessor accessor = _manager.getHelixDataAccessor();
-      PropertyKey propertyKey = accessor.keyBuilder().idealStates(jobName);
-      accessor.setProperty(propertyKey, taskIs);
-    }
-
-    // Update rebalancer context, previous ideal state.
-    TaskUtil.setJobContext(_manager, jobName, jobCtx);
-    TaskUtil.setWorkflowContext(_manager, workflowResource, workflowCtx);
-    setPrevResourceAssignment(jobName, newAssignment);
-
-    LOG.debug("Job " + jobName + " new assignment " + Arrays
-        .toString(newAssignment.getMappedPartitions().toArray()));
-    return newAssignment;
-  }
-
-  private Set<String> getInstancesAssignedToOtherJobs(String currentJobName,
-      WorkflowConfig workflowCfg) {
-    Set<String> ret = new HashSet<String>();
-    for (String jobName : workflowCfg.getJobDag().getAllNodes()) {
-      if (jobName.equals(currentJobName)) {
-        continue;
-      }
-      JobContext jobContext = TaskUtil.getJobContext(_manager, jobName);
-      if (jobContext == null) {
-        continue;
-      }
-      for (int partition : jobContext.getPartitionSet()) {
-        TaskPartitionState partitionState = jobContext.getPartitionState(partition);
-        if (partitionState == TaskPartitionState.INIT ||
-            partitionState == TaskPartitionState.RUNNING) {
-          ret.add(jobContext.getAssignedParticipant(partition));
-        }
-      }
-    }
-
-    return ret;
-  }
-
-  private ResourceAssignment computeResourceMapping(String jobResource,
-      WorkflowConfig workflowConfig, JobConfig jobCfg, ResourceAssignment prevAssignment,
-      Collection<String> liveInstances, CurrentStateOutput currStateOutput,
-      WorkflowContext workflowCtx, JobContext jobCtx, Set<Integer> partitionsToDropFromIs,
-      ClusterDataCache cache) {
-    TargetState jobTgtState = workflowConfig.getTargetState();
-    // Update running status in workflow context
-    if (jobTgtState == TargetState.STOP) {
-      workflowCtx.setJobState(jobResource, TaskState.STOPPED);
-      // Workflow has been stopped if all in progress jobs are stopped
-      if (isWorkflowStopped(workflowCtx, workflowConfig)) {
-        workflowCtx.setWorkflowState(TaskState.STOPPED);
-      }
-    } else {
-      workflowCtx.setJobState(jobResource, TaskState.IN_PROGRESS);
-      // Workflow is in progress if any task is in progress
-      workflowCtx.setWorkflowState(TaskState.IN_PROGRESS);
-    }
-
-    // Used to keep track of tasks that have already been assigned to instances.
-    Set<Integer> assignedPartitions = new HashSet<Integer>();
-
-    // Used to keep track of tasks that have failed, but whose failure is acceptable
-    Set<Integer> skippedPartitions = new HashSet<Integer>();
-
-    // Keeps a mapping of (partition) -> (instance, state)
-    Map<Integer, PartitionAssignment> paMap = new TreeMap<Integer, PartitionAssignment>();
-
-    // Keeps a mapping of (partition) -> (instance, state) of partitions have have been relocated
-    Map<Integer, PartitionAssignment> relocatedPaMap = new TreeMap<Integer, PartitionAssignment>();
-
-    Set<String> excludedInstances = getInstancesAssignedToOtherJobs(jobResource, workflowConfig);
-
-    // Process all the current assignments of tasks.
-    TaskAssignmentCalculator taskAssignmentCal = getAssignmentCalulator(jobCfg);
-    Set<Integer> allPartitions = taskAssignmentCal
-        .getAllTaskPartitions(jobCfg, jobCtx, workflowConfig, workflowCtx, cache.getIdealStates());
-
-    if (allPartitions == null || allPartitions.isEmpty()) {
-      // Empty target partitions, mark the job as FAILED.
-      String failureMsg = "Empty task partition mapping for job " + jobResource + ", marked the job as FAILED!";
-      LOG.info(failureMsg);
-      jobCtx.setInfo(failureMsg);
-      markJobFailed(jobResource, jobCtx, workflowConfig, workflowCtx);
-      markAllPartitionsError(jobCtx, TaskPartitionState.ERROR, false);
-      _clusterStatusMonitor.updateJobCounters(jobCfg, TaskState.FAILED);
-      return new ResourceAssignment(jobResource);
-    }
-
-    Map<String, SortedSet<Integer>> taskAssignments =
-        getTaskPartitionAssignments(liveInstances, prevAssignment, allPartitions);
-    long currentTime = System.currentTimeMillis();
-
-    LOG.debug("All partitions: " + allPartitions + " taskAssignment: " + taskAssignments
-        + " excludedInstances: " + excludedInstances);
-
-    for (Map.Entry<String, SortedSet<Integer>> entryInstance : taskAssignments.entrySet()) {
-      String instance = entryInstance.getKey();
-      if (excludedInstances.contains(instance)) {
-        continue;
-      }
-
-      Set<Integer> pSet = entryInstance.getValue();
-      // Used to keep track of partitions that are in one of the final states: COMPLETED, TIMED_OUT,
-      // TASK_ERROR, ERROR.
-      Set<Integer> donePartitions = new TreeSet<Integer>();
-      for (int pId : pSet) {
-        final String pName = pName(jobResource, pId);
-
-        // Check for pending state transitions on this (partition, instance).
-        Message pendingMessage =
-            currStateOutput.getPendingState(jobResource, new Partition(pName), instance);
-        if (pendingMessage != null) {
-          // There is a pending state transition for this (partition, instance). Just copy forward
-          // the state assignment from the previous ideal state.
-          Map<String, String> stateMap = prevAssignment.getReplicaMap(new Partition(pName));
-          if (stateMap != null) {
-            String prevState = stateMap.get(instance);
-            paMap.put(pId, new PartitionAssignment(instance, prevState));
-            assignedPartitions.add(pId);
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(String.format(
-                  "Task partition %s has a pending state transition on instance %s. Using the previous ideal state which was %s.",
-                  pName, instance, prevState));
-            }
-          }
-
-          continue;
-        }
-
-        TaskPartitionState currState =
-            TaskPartitionState.valueOf(currStateOutput.getCurrentState(jobResource, new Partition(
-                pName), instance));
-        jobCtx.setPartitionState(pId, currState);
-
-        String taskMsg = currStateOutput.getInfo(jobResource, new Partition(
-            pName), instance);
-        if (taskMsg != null) {
-          jobCtx.setPartitionInfo(pId, taskMsg);
-        }
-
-        // Process any requested state transitions.
-        String requestedStateStr =
-            currStateOutput.getRequestedState(jobResource, new Partition(pName), instance);
-        if (requestedStateStr != null && !requestedStateStr.isEmpty()) {
-          TaskPartitionState requestedState = TaskPartitionState.valueOf(requestedStateStr);
-          if (requestedState.equals(currState)) {
-            LOG.warn(String.format(
-                "Requested state %s is the same as the current state for instance %s.",
-                requestedState, instance));
-          }
-
-          paMap.put(pId, new PartitionAssignment(instance, requestedState.name()));
-          assignedPartitions.add(pId);
-          LOG.debug(String.format(
-              "Instance %s requested a state transition to %s for partition %s.", instance,
-              requestedState, pName));
-          continue;
-        }
-
-        switch (currState) {
-          case RUNNING:
-          case STOPPED: {
-            TaskPartitionState nextState;
-            if (jobTgtState == TargetState.START) {
-              nextState = TaskPartitionState.RUNNING;
-            } else {
-              nextState = TaskPartitionState.STOPPED;
-            }
-
-            paMap.put(pId, new PartitionAssignment(instance, nextState.name()));
-            assignedPartitions.add(pId);
-            LOG.debug(String.format("Setting task partition %s state to %s on instance %s.", pName,
-                nextState, instance));
-          }
-          break;
-          case COMPLETED: {
-            // The task has completed on this partition. Mark as such in the context object.
-            donePartitions.add(pId);
-            LOG.debug(String
-                .format(
-                    "Task partition %s has completed with state %s. Marking as such in rebalancer context.",
-                    pName, currState));
-            partitionsToDropFromIs.add(pId);
-            markPartitionCompleted(jobCtx, pId);
-          }
-          break;
-          case TIMED_OUT:
-          case TASK_ERROR:
-          case TASK_ABORTED:
-          case ERROR: {
-            donePartitions.add(pId); // The task may be rescheduled on a different instance.
-            LOG.debug(String.format(
-                "Task partition %s has error state %s with msg %s. Marking as such in rebalancer context.", pName,
-                currState, taskMsg));
-            markPartitionError(jobCtx, pId, currState, true);
-            // The error policy is to fail the task as soon a single partition fails for a specified
-            // maximum number of attempts or task is in ABORTED state.
-            if (jobCtx.getPartitionNumAttempts(pId) >= jobCfg.getMaxAttemptsPerTask() ||
-                currState.equals(TaskPartitionState.TASK_ABORTED)) {
-              // If we have some leeway for how many tasks we can fail, then we don't have
-              // to fail the job immediately
-              if (skippedPartitions.size() >= jobCfg.getFailureThreshold()) {
-                markJobFailed(jobResource, jobCtx, workflowConfig, workflowCtx);
-                _clusterStatusMonitor.updateJobCounters(jobCfg, TaskState.FAILED);
-                markAllPartitionsError(jobCtx, currState, false);
-                addAllPartitions(allPartitions, partitionsToDropFromIs);
-
-                // remove IdealState of this job
-                cleanupIdealStateExtView(_manager.getHelixDataAccessor(), jobResource);
-                return buildEmptyAssignment(jobResource, currStateOutput);
-              } else {
-                skippedPartitions.add(pId);
-                partitionsToDropFromIs.add(pId);
-              }
-
-              LOG.debug("skippedPartitions:" + skippedPartitions);
-            } else {
-              // Mark the task to be started at some later time (if enabled)
-              markPartitionDelayed(jobCfg, jobCtx, pId);
-            }
-          }
-          break;
-          case INIT:
-          case DROPPED: {
-            // currState in [INIT, DROPPED]. Do nothing, the partition is eligible to be reassigned.
-            donePartitions.add(pId);
-            LOG.debug(String.format(
-                "Task partition %s has state %s. It will be dropped from the current ideal state.",
-                pName, currState));
-          }
-          break;
-          default:
-            throw new AssertionError("Unknown enum symbol: " + currState);
-        }
-      }
-
-      // Remove the set of task partitions that are completed or in one of the error states.
-      pSet.removeAll(donePartitions);
-    }
-
-    // For delayed tasks, trigger a rebalance event for the closest upcoming ready time
-    scheduleForNextTask(jobResource, jobCtx, currentTime);
-
-    if (isJobComplete(jobCtx, allPartitions, skippedPartitions, jobCfg)) {
-      markJobComplete(jobResource, jobCtx, workflowConfig, workflowCtx);
-      _clusterStatusMonitor.updateJobCounters(jobCfg, TaskState.COMPLETED);
-      // remove IdealState of this job
-      cleanupIdealStateExtView(_manager.getHelixDataAccessor(), jobResource);
-    }
-
-    // Make additional task assignments if needed.
-    if (jobTgtState == TargetState.START) {
-      // Contains the set of task partitions that must be excluded from consideration when making
-      // any new assignments.
-      // This includes all completed, failed, delayed, and already assigned partitions.
-      //Set<Integer> excludeSet = Sets.newTreeSet(assignedPartitions);
-      //HACK: Modify excludeSet to be empty
-      Set<Integer> excludeSet = Sets.newTreeSet();
-      addCompletedTasks(excludeSet, jobCtx, allPartitions);
-      addGiveupPartitions(excludeSet, jobCtx, allPartitions, jobCfg);
-      excludeSet.addAll(skippedPartitions);
-      excludeSet.addAll(getNonReadyPartitions(jobCtx, currentTime));
-      // Get instance->[partition, ...] mappings for the target resource.
-      Map<String, SortedSet<Integer>> tgtPartitionAssignments = taskAssignmentCal
-          .getTaskAssignment(currStateOutput, prevAssignment, liveInstances, jobCfg, jobCtx,
-              workflowConfig, workflowCtx, allPartitions, cache.getIdealStates());
-      for (Map.Entry<String, SortedSet<Integer>> entry : taskAssignments.entrySet()) {
-        String instance = entry.getKey();
-        if (!tgtPartitionAssignments.containsKey(instance) || excludedInstances
-            .contains(instance)) {
-          continue;
-        }
-
-        // Contains the set of task partitions currently assigned to the instance.
-        Set<Integer> pSet = entry.getValue();
-        int numToAssign = jobCfg.getNumConcurrentTasksPerInstance() - pSet.size();
-        if (numToAssign > 0) {
-          List<Integer> nextPartitions =
-              getNextPartitions(tgtPartitionAssignments.get(instance), excludeSet, numToAssign);
-          for (Integer pId : nextPartitions) {
-            // if partition is not currently assigned to instance then it may have been moved
-            if (!pSet.contains(pId)) {
-              // look at current assignment to see if task is running on another instance
-              for (Map.Entry<String, SortedSet<Integer>> currentEntry : taskAssignments.entrySet()) {
-                String currentInstance = currentEntry.getKey();
-                Set<Integer> currentpSet = currentEntry.getValue();
-
-                // task is being moved, so transition to STOPPED state
-                if (!instance.equals(currentInstance) && currentpSet.contains(pId)) {
-                  relocatedPaMap.put(pId, new PartitionAssignment(currentInstance, TaskPartitionState.STOPPED.name()));
-                  break;
-                }
-              }
-            }
-
-            String pName = pName(jobResource, pId);
-            paMap.put(pId, new PartitionAssignment(instance, TaskPartitionState.RUNNING.name()));
-            excludeSet.add(pId);
-            jobCtx.setAssignedParticipant(pId, instance);
-            jobCtx.setPartitionState(pId, TaskPartitionState.INIT);
-            jobCtx.setPartitionStartTime(pId, System.currentTimeMillis());
-            LOG.debug(String.format("Setting task partition %s state to %s on instance %s.", pName,
-                TaskPartitionState.RUNNING, instance));
-          }
-        }
-      }
-    }
-
-    // Construct a ResourceAssignment object from the map of partition assignments.
-    ResourceAssignment ra = new ResourceAssignment(jobResource);
-    for (Map.Entry<Integer, PartitionAssignment> e : paMap.entrySet()) {
-      PartitionAssignment pa = e.getValue();
-
-      if (relocatedPaMap.containsKey(e.getKey())) {
-        PartitionAssignment currentPa = relocatedPaMap.get(e.getKey());
-
-        ra.addReplicaMap(new Partition(pName(jobResource, e.getKey())),
-            ImmutableMap.of(pa._instance, pa._state, currentPa._instance, currentPa._state));
-      } else {
-        ra.addReplicaMap(new Partition(pName(jobResource, e.getKey())), ImmutableMap.of(pa._instance, pa._state));
-      }
-    }
-
-    return ra;
-  }
-
-  private void markJobComplete(String jobName, JobContext jobContext,
-      WorkflowConfig workflowConfig, WorkflowContext workflowContext) {
-    long currentTime = System.currentTimeMillis();
-    workflowContext.setJobState(jobName, TaskState.COMPLETED);
-    jobContext.setFinishTime(currentTime);
-    if (isWorkflowFinished(workflowContext, workflowConfig)) {
-      workflowContext.setFinishTime(currentTime);
-    }
-  }
-
-  private void scheduleForNextTask(String job, JobContext jobCtx, long now) {
-    // Clear current entries if they exist and are expired
-    long currentTime = now;
-    long scheduledTime = _scheduledRebalancer.getRebalanceTime(job);
-    if (scheduledTime > 0 && currentTime > scheduledTime) {
-      _scheduledRebalancer.removeScheduledRebalance(job);
-    }
-
-    // Figure out the earliest schedulable time in the future of a non-complete job
-    boolean shouldSchedule = false;
-    long earliestTime = Long.MAX_VALUE;
-    for (int p : jobCtx.getPartitionSet()) {
-      long retryTime = jobCtx.getNextRetryTime(p);
-      TaskPartitionState state = jobCtx.getPartitionState(p);
-      state = (state != null) ? state : TaskPartitionState.INIT;
-      Set<TaskPartitionState> errorStates =
-          Sets.newHashSet(TaskPartitionState.ERROR, TaskPartitionState.TASK_ERROR,
-              TaskPartitionState.TIMED_OUT);
-      if (errorStates.contains(state) && retryTime > currentTime && retryTime < earliestTime) {
-        earliestTime = retryTime;
-        shouldSchedule = true;
-      }
-    }
-
-    // If any was found, then schedule it
-    if (shouldSchedule) {
-      _scheduledRebalancer.scheduleRebalance(_manager, job, earliestTime);
-    }
-  }
-
-  /**
-   * Get the last task assignment for a given job
-   *
-   * @param resourceName the name of the job
-   * @return {@link ResourceAssignment} instance, or null if no assignment is available
-   */
-  private ResourceAssignment getPrevResourceAssignment(String resourceName) {
-    ZNRecord r = _manager.getHelixPropertyStore()
-        .get(Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, resourceName, PREV_RA_NODE),
-            null, AccessOption.PERSISTENT);
-    return r != null ? new ResourceAssignment(r) : null;
-  }
-
-  /**
-   * Set the last task assignment for a given job
-   *
-   * @param resourceName the name of the job
-   * @param ra           {@link ResourceAssignment} containing the task assignment
-   */
-  private void setPrevResourceAssignment(String resourceName,
-      ResourceAssignment ra) {
-    _manager.getHelixPropertyStore()
-        .set(Joiner.on("/").join(TaskConstants.REBALANCER_CONTEXT_ROOT, resourceName, PREV_RA_NODE),
-            ra.getRecord(), AccessOption.PERSISTENT);
-  }
-
-  /**
-   * Checks if the job has completed.
-   * @param ctx The rebalancer context.
-   * @param allPartitions The set of partitions to check.
-   * @param skippedPartitions partitions that failed, but whose failure is acceptable
-   * @return true if all task partitions have been marked with status
-   *         {@link TaskPartitionState#COMPLETED} in the rebalancer
-   *         context, false otherwise.
-   */
-  private static boolean isJobComplete(JobContext ctx, Set<Integer> allPartitions,
-      Set<Integer> skippedPartitions, JobConfig cfg) {
-    for (Integer pId : allPartitions) {
-      TaskPartitionState state = ctx.getPartitionState(pId);
-      if (!skippedPartitions.contains(pId) && state != TaskPartitionState.COMPLETED
-          && !isTaskGivenup(ctx, cfg, pId)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-
-  private static void addAllPartitions(Set<Integer> toAdd, Set<Integer> destination) {
-    for (Integer pId : toAdd) {
-      destination.add(pId);
-    }
-  }
-
-  private static void addCompletedTasks(Set<Integer> set, JobContext ctx,
-      Iterable<Integer> pIds) {
-    for (Integer pId : pIds) {
-      TaskPartitionState state = ctx.getPartitionState(pId);
-      if (state == TaskPartitionState.COMPLETED) {
-        set.add(pId);
-      }
-    }
-  }
-
-  private static boolean isTaskGivenup(JobContext ctx, JobConfig cfg, int pId) {
-    TaskPartitionState state = ctx.getPartitionState(pId);
-    if ((state != null) && (state.equals(TaskPartitionState.TASK_ABORTED) || state
-        .equals(TaskPartitionState.ERROR))) {
-      return true;
-    }
-    return ctx.getPartitionNumAttempts(pId) >= cfg.getMaxAttemptsPerTask();
-  }
-
-  // add all partitions that have been tried maxNumberAttempts
-  private static void addGiveupPartitions(Set<Integer> set, JobContext ctx, Iterable<Integer> pIds,
-      JobConfig cfg) {
-    for (Integer pId : pIds) {
-      if (isTaskGivenup(ctx, cfg, pId)) {
-        set.add(pId);
-      }
-    }
-  }
-
-  private static List<Integer> getNextPartitions(SortedSet<Integer> candidatePartitions,
-      Set<Integer> excluded, int n) {
-    List<Integer> result = new ArrayList<Integer>();
-    for (Integer pId : candidatePartitions) {
-      if (result.size() >= n) {
-        break;
-      }
-
-      if (!excluded.contains(pId)) {
-        result.add(pId);
-      }
-    }
-
-    return result;
-  }
-
-  private static void markPartitionDelayed(JobConfig cfg, JobContext ctx, int p) {
-    long delayInterval = cfg.getTaskRetryDelay();
-    if (delayInterval <= 0) {
-      return;
-    }
-    long nextStartTime = ctx.getPartitionFinishTime(p) + delayInterval;
-    ctx.setNextRetryTime(p, nextStartTime);
-  }
-
-  private static void markPartitionCompleted(JobContext ctx, int pId) {
-    ctx.setPartitionState(pId, TaskPartitionState.COMPLETED);
-    ctx.setPartitionFinishTime(pId, System.currentTimeMillis());
-    ctx.incrementNumAttempts(pId);
-  }
-
-  private static void markPartitionError(JobContext ctx, int pId, TaskPartitionState state,
-      boolean incrementAttempts) {
-    ctx.setPartitionState(pId, state);
-    ctx.setPartitionFinishTime(pId, System.currentTimeMillis());
-    if (incrementAttempts) {
-      ctx.incrementNumAttempts(pId);
-    }
-  }
-
-  private static void markAllPartitionsError(JobContext ctx, TaskPartitionState state,
-      boolean incrementAttempts) {
-    for (int pId : ctx.getPartitionSet()) {
-      markPartitionError(ctx, pId, state, incrementAttempts);
-    }
-  }
-
-  /**
-   * Return the assignment of task partitions per instance.
-   */
-  private static Map<String, SortedSet<Integer>> getTaskPartitionAssignments(
-      Iterable<String> instanceList, ResourceAssignment assignment, Set<Integer> includeSet) {
-    Map<String, SortedSet<Integer>> result = new HashMap<String, SortedSet<Integer>>();
-    for (String instance : instanceList) {
-      result.put(instance, new TreeSet<Integer>());
-    }
-
-    for (Partition partition : assignment.getMappedPartitions()) {
-      int pId = TaskUtil.getPartitionId(partition.getPartitionName());
-      if (includeSet.contains(pId)) {
-        Map<String, String> replicaMap = assignment.getReplicaMap(partition);
-        for (String instance : replicaMap.keySet()) {
-          SortedSet<Integer> pList = result.get(instance);
-          if (pList != null) {
-            pList.add(pId);
-          }
-        }
-      }
-    }
-    return result;
-  }
-
-  private static Set<Integer> getNonReadyPartitions(JobContext ctx, long now) {
-    Set<Integer> nonReadyPartitions = Sets.newHashSet();
-    for (int p : ctx.getPartitionSet()) {
-      long toStart = ctx.getNextRetryTime(p);
-      if (now < toStart) {
-        nonReadyPartitions.add(p);
-      }
-    }
-    return nonReadyPartitions;
-  }
-
-  private TaskAssignmentCalculator getAssignmentCalulator(JobConfig jobConfig) {
-    Map<String, TaskConfig> taskConfigMap = jobConfig.getTaskConfigMap();
-    if (taskConfigMap != null && !taskConfigMap.isEmpty()) {
-      return genericTaskAssignmentCal;
-    } else {
-      return fixTaskAssignmentCal;
-    }
-  }
-
-  /**
-   * Computes the partition name given the resource name and partition id.
-   */
-  private String pName(String resource, int pId) {
-    return resource + "_" + pId;
-  }
-
-  /**
-   * An (instance, state) pair.
-   */
-  private static class PartitionAssignment {
-    private final String _instance;
-    private final String _state;
-
-    private PartitionAssignment(String instance, String state) {
-      _instance = instance;
-      _state = state;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/d29b72f4/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobLauncherTest.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobLauncherTest.java b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobLauncherTest.java
index aaf5f05..10ef3db 100644
--- a/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobLauncherTest.java
+++ b/gobblin-cluster/src/test/java/org/apache/gobblin/cluster/GobblinHelixJobLauncherTest.java
@@ -307,6 +307,9 @@ public class GobblinHelixJobLauncherTest {
 
     gobblinHelixJobLauncher.close();
 
+    // job queue deleted asynchronously after close
+    waitForQueueCleanup(taskDriver, jobName);
+
     jobContext = taskDriver.getJobContext(jobContextName);
 
     // job context should have been deleted
@@ -325,7 +328,9 @@ public class GobblinHelixJobLauncherTest {
 
     gobblinHelixJobLauncher2.close();
 
-    // job queue deleted after close
+    // job queue deleted asynchronously after close
+    waitForQueueCleanup(taskDriver, jobName2);
+
     workflowConfig  = taskDriver.getWorkflowConfig(jobName2);
     Assert.assertNull(workflowConfig);
 
@@ -358,4 +363,19 @@ public class GobblinHelixJobLauncherTest {
       this.closer.close();
     }
   }
+
+   private void waitForQueueCleanup(TaskDriver taskDriver, String queueName) {
+     for (int i = 0; i < 60; i++) {
+       WorkflowConfig workflowConfig  = taskDriver.getWorkflowConfig(queueName);
+
+       if (workflowConfig == null) {
+         break;
+       }
+
+       try {
+         Thread.sleep(1000);
+       } catch (InterruptedException e) {
+       }
+     }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/d29b72f4/gradle/scripts/computeVersions.gradle
----------------------------------------------------------------------
diff --git a/gradle/scripts/computeVersions.gradle b/gradle/scripts/computeVersions.gradle
index 2c05350..cf7baaa 100644
--- a/gradle/scripts/computeVersions.gradle
+++ b/gradle/scripts/computeVersions.gradle
@@ -62,7 +62,7 @@ ext.gradleVersionMajor = Integer.parseInt(gradleVersions[0])
 ext.gradleVersionMinor = Integer.parseInt(gradleVersions[1])
 println "Detected Gradle version major=" + gradleVersionMajor + " minor=" + gradleVersionMinor
 
-ext.dropwizardMetricsVersion = '3.1.0'
+ext.dropwizardMetricsVersion = '3.2.3'
 ext.findBugsVersion = '3.0.0'
 ext.googleVersion = '1.22.0'
 ext.slf4jVersion = '1.7.21'

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/d29b72f4/gradle/scripts/dependencyDefinitions.gradle
----------------------------------------------------------------------
diff --git a/gradle/scripts/dependencyDefinitions.gradle b/gradle/scripts/dependencyDefinitions.gradle
index d138a55..ad1c1cf 100644
--- a/gradle/scripts/dependencyDefinitions.gradle
+++ b/gradle/scripts/dependencyDefinitions.gradle
@@ -61,7 +61,7 @@ ext.externalDependency = [
     "hadoopYarnMiniCluster": "org.apache.hadoop:hadoop-minicluster:" + hadoopVersion,
     "hadoopAnnotations": "org.apache.hadoop:hadoop-annotations:" + hadoopVersion,
     "hadoopAws": "org.apache.hadoop:hadoop-aws:2.6.0",
-    "helix": "org.apache.helix:helix-core:0.6.7",
+    "helix": "org.apache.helix:helix-core:0.6.9",
     "hiveCommon": "org.apache.hive:hive-common:" + hiveVersion,
     "hiveService": "org.apache.hive:hive-service:" + hiveVersion,
     "hiveJdbc": "org.apache.hive:hive-jdbc:" + hiveVersion,


[23/50] incubator-gobblin git commit: postgresql extractor fixes * covering more data types * adding source.conn.database which allows you to connect to a database that has a different name than the source schema (i.e. this is 3-part naming convention).

Posted by ab...@apache.org.
postgresql extractor fixes
* covering more data types
* adding source.conn.database which allows you to connect to a database
that has a different name than the source schema (i.e. this is 3-part
naming convention).  Without this you cannot connect to a database
"workgroup" to pull the table "public.http_requests"

Closes #2152 from DepthDeluxe/master


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/34de6bfd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/34de6bfd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/34de6bfd

Branch: refs/heads/0.12.0
Commit: 34de6bfd3c06399a29937249390454fc5fa57f00
Parents: 3598d10
Author: Colin Heinzmann <ch...@linkedin.com>
Authored: Mon Feb 5 12:05:55 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Mon Feb 5 12:05:55 2018 -0800

----------------------------------------------------------------------
 .../apache/gobblin/source/jdbc/PostgresqlExtractor.java | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/34de6bfd/gobblin-modules/gobblin-sql/src/main/java/org/apache/gobblin/source/jdbc/PostgresqlExtractor.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-sql/src/main/java/org/apache/gobblin/source/jdbc/PostgresqlExtractor.java b/gobblin-modules/gobblin-sql/src/main/java/org/apache/gobblin/source/jdbc/PostgresqlExtractor.java
index 33b4e00..6876739 100644
--- a/gobblin-modules/gobblin-sql/src/main/java/org/apache/gobblin/source/jdbc/PostgresqlExtractor.java
+++ b/gobblin-modules/gobblin-sql/src/main/java/org/apache/gobblin/source/jdbc/PostgresqlExtractor.java
@@ -44,6 +44,7 @@ import lombok.extern.slf4j.Slf4j;
 
 @Slf4j
 public class PostgresqlExtractor extends JdbcExtractor {
+  private static final String CONNECTION_DATABASE = "source.conn.database";
   private static final String POSTGRES_TIMESTAMP_FORMAT = "yyyy-MM-dd HH:mm:ss";
   private static final String POSTGRES_DATE_FORMAT = "yyyy-MM-dd";
   private static final String POSTGRES_HOUR_FORMAT = "HH";
@@ -168,7 +169,8 @@ public class PostgresqlExtractor extends JdbcExtractor {
   public String getConnectionUrl() {
     String host = this.workUnitState.getProp(ConfigurationKeys.SOURCE_CONN_HOST_NAME);
     String port = this.workUnitState.getProp(ConfigurationKeys.SOURCE_CONN_PORT);
-    String database = this.workUnitState.getProp(ConfigurationKeys.SOURCE_QUERYBASED_SCHEMA);
+    String database = this.workUnitState.getProp(CONNECTION_DATABASE);
+
     return "jdbc:postgresql://" + host.trim() + ":" + port + "/" + database.trim();
   }
 
@@ -182,13 +184,15 @@ public class PostgresqlExtractor extends JdbcExtractor {
   public Map<String, String> getDataTypeMap() {
     Map<String, String> dataTypeMap =
         ImmutableMap.<String, String>builder().put("tinyint", "int").put("smallint", "int").put("mediumint", "int")
-            .put("int", "int").put("bigint", "long").put("float", "float").put("double", "double")
-            .put("decimal", "double").put("numeric", "double").put("date", "date").put("timestamp", "timestamp")
+            .put("integer", "int").put("int", "int").put("bigint", "long").put("float", "float").put("double", "double")
+            .put("double precision", "double").put("decimal", "double").put("numeric", "double").put("date", "date").put("timestamp", "timestamp")
+            .put("timestamp without time zone", "timestamp").put("timestamp with time zone", "timestamp")
             .put("datetime", "timestamp").put("time", "time").put("char", "string").put("varchar", "string")
             .put("varbinary", "string").put("text", "string").put("tinytext", "string").put("mediumtext", "string")
-            .put("longtext", "string").put("blob", "string").put("tinyblob", "string").put("mediumblob", "string")
+            .put("character varying", "string").put("longtext", "string").put("blob", "string").put("tinyblob", "string").put("mediumblob", "string")
             .put("longblob", "string").put("enum", "string").build();
     return dataTypeMap;
+
   }
 
   @Override


[14/50] incubator-gobblin git commit: [GOBBLIN-401] Provide a constructor for CombineSelectionPolicy with only the selection config as argument[]

Posted by ab...@apache.org.
[GOBBLIN-401] Provide a constructor for CombineSelectionPolicy with only the selection config as argument[]

Closes #2275 from sv2000/gobblin-401


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/8879cdec
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/8879cdec
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/8879cdec

Branch: refs/heads/0.12.0
Commit: 8879cdec2680e9e748d7e89f5cef5d72110c1c07
Parents: fd3a547
Author: suvasude <su...@linkedin.biz>
Authored: Fri Feb 2 07:54:44 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Fri Feb 2 07:54:44 2018 -0800

----------------------------------------------------------------------
 .../policy/CombineSelectionPolicy.java          | 24 ++++++++++++--------
 1 file changed, 14 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/8879cdec/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/policy/CombineSelectionPolicy.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/policy/CombineSelectionPolicy.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/policy/CombineSelectionPolicy.java
index b6377a0..c3c70f8 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/policy/CombineSelectionPolicy.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/policy/CombineSelectionPolicy.java
@@ -71,13 +71,17 @@ public class CombineSelectionPolicy implements VersionSelectionPolicy<DatasetVer
   public static final String VERSION_SELECTION_COMBINE_OPERATION = "selection.combine.operation";
 
   public enum CombineOperation {
-    INTERSECT,
-    UNION
+    INTERSECT, UNION
   }
 
   private final List<VersionSelectionPolicy<DatasetVersion>> selectionPolicies;
   private final CombineOperation combineOperation;
 
+  public CombineSelectionPolicy(Config config)
+      throws IOException {
+    this(config, new Properties());
+  }
+
   public CombineSelectionPolicy(List<VersionSelectionPolicy<DatasetVersion>> selectionPolicies,
       CombineOperation combineOperation) {
     this.combineOperation = combineOperation;
@@ -85,17 +89,18 @@ public class CombineSelectionPolicy implements VersionSelectionPolicy<DatasetVer
   }
 
   @SuppressWarnings("unchecked")
-  public CombineSelectionPolicy(Config config, Properties jobProps) throws IOException {
+  public CombineSelectionPolicy(Config config, Properties jobProps)
+      throws IOException {
     Preconditions.checkArgument(config.hasPath(VERSION_SELECTION_POLICIES_PREFIX), "Combine operation not specified.");
 
     ImmutableList.Builder<VersionSelectionPolicy<DatasetVersion>> builder = ImmutableList.builder();
 
     for (String combineClassName : config.getStringList(VERSION_SELECTION_POLICIES_PREFIX)) {
       try {
-        builder.add((VersionSelectionPolicy<DatasetVersion>) GobblinConstructorUtils.invokeFirstConstructor(
-            Class.forName(combineClassName), ImmutableList.<Object> of(config), ImmutableList.<Object> of(jobProps)));
-      } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException
-          | ClassNotFoundException e) {
+        builder.add((VersionSelectionPolicy<DatasetVersion>) GobblinConstructorUtils
+            .invokeFirstConstructor(Class.forName(combineClassName), ImmutableList.<Object>of(config),
+                ImmutableList.<Object>of(jobProps)));
+      } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException | ClassNotFoundException e) {
         throw new IllegalArgumentException(e);
       }
     }
@@ -109,9 +114,9 @@ public class CombineSelectionPolicy implements VersionSelectionPolicy<DatasetVer
         CombineOperation.valueOf(config.getString(VERSION_SELECTION_COMBINE_OPERATION).toUpperCase());
   }
 
-  public CombineSelectionPolicy(Properties props) throws IOException {
+  public CombineSelectionPolicy(Properties props)
+      throws IOException {
     this(ConfigFactory.parseProperties(props), props);
-
   }
 
   /**
@@ -150,7 +155,6 @@ public class CombineSelectionPolicy implements VersionSelectionPolicy<DatasetVer
       default:
         throw new RuntimeException("Combine operation " + this.combineOperation + " not recognized.");
     }
-
   }
 
   @VisibleForTesting


[27/50] incubator-gobblin git commit: [GOBBLIN-396] add date partitioned based json source

Posted by ab...@apache.org.
[GOBBLIN-396] add date partitioned based json source

Closes #2270 from
arjun4084346/jsonDatePartitionedSource


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/5c678d9b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/5c678d9b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/5c678d9b

Branch: refs/heads/0.12.0
Commit: 5c678d9b6b008c9fef0eeea731f2bf19e55e1cea
Parents: 457ede2
Author: Arjun <ab...@linkedin.com>
Authored: Thu Feb 8 11:40:19 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Thu Feb 8 11:40:19 2018 -0800

----------------------------------------------------------------------
 .../configuration/ConfigurationKeys.java        |   6 +
 .../JsonRecordAvroSchemaToAvroConverter.java    |  11 +-
 .../source/DatePartitionedJsonFileSource.java   |  39 ++++++
 .../source/DatePartitionedNestedRetriever.java  |   9 +-
 .../source/PartitionedFileSourceBase.java       |  20 +++-
 .../source/RegexBasedPartitionedRetriever.java  |   9 +-
 .../DatePartitionedJsonFileExtractor.java       |  30 +++++
 .../source/extractor/SimpleJsonExtractor.java   | 118 +++++++++++++++++++
 ...JsonRecordAvroSchemaToAvroConverterTest.java |   5 +-
 .../filebased/FileBasedSourceTest.java          |  23 ++++
 .../test/resources/source/2017-12/metadata.json |   1 +
 .../resources/source/2017-12/simplejson.json    |   3 +
 .../test/resources/source/2018-01/metadata.json |   1 +
 .../resources/source/2018-01/simplejson.json    |   3 +
 .../resources/source/2018-01/simplejson2.json   |   3 +
 15 files changed, 268 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
index 267a17e..d07d740 100644
--- a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
+++ b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
@@ -302,6 +302,8 @@ public class ConfigurationKeys {
   public static final String CONVERTER_AVRO_FIELD_PICK_FIELDS = "converter.avro.fields";
   public static final String CONVERTER_AVRO_JDBC_ENTRY_FIELDS_PAIRS = "converter.avro.jdbc.entry_fields_pairs";
   public static final String CONVERTER_SKIP_FAILED_RECORD = "converter.skipFailedRecord";
+  public static final String CONVERTER_AVRO_SCHEMA_KEY = "converter.avroSchema";
+  public static final String CONVERTER_IGNORE_FIELDS = "converter.ignoreFields";
 
   /**
    * Fork operator configuration properties.
@@ -452,6 +454,10 @@ public class ConfigurationKeys {
    * Configuration properties used by the extractor.
    */
   public static final String SOURCE_ENTITY = "source.entity";
+  public static final String SCHEMA_IN_SOURCE_DIR = "schema.in.source.dir";
+  public static final boolean DEFAULT_SCHEMA_IN_SOURCE_DIR = false;
+  public static final String SCHEMA_FILENAME = "schema.filename";
+  public static final String DEFAULT_SCHEMA_FILENAME = "metadata.json";
 
   // Comma-separated source entity names
   public static final String SOURCE_ENTITIES = "source.entities";

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverter.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverter.java b/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverter.java
index 11f85f4..8e25975 100644
--- a/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverter.java
+++ b/gobblin-core/src/main/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverter.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericRecord;
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.WorkUnitState;
 import org.apache.gobblin.converter.Converter;
 import org.apache.gobblin.converter.DataConversionException;
@@ -42,18 +43,12 @@ import com.google.common.base.Splitter;
 public class JsonRecordAvroSchemaToAvroConverter<SI> extends ToAvroConverterBase<SI, JsonObject> {
 
   private static final Splitter SPLITTER_ON_COMMA = Splitter.on(',').trimResults().omitEmptyStrings();
-
-  public static final String AVRO_SCHEMA_KEY = "converter.avroSchema";
-  public static final String IGNORE_FIELDS = "converter.ignoreFields";
-
   private Schema schema;
   private List<String> ignoreFields;
 
   public ToAvroConverterBase<SI, JsonObject> init(WorkUnitState workUnit) {
     super.init(workUnit);
-    Preconditions.checkArgument(workUnit.contains(AVRO_SCHEMA_KEY));
-    this.schema = new Schema.Parser().parse(workUnit.getProp(AVRO_SCHEMA_KEY));
-    this.ignoreFields = SPLITTER_ON_COMMA.splitToList(workUnit.getProp(IGNORE_FIELDS, ""));
+    this.ignoreFields = SPLITTER_ON_COMMA.splitToList(workUnit.getProp(ConfigurationKeys.CONVERTER_IGNORE_FIELDS, ""));
     return this;
   }
 
@@ -62,6 +57,8 @@ public class JsonRecordAvroSchemaToAvroConverter<SI> extends ToAvroConverterBase
    */
   @Override
   public Schema convertSchema(SI inputSchema, WorkUnitState workUnit) throws SchemaConversionException {
+    Preconditions.checkArgument(workUnit.contains(ConfigurationKeys.CONVERTER_AVRO_SCHEMA_KEY));
+    this.schema = new Schema.Parser().parse(workUnit.getProp(ConfigurationKeys.CONVERTER_AVRO_SCHEMA_KEY));
     return this.schema;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/main/java/org/apache/gobblin/source/DatePartitionedJsonFileSource.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/source/DatePartitionedJsonFileSource.java b/gobblin-core/src/main/java/org/apache/gobblin/source/DatePartitionedJsonFileSource.java
new file mode 100644
index 0000000..cc5167a
--- /dev/null
+++ b/gobblin-core/src/main/java/org/apache/gobblin/source/DatePartitionedJsonFileSource.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.source;
+
+import java.io.IOException;
+
+import org.apache.gobblin.configuration.WorkUnitState;
+import org.apache.gobblin.source.extractor.DatePartitionedJsonFileExtractor;
+import org.apache.gobblin.source.extractor.Extractor;
+
+import com.google.gson.JsonObject;
+
+public class DatePartitionedJsonFileSource extends PartitionedFileSourceBase<String, JsonObject> {
+
+  public DatePartitionedJsonFileSource() {
+    super(new DatePartitionedNestedRetriever(".json"));
+  }
+
+  @Override
+  public Extractor<String, JsonObject> getExtractor(WorkUnitState state)
+      throws IOException {
+    return new DatePartitionedJsonFileExtractor(state);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/main/java/org/apache/gobblin/source/DatePartitionedNestedRetriever.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/source/DatePartitionedNestedRetriever.java b/gobblin-core/src/main/java/org/apache/gobblin/source/DatePartitionedNestedRetriever.java
index a9ff257..4c33555 100644
--- a/gobblin-core/src/main/java/org/apache/gobblin/source/DatePartitionedNestedRetriever.java
+++ b/gobblin-core/src/main/java/org/apache/gobblin/source/DatePartitionedNestedRetriever.java
@@ -72,6 +72,8 @@ public class DatePartitionedNestedRetriever implements PartitionAwareFileRetriev
   private HadoopFsHelper helper;
   private final String expectedExtension;
   private Duration leadTimeDuration;
+  private boolean schemaInSourceDir;
+  private String schemaFile;
 
   public DatePartitionedNestedRetriever(String expectedExtension) {
     this.expectedExtension = expectedExtension;
@@ -91,6 +93,10 @@ public class DatePartitionedNestedRetriever implements PartitionAwareFileRetriev
     this.sourceDir = new Path(state.getProp(ConfigurationKeys.SOURCE_FILEBASED_DATA_DIRECTORY));
     this.leadTimeDuration = PartitionAwareFileRetrieverUtils.getLeadTimeDurationFromConfig(state);
     this.helper = new HadoopFsHelper(state);
+    this.schemaInSourceDir = state.getPropAsBoolean(ConfigurationKeys.SCHEMA_IN_SOURCE_DIR,
+        ConfigurationKeys.DEFAULT_SCHEMA_IN_SOURCE_DIR);
+    this.schemaFile = this.schemaInSourceDir ? state.getProp(ConfigurationKeys.SCHEMA_FILENAME,
+        ConfigurationKeys.DEFAULT_SCHEMA_FILENAME) : "";
   }
 
   @Override
@@ -201,7 +207,8 @@ public class DatePartitionedNestedRetriever implements PartitionAwareFileRetriev
     return new PathFilter() {
       @Override
       public boolean accept(Path path) {
-        return path.getName().endsWith(extension);
+        return path.getName().endsWith(extension) &&
+            !(schemaInSourceDir && path.getName().equals(schemaFile)) ;
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/main/java/org/apache/gobblin/source/PartitionedFileSourceBase.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/source/PartitionedFileSourceBase.java b/gobblin-core/src/main/java/org/apache/gobblin/source/PartitionedFileSourceBase.java
index 9ec7707..1b54895 100644
--- a/gobblin-core/src/main/java/org/apache/gobblin/source/PartitionedFileSourceBase.java
+++ b/gobblin-core/src/main/java/org/apache/gobblin/source/PartitionedFileSourceBase.java
@@ -69,7 +69,7 @@ public abstract class PartitionedFileSourceBase<SCHEMA, DATA> extends FileBasedS
   public static final String DATE_PARTITIONED_SOURCE_PARTITION_SUFFIX =
       DATE_PARTITIONED_SOURCE_PREFIX + ".partition.suffix";
 
-  static final String DATE_PARTITIONED_SOURCE_PARTITION_PATTERN =
+  public static final String DATE_PARTITIONED_SOURCE_PARTITION_PATTERN =
       DATE_PARTITIONED_SOURCE_PREFIX + ".partition.pattern";
 
   public static final String DATE_PARTITIONED_SOURCE_PARTITION_GRANULARITY =
@@ -99,7 +99,7 @@ public abstract class PartitionedFileSourceBase<SCHEMA, DATA> extends FileBasedS
   * If this parameter is not specified the job will start reading data from
   * the beginning of Unix time.
   */
-  private static final String DATE_PARTITIONED_SOURCE_MIN_WATERMARK_VALUE =
+  public static final String DATE_PARTITIONED_SOURCE_MIN_WATERMARK_VALUE =
       DATE_PARTITIONED_SOURCE_PREFIX + ".min.watermark.value";
 
   /**
@@ -291,6 +291,11 @@ public abstract class PartitionedFileSourceBase<SCHEMA, DATA> extends FileBasedS
         singleWorkUnit.setProp(ConfigurationKeys.WORK_UNIT_HIGH_WATER_MARK_KEY, file.getWatermarkMsSinceEpoch());
         singleWorkUnit.setProp(ConfigurationKeys.WORK_UNIT_DATE_PARTITION_KEY, file.getWatermarkMsSinceEpoch());
 
+        if (this.sourceState.getPropAsBoolean(ConfigurationKeys.SCHEMA_IN_SOURCE_DIR,
+            ConfigurationKeys.DEFAULT_SCHEMA_IN_SOURCE_DIR)) {
+          addSchemaFile(file, singleWorkUnit);
+        }
+
         multiWorkUnitWeightedQueue.addWorkUnit(singleWorkUnit, file.getFileSize());
 
         this.fileCount++;
@@ -302,6 +307,17 @@ public abstract class PartitionedFileSourceBase<SCHEMA, DATA> extends FileBasedS
     }
   }
 
+  private void addSchemaFile(PartitionAwareFileRetriever.FileInfo dataFile, WorkUnit workUnit)
+      throws IOException {
+    Path schemaFile = new Path(new Path(dataFile.getFilePath()).getParent(),
+        workUnit.getProp(ConfigurationKeys.SCHEMA_FILENAME, ConfigurationKeys.DEFAULT_SCHEMA_FILENAME));
+    if (fs.exists(schemaFile)) {
+      workUnit.setProp(ConfigurationKeys.SOURCE_SCHEMA, schemaFile.toString());
+    } else {
+      throw new IOException("Schema file " + schemaFile + " does not exist.");
+    }
+  }
+
   /**
    * Gets the LWM for this job runs. The new LWM is the HWM of the previous run + 1 unit (day,hour,minute..etc).
    * If there was no previous execution then it is set to the given lowWaterMark + 1 unit.

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/main/java/org/apache/gobblin/source/RegexBasedPartitionedRetriever.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/source/RegexBasedPartitionedRetriever.java b/gobblin-core/src/main/java/org/apache/gobblin/source/RegexBasedPartitionedRetriever.java
index e082bda..7d3ad92 100644
--- a/gobblin-core/src/main/java/org/apache/gobblin/source/RegexBasedPartitionedRetriever.java
+++ b/gobblin-core/src/main/java/org/apache/gobblin/source/RegexBasedPartitionedRetriever.java
@@ -48,6 +48,8 @@ public class RegexBasedPartitionedRetriever implements PartitionAwareFileRetriev
   private Path sourceDir;
   private final String expectedExtension;
   private Duration leadTime;
+  private boolean schemaInSourceDir;
+  private String schemaFile;
 
   public RegexBasedPartitionedRetriever(String expectedExtension) {
     this.expectedExtension = expectedExtension;
@@ -64,6 +66,10 @@ public class RegexBasedPartitionedRetriever implements PartitionAwareFileRetriev
     this.pattern = Pattern.compile(regexPattern);
     this.helper = new HadoopFsHelper(state);
     this.sourceDir = new Path(state.getProp(ConfigurationKeys.SOURCE_FILEBASED_DATA_DIRECTORY));
+    this.schemaInSourceDir = state.getPropAsBoolean(ConfigurationKeys.SCHEMA_IN_SOURCE_DIR,
+        ConfigurationKeys.DEFAULT_SCHEMA_IN_SOURCE_DIR);
+    this.schemaFile = this.schemaInSourceDir ? state.getProp(ConfigurationKeys.SCHEMA_FILENAME,
+        ConfigurationKeys.DEFAULT_SCHEMA_FILENAME) : "";
   }
 
   @Override
@@ -175,7 +181,8 @@ public class RegexBasedPartitionedRetriever implements PartitionAwareFileRetriev
     return new PathFilter() {
       @Override
       public boolean accept(Path path) {
-        return path.getName().endsWith(extension);
+        return path.getName().endsWith(extension) &&
+            !(schemaInSourceDir && path.getName().equals(schemaFile)) ;
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/DatePartitionedJsonFileExtractor.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/DatePartitionedJsonFileExtractor.java b/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/DatePartitionedJsonFileExtractor.java
new file mode 100644
index 0000000..b165a35
--- /dev/null
+++ b/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/DatePartitionedJsonFileExtractor.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.source.extractor;
+
+import java.io.IOException;
+
+import org.apache.gobblin.configuration.WorkUnitState;
+
+public class DatePartitionedJsonFileExtractor extends SimpleJsonExtractor {
+
+  public DatePartitionedJsonFileExtractor(WorkUnitState workUnitState)
+      throws IOException {
+    super(workUnitState);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/SimpleJsonExtractor.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/SimpleJsonExtractor.java b/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/SimpleJsonExtractor.java
new file mode 100644
index 0000000..412a06e
--- /dev/null
+++ b/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/SimpleJsonExtractor.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.source.extractor;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import org.apache.gobblin.source.extractor.hadoop.HadoopFsHelper;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.io.Closer;
+import com.google.gson.JsonObject;
+import com.google.gson.Gson;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.configuration.WorkUnitState;
+
+/**
+ * An implementation of {@link Extractor}.
+ *
+ * <p>
+ *   This extractor reads the assigned input file storing
+ *   json documents confirming to a schema. Each line of the file is a json document.
+ * </p>
+ */
+public class SimpleJsonExtractor implements Extractor<String, JsonObject> {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SimpleJsonExtractor.class);
+  private final WorkUnitState workUnitState;
+  private final FileSystem fs;
+  private final BufferedReader bufferedReader;
+  private final Closer closer = Closer.create();
+  private static final Gson GSON = new Gson();
+
+  public SimpleJsonExtractor(WorkUnitState workUnitState) throws IOException {
+    this.workUnitState = workUnitState;
+
+    HadoopFsHelper fsHelper = new HadoopFsHelper(workUnitState);
+    try {
+      fsHelper.connect();
+    } catch (Exception e) {
+      throw new IOException("Exception at SimpleJsonExtractor");
+    }
+    // Source is responsible to set SOURCE_FILEBASED_FILES_TO_PULL
+    this.fs = fsHelper.getFileSystem();
+    InputStreamReader isr = new InputStreamReader(this.fs.open(
+        new Path(workUnitState.getProp(ConfigurationKeys.SOURCE_FILEBASED_FILES_TO_PULL))), StandardCharsets.UTF_8);
+
+    this.bufferedReader =
+        this.closer.register(new BufferedReader(isr));
+  }
+
+  @Override
+  public String getSchema() throws IOException {
+    // Source is responsible to set SOURCE_SCHEMA
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    IOUtils.copyBytes(fs.open(
+        new Path(workUnitState.getProp(ConfigurationKeys.SOURCE_SCHEMA))), outputStream, 4096, false);
+    String schema = new String(outputStream.toByteArray(), StandardCharsets.UTF_8);
+    workUnitState.setProp((ConfigurationKeys.CONVERTER_AVRO_SCHEMA_KEY), schema);
+    return schema;
+  }
+
+  @Override
+  public JsonObject readRecord(@Deprecated JsonObject reuse) throws DataRecordException, IOException {
+    String jsonString = this.bufferedReader.readLine();
+    return GSON.fromJson(jsonString, JsonObject.class);
+  }
+
+  @Override
+  public long getExpectedRecordCount() {
+    // We don't know how many records are in the file before actually reading them
+    return 0;
+  }
+
+  @Override
+  public long getHighWatermark() {
+    // Watermark is not applicable for this type of extractor
+    return 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      this.closer.close();
+    } catch (IOException ioe) {
+      LOGGER.error("Failed to close the input stream", ioe);
+    }
+
+    try {
+      fs.close();
+    } catch (IOException ioe) {
+      LOGGER.error("Failed to close the file object", ioe);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/test/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverterTest.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverterTest.java b/gobblin-core/src/test/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverterTest.java
index 9971d83..4cf6898 100644
--- a/gobblin-core/src/test/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverterTest.java
+++ b/gobblin-core/src/test/java/org/apache/gobblin/converter/avro/JsonRecordAvroSchemaToAvroConverterTest.java
@@ -23,6 +23,7 @@ import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericArray;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.commons.io.IOUtils;
+import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.SourceState;
 import org.apache.gobblin.configuration.WorkUnitState;
 import org.apache.gobblin.source.workunit.Extract.TableType;
@@ -52,8 +53,8 @@ public class JsonRecordAvroSchemaToAvroConverterTest {
     SourceState source = new SourceState();
     this.state = new WorkUnitState(
         source.createWorkUnit(source.createExtract(TableType.SNAPSHOT_ONLY, "test_table", "test_namespace")));
-    this.state.setProp(JsonRecordAvroSchemaToAvroConverter.AVRO_SCHEMA_KEY, avroSchemaString);
-    this.state.setProp(JsonRecordAvroSchemaToAvroConverter.IGNORE_FIELDS, "fieldToIgnore");
+    this.state.setProp(ConfigurationKeys.CONVERTER_AVRO_SCHEMA_KEY, avroSchemaString);
+    this.state.setProp(ConfigurationKeys.CONVERTER_IGNORE_FIELDS, "fieldToIgnore");
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/test/java/org/apache/gobblin/source/extractor/filebased/FileBasedSourceTest.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/java/org/apache/gobblin/source/extractor/filebased/FileBasedSourceTest.java b/gobblin-core/src/test/java/org/apache/gobblin/source/extractor/filebased/FileBasedSourceTest.java
index 8cceff2..95b3656 100644
--- a/gobblin-core/src/test/java/org/apache/gobblin/source/extractor/filebased/FileBasedSourceTest.java
+++ b/gobblin-core/src/test/java/org/apache/gobblin/source/extractor/filebased/FileBasedSourceTest.java
@@ -21,10 +21,13 @@ import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.SourceState;
 import org.apache.gobblin.configuration.State;
 import org.apache.gobblin.configuration.WorkUnitState;
+import org.apache.gobblin.source.DatePartitionedJsonFileSource;
+import org.apache.gobblin.source.PartitionedFileSourceBase;
 import org.apache.gobblin.source.extractor.DataRecordException;
 import org.apache.gobblin.source.extractor.Extractor;
 import org.apache.gobblin.source.workunit.Extract;
 import org.apache.gobblin.source.workunit.WorkUnit;
+import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
 import org.testng.annotations.Test;
 import org.testng.collections.Lists;
@@ -32,6 +35,7 @@ import org.testng.collections.Lists;
 import java.io.IOException;
 import java.util.List;
 
+
 @Test
 public class FileBasedSourceTest {
     @Test
@@ -57,6 +61,25 @@ public class FileBasedSourceTest {
         }
     }
 
+    @Test void numberOfWorkUnits() throws IOException {
+        SourceState sourceState = new SourceState();
+        DatePartitionedJsonFileSource source = new DatePartitionedJsonFileSource();
+        initState(sourceState);
+        List<WorkUnit> workUnits = source.getWorkunits(sourceState);
+        Assert.assertEquals(3, workUnits.size());
+    }
+
+    private void initState(State state) {
+        state.setProp(ConfigurationKeys.SOURCE_FILEBASED_DATA_DIRECTORY,
+            new Path(getClass().getResource("/source").toString()).toString());
+        state.setProp(PartitionedFileSourceBase.DATE_PARTITIONED_SOURCE_PARTITION_PATTERN, "yyyy-MM");
+        state.setProp(PartitionedFileSourceBase.DATE_PARTITIONED_SOURCE_MIN_WATERMARK_VALUE, "2017-11");
+        state.setProp(ConfigurationKeys.EXTRACT_TABLE_TYPE_KEY, "snapshot_only");
+        state.setProp(ConfigurationKeys.SOURCE_FILEBASED_FS_URI, "file:///");
+        state.setProp(ConfigurationKeys.SCHEMA_IN_SOURCE_DIR, "true");
+        state.setProp(ConfigurationKeys.SCHEMA_FILENAME, "metadata.json");
+    }
+
     private static class DummyFileBasedSource extends FileBasedSource<String, String> {
         @Override
         public void initFileSystemHelper(State state) throws FileBasedHelperException {

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/test/resources/source/2017-12/metadata.json
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/source/2017-12/metadata.json b/gobblin-core/src/test/resources/source/2017-12/metadata.json
new file mode 100644
index 0000000..0003f63
--- /dev/null
+++ b/gobblin-core/src/test/resources/source/2017-12/metadata.json
@@ -0,0 +1 @@
+{"namespace":"example.avro", "type":"record", "name":"User", "fields":[{"name":"name", "type":"string"}, {"name":"favorite_number",  "type":"int"}, {"name":"favorite_color", "type":"string"}]}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/test/resources/source/2017-12/simplejson.json
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/source/2017-12/simplejson.json b/gobblin-core/src/test/resources/source/2017-12/simplejson.json
new file mode 100644
index 0000000..c325df0
--- /dev/null
+++ b/gobblin-core/src/test/resources/source/2017-12/simplejson.json
@@ -0,0 +1,3 @@
+{"name": "Alyssa", "favorite_number": 256, "favorite_color": "yellow"}
+{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
+{"name": "Charlie", "favorite_number": 68, "favorite_color": "blue"}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/test/resources/source/2018-01/metadata.json
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/source/2018-01/metadata.json b/gobblin-core/src/test/resources/source/2018-01/metadata.json
new file mode 100644
index 0000000..0003f63
--- /dev/null
+++ b/gobblin-core/src/test/resources/source/2018-01/metadata.json
@@ -0,0 +1 @@
+{"namespace":"example.avro", "type":"record", "name":"User", "fields":[{"name":"name", "type":"string"}, {"name":"favorite_number",  "type":"int"}, {"name":"favorite_color", "type":"string"}]}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/test/resources/source/2018-01/simplejson.json
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/source/2018-01/simplejson.json b/gobblin-core/src/test/resources/source/2018-01/simplejson.json
new file mode 100644
index 0000000..c325df0
--- /dev/null
+++ b/gobblin-core/src/test/resources/source/2018-01/simplejson.json
@@ -0,0 +1,3 @@
+{"name": "Alyssa", "favorite_number": 256, "favorite_color": "yellow"}
+{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
+{"name": "Charlie", "favorite_number": 68, "favorite_color": "blue"}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/5c678d9b/gobblin-core/src/test/resources/source/2018-01/simplejson2.json
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/source/2018-01/simplejson2.json b/gobblin-core/src/test/resources/source/2018-01/simplejson2.json
new file mode 100644
index 0000000..c325df0
--- /dev/null
+++ b/gobblin-core/src/test/resources/source/2018-01/simplejson2.json
@@ -0,0 +1,3 @@
+{"name": "Alyssa", "favorite_number": 256, "favorite_color": "yellow"}
+{"name": "Ben", "favorite_number": 7, "favorite_color": "red"}
+{"name": "Charlie", "favorite_number": 68, "favorite_color": "blue"}


[45/50] incubator-gobblin git commit: [GOBBLIN-422] Update fs snapshot in previously failed workunits with the current effectiveSnapshot

Posted by ab...@apache.org.
[GOBBLIN-422] Update fs snapshot in previously failed workunits with the current effectiveSnapshot

Closes #2299 from ragepati/ragepati-
filebasedsource-prevfssnapshot


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/979ad2a0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/979ad2a0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/979ad2a0

Branch: refs/heads/0.12.0
Commit: 979ad2a090600495ad3b47de462d39b2f2ab33ea
Parents: 8636b0c
Author: Raul Agepati <ra...@users.noreply.github.com>
Authored: Mon Mar 5 15:05:10 2018 -0800
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Mon Mar 5 15:05:10 2018 -0800

----------------------------------------------------------------------
 .../gobblin/source/extractor/filebased/FileBasedSource.java     | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/979ad2a0/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/filebased/FileBasedSource.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/filebased/FileBasedSource.java b/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/filebased/FileBasedSource.java
index 46a0de0..e34a28f 100644
--- a/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/filebased/FileBasedSource.java
+++ b/gobblin-core/src/main/java/org/apache/gobblin/source/extractor/filebased/FileBasedSource.java
@@ -152,6 +152,11 @@ public abstract class FileBasedSource<S, D> extends AbstractSource<S, D> {
         // file is not pulled this run
       }
     }
+    // Update the snapshot from the previous run with the new files processed in this run
+    // Otherwise a corrupt file could cause re-processing of already processed files
+    for (WorkUnit workUnit : previousWorkUnitsForRetry) {
+      workUnit.setProp(ConfigurationKeys.SOURCE_FILEBASED_FS_SNAPSHOT, StringUtils.join(effectiveSnapshot, ","));
+    }
 
     if (!filesToPull.isEmpty()) {
       logFilesToPull(filesToPull);


[15/50] incubator-gobblin git commit: [GOBBLIN-402] Add more metrics for gobblin cluster and fix the getJobs slowness issue

Posted by ab...@apache.org.
[GOBBLIN-402] Add more metrics for gobblin cluster and fix the getJobs slowness issue

Closes #2276 from yukuai518/morem


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/de83a3fb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/de83a3fb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/de83a3fb

Branch: refs/heads/0.12.0
Commit: de83a3fb5f2644e9657b546c7415d1e8f5336a2c
Parents: 8879cde
Author: Kuai Yu <ku...@linkedin.com>
Authored: Fri Feb 2 13:25:53 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Fri Feb 2 13:25:53 2018 -0800

----------------------------------------------------------------------
 .../gobblin/cluster/GobblinClusterManager.java  |  6 +-
 .../cluster/GobblinHelixJobScheduler.java       | 33 ++++++-
 .../StreamingJobConfigurationManager.java       |  3 +
 .../service/StreamingKafkaSpecConsumer.java     | 92 +++++++++++++++++++-
 .../apache/gobblin/runtime/api/JobCatalog.java  |  7 +-
 .../runtime/api/JobExecutionLauncher.java       |  6 ++
 .../runtime/job_monitor/KafkaJobMonitor.java    |  3 +
 7 files changed, 139 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/de83a3fb/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
index 77e511e..3393df6 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterManager.java
@@ -153,6 +153,9 @@ public class GobblinClusterManager implements ApplicationLauncher, StandardMetri
   private MutableJobCatalog jobCatalog;
   @Getter
   private GobblinHelixJobScheduler jobScheduler;
+  @Getter
+  private JobConfigurationManager jobConfigurationManager;
+  
   private final String clusterName;
   private final Config config;
   private final MetricContext metricContext;
@@ -209,7 +212,8 @@ public class GobblinClusterManager implements ApplicationLauncher, StandardMetri
     this.jobScheduler = buildGobblinHelixJobScheduler(config, this.appWorkDir, getMetadataTags(clusterName, applicationId),
         schedulerService);
     this.applicationLauncher.addService(this.jobScheduler);
-    this.applicationLauncher.addService(buildJobConfigurationManager(config));
+    this.jobConfigurationManager = buildJobConfigurationManager(config);
+    this.applicationLauncher.addService(this.jobConfigurationManager);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/de83a3fb/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java
index 36ba542..141e3d1 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobScheduler.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -34,6 +35,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.eventbus.EventBus;
 import com.google.common.eventbus.Subscribe;
@@ -157,7 +159,17 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe
     private final ContextAwareTimer timeBeforeJobScheduling;
     private final ContextAwareTimer timeBeforeJobLaunching;
 
+    private final ThreadPoolExecutor threadPoolExecutor;
+    private final ContextAwareGauge<Integer> executorActiveCount;
+    private final ContextAwareGauge<Integer> executorMaximumPoolSize;
+    private final ContextAwareGauge<Integer> executorPoolSize;
+    private final ContextAwareGauge<Integer> executorCorePoolSize;
+    private final ContextAwareGauge<Integer> executorQueueSize;
+
     public Metrics(final MetricContext metricContext) {
+      // Thread executor reference from job scheduler
+      this.threadPoolExecutor = (ThreadPoolExecutor)GobblinHelixJobScheduler.this.jobExecutor;
+
       // All historical counters
       this.totalJobsLaunched = new AtomicLong(0);
       this.totalJobsCompleted = new AtomicLong(0);
@@ -177,6 +189,13 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe
       this.timeForJobFailure = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_FOR_JOB_FAILURE,1, TimeUnit.MINUTES);
       this.timeBeforeJobScheduling = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_BEFORE_JOB_SCHEDULING, 1, TimeUnit.MINUTES);
       this.timeBeforeJobLaunching = metricContext.contextAwareTimer(JobExecutionLauncher.StandardMetrics.TIMER_BEFORE_JOB_LAUNCHING, 1, TimeUnit.MINUTES);
+
+      // executor metrics
+      this.executorActiveCount = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_ACTIVE_COUNT, ()->this.threadPoolExecutor.getActiveCount());
+      this.executorMaximumPoolSize = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_MAX_POOL_SIZE, ()->this.threadPoolExecutor.getMaximumPoolSize());
+      this.executorPoolSize = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_POOL_SIZE, ()->this.threadPoolExecutor.getPoolSize());
+      this.executorCorePoolSize =  metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_CORE_POOL_SIZE, ()->this.threadPoolExecutor.getCorePoolSize());
+      this.executorQueueSize = metricContext.newContextAwareGauge(JobExecutionLauncher.StandardMetrics.EXECUTOR_QUEUE_SIZE, ()->this.threadPoolExecutor.getQueue().size());
     }
 
     private void updateTimeBeforeJobScheduling (Properties jobConfig) {
@@ -196,7 +215,19 @@ public class GobblinHelixJobScheduler extends JobScheduler implements StandardMe
 
     @Override
     public Collection<ContextAwareGauge<?>> getGauges() {
-      return ImmutableList.of(numJobsRunning, numJobsLaunched, numJobsCompleted, numJobsCommitted, numJobsFailed, numJobsCancelled);
+      List<ContextAwareGauge<?>> list = Lists.newArrayList();
+      list.add(numJobsRunning);
+      list.add(numJobsLaunched);
+      list.add(numJobsCompleted);
+      list.add(numJobsCommitted);
+      list.add(numJobsFailed);
+      list.add(numJobsCancelled);
+      list.add(executorActiveCount);
+      list.add(executorMaximumPoolSize);
+      list.add(executorPoolSize);
+      list.add(executorCorePoolSize);
+      list.add(executorQueueSize);
+      return list;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/de83a3fb/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/StreamingJobConfigurationManager.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/StreamingJobConfigurationManager.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/StreamingJobConfigurationManager.java
index 849dd6a..3c01704 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/StreamingJobConfigurationManager.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/StreamingJobConfigurationManager.java
@@ -45,6 +45,8 @@ import org.apache.gobblin.util.ExecutorsUtils;
 import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
 import org.apache.gobblin.runtime.api.SpecConsumer;
 
+import lombok.Getter;
+
 
 /**
  * A {@link JobConfigurationManager} that fetches job specs from a {@link SpecConsumer} in a loop
@@ -56,6 +58,7 @@ public class StreamingJobConfigurationManager extends JobConfigurationManager {
 
   private final ExecutorService fetchJobSpecExecutor;
 
+  @Getter
   private final SpecConsumer specConsumer;
 
   private final long stopTimeoutSeconds;

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/de83a3fb/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java b/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
index 23966e9..4764603 100644
--- a/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
+++ b/gobblin-modules/gobblin-service-kafka/src/main/java/org/apache/gobblin/service/StreamingKafkaSpecConsumer.java
@@ -21,18 +21,29 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.instrumented.StandardMetricsBridge;
+import org.apache.gobblin.metrics.ContextAwareGauge;
+import org.apache.gobblin.metrics.GobblinMetrics;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.Tag;
+
 import org.slf4j.Logger;
 
 import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.AbstractIdleService;
 import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
@@ -49,6 +60,9 @@ import org.apache.gobblin.util.CompletedFuture;
 import org.apache.gobblin.util.ConfigUtils;
 
 import static org.apache.gobblin.service.SimpleKafkaSpecExecutor.SPEC_KAFKA_TOPICS_KEY;
+
+import javax.annotation.Nonnull;
+import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
 
 @Slf4j
@@ -56,12 +70,16 @@ import lombok.extern.slf4j.Slf4j;
  * SpecConsumer that consumes from kafka in a streaming manner
  * Implemented {@link AbstractIdleService} for starting up and shutting down.
  */
-public class StreamingKafkaSpecConsumer extends AbstractIdleService implements SpecConsumer<Spec>, Closeable {
+public class StreamingKafkaSpecConsumer extends AbstractIdleService implements SpecConsumer<Spec>, Closeable, StandardMetricsBridge {
   public static final String SPEC_STREAMING_BLOCKING_QUEUE_SIZE = "spec.StreamingBlockingQueueSize";
   private static final int DEFAULT_SPEC_STREAMING_BLOCKING_QUEUE_SIZE = 100;
+  @Getter
   private final AvroJobSpecKafkaJobMonitor _jobMonitor;
   private final BlockingQueue<ImmutablePair<SpecExecutor.Verb, Spec>> _jobSpecQueue;
   private final MutableJobCatalog _jobCatalog;
+  private final MetricContext _metricContext;
+  private final Metrics _metrics;
+  private final boolean _isInstrumentedEnabled;
   public StreamingKafkaSpecConsumer(Config config, MutableJobCatalog jobCatalog, Optional<Logger> log) {
     String topic = config.getString(SPEC_KAFKA_TOPICS_KEY);
     Config defaults = ConfigFactory.parseMap(ImmutableMap.of(AvroJobSpecKafkaJobMonitor.TOPIC_KEY, topic,
@@ -73,10 +91,12 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
     } catch (IOException e) {
       throw new RuntimeException("Could not create job monitor", e);
     }
-
+    _isInstrumentedEnabled = GobblinMetrics.isEnabled(ConfigUtils.configToProperties(config));
     _jobCatalog = jobCatalog;
     _jobSpecQueue = new LinkedBlockingQueue<>(ConfigUtils.getInt(config, "SPEC_STREAMING_BLOCKING_QUEUE_SIZE",
         DEFAULT_SPEC_STREAMING_BLOCKING_QUEUE_SIZE));
+    _metricContext = Instrumented.getMetricContext(ConfigUtils.configToState(config), this.getClass());
+    _metrics = new Metrics(this._metricContext);
   }
 
   public StreamingKafkaSpecConsumer(Config config, MutableJobCatalog jobCatalog, Logger log) {
@@ -98,7 +118,7 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
 
     try {
       Pair<SpecExecutor.Verb, Spec> specPair = _jobSpecQueue.take();
-
+      _metrics.jobSpecDeqCount.incrementAndGet();
       do {
         changesSpecs.add(specPair);
 
@@ -145,6 +165,7 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
 
       try {
         _jobSpecQueue.put(new ImmutablePair<SpecExecutor.Verb, Spec>(SpecExecutor.Verb.ADD, addedJob));
+        _metrics.jobSpecEnqCount.incrementAndGet();
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
       }
@@ -159,6 +180,7 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
         jobSpecBuilder.withVersion(deletedJobVersion).withConfigAsProperties(props);
 
         _jobSpecQueue.put(new ImmutablePair<SpecExecutor.Verb, Spec>(SpecExecutor.Verb.DELETE, jobSpecBuilder.build()));
+        _metrics.jobSpecEnqCount.incrementAndGet();
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
       }
@@ -169,9 +191,73 @@ public class StreamingKafkaSpecConsumer extends AbstractIdleService implements S
 
       try {
         _jobSpecQueue.put(new ImmutablePair<SpecExecutor.Verb, Spec>(SpecExecutor.Verb.UPDATE, updatedJob));
+        _metrics.jobSpecEnqCount.incrementAndGet();
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
       }
     }
   }
+
+  private class Metrics extends StandardMetricsBridge.StandardMetrics {
+    private ContextAwareGauge<Integer> jobSpecQueueSize;
+    private ContextAwareGauge<Long> jobSpecEnq;
+    private ContextAwareGauge<Long> jobSpecDeq;
+    private ContextAwareGauge<Long> jobSpecConsumed;
+    private AtomicLong jobSpecEnqCount = new AtomicLong(0);
+    private AtomicLong jobSpecDeqCount = new AtomicLong(0);
+
+    public static final String SPEC_CONSUMER_JOB_SPEC_QUEUE_SIZE = "specConsumerJobSpecQueueSize";
+    public static final String SPEC_CONSUMER_JOB_SPEC_ENQ = "specConsumerJobSpecEnq";
+    public static final String SPEC_CONSUMER_JOB_SPEC_DEQ = "specConsumerJobSpecDeq";
+    public static final String SPEC_CONSUMER_JOB_SPEC_CONSUMED = "specConsumerJobSpecConsumed";
+
+
+    public Metrics(MetricContext context) {
+      this.jobSpecQueueSize = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_QUEUE_SIZE, ()->StreamingKafkaSpecConsumer.this._jobSpecQueue.size());
+      this.jobSpecEnq = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_ENQ, ()->jobSpecEnqCount.get());
+      this.jobSpecDeq = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_DEQ, ()->jobSpecDeqCount.get());
+      this.jobSpecConsumed = context.newContextAwareGauge(SPEC_CONSUMER_JOB_SPEC_CONSUMED,
+          ()->StreamingKafkaSpecConsumer.this._jobMonitor.getNewSpecs().getCount() + StreamingKafkaSpecConsumer.this._jobMonitor.getRemmovedSpecs().getCount());
+    }
+
+    public Collection<ContextAwareGauge<?>> getGauges() {
+      List list = Lists.newArrayList();
+      list.add(jobSpecQueueSize);
+      list.add(jobSpecEnq);
+      list.add(jobSpecDeq);
+      list.add(jobSpecConsumed);
+      return list;
+    }
+  }
+
+  @Override
+  public StandardMetrics getStandardMetrics() {
+    throw new UnsupportedOperationException("Implemented in sub class");
+  }
+
+  @Nonnull
+  @Override
+  public MetricContext getMetricContext() {
+    return _metricContext;
+  }
+
+  @Override
+  public boolean isInstrumentationEnabled() {
+    return _isInstrumentedEnabled;
+  }
+
+  @Override
+  public List<Tag<?>> generateTags(org.apache.gobblin.configuration.State state) {
+    return ImmutableList.of();
+  }
+
+  @Override
+  public void switchMetricContext(List<Tag<?>> tags) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void switchMetricContext(MetricContext context) {
+    throw new UnsupportedOperationException();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/de83a3fb/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobCatalog.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobCatalog.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobCatalog.java
index 950b86d..42ecef3 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobCatalog.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobCatalog.java
@@ -95,12 +95,7 @@ public interface JobCatalog extends JobCatalogListenersContainer, Instrumentable
       this.totalAddCalls = metricsContext.newContextAwareGauge(TOTAL_ADD_CALLS, ()->this.totalAddedJobs.get());
       this.totalUpdateCalls = metricsContext.newContextAwareGauge(TOTAL_UPDATE_CALLS, ()->this.totalUpdatedJobs.get());
       this.totalDeleteCalls = metricsContext.newContextAwareGauge(TOTAL_DELETE_CALLS, ()->this.totalDeletedJobs.get());
-      this.numActiveJobs = metricsContext.newContextAwareGauge(NUM_ACTIVE_JOBS_NAME, ()->{
-          long startTime = System.currentTimeMillis();
-          int size = jobCatalog.getJobs().size();
-          updateGetJobTime(startTime);
-          return size;
-      });
+      this.numActiveJobs = metricsContext.newContextAwareGauge(NUM_ACTIVE_JOBS_NAME, ()->(int)(totalAddedJobs.get() - totalDeletedJobs.get()));
     }
 
     public void updateGetJobTime(long startTime) {

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/de83a3fb/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobExecutionLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobExecutionLauncher.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobExecutionLauncher.java
index a7e5878..3f50ee7 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobExecutionLauncher.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/api/JobExecutionLauncher.java
@@ -49,6 +49,12 @@ public interface JobExecutionLauncher extends Instrumentable {
     public static final String TIMER_BEFORE_JOB_SCHEDULING = "timerBeforeJobScheduling";
     public static final String TIMER_BEFORE_JOB_LAUNCHING = "timerBeforeJobLaunching";
 
+    public static final String EXECUTOR_ACTIVE_COUNT = "executorActiveCount";
+    public static final String EXECUTOR_MAX_POOL_SIZE = "executorMaximumPoolSize";
+    public static final String EXECUTOR_POOL_SIZE = "executorPoolSize";
+    public static final String EXECUTOR_CORE_POOL_SIZE = "executorCorePoolSize";
+    public static final String EXECUTOR_QUEUE_SIZE = "executorQueueSize";
+
     public static final String TRACKING_EVENT_NAME = "JobExecutionLauncherEvent";
     public static final String JOB_EXECID_META = "jobExecId";
     public static final String JOB_LAUNCHED_OPERATION_TYPE = "JobLaunched";

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/de83a3fb/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
index 8181c16..ba79305 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
@@ -35,6 +35,7 @@ import org.apache.gobblin.util.ConfigUtils;
 import org.apache.gobblin.util.Either;
 
 import kafka.message.MessageAndMetadata;
+import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
 
 
@@ -51,7 +52,9 @@ public abstract class KafkaJobMonitor extends HighLevelConsumer<byte[], byte[]>
   public static final String KAFKA_AUTO_OFFSET_RESET_LARGEST = "largest";
 
   private final MutableJobCatalog jobCatalog;
+  @Getter
   private Counter newSpecs;
+  @Getter
   private Counter remmovedSpecs;
 
   /**


[13/50] incubator-gobblin git commit: [GOBBLIN-400] Allow skipping execution of MR job in MR tasks.

Posted by ab...@apache.org.
[GOBBLIN-400] Allow skipping execution of MR job in MR tasks.

Closes #2274 from ibuenros/mr-job-ski


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/fd3a547e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/fd3a547e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/fd3a547e

Branch: refs/heads/0.12.0
Commit: fd3a547ec2ea5deac87febc8b95268ed63d78240
Parents: af68d7e
Author: ibuenros <is...@gmail.com>
Authored: Thu Feb 1 13:06:16 2018 -0800
Committer: Issac Buenrostro <ib...@apache.org>
Committed: Thu Feb 1 13:06:16 2018 -0800

----------------------------------------------------------------------
 .../gobblin/runtime/mapreduce/MRTask.java       | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/fd3a547e/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/mapreduce/MRTask.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/mapreduce/MRTask.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/mapreduce/MRTask.java
index 3abee65..a2e56d7 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/mapreduce/MRTask.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/mapreduce/MRTask.java
@@ -49,6 +49,7 @@ public class MRTask extends BaseAbstractTask {
     public static final String MR_JOB_STARTED_EVENT = "MRJobStarted";
     public static final String MR_JOB_SUCCESSFUL = "MRJobSuccessful";
     public static final String MR_JOB_FAILED = "MRJobFailed";
+    public static final String MR_JOB_SKIPPED = "MRJobSkipped";
 
     public static final String JOB_URL = "jobTrackingUrl";
     public static final String FAILURE_CONTEXT = "failureContext";
@@ -93,6 +94,14 @@ public class MRTask extends BaseAbstractTask {
     try {
       Job job = createJob();
 
+      if (job == null) {
+        log.info("No MR job created. Skipping.");
+        this.workingState = WorkUnitState.WorkingState.SUCCESSFUL;
+        this.eventSubmitter.submit(Events.MR_JOB_SKIPPED);
+        onSkippedMRJob();
+        return;
+      }
+
       job.submit();
       this.eventSubmitter.submit(Events.MR_JOB_STARTED_EVENT, Events.JOB_URL, job.getTrackingURL());
       job.waitForCompletion(false);
@@ -116,6 +125,10 @@ public class MRTask extends BaseAbstractTask {
     return Maps.newHashMap();
   }
 
+  /**
+   * Create the {@link Job} to run in this task.
+   * @return the {@link Job} to run. If this method returns null, no job will be run and the task will be marked as successful.
+   */
   protected Job createJob() throws IOException {
     Job job = Job.getInstance(new Configuration());
     for (Map.Entry<Object, Object> entry : this.taskContext.getTaskState().getProperties().entrySet()) {
@@ -127,4 +140,11 @@ public class MRTask extends BaseAbstractTask {
     return job;
   }
 
+  /**
+   * Called when a job is skipped (because {@link #createJob()} returned null).
+   */
+  protected void onSkippedMRJob() {
+    // do nothing
+  }
+
 }


[06/50] incubator-gobblin git commit: [GOBBLIN-390] Add log configs to child process

Posted by ab...@apache.org.
[GOBBLIN-390] Add log configs to child process

Closes #2266 from yukuai518/log


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/41fd2b92
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/41fd2b92
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/41fd2b92

Branch: refs/heads/0.12.0
Commit: 41fd2b920fe0dc5b282e2ee26a0a27acd6baa284
Parents: 6a31ef8
Author: Kuai Yu <ku...@linkedin.com>
Authored: Thu Jan 25 15:47:12 2018 -0800
Committer: Hung Tran <hu...@linkedin.com>
Committed: Thu Jan 25 15:47:12 2018 -0800

----------------------------------------------------------------------
 .../gobblin/cluster/GobblinClusterConfigurationKeys.java     | 2 ++
 .../java/org/apache/gobblin/cluster/SingleTaskLauncher.java  | 8 ++++++++
 .../java/org/apache/gobblin/cluster/SingleTaskRunner.java    | 1 -
 3 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/41fd2b92/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
index de501f1..68a8208 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
@@ -35,6 +35,8 @@ public class GobblinClusterConfigurationKeys {
       GOBBLIN_CLUSTER_PREFIX + "enableTaskInSeparateProcess";
   public static final String TASK_CLASSPATH =
       GOBBLIN_CLUSTER_PREFIX + "task.classpath";
+  public static final String TASK_LOG_CONFIG =
+      GOBBLIN_CLUSTER_PREFIX + "task.log.config";
 
   // General Gobblin Cluster application configuration properties.
   public static final String APPLICATION_NAME_OPTION_NAME = "app_name";

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/41fd2b92/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
index 1fe3eaf..e6fac31 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskLauncher.java
@@ -80,6 +80,7 @@ class SingleTaskLauncher {
     List<String> build() {
       addJavaBin();
       addClassPath();
+      addLogConfig();
       addClassName();
       addOptions();
       return this.cmd;
@@ -96,6 +97,13 @@ class SingleTaskLauncher {
       this.cmd.add(javaBinPath.toString());
     }
 
+    private void addLogConfig() {
+      if (sysConfig.hasPath(GobblinClusterConfigurationKeys.TASK_LOG_CONFIG)) {
+        String logConfig = sysConfig.getString(GobblinClusterConfigurationKeys.TASK_LOG_CONFIG);
+        this.cmd.add(logConfig);
+      }
+    }
+
     private void addClassPath() {
       this.cmd.add("-cp");
       String classPath;

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/41fd2b92/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskRunner.java
----------------------------------------------------------------------
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskRunner.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskRunner.java
index 6226cf1..7eb87c5 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskRunner.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTaskRunner.java
@@ -38,7 +38,6 @@ import com.typesafe.config.Config;
 import com.typesafe.config.ConfigFactory;
 
 import org.apache.gobblin.configuration.ConfigurationKeys;
-import org.apache.gobblin.runtime.AbstractJobLauncher;
 import org.apache.gobblin.runtime.TaskExecutor;
 import org.apache.gobblin.runtime.util.StateStores;
 import org.apache.gobblin.util.ConfigUtils;


[48/50] incubator-gobblin git commit: [GOBBLIN-428] fix delete spec not propagating to cluster

Posted by ab...@apache.org.
[GOBBLIN-428] fix delete spec not propagating to cluster

Closes #2305 from arjun4084346/fixDeleteInCluster


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/7e1d8433
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/7e1d8433
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/7e1d8433

Branch: refs/heads/0.12.0
Commit: 7e1d8433aee46c2d3f97c068b4348fc051bef654
Parents: 2e61a10
Author: Arjun <ab...@linkedin.com>
Authored: Fri Mar 16 09:27:15 2018 -0700
Committer: Hung Tran <hu...@linkedin.com>
Committed: Fri Mar 16 09:27:15 2018 -0700

----------------------------------------------------------------------
 .../gobblin/runtime/job_monitor/KafkaJobMonitor.java      | 10 +++-------
 1 file changed, 3 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/7e1d8433/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
----------------------------------------------------------------------
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
index 0bb4f14..6902eae 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/job_monitor/KafkaJobMonitor.java
@@ -71,13 +71,9 @@ public abstract class KafkaJobMonitor extends HighLevelConsumer<byte[], byte[]>
     super(topic, ConfigUtils.getConfigOrEmpty(config, KAFKA_JOB_MONITOR_PREFIX), 1);
     this.jobCatalog = catalog;
     try {
-      if (config.hasPath(ConfigurationKeys.STATE_STORE_ENABLED) &&
-          config.getBoolean(ConfigurationKeys.STATE_STORE_ENABLED) &&
-          config.hasPath(ConfigurationKeys.STATE_STORE_ROOT_DIR_KEY)) {
-        this.datasetStateStore = DatasetStateStore.buildDatasetStateStore(config);
-      }
-    } catch (IOException e) {
-      log.warn("DatasetStateStore could not be created.");
+      this.datasetStateStore = DatasetStateStore.buildDatasetStateStore(config);
+    } catch (Exception e) {
+      log.warn("DatasetStateStore could not be created.", e);
     }
   }
 


[50/50] incubator-gobblin git commit: [GOBBLIN-427] Add a decryption converter

Posted by ab...@apache.org.
[GOBBLIN-427] Add a decryption converter

Closes #2304 from xzhang27/master


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/b7f123f7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/b7f123f7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/b7f123f7

Branch: refs/heads/0.12.0
Commit: b7f123f77a58c690a9acf89f6d3168aeda259a17
Parents: 0795fa7
Author: Xiang <xn...@linkedin.com>
Authored: Mon Mar 19 11:39:46 2018 -0700
Committer: Hung Tran <hu...@linkedin.com>
Committed: Mon Mar 19 11:39:55 2018 -0700

----------------------------------------------------------------------
 .../gobblin/crypto/EncryptionConfigParser.java  |  4 +-
 .../crypto/EncryptionConfigParserTest.java      |  4 +-
 .../copy/converter/DecryptConverter.java        |  2 +-
 ...alizedRecordToSerializedRecordConverter.java | 44 +++++++++
 ...ordToEncryptedSerializedRecordConverter.java |  2 +-
 .../StringFieldEncryptorConverter.java          |  2 +-
 ...edRecordToSerializedRecordConverterTest.java | 95 ++++++++++++++++++++
 ...edRecordToSerializedRecordConverterBase.java | 79 ++++++++++++++++
 ...dRecordWithMetadataToRecordWithMetadata.java | 81 +++++++++++++++++
 ...ordWithMetadataToRecordWithMetadataTest.java | 75 ++++++++++++++++
 10 files changed, 382 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b7f123f7/gobblin-core-base/src/main/java/org/apache/gobblin/crypto/EncryptionConfigParser.java
----------------------------------------------------------------------
diff --git a/gobblin-core-base/src/main/java/org/apache/gobblin/crypto/EncryptionConfigParser.java b/gobblin-core-base/src/main/java/org/apache/gobblin/crypto/EncryptionConfigParser.java
index 324365a..900b616 100644
--- a/gobblin-core-base/src/main/java/org/apache/gobblin/crypto/EncryptionConfigParser.java
+++ b/gobblin-core-base/src/main/java/org/apache/gobblin/crypto/EncryptionConfigParser.java
@@ -52,6 +52,7 @@ public class EncryptionConfigParser {
    */
   static final String WRITER_ENCRYPT_PREFIX = ConfigurationKeys.WRITER_PREFIX + ".encrypt";
   static final String CONVERTER_ENCRYPT_PREFIX = "converter.encrypt";
+  static final String CONVERTER_DECRYPT_PREFIX = "converter.decrypt";
 
   public static final String ENCRYPTION_ALGORITHM_KEY = "algorithm";
   public static final String ENCRYPTION_KEYSTORE_PATH_KEY = "keystore_path";
@@ -72,7 +73,8 @@ public class EncryptionConfigParser {
    * enum maps entity type to a configuration prefix.
    */
   public enum EntityType {
-    CONVERTER(CONVERTER_ENCRYPT_PREFIX),
+    CONVERTER_ENCRYPT(CONVERTER_ENCRYPT_PREFIX),
+    CONVERTER_DECRYPT(CONVERTER_DECRYPT_PREFIX),
     WRITER(WRITER_ENCRYPT_PREFIX);
 
     private final String configPrefix;

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b7f123f7/gobblin-core-base/src/test/java/org/apache/gobblin/crypto/EncryptionConfigParserTest.java
----------------------------------------------------------------------
diff --git a/gobblin-core-base/src/test/java/org/apache/gobblin/crypto/EncryptionConfigParserTest.java b/gobblin-core-base/src/test/java/org/apache/gobblin/crypto/EncryptionConfigParserTest.java
index 68fbf37..54b52d4 100644
--- a/gobblin-core-base/src/test/java/org/apache/gobblin/crypto/EncryptionConfigParserTest.java
+++ b/gobblin-core-base/src/test/java/org/apache/gobblin/crypto/EncryptionConfigParserTest.java
@@ -97,7 +97,7 @@ public class EncryptionConfigParserTest {
         "keyname");
     wuState.setProp(EncryptionConfigParser.CONVERTER_ENCRYPT_PREFIX + "abc.def", "foobar");
 
-    Map<String, Object> parsedProperties = EncryptionConfigParser.getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER, wuState);
+    Map<String, Object> parsedProperties = EncryptionConfigParser.getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER_ENCRYPT, wuState);
     Assert.assertNotNull(parsedProperties, "Expected parser to only return one record");
     Assert.assertEquals(parsedProperties.size(), 4, "Did not expect abc.def to be picked up in config");
 
@@ -125,7 +125,7 @@ public class EncryptionConfigParserTest {
         "keyname");
     wuState.setProp(EncryptionConfigParser.CONVERTER_ENCRYPT_PREFIX + "abc.def", "foobar");
 
-    Map<String, Object> parsedProperties = EncryptionConfigParser.getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER, entityName, wuState);
+    Map<String, Object> parsedProperties = EncryptionConfigParser.getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER_ENCRYPT, entityName, wuState);
     Assert.assertNotNull(parsedProperties, "Expected parser to only return one record");
     Assert.assertEquals(parsedProperties.size(), 4, "Did not expect abc.def to be picked up in config");
     Assert.assertEquals(EncryptionConfigParser.getEncryptionType(parsedProperties), "aes_rotating");

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b7f123f7/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/converter/DecryptConverter.java
----------------------------------------------------------------------
diff --git a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/converter/DecryptConverter.java b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/converter/DecryptConverter.java
index d074046..d9c6353 100644
--- a/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/converter/DecryptConverter.java
+++ b/gobblin-data-management/src/main/java/org/apache/gobblin/data/management/copy/converter/DecryptConverter.java
@@ -59,7 +59,7 @@ public class DecryptConverter extends DistcpConverter {
   @Override
   public Converter<String, String, FileAwareInputStream, FileAwareInputStream> init(WorkUnitState workUnit) {
     Map<String, Object> config =
-        EncryptionConfigParser.getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER, workUnit);
+        EncryptionConfigParser.getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER_ENCRYPT, workUnit);
 
    if (config == null) {
      // Backwards compatibility check: if no config was passed in via the standard config, revert back to GPG

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b7f123f7/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverter.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverter.java b/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverter.java
new file mode 100644
index 0000000..2088eee
--- /dev/null
+++ b/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverter.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.converter;
+
+import java.util.Map;
+import org.apache.gobblin.codec.StreamCodec;
+import org.apache.gobblin.configuration.WorkUnitState;
+import org.apache.gobblin.crypto.EncryptionConfigParser;
+import org.apache.gobblin.crypto.EncryptionFactory;
+
+
+/**
+ * Specific implementation of {@link EncryptedSerializedRecordToSerializedRecordConverterBase} that uses Gobblin's
+ * {@link EncryptionFactory} to build the proper decryption codec based on config.
+ */
+public class EncryptedSerializedRecordToSerializedRecordConverter extends EncryptedSerializedRecordToSerializedRecordConverterBase {
+  @Override
+  protected StreamCodec buildDecryptor(WorkUnitState config) {
+    Map<String, Object> decryptionConfig =
+        EncryptionConfigParser.getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER_DECRYPT,
+            getClass().getSimpleName(), config);
+    if (decryptionConfig == null) {
+      throw new IllegalStateException("No decryption config specified in job - can't decrypt!");
+    }
+
+    return EncryptionFactory.buildStreamCryptoProvider(decryptionConfig);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b7f123f7/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/SerializedRecordToEncryptedSerializedRecordConverter.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/SerializedRecordToEncryptedSerializedRecordConverter.java b/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/SerializedRecordToEncryptedSerializedRecordConverter.java
index 2c830a9..be550c7 100644
--- a/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/SerializedRecordToEncryptedSerializedRecordConverter.java
+++ b/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/SerializedRecordToEncryptedSerializedRecordConverter.java
@@ -32,7 +32,7 @@ public class SerializedRecordToEncryptedSerializedRecordConverter extends Serial
   @Override
   protected StreamCodec buildEncryptor(WorkUnitState config) {
     Map<String, Object> encryptionConfig =
-        EncryptionConfigParser.getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER, getClass().getSimpleName(), config);
+        EncryptionConfigParser.getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER_ENCRYPT, getClass().getSimpleName(), config);
     if (encryptionConfig == null) {
       throw new IllegalStateException("No encryption config specified in job - can't encrypt!");
     }

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b7f123f7/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/StringFieldEncryptorConverter.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/StringFieldEncryptorConverter.java b/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/StringFieldEncryptorConverter.java
index d6e8de1..f242ec5 100644
--- a/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/StringFieldEncryptorConverter.java
+++ b/gobblin-modules/gobblin-crypto-provider/src/main/java/org/apache/gobblin/converter/StringFieldEncryptorConverter.java
@@ -47,7 +47,7 @@ public abstract class StringFieldEncryptorConverter<SCHEMA, DATA> extends Conver
   public Converter<SCHEMA, SCHEMA, DATA, DATA> init(WorkUnitState workUnit) {
     super.init(workUnit);
     Map<String, Object> config = EncryptionConfigParser
-        .getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER, getClass().getSimpleName(), workUnit);
+        .getConfigForBranch(EncryptionConfigParser.EntityType.CONVERTER_ENCRYPT, getClass().getSimpleName(), workUnit);
     encryptor = EncryptionFactory.buildStreamCryptoProvider(config);
 
     String fieldsToEncryptConfig = workUnit.getProp(FIELDS_TO_ENCRYPT_CONFIG_NAME, null);

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b7f123f7/gobblin-modules/gobblin-crypto-provider/src/test/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverterTest.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-crypto-provider/src/test/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverterTest.java b/gobblin-modules/gobblin-crypto-provider/src/test/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverterTest.java
new file mode 100644
index 0000000..efaeb16
--- /dev/null
+++ b/gobblin-modules/gobblin-crypto-provider/src/test/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverterTest.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.converter;
+
+import java.util.Iterator;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.configuration.WorkUnitState;
+import org.apache.gobblin.crypto.EncryptionConfigParser;
+import org.apache.gobblin.metadata.types.Metadata;
+import org.apache.gobblin.test.crypto.InsecureShiftCodec;
+import org.apache.gobblin.type.RecordWithMetadata;
+import org.testng.Assert;
+import org.testng.annotations.BeforeTest;
+import org.testng.annotations.Test;
+
+
+public class EncryptedSerializedRecordToSerializedRecordConverterTest {
+
+  private WorkUnitState workUnitState;
+  private EncryptedSerializedRecordToSerializedRecordConverter converter;
+  private RecordWithMetadata<byte[]> sampleRecord;
+  private byte[] shiftedValue;
+  private String insecureShiftTag;
+
+  private final String DECRYPT_PREFIX = "converter.decrypt.";
+
+  @BeforeTest
+  public void setUp() {
+    workUnitState = new WorkUnitState();
+    converter = new EncryptedSerializedRecordToSerializedRecordConverter();
+    sampleRecord = new RecordWithMetadata<>(new byte[]{'b', 'c', 'd', 'e'}, new Metadata());
+    shiftedValue = new byte[]{'a', 'b', 'c', 'd'};
+    insecureShiftTag = InsecureShiftCodec.TAG;
+  }
+
+  @Test(expectedExceptions = IllegalStateException.class)
+  public void throwsIfMisconfigured()
+      throws DataConversionException {
+    converter.init(workUnitState);
+    converter.convertRecord("", sampleRecord, workUnitState);
+  }
+
+  @Test
+  public void worksWithFork()
+      throws DataConversionException {
+    workUnitState.setProp(ConfigurationKeys.FORK_BRANCH_ID_KEY, 2);
+    workUnitState.getJobState()
+        .setProp(DECRYPT_PREFIX + EncryptionConfigParser.ENCRYPTION_ALGORITHM_KEY + ".2",
+            "insecure_shift");
+
+    converter.init(workUnitState);
+    Iterable<RecordWithMetadata<byte[]>> records = converter.convertRecord("", sampleRecord, workUnitState);
+    Iterator<RecordWithMetadata<byte[]>> recordIterator = records.iterator();
+    Assert.assertTrue(recordIterator.hasNext());
+
+    RecordWithMetadata<byte[]> record = recordIterator.next();
+
+    Assert.assertFalse(recordIterator.hasNext());
+    Assert.assertEquals(record.getMetadata().getGlobalMetadata().getTransferEncoding().get(0), insecureShiftTag);
+    Assert.assertEquals(record.getRecord(), shiftedValue);
+  }
+
+  @Test
+  public void worksNoFork()
+      throws DataConversionException {
+    workUnitState.getJobState()
+        .setProp(DECRYPT_PREFIX + EncryptionConfigParser.ENCRYPTION_ALGORITHM_KEY,
+            "insecure_shift");
+    converter.init(workUnitState);
+    Iterable<RecordWithMetadata<byte[]>> records = converter.convertRecord("", sampleRecord, workUnitState);
+    Iterator<RecordWithMetadata<byte[]>> recordIterator = records.iterator();
+    Assert.assertTrue(recordIterator.hasNext());
+
+    RecordWithMetadata<byte[]> record = recordIterator.next();
+
+    Assert.assertFalse(recordIterator.hasNext());
+    Assert.assertEquals(record.getMetadata().getGlobalMetadata().getTransferEncoding().get(0), insecureShiftTag);
+    Assert.assertEquals(record.getRecord(), shiftedValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b7f123f7/gobblin-modules/gobblin-crypto/src/main/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverterBase.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-crypto/src/main/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverterBase.java b/gobblin-modules/gobblin-crypto/src/main/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverterBase.java
new file mode 100644
index 0000000..7b6baca
--- /dev/null
+++ b/gobblin-modules/gobblin-crypto/src/main/java/org/apache/gobblin/converter/EncryptedSerializedRecordToSerializedRecordConverterBase.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.converter;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.util.Collections;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.io.IOUtils;
+import org.apache.gobblin.codec.StreamCodec;
+import org.apache.gobblin.configuration.WorkUnitState;
+import org.apache.gobblin.type.RecordWithMetadata;
+
+
+/**
+ * A converter that converts a encrypted {@link org.apache.gobblin.type.SerializedRecordWithMetadata} to
+ * a {@link org.apache.gobblin.type.SerializedRecordWithMetadata}. The decryption algorithm used will be
+ * appended to the Transfer-Encoding of the new record.
+ */
+@Slf4j
+public abstract class EncryptedSerializedRecordToSerializedRecordConverterBase extends Converter<String, String, RecordWithMetadata<byte[]>, RecordWithMetadata<byte[]>> {
+  private StreamCodec decryptor;
+
+  @Override
+  public Converter<String, String, RecordWithMetadata<byte[]>, RecordWithMetadata<byte[]>> init(
+      WorkUnitState workUnit) {
+    super.init(workUnit);
+    decryptor = buildDecryptor(workUnit);
+    return this;
+  }
+
+  /**
+   * Build the StreamCodec that will be used to decrypt each byte record. Must be provided by concrete
+   * implementations of this class.
+   */
+  protected abstract StreamCodec buildDecryptor(WorkUnitState config);
+
+  @Override
+  public String convertSchema(String inputSchema, WorkUnitState workUnit)
+      throws SchemaConversionException {
+    return "";
+  }
+
+  @Override
+  public Iterable<RecordWithMetadata<byte[]>> convertRecord(String outputSchema, RecordWithMetadata<byte[]> inputRecord,
+      WorkUnitState workUnit)
+      throws DataConversionException {
+    try {
+      ByteArrayInputStream inputStream = new ByteArrayInputStream(inputRecord.getRecord());
+      byte[] decryptedBytes;
+      try (InputStream decryptedInputStream = decryptor.decodeInputStream(inputStream)) {
+        decryptedBytes = IOUtils.toByteArray(decryptedInputStream);
+      }
+      inputRecord.getMetadata().getGlobalMetadata().addTransferEncoding(decryptor.getTag());
+
+      RecordWithMetadata<byte[]> serializedRecord =
+          new RecordWithMetadata<byte[]>(decryptedBytes, inputRecord.getMetadata());
+      return Collections.singleton(serializedRecord);
+    } catch (Exception e) {
+      throw new DataConversionException(e);
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b7f123f7/gobblin-modules/gobblin-metadata/src/main/java/org/apache/gobblin/converter/EnvelopedRecordWithMetadataToRecordWithMetadata.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-metadata/src/main/java/org/apache/gobblin/converter/EnvelopedRecordWithMetadataToRecordWithMetadata.java b/gobblin-modules/gobblin-metadata/src/main/java/org/apache/gobblin/converter/EnvelopedRecordWithMetadataToRecordWithMetadata.java
new file mode 100644
index 0000000..c686091
--- /dev/null
+++ b/gobblin-modules/gobblin-metadata/src/main/java/org/apache/gobblin/converter/EnvelopedRecordWithMetadataToRecordWithMetadata.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.converter;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.gobblin.configuration.WorkUnitState;
+import org.apache.gobblin.metadata.types.Metadata;
+import org.apache.gobblin.type.RecordWithMetadata;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.map.DeserializationConfig;
+import org.codehaus.jackson.map.ObjectMapper;
+
+/**
+ * A converter that takes a {@link RecordWithMetadata} and deserializes it by trying to parse it into a
+ * json format. It looks up two fields: "rMd" for record metadata and "r" for record details represented
+ * as a string.
+ */
+public class EnvelopedRecordWithMetadataToRecordWithMetadata extends Converter<String, Object, RecordWithMetadata<byte[]>, RecordWithMetadata<?>>  {
+
+  private static final String RECORD_KEY = "r";
+  private static final String METADATA_KEY = "rMd";
+  private static final String METADATA_RECORD_KEY = "recordMetadata";
+
+  private static final ObjectMapper objectMapper = new ObjectMapper().configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+  private static final JsonFactory jsonFactory = new JsonFactory();
+
+  @Override
+  public String convertSchema(String inputSchema, WorkUnitState workUnit)
+      throws SchemaConversionException {
+    return "";
+  }
+
+  @Override
+  public Iterable<RecordWithMetadata<?>> convertRecord(Object outputSchema, RecordWithMetadata<byte[]> inputRecord,
+      WorkUnitState workUnit)
+      throws DataConversionException {
+
+    try {
+      try (JsonParser parser = jsonFactory.createJsonParser(inputRecord.getRecord())) {
+        parser.setCodec(objectMapper);
+        JsonNode jsonNode = parser.readValueAsTree();
+
+        // extracts required record
+        if (!jsonNode.has(RECORD_KEY)) {
+          throw new DataConversionException("Input data does not have record.");
+        }
+        String record = jsonNode.get(RECORD_KEY).getTextValue();
+
+        // Extract metadata field
+        Metadata md = new Metadata();
+        if (jsonNode.has(METADATA_KEY) && jsonNode.get(METADATA_KEY).has(METADATA_RECORD_KEY)) {
+          md.getRecordMetadata().putAll(objectMapper.readValue(jsonNode.get(METADATA_KEY).get(METADATA_RECORD_KEY), Map.class));
+        }
+
+        return Collections.singleton(new RecordWithMetadata<>(record, md));
+      }
+    } catch (IOException e) {
+      throw new DataConversionException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/b7f123f7/gobblin-modules/gobblin-metadata/src/test/java/org/apache/gobblin/converter/EnvelopedRecordWithMetadataToRecordWithMetadataTest.java
----------------------------------------------------------------------
diff --git a/gobblin-modules/gobblin-metadata/src/test/java/org/apache/gobblin/converter/EnvelopedRecordWithMetadataToRecordWithMetadataTest.java b/gobblin-modules/gobblin-metadata/src/test/java/org/apache/gobblin/converter/EnvelopedRecordWithMetadataToRecordWithMetadataTest.java
new file mode 100644
index 0000000..8d247a7
--- /dev/null
+++ b/gobblin-modules/gobblin-metadata/src/test/java/org/apache/gobblin/converter/EnvelopedRecordWithMetadataToRecordWithMetadataTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.gobblin.converter;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import org.apache.gobblin.metadata.types.Metadata;
+import org.apache.gobblin.type.RecordWithMetadata;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Test
+public class EnvelopedRecordWithMetadataToRecordWithMetadataTest {
+
+  @Test
+  public void testSuccessWithRecord() throws DataConversionException, IOException {
+    ObjectMapper objectMapper = new ObjectMapper();
+    String innerRecord = "abracadabra";
+
+    // Build the input record
+    HashMap<String, Object> map = new HashMap<>();
+    map.put("r", innerRecord);
+    Metadata md = new Metadata();
+    md.getRecordMetadata().put("test1", "test2");
+    map.put("rMd", md);
+    JsonNode jsonNode = objectMapper.valueToTree(map);
+    RecordWithMetadata<byte[]> inputRecord = new RecordWithMetadata<>(jsonNode.toString().getBytes(), null);
+
+    EnvelopedRecordWithMetadataToRecordWithMetadata converter = new EnvelopedRecordWithMetadataToRecordWithMetadata();
+    Iterator<RecordWithMetadata<?>> iterator =
+        converter.convertRecord(null, inputRecord, null).iterator();
+
+    Assert.assertTrue(iterator.hasNext());
+
+    RecordWithMetadata<?> outputRecord = iterator.next();
+
+    Assert.assertEquals(outputRecord.getRecord(), innerRecord);
+    Assert.assertEquals(outputRecord.getMetadata().getRecordMetadata().get("test1"), "test2");
+  }
+
+  @Test(expectedExceptions = DataConversionException.class, expectedExceptionsMessageRegExp = "Input data does not have record.")
+  public void testFailureWithoutRecord() throws DataConversionException, IOException {
+    ObjectMapper objectMapper = new ObjectMapper();
+
+    // Build the input record without data
+    HashMap<String, Object> map = new HashMap<>();
+    Metadata md = new Metadata();
+    md.getRecordMetadata().put("test1", "test2");
+    map.put("rMd", md);
+    JsonNode jsonNode = objectMapper.valueToTree(map);
+    RecordWithMetadata<byte[]> inputRecord = new RecordWithMetadata<>(jsonNode.toString().getBytes(), null);
+
+    EnvelopedRecordWithMetadataToRecordWithMetadata converter = new EnvelopedRecordWithMetadataToRecordWithMetadata();
+    converter.convertRecord(null, inputRecord, null);
+  }
+
+}