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 2017/09/22 20:03:31 UTC

incubator-gobblin git commit: [GOBBLIN-231] Converter to convert text data to json based on a Grok pattern.

Repository: incubator-gobblin
Updated Branches:
  refs/heads/master de55592c6 -> e65f1316d


[GOBBLIN-231] Converter to convert text data to json based on a Grok pattern.

Closes #2081 from treff7es/converter_grok


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

Branch: refs/heads/master
Commit: e65f1316d8f9a0535b020c89941ceef9709615e9
Parents: de55592
Author: treff7es <ta...@prezi.com>
Authored: Fri Sep 22 13:03:26 2017 -0700
Committer: Abhishek Tiwari <ab...@gmail.com>
Committed: Fri Sep 22 13:03:26 2017 -0700

----------------------------------------------------------------------
 gobblin-core/build.gradle                       |   1 +
 .../converter/grok/GrokToJsonConverter.java     | 219 +++++++++++++++++++
 .../src/main/resources/grok/grok-base-patterns  |  97 ++++++++
 .../converter/grok/GrokToJsonConverterTest.java | 118 ++++++++++
 .../converter/grok/convertedRecord.json         |   1 +
 .../grok/convertedS3AccessLogRecord.json        |   1 +
 .../converter/grok/s3AccessLogSchema.json       | 138 ++++++++++++
 .../grok/schemaWithNonNullableFields.json       |  66 ++++++
 .../grok/schemaWithNullableFields.json          |  66 ++++++
 .../src/test/resources/grok/grok-patterns       |  97 ++++++++
 gradle/scripts/dependencyDefinitions.gradle     |   1 +
 11 files changed, 805 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gobblin-core/build.gradle
----------------------------------------------------------------------
diff --git a/gobblin-core/build.gradle b/gobblin-core/build.gradle
index ee7a77c..9eeae56 100644
--- a/gobblin-core/build.gradle
+++ b/gobblin-core/build.gradle
@@ -56,6 +56,7 @@ dependencies {
   compile externalDependency.findBugsAnnotations
   compile externalDependency.oltu
   compile externalDependency.opencsv
+  compile externalDependency.grok
   compile externalDependency.hadoopHdfs
 
   runtime externalDependency.protobuf

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gobblin-core/src/main/java/org/apache/gobblin/converter/grok/GrokToJsonConverter.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/converter/grok/GrokToJsonConverter.java b/gobblin-core/src/main/java/org/apache/gobblin/converter/grok/GrokToJsonConverter.java
new file mode 100644
index 0000000..1568eb7
--- /dev/null
+++ b/gobblin-core/src/main/java/org/apache/gobblin/converter/grok/GrokToJsonConverter.java
@@ -0,0 +1,219 @@
+/*
+ * 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.grok;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.gson.JsonArray;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonNull;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+
+import io.thekraken.grok.api.Grok;
+import io.thekraken.grok.api.Match;
+import io.thekraken.grok.api.exception.GrokException;
+
+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 org.apache.gobblin.util.DatasetFilterUtils;
+
+
+/**
+ * GrokToJsonConverter accepts already deserialized text row, String, where you can use.
+ *
+ * Converts Text to JSON based on Grok pattern. Schema is represented by the form of JsonArray same interface being used by CsvToJonConverter.
+ * Each text record is represented by a String.
+ * The converter only supports Grok patterns where every group is named because it uses the group names as column names.
+ *
+ * The following config properties can be set:
+ * The grok pattern to use for the conversion:
+ * converter.grokToJsonConverter.grokPattern ="^%{IPORHOST:clientip} (?:-|%{USER:ident}) (?:-|%{USER:auth}) \[%{HTTPDATE:timestamp}\] \"(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|-)\" %{NUMBER:response} (?:-|%{NUMBER:bytes})"
+ *
+ * Path to the file which contains the base grok patterns which can be used in the converter's GROK pattern (if not set it will use the default ones):
+ * converter.grokToJsonConverter.baseGrokPatternsFile=
+ **
+ * Specify a comma separated list of regexes which will be applied on the fields and matched one will be converted to json null:
+ * converter.grokToJsonConverter.nullStringRegexes="[-\s]"
+ *
+ * Example of schema:
+ * [
+ {
+ "columnName": "Day",
+ "comment": "",
+ "isNullable": "true",
+ "dataType": {
+ "type": "string"
+ }
+ },
+ {
+ "columnName": "Pageviews",
+ "comment": "",
+ "isNullable": "true",
+ "dataType": {
+ "type": "long"
+ }
+ }
+ ]
+ */
+public class GrokToJsonConverter extends Converter<String, JsonArray, String, JsonObject> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GrokToJsonConverter.class);
+  private static final JsonParser JSON_PARSER = new JsonParser();
+  private static final String COLUMN_NAME_KEY = "columnName";
+  private static final String DATA_TYPE = "dataType";
+  private static final String TYPE_KEY = "type";
+  private static final String NULLABLE = "isNullable";
+
+  public static final String GROK_PATTERN = "converter.grokToJsonConverter.grokPattern";
+  public static final String BASE_PATTERNS_FILE = "converter.grokToJsonConverter.baseGrokPatternsFile";
+  public static final String NULLSTRING_REGEXES = "converter.grokToJsonConverter.nullStringRegexes";
+
+  public static final String DEFAULT_GROK_PATTERNS_FILE = "/grok/grok-patterns";
+
+  private List<Pattern> nullStringRegexes;
+
+  private Grok grok;
+
+  @Override
+  public Converter<String, JsonArray, String, JsonObject> init(WorkUnitState workUnit) {
+    super.init(workUnit);
+    String pattern = workUnit.getProp(GROK_PATTERN);
+    String patternsFile = workUnit.getProp(BASE_PATTERNS_FILE);
+    this.nullStringRegexes = DatasetFilterUtils.getPatternsFromStrings(workUnit.getPropAsList(NULLSTRING_REGEXES, ""));
+
+    InputStreamReader grokPatterns;
+    try {
+      if (patternsFile == null) {
+        grokPatterns = new InputStreamReader(getClass().getResourceAsStream("/grok/grok-base-patterns"), "UTF8");
+      } else {
+        grokPatterns = new InputStreamReader(new FileInputStream(patternsFile), "UTF8");
+      }
+      grok = new Grok();
+      grok.addPatternFromReader(grokPatterns);
+      grok.compile(pattern);
+    } catch (GrokException | FileNotFoundException | UnsupportedEncodingException e) {
+      throw new RuntimeException("Error initializing GROK: " + e);
+    }
+
+    return this;
+  }
+
+  @Override
+  public JsonArray convertSchema(String inputSchema, WorkUnitState workUnit)
+      throws SchemaConversionException {
+    Preconditions.checkNotNull(inputSchema, "inputSchema is required.");
+    return JSON_PARSER.parse(inputSchema).getAsJsonArray();
+  }
+
+  /**
+   * Converts Text (String) to JSON based on a Grok regexp expression.
+   * By default, fields between Text and JSON are mapped by Grok SEMANTIC which is the identifier you give to the piece of text being matched in your Grok expression.
+   *
+   *
+   * e.g:
+   * {@inheritDoc}
+   * @see Converter#convertRecord(Object, Object, WorkUnitState)
+   */
+  @Override
+  public Iterable<JsonObject> convertRecord(JsonArray outputSchema, String inputRecord, WorkUnitState workUnit)
+      throws DataConversionException {
+
+    JsonObject outputRecord = createOutput(outputSchema, inputRecord);
+
+    LOG.debug("Converted into " + outputRecord);
+
+    return new SingleRecordIterable<JsonObject>(outputRecord);
+  }
+
+  @VisibleForTesting
+  JsonObject createOutput(JsonArray outputSchema, String inputRecord)
+      throws DataConversionException {
+    JsonObject outputRecord = new JsonObject();
+
+    Match gm = grok.match(inputRecord);
+    gm.captures();
+
+    JsonElement capturesJson = JSON_PARSER.parse(gm.toJson());
+
+    for (JsonElement anOutputSchema : outputSchema) {
+      JsonObject outputSchemaJsonObject = anOutputSchema.getAsJsonObject();
+      String key = outputSchemaJsonObject.get(COLUMN_NAME_KEY).getAsString();
+      String type = outputSchemaJsonObject.getAsJsonObject(DATA_TYPE).get(TYPE_KEY).getAsString();
+
+      if (isFieldNull(capturesJson, key)) {
+        if (!outputSchemaJsonObject.get(NULLABLE).getAsBoolean()) {
+          throw new DataConversionException(
+              "Field " + key + " is null or not exists but it is non-nullable by the schema.");
+        }
+        outputRecord.add(key, JsonNull.INSTANCE);
+      } else {
+        JsonElement jsonElement = capturesJson.getAsJsonObject().get(key);
+        switch (type) {
+          case "int":
+            outputRecord.addProperty(key, jsonElement.getAsInt());
+            break;
+          case "long":
+            outputRecord.addProperty(key, jsonElement.getAsLong());
+            break;
+          case "double":
+            outputRecord.addProperty(key, jsonElement.getAsDouble());
+            break;
+          case "float":
+            outputRecord.addProperty(key, jsonElement.getAsFloat());
+            break;
+          case "boolean":
+            outputRecord.addProperty(key, jsonElement.getAsBoolean());
+            break;
+          case "string":
+          default:
+            outputRecord.addProperty(key, jsonElement.getAsString());
+        }
+      }
+    }
+    return outputRecord;
+  }
+
+  private boolean isFieldNull(JsonElement capturesJson, String key) {
+    JsonObject jsonObject = capturesJson.getAsJsonObject();
+
+    if (!jsonObject.has(key)) {
+      return true;
+    }
+
+    for (Pattern pattern : this.nullStringRegexes) {
+      if (pattern.matcher(jsonObject.get(key).getAsString()).matches()) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gobblin-core/src/main/resources/grok/grok-base-patterns
----------------------------------------------------------------------
diff --git a/gobblin-core/src/main/resources/grok/grok-base-patterns b/gobblin-core/src/main/resources/grok/grok-base-patterns
new file mode 100644
index 0000000..3793e02
--- /dev/null
+++ b/gobblin-core/src/main/resources/grok/grok-base-patterns
@@ -0,0 +1,97 @@
+#Forked from https://github.com/logstash-plugins/logstash-patterns-core/blob/master/patterns/grok-patterns
+
+USERNAME [a-zA-Z0-9._-]+
+USER %{USERNAME}
+EMAILLOCALPART [a-zA-Z][a-zA-Z0-9_.+-=:]+
+EMAILADDRESS %{EMAILLOCALPART}@%{HOSTNAME}
+INT (?:[+-]?(?:[0-9]+))
+BASE10NUM (?<![0-9.+-])(?>[+-]?(?:(?:[0-9]+(?:\.[0-9]+)?)|(?:\.[0-9]+)))
+NUMBER (?:%{BASE10NUM})
+BASE16NUM (?<![0-9A-Fa-f])(?:[+-]?(?:0x)?(?:[0-9A-Fa-f]+))
+BASE16FLOAT \b(?<![0-9A-Fa-f.])(?:[+-]?(?:0x)?(?:(?:[0-9A-Fa-f]+(?:\.[0-9A-Fa-f]*)?)|(?:\.[0-9A-Fa-f]+)))\b
+
+POSINT \b(?:[1-9][0-9]*)\b
+NONNEGINT \b(?:[0-9]+)\b
+WORD \b\w+\b
+NOTSPACE \S+
+SPACE \s*
+DATA .*?
+GREEDYDATA .*
+QUOTEDSTRING (?>(?<!\\)(?>"(?>\\.|[^\\"]+)+"|""|(?>'(?>\\.|[^\\']+)+')|''|(?>`(?>\\.|[^\\`]+)+`)|``))
+UUID [A-Fa-f0-9]{8}-(?:[A-Fa-f0-9]{4}-){3}[A-Fa-f0-9]{12}
+# URN, allowing use of RFC 2141 section 2.3 reserved characters
+URN urn:[0-9A-Za-z][0-9A-Za-z-]{0,31}:(?:%[0-9a-fA-F]{2}|[0-9A-Za-z()+,.:=@;$_!*'/?#-])+
+
+# Networking
+MAC (?:%{CISCOMAC}|%{WINDOWSMAC}|%{COMMONMAC})
+CISCOMAC (?:(?:[A-Fa-f0-9]{4}\.){2}[A-Fa-f0-9]{4})
+WINDOWSMAC (?:(?:[A-Fa-f0-9]{2}-){5}[A-Fa-f0-9]{2})
+COMMONMAC (?:(?:[A-Fa-f0-9]{2}:){5}[A-Fa-f0-9]{2})
+IPV6 ((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5
 ]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:)))(%.+)?
+IPV4 (?<![0-9])(?:(?:[0-1]?[0-9]{1,2}|2[0-4][0-9]|25[0-5])[.](?:[0-1]?[0-9]{1,2}|2[0-4][0-9]|25[0-5])[.](?:[0-1]?[0-9]{1,2}|2[0-4][0-9]|25[0-5])[.](?:[0-1]?[0-9]{1,2}|2[0-4][0-9]|25[0-5]))(?![0-9])
+IP (?:%{IPV6}|%{IPV4})
+HOSTNAME \b(?:[0-9A-Za-z][0-9A-Za-z-]{0,62})(?:\.(?:[0-9A-Za-z][0-9A-Za-z-]{0,62}))*(\.?|\b)
+IPORHOST (?:%{IP}|%{HOSTNAME})
+HOSTPORT %{IPORHOST}:%{POSINT}
+
+# paths
+PATH (?:%{UNIXPATH}|%{WINPATH})
+UNIXPATH (/([\w_%!$@:.,+~-]+|\\.)*)+
+TTY (?:/dev/(pts|tty([pq])?)(\w+)?/?(?:[0-9]+))
+WINPATH (?>[A-Za-z]+:|\\)(?:\\[^\\?*]*)+
+URIPROTO [A-Za-z]([A-Za-z0-9+\-.]+)+
+URIHOST %{IPORHOST}(?::%{POSINT:port})?
+# uripath comes loosely from RFC1738, but mostly from what Firefox
+# doesn't turn into %XX
+URIPATH (?:/[A-Za-z0-9$.+!*'(){},~:;=@#%&_\-]*)+
+#URIPARAM \?(?:[A-Za-z0-9]+(?:=(?:[^&]*))?(?:&(?:[A-Za-z0-9]+(?:=(?:[^&]*))?)?)*)?
+URIPARAM \?[A-Za-z0-9$.+!*'|(){},~@#%&/=:;_?\-\[\]<>]*
+URIPATHPARAM %{URIPATH}(?:%{URIPARAM})?
+URI %{URIPROTO}://(?:%{USER}(?::[^@]*)?@)?(?:%{URIHOST})?(?:%{URIPATHPARAM})?
+
+# Months: January, Feb, 3, 03, 12, December
+MONTH \b(?:[Jj]an(?:uary|uar)?|[Ff]eb(?:ruary|ruar)?|[Mm](?:a|ä)?r(?:ch|z)?|[Aa]pr(?:il)?|[Mm]a(?:y|i)?|[Jj]un(?:e|i)?|[Jj]ul(?:y)?|[Aa]ug(?:ust)?|[Ss]ep(?:tember)?|[Oo](?:c|k)?t(?:ober)?|[Nn]ov(?:ember)?|[Dd]e(?:c|z)(?:ember)?)\b
+MONTHNUM (?:0?[1-9]|1[0-2])
+MONTHNUM2 (?:0[1-9]|1[0-2])
+MONTHDAY (?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9])
+
+# Days: Monday, Tue, Thu, etc...
+DAY (?:Mon(?:day)?|Tue(?:sday)?|Wed(?:nesday)?|Thu(?:rsday)?|Fri(?:day)?|Sat(?:urday)?|Sun(?:day)?)
+
+# Years?
+YEAR (?>\d\d){1,2}
+HOUR (?:2[0123]|[01]?[0-9])
+MINUTE (?:[0-5][0-9])
+# '60' is a leap second in most time standards and thus is valid.
+SECOND (?:(?:[0-5]?[0-9]|60)(?:[:.,][0-9]+)?)
+TIME (?!<[0-9])%{HOUR}:%{MINUTE}(?::%{SECOND})(?![0-9])
+# datestamp is YYYY/MM/DD-HH:MM:SS.UUUU (or something like it)
+DATE_US %{MONTHNUM}[/-]%{MONTHDAY}[/-]%{YEAR}
+DATE_EU %{MONTHDAY}[./-]%{MONTHNUM}[./-]%{YEAR}
+ISO8601_TIMEZONE (?:Z|[+-]%{HOUR}(?::?%{MINUTE}))
+ISO8601_SECOND (?:%{SECOND}|60)
+TIMESTAMP_ISO8601 %{YEAR}-%{MONTHNUM}-%{MONTHDAY}[T ]%{HOUR}:?%{MINUTE}(?::?%{SECOND})?%{ISO8601_TIMEZONE}?
+DATE %{DATE_US}|%{DATE_EU}
+DATESTAMP %{DATE}[- ]%{TIME}
+TZ (?:[APMCE][SD]T|UTC)
+DATESTAMP_RFC822 %{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{TIME} %{TZ}
+DATESTAMP_RFC2822 %{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{TIME} %{ISO8601_TIMEZONE}
+DATESTAMP_OTHER %{DAY} %{MONTH} %{MONTHDAY} %{TIME} %{TZ} %{YEAR}
+DATESTAMP_EVENTLOG %{YEAR}%{MONTHNUM2}%{MONTHDAY}%{HOUR}%{MINUTE}%{SECOND}
+
+# Syslog Dates: Month Day HH:MM:SS
+SYSLOGTIMESTAMP %{MONTH} +%{MONTHDAY} %{TIME}
+PROG [\x21-\x5a\x5c\x5e-\x7e]+
+SYSLOGPROG %{PROG:program}(?:\[%{POSINT:pid}\])?
+SYSLOGHOST %{IPORHOST}
+SYSLOGFACILITY <%{NONNEGINT:facility}.%{NONNEGINT:priority}>
+HTTPDATE %{MONTHDAY}/%{MONTH}/%{YEAR}:%{TIME} %{INT}
+
+# Shortcuts
+QS %{QUOTEDSTRING}
+
+# Log formats
+SYSLOGBASE %{SYSLOGTIMESTAMP:timestamp} (?:%{SYSLOGFACILITY} )?%{SYSLOGHOST:logsource} %{SYSLOGPROG}:
+
+# Log Levels
+LOGLEVEL ([Aa]lert|ALERT|[Tt]race|TRACE|[Dd]ebug|DEBUG|[Nn]otice|NOTICE|[Ii]nfo|INFO|[Ww]arn?(?:ing)?|WARN?(?:ING)?|[Ee]rr?(?:or)?|ERR?(?:OR)?|[Cc]rit?(?:ical)?|CRIT?(?:ICAL)?|[Ff]atal|FATAL|[Ss]evere|SEVERE|EMERG(?:ENCY)?|[Ee]merg(?:ency)?)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gobblin-core/src/test/java/org/apache/gobblin/converter/grok/GrokToJsonConverterTest.java
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/java/org/apache/gobblin/converter/grok/GrokToJsonConverterTest.java b/gobblin-core/src/test/java/org/apache/gobblin/converter/grok/GrokToJsonConverterTest.java
new file mode 100644
index 0000000..3a4b78b
--- /dev/null
+++ b/gobblin-core/src/test/java/org/apache/gobblin/converter/grok/GrokToJsonConverterTest.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.converter.grok;
+
+import java.io.InputStreamReader;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.gson.JsonArray;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+
+import gobblin.configuration.WorkUnitState;
+
+import org.apache.gobblin.converter.DataConversionException;
+
+
+@Test(groups = {"gobblin.converter"})
+public class GrokToJsonConverterTest {
+  @Test
+  public void convertOutputWithNullableFields()
+      throws Exception {
+    JsonParser parser = new JsonParser();
+
+    String inputRecord =
+        "10.121.123.104 - - [01/Nov/2012:21:01:17 +0100] \"GET /cpc/auth.do?loginsetup=true&targetPage=%2Fcpc%2F HTTP/1.1\" 302 466";
+
+    JsonElement jsonElement = parser
+        .parse(new InputStreamReader(getClass().getResourceAsStream("/converter/grok/schemaWithNullableFields.json")));
+    JsonArray outputSchema = jsonElement.getAsJsonArray();
+
+    GrokToJsonConverter grokToJsonConverter = new GrokToJsonConverter();
+    WorkUnitState workUnitState = new WorkUnitState();
+    workUnitState.setProp(GrokToJsonConverter.GROK_PATTERN,
+        "^%{IPORHOST:clientip} (?:-|%{USER:ident}) (?:-|%{USER:auth}) \\[%{HTTPDATE:timestamp}\\] \\\"(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|-)\\\" %{NUMBER:response} (?:-|%{NUMBER:bytes})");
+
+    grokToJsonConverter.init(workUnitState);
+    JsonObject actual = grokToJsonConverter.convertRecord(outputSchema, inputRecord, workUnitState).iterator().next();
+
+    JsonObject expected =
+        parser.parse(new InputStreamReader(getClass().getResourceAsStream("/converter/grok/convertedRecord.json")))
+            .getAsJsonObject();
+    Assert.assertEquals(actual, expected);
+    grokToJsonConverter.close();
+  }
+
+  @Test(expectedExceptions = DataConversionException.class)
+  public void convertOutputWithNonNullableFieldsShouldThrowDataConversionException()
+      throws Exception {
+    JsonParser parser = new JsonParser();
+
+    String inputRecord =
+        "10.121.123.104 - - [01/Nov/2012:21:01:17 +0100] \"GET /cpc/auth.do?loginsetup=true&targetPage=%2Fcpc%2F HTTP/1.1\" 302 466";
+
+    JsonElement jsonElement = parser.parse(
+        new InputStreamReader(getClass().getResourceAsStream("/converter/grok/schemaWithNonNullableFields.json")));
+    JsonArray outputSchema = jsonElement.getAsJsonArray();
+
+    GrokToJsonConverter grokToJsonConverter = new GrokToJsonConverter();
+    WorkUnitState workUnitState = new WorkUnitState();
+    workUnitState.setProp(GrokToJsonConverter.GROK_PATTERN,
+        "^%{IPORHOST:clientip} (?:-|%{USER:ident}) (?:-|%{USER:auth}) \\[%{HTTPDATE:timestamp}\\] \\\"(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|-)\\\" %{NUMBER:response} (?:-|%{NUMBER:bytes})");
+
+    grokToJsonConverter.init(workUnitState);
+    JsonObject actual = grokToJsonConverter.convertRecord(outputSchema, inputRecord, workUnitState).iterator().next();
+
+    JsonObject expected =
+        parser.parse(new InputStreamReader(getClass().getResourceAsStream("/converter/grok/convertedRecord.json")))
+            .getAsJsonObject();
+    grokToJsonConverter.close();
+  }
+
+  @Test
+  public void convertWithNullStringSet()
+      throws Exception {
+    JsonParser parser = new JsonParser();
+
+    String inputRecord =
+        "79a59df900b949e55d96a1e698fbacedfd6e09d98eacf8f8d5218e7cd47ef2be mybucket [06/Feb/2014:00:00:38 +0000] 192.0.2.3 79a59df900b949e55d96a1e698fbacedfd6e09d98eacf8f8d5218e7cd47ef2be 3E57427F3EXAMPLE REST.GET.VERSIONING - \"GET /mybucket?versioning HTTP/1.1\" 200 - 113 - 7 - \"-\" \"S3Console/0.4\" -";
+
+    JsonElement jsonElement =
+        parser.parse(new InputStreamReader(getClass().getResourceAsStream("/converter/grok/s3AccessLogSchema.json")));
+    JsonArray outputSchema = jsonElement.getAsJsonArray();
+
+    GrokToJsonConverter grokToJsonConverter = new GrokToJsonConverter();
+    WorkUnitState workUnitState = new WorkUnitState();
+    //Grok expression was taken from https://github.com/logstash-plugins/logstash-patterns-core/blob/master/patterns/aws
+    workUnitState.setProp(GrokToJsonConverter.GROK_PATTERN,
+        "%{WORD:owner} %{NOTSPACE:bucket} \\[%{HTTPDATE:timestamp}\\] %{IP:clientip} %{NOTSPACE:requester} %{NOTSPACE:request_id} %{NOTSPACE:operation} %{NOTSPACE:key} (?:\"(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest})\"|-) (?:%{INT:response:int}|-) (?:-|%{NOTSPACE:error_code}) (?:%{INT:bytes:int}|-) (?:%{INT:object_size:int}|-) (?:%{INT:request_time_ms:int}|-) (?:%{INT:turnaround_time_ms:int}|-) (?:%{QS:referrer}|-) (?:\"?%{QS:agent}\"?|-) (?:-|%{NOTSPACE:version_id})");
+    workUnitState.setProp(GrokToJsonConverter.NULLSTRING_REGEXES, "[\\s-]");
+
+    grokToJsonConverter.init(workUnitState);
+    JsonObject actual = grokToJsonConverter.convertRecord(outputSchema, inputRecord, workUnitState).iterator().next();
+
+    JsonObject expected = parser
+        .parse(new InputStreamReader(getClass().getResourceAsStream("/converter/grok/convertedS3AccessLogRecord.json")))
+        .getAsJsonObject();
+    Assert.assertEquals(actual, expected);
+    grokToJsonConverter.close();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gobblin-core/src/test/resources/converter/grok/convertedRecord.json
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/converter/grok/convertedRecord.json b/gobblin-core/src/test/resources/converter/grok/convertedRecord.json
new file mode 100644
index 0000000..69f45f1
--- /dev/null
+++ b/gobblin-core/src/test/resources/converter/grok/convertedRecord.json
@@ -0,0 +1 @@
+{"clientip":"10.121.123.104","ident":null,"auth":null,"timestamp":"01/Nov/2012:21:01:17 +0100","request":"/cpc/auth.do?loginsetup=true&targetPage=%2Fcpc%2F","httpversion":1.1,"response":302,"bytes":466}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gobblin-core/src/test/resources/converter/grok/convertedS3AccessLogRecord.json
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/converter/grok/convertedS3AccessLogRecord.json b/gobblin-core/src/test/resources/converter/grok/convertedS3AccessLogRecord.json
new file mode 100644
index 0000000..29da948
--- /dev/null
+++ b/gobblin-core/src/test/resources/converter/grok/convertedS3AccessLogRecord.json
@@ -0,0 +1 @@
+{"owner":"79a59df900b949e55d96a1e698fbacedfd6e09d98eacf8f8d5218e7cd47ef2be","bucket":"mybucket","timestamp":"06/Feb/2014:00:00:38 +0000","clientip":"192.0.2.3","requester":"79a59df900b949e55d96a1e698fbacedfd6e09d98eacf8f8d5218e7cd47ef2be","request_id":"3E57427F3EXAMPLE","operation":"REST.GET.VERSIONING","key":null,"response":200,"error_code":null,"bytes":113,"object_size":null,"request_time_ms":7,"turnaround_time_ms":null,"referrer":null,"agent":"S3Console/0.4","version_id":null}

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gobblin-core/src/test/resources/converter/grok/s3AccessLogSchema.json
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/converter/grok/s3AccessLogSchema.json b/gobblin-core/src/test/resources/converter/grok/s3AccessLogSchema.json
new file mode 100644
index 0000000..934399c
--- /dev/null
+++ b/gobblin-core/src/test/resources/converter/grok/s3AccessLogSchema.json
@@ -0,0 +1,138 @@
+[
+  {
+    "columnName": "owner",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "bucket",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "timestamp",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "clientip",
+    "comment": "",
+    "isNullable": "false",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "requester",
+    "comment": "",
+    "isNullable": "false",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "request_id",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "operation",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "key",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "response",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "int"
+    }
+  },
+  {
+    "columnName": "error_code",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "int"
+    }
+  },
+  {
+    "columnName": "bytes",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "int"
+    }
+  },
+  {
+    "columnName": "object_size",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "int"
+    }
+  },
+  {
+    "columnName": "request_time_ms",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "int"
+    }
+  },
+  {
+    "columnName": "turnaround_time_ms",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "int"
+    }
+  },
+  {
+    "columnName": "referrer",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "agent",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "version_id",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  }
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gobblin-core/src/test/resources/converter/grok/schemaWithNonNullableFields.json
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/converter/grok/schemaWithNonNullableFields.json b/gobblin-core/src/test/resources/converter/grok/schemaWithNonNullableFields.json
new file mode 100644
index 0000000..a1ce180
--- /dev/null
+++ b/gobblin-core/src/test/resources/converter/grok/schemaWithNonNullableFields.json
@@ -0,0 +1,66 @@
+[
+  {
+    "columnName": "clientip",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "ident",
+    "comment": "",
+    "isNullable": "false",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "auth",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "timestamp",
+    "comment": "",
+    "isNullable": "false",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "request",
+    "comment": "",
+    "isNullable": "false",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "httpversion",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "double"
+    }
+  },
+  {
+    "columnName": "response",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "int"
+    }
+  },
+  {
+    "columnName": "bytes",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "int"
+    }
+  }
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gobblin-core/src/test/resources/converter/grok/schemaWithNullableFields.json
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/converter/grok/schemaWithNullableFields.json b/gobblin-core/src/test/resources/converter/grok/schemaWithNullableFields.json
new file mode 100644
index 0000000..b8b0536
--- /dev/null
+++ b/gobblin-core/src/test/resources/converter/grok/schemaWithNullableFields.json
@@ -0,0 +1,66 @@
+[
+  {
+    "columnName": "clientip",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "ident",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "auth",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "timestamp",
+    "comment": "",
+    "isNullable": "false",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "request",
+    "comment": "",
+    "isNullable": "false",
+    "dataType": {
+      "type": "string"
+    }
+  },
+  {
+    "columnName": "httpversion",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "double"
+    }
+  },
+  {
+    "columnName": "response",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "int"
+    }
+  },
+  {
+    "columnName": "bytes",
+    "comment": "",
+    "isNullable": "true",
+    "dataType": {
+      "type": "int"
+    }
+  }
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gobblin-core/src/test/resources/grok/grok-patterns
----------------------------------------------------------------------
diff --git a/gobblin-core/src/test/resources/grok/grok-patterns b/gobblin-core/src/test/resources/grok/grok-patterns
new file mode 100644
index 0000000..3793e02
--- /dev/null
+++ b/gobblin-core/src/test/resources/grok/grok-patterns
@@ -0,0 +1,97 @@
+#Forked from https://github.com/logstash-plugins/logstash-patterns-core/blob/master/patterns/grok-patterns
+
+USERNAME [a-zA-Z0-9._-]+
+USER %{USERNAME}
+EMAILLOCALPART [a-zA-Z][a-zA-Z0-9_.+-=:]+
+EMAILADDRESS %{EMAILLOCALPART}@%{HOSTNAME}
+INT (?:[+-]?(?:[0-9]+))
+BASE10NUM (?<![0-9.+-])(?>[+-]?(?:(?:[0-9]+(?:\.[0-9]+)?)|(?:\.[0-9]+)))
+NUMBER (?:%{BASE10NUM})
+BASE16NUM (?<![0-9A-Fa-f])(?:[+-]?(?:0x)?(?:[0-9A-Fa-f]+))
+BASE16FLOAT \b(?<![0-9A-Fa-f.])(?:[+-]?(?:0x)?(?:(?:[0-9A-Fa-f]+(?:\.[0-9A-Fa-f]*)?)|(?:\.[0-9A-Fa-f]+)))\b
+
+POSINT \b(?:[1-9][0-9]*)\b
+NONNEGINT \b(?:[0-9]+)\b
+WORD \b\w+\b
+NOTSPACE \S+
+SPACE \s*
+DATA .*?
+GREEDYDATA .*
+QUOTEDSTRING (?>(?<!\\)(?>"(?>\\.|[^\\"]+)+"|""|(?>'(?>\\.|[^\\']+)+')|''|(?>`(?>\\.|[^\\`]+)+`)|``))
+UUID [A-Fa-f0-9]{8}-(?:[A-Fa-f0-9]{4}-){3}[A-Fa-f0-9]{12}
+# URN, allowing use of RFC 2141 section 2.3 reserved characters
+URN urn:[0-9A-Za-z][0-9A-Za-z-]{0,31}:(?:%[0-9a-fA-F]{2}|[0-9A-Za-z()+,.:=@;$_!*'/?#-])+
+
+# Networking
+MAC (?:%{CISCOMAC}|%{WINDOWSMAC}|%{COMMONMAC})
+CISCOMAC (?:(?:[A-Fa-f0-9]{4}\.){2}[A-Fa-f0-9]{4})
+WINDOWSMAC (?:(?:[A-Fa-f0-9]{2}-){5}[A-Fa-f0-9]{2})
+COMMONMAC (?:(?:[A-Fa-f0-9]{2}:){5}[A-Fa-f0-9]{2})
+IPV6 ((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5
 ]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:)))(%.+)?
+IPV4 (?<![0-9])(?:(?:[0-1]?[0-9]{1,2}|2[0-4][0-9]|25[0-5])[.](?:[0-1]?[0-9]{1,2}|2[0-4][0-9]|25[0-5])[.](?:[0-1]?[0-9]{1,2}|2[0-4][0-9]|25[0-5])[.](?:[0-1]?[0-9]{1,2}|2[0-4][0-9]|25[0-5]))(?![0-9])
+IP (?:%{IPV6}|%{IPV4})
+HOSTNAME \b(?:[0-9A-Za-z][0-9A-Za-z-]{0,62})(?:\.(?:[0-9A-Za-z][0-9A-Za-z-]{0,62}))*(\.?|\b)
+IPORHOST (?:%{IP}|%{HOSTNAME})
+HOSTPORT %{IPORHOST}:%{POSINT}
+
+# paths
+PATH (?:%{UNIXPATH}|%{WINPATH})
+UNIXPATH (/([\w_%!$@:.,+~-]+|\\.)*)+
+TTY (?:/dev/(pts|tty([pq])?)(\w+)?/?(?:[0-9]+))
+WINPATH (?>[A-Za-z]+:|\\)(?:\\[^\\?*]*)+
+URIPROTO [A-Za-z]([A-Za-z0-9+\-.]+)+
+URIHOST %{IPORHOST}(?::%{POSINT:port})?
+# uripath comes loosely from RFC1738, but mostly from what Firefox
+# doesn't turn into %XX
+URIPATH (?:/[A-Za-z0-9$.+!*'(){},~:;=@#%&_\-]*)+
+#URIPARAM \?(?:[A-Za-z0-9]+(?:=(?:[^&]*))?(?:&(?:[A-Za-z0-9]+(?:=(?:[^&]*))?)?)*)?
+URIPARAM \?[A-Za-z0-9$.+!*'|(){},~@#%&/=:;_?\-\[\]<>]*
+URIPATHPARAM %{URIPATH}(?:%{URIPARAM})?
+URI %{URIPROTO}://(?:%{USER}(?::[^@]*)?@)?(?:%{URIHOST})?(?:%{URIPATHPARAM})?
+
+# Months: January, Feb, 3, 03, 12, December
+MONTH \b(?:[Jj]an(?:uary|uar)?|[Ff]eb(?:ruary|ruar)?|[Mm](?:a|ä)?r(?:ch|z)?|[Aa]pr(?:il)?|[Mm]a(?:y|i)?|[Jj]un(?:e|i)?|[Jj]ul(?:y)?|[Aa]ug(?:ust)?|[Ss]ep(?:tember)?|[Oo](?:c|k)?t(?:ober)?|[Nn]ov(?:ember)?|[Dd]e(?:c|z)(?:ember)?)\b
+MONTHNUM (?:0?[1-9]|1[0-2])
+MONTHNUM2 (?:0[1-9]|1[0-2])
+MONTHDAY (?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9])
+
+# Days: Monday, Tue, Thu, etc...
+DAY (?:Mon(?:day)?|Tue(?:sday)?|Wed(?:nesday)?|Thu(?:rsday)?|Fri(?:day)?|Sat(?:urday)?|Sun(?:day)?)
+
+# Years?
+YEAR (?>\d\d){1,2}
+HOUR (?:2[0123]|[01]?[0-9])
+MINUTE (?:[0-5][0-9])
+# '60' is a leap second in most time standards and thus is valid.
+SECOND (?:(?:[0-5]?[0-9]|60)(?:[:.,][0-9]+)?)
+TIME (?!<[0-9])%{HOUR}:%{MINUTE}(?::%{SECOND})(?![0-9])
+# datestamp is YYYY/MM/DD-HH:MM:SS.UUUU (or something like it)
+DATE_US %{MONTHNUM}[/-]%{MONTHDAY}[/-]%{YEAR}
+DATE_EU %{MONTHDAY}[./-]%{MONTHNUM}[./-]%{YEAR}
+ISO8601_TIMEZONE (?:Z|[+-]%{HOUR}(?::?%{MINUTE}))
+ISO8601_SECOND (?:%{SECOND}|60)
+TIMESTAMP_ISO8601 %{YEAR}-%{MONTHNUM}-%{MONTHDAY}[T ]%{HOUR}:?%{MINUTE}(?::?%{SECOND})?%{ISO8601_TIMEZONE}?
+DATE %{DATE_US}|%{DATE_EU}
+DATESTAMP %{DATE}[- ]%{TIME}
+TZ (?:[APMCE][SD]T|UTC)
+DATESTAMP_RFC822 %{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{TIME} %{TZ}
+DATESTAMP_RFC2822 %{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{TIME} %{ISO8601_TIMEZONE}
+DATESTAMP_OTHER %{DAY} %{MONTH} %{MONTHDAY} %{TIME} %{TZ} %{YEAR}
+DATESTAMP_EVENTLOG %{YEAR}%{MONTHNUM2}%{MONTHDAY}%{HOUR}%{MINUTE}%{SECOND}
+
+# Syslog Dates: Month Day HH:MM:SS
+SYSLOGTIMESTAMP %{MONTH} +%{MONTHDAY} %{TIME}
+PROG [\x21-\x5a\x5c\x5e-\x7e]+
+SYSLOGPROG %{PROG:program}(?:\[%{POSINT:pid}\])?
+SYSLOGHOST %{IPORHOST}
+SYSLOGFACILITY <%{NONNEGINT:facility}.%{NONNEGINT:priority}>
+HTTPDATE %{MONTHDAY}/%{MONTH}/%{YEAR}:%{TIME} %{INT}
+
+# Shortcuts
+QS %{QUOTEDSTRING}
+
+# Log formats
+SYSLOGBASE %{SYSLOGTIMESTAMP:timestamp} (?:%{SYSLOGFACILITY} )?%{SYSLOGHOST:logsource} %{SYSLOGPROG}:
+
+# Log Levels
+LOGLEVEL ([Aa]lert|ALERT|[Tt]race|TRACE|[Dd]ebug|DEBUG|[Nn]otice|NOTICE|[Ii]nfo|INFO|[Ww]arn?(?:ing)?|WARN?(?:ING)?|[Ee]rr?(?:or)?|ERR?(?:OR)?|[Cc]rit?(?:ical)?|CRIT?(?:ICAL)?|[Ff]atal|FATAL|[Ss]evere|SEVERE|EMERG(?:ENCY)?|[Ee]merg(?:ency)?)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/e65f1316/gradle/scripts/dependencyDefinitions.gradle
----------------------------------------------------------------------
diff --git a/gradle/scripts/dependencyDefinitions.gradle b/gradle/scripts/dependencyDefinitions.gradle
index cde546c..096d107 100644
--- a/gradle/scripts/dependencyDefinitions.gradle
+++ b/gradle/scripts/dependencyDefinitions.gradle
@@ -164,6 +164,7 @@ ext.externalDependency = [
     "googleOauthClient": "com.google.oauth-client:google-oauth-client:" + googleVersion,
     "googleApiClient": "com.google.api-client:google-api-client:" + googleVersion,
     "opencsv": "com.opencsv:opencsv:3.8",
+    "grok": "io.thekraken:grok:0.1.5",
     "hadoopAdl" : "org.apache.hadoop:hadoop-azure-datalake:3.0.0-alpha2",
     'parquet': 'com.twitter:parquet-hadoop-bundle:1.5.0',
     'reactivex': 'io.reactivex.rxjava2:rxjava:2.1.0',