You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ot...@apache.org on 2018/04/18 14:59:43 UTC

[13/52] [abbrv] metron git commit: METRON-1397 Support for JSON Path and complex documents in JSONMapParser closes apache/incubator-metron#914

METRON-1397 Support for JSON Path and complex documents in JSONMapParser closes apache/incubator-metron#914


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

Branch: refs/heads/feature/METRON-1211-extensions-parsers-gradual
Commit: 9c5d9d76644fc07bae36644906f52e0422f33d0e
Parents: 85d1247
Author: ottobackwards <ot...@gmail.com>
Authored: Thu Mar 15 14:17:31 2018 -0400
Committer: cstella <ce...@gmail.com>
Committed: Thu Mar 15 14:17:31 2018 -0400

----------------------------------------------------------------------
 dependencies_with_url.csv                       |   3 +
 .../docker/rpm-docker/SPECS/metron.spec         |   1 +
 .../jsonMapQuery/parsed/jsonMapExampleParsed    |   2 +
 .../data/jsonMapQuery/raw/jsonMapExampleOutput  |   1 +
 metron-platform/metron-parsers/README.md        |  12 ++
 metron-platform/metron-parsers/pom.xml          |   5 +
 .../config/zookeeper/parsers/jsonMapQuery.json  |   5 +
 .../metron/parsers/json/JSONMapParser.java      | 145 +++++++++----
 .../JSONMapQueryIntegrationTest.java            |  36 ++++
 .../validation/SampleDataValidation.java        |   2 +-
 .../parsers/json/JSONMapParserQueryTest.java    | 201 +++++++++++++++++++
 .../metron/test/utils/ValidationUtils.java      |  46 ++++-
 12 files changed, 406 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/dependencies_with_url.csv
----------------------------------------------------------------------
diff --git a/dependencies_with_url.csv b/dependencies_with_url.csv
index e2b947b..1e73eb1 100644
--- a/dependencies_with_url.csv
+++ b/dependencies_with_url.csv
@@ -22,6 +22,9 @@ com.flipkart.zjsonpatch:zjsonpatch:jar:0.3.4:compile,Apache v2, https://github.c
 com.google.protobuf:protobuf-java:jar:2.5.0:compile,New BSD license,http://code.google.com/p/protobuf
 com.google.protobuf:protobuf-java:jar:2.6.1:compile,New BSD license,http://code.google.com/p/protobuf
 com.jcraft:jsch:jar:0.1.42:compile,BSD,http://www.jcraft.com/jsch/
+com.jayway.jsonpath:json-path:jar:2.3.0:compile,Apache v2,https://github.com/json-path/JsonPath
+net.minidev:accessors-smart:jar:1.2:compile,Apache v2,https://github.com/netplex/json-smart-v2
+net.minidev:json-smart:jar:2.3:compile,Apache v2,https://github.com/netplex/json-smart-v2
 com.maxmind.db:maxmind-db:jar:1.2.1:compile,CC-BY-SA 3.0,https://github.com/maxmind/MaxMind-DB
 com.maxmind.geoip2:geoip2:jar:2.8.0:compile,Apache v2,https://github.com/maxmind/GeoIP2-java
 com.sun.xml.bind:jaxb-impl:jar:2.2.3-1:compile,CDDL,http://jaxb.java.net/

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec
----------------------------------------------------------------------
diff --git a/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec b/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec
index 265d595..cc01d7c 100644
--- a/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec
+++ b/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec
@@ -147,6 +147,7 @@ This package installs the Metron Parser files
 %{metron_home}/bin/start_parser_topology.sh
 %{metron_home}/config/zookeeper/parsers/bro.json
 %{metron_home}/config/zookeeper/parsers/jsonMap.json
+%{metron_home}/config/zookeeper/parsers/jsonMapQuery.json
 %{metron_home}/config/zookeeper/parsers/snort.json
 %{metron_home}/config/zookeeper/parsers/squid.json
 %{metron_home}/config/zookeeper/parsers/websphere.json

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-platform/metron-integration-test/src/main/sample/data/jsonMapQuery/parsed/jsonMapExampleParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/jsonMapQuery/parsed/jsonMapExampleParsed b/metron-platform/metron-integration-test/src/main/sample/data/jsonMapQuery/parsed/jsonMapExampleParsed
new file mode 100644
index 0000000..e614bda
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/jsonMapQuery/parsed/jsonMapExampleParsed
@@ -0,0 +1,2 @@
+{ "string" : "bar", "number" : 2, "ignored" : [ "blah" ], "original_string":"{ \"string\" : \"bar\", \"number\" : 2, \"ignored\" : [ \"blah\" ] }","timestamp":1000000000000, "source.type":"jsonMapQuery","guid":"this-is-random-uuid-will-be-36-chars" }
+{ "number" : 7 , "original_string" : "{ \"number\" : 7 }", "source.type":"jsonMapQuery","timestamp":1000000000000,"guid":"this-is-random-uuid-will-be-36-chars"}

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-platform/metron-integration-test/src/main/sample/data/jsonMapQuery/raw/jsonMapExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/jsonMapQuery/raw/jsonMapExampleOutput b/metron-platform/metron-integration-test/src/main/sample/data/jsonMapQuery/raw/jsonMapExampleOutput
new file mode 100644
index 0000000..8f25f4f
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/jsonMapQuery/raw/jsonMapExampleOutput
@@ -0,0 +1 @@
+{"foo":[{ "string" : "bar", "number" : 2, "ignored" : [ "blah" ] },{ "number" : 7 }]}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-platform/metron-parsers/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/README.md b/metron-platform/metron-parsers/README.md
index ade0f51..3d9fdfe 100644
--- a/metron-platform/metron-parsers/README.md
+++ b/metron-platform/metron-parsers/README.md
@@ -43,6 +43,7 @@ There are two general types types of parsers:
       * `UNFOLD` : Unfold inner maps.  So `{ "foo" : { "bar" : 1} }` would turn into `{"foo.bar" : 1}`
       * `ALLOW` : Allow multidimensional maps
       * `ERROR` : Throw an error when a multidimensional map is encountered
+    * `jsonpQuery` : A [JSON Path](#json_path) query string. If present, the result of the JSON Path query should be a list of messages. This is useful if you have a JSON document which contains a list or array of messages embedded in it, and you do not have another means of splitting the message.
     * A field called `timestamp` is expected to exist and, if it does not, then current time is inserted.  
     
 ## Parser Architecture
@@ -520,3 +521,14 @@ be customized by modifying the arguments sent to this utility.
  
 Finally, if workers and executors are new to you, the following might be of use to you:
 * [Understanding the Parallelism of a Storm Topology](http://www.michael-noll.com/blog/2012/10/16/understanding-the-parallelism-of-a-storm-topology/)
+
+## JSON Path
+
+> "JSONPath expressions always refer to a JSON structure in the same way as XPath expression are used in combination with an XML document."
+> ~ Stefan Goessner
+
+
+- [JSON Path concept](http://goessner.net/articles/JsonPath/)
+- [Read about JSON Path library Apache Metron uses](https://github.com/json-path/JsonPath)
+- [Try JSON Path expressions online](http://jsonpath.herokuapp.com)
+

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-platform/metron-parsers/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/pom.xml b/metron-platform/metron-parsers/pom.xml
index f856654..c481864 100644
--- a/metron-platform/metron-parsers/pom.xml
+++ b/metron-platform/metron-parsers/pom.xml
@@ -256,6 +256,11 @@
             <version>2.2.6</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>com.jayway.jsonpath</groupId>
+            <artifactId>json-path</artifactId>
+            <version>2.3.0</version>
+        </dependency>
     </dependencies>
     <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/jsonMapQuery.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/jsonMapQuery.json b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/jsonMapQuery.json
new file mode 100644
index 0000000..7dad779
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/jsonMapQuery.json
@@ -0,0 +1,5 @@
+{
+  "parserClassName":"org.apache.metron.parsers.json.JSONMapParser",
+  "sensorTopic":"jsonMapQuery",
+  "parserConfig": {"jsonpQuery":"$.foo"}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/json/JSONMapParser.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/json/JSONMapParser.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/json/JSONMapParser.java
index 7e5468f..bddf35d 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/json/JSONMapParser.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/json/JSONMapParser.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,65 +15,116 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.metron.parsers.json;
 
 import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableList;
-import org.apache.metron.common.utils.JSONUtils;
-import org.apache.metron.parsers.BasicParser;
-import org.json.simple.JSONObject;
-
+import com.jayway.jsonpath.Configuration;
+import com.jayway.jsonpath.JsonPath;
+import com.jayway.jsonpath.Option;
+import com.jayway.jsonpath.TypeRef;
+import com.jayway.jsonpath.spi.cache.CacheProvider;
+import com.jayway.jsonpath.spi.cache.LRUCache;
+import com.jayway.jsonpath.spi.json.JacksonJsonProvider;
+import com.jayway.jsonpath.spi.json.JsonProvider;
+import com.jayway.jsonpath.spi.mapper.JacksonMappingProvider;
+import com.jayway.jsonpath.spi.mapper.MappingProvider;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.metron.common.utils.JSONUtils;
+import org.apache.metron.parsers.BasicParser;
+import org.json.simple.JSONObject;
 
 public class JSONMapParser extends BasicParser {
-  private static interface Handler {
+
+  private interface Handler {
+
     JSONObject handle(String key, Map value, JSONObject obj);
   }
-  public static enum MapStrategy implements Handler {
-     DROP((key, value, obj) -> obj)
-    ,UNFOLD( (key, value, obj) -> {
-      return recursiveUnfold(key,value,obj);
-    })
-    ,ALLOW((key, value, obj) -> {
+
+  @SuppressWarnings("unchecked")
+  public enum MapStrategy implements Handler {
+    DROP((key, value, obj) -> obj), UNFOLD((key, value, obj) -> {
+      return recursiveUnfold(key, value, obj);
+    }), ALLOW((key, value, obj) -> {
       obj.put(key, value);
       return obj;
-    })
-    ,ERROR((key, value, obj) -> {
-      throw new IllegalStateException("Unable to process " + key + " => " + value + " because value is a map.");
-    })
-    ;
+    }), ERROR((key, value, obj) -> {
+      throw new IllegalStateException(
+          "Unable to process " + key + " => " + value + " because value is a map.");
+    });
     Handler handler;
+
     MapStrategy(Handler handler) {
       this.handler = handler;
     }
 
-    private static JSONObject recursiveUnfold(String key, Map value, JSONObject obj){
+    @SuppressWarnings("unchecked")
+    private static JSONObject recursiveUnfold(String key, Map value, JSONObject obj) {
       Set<Map.Entry<Object, Object>> entrySet = value.entrySet();
-      for(Map.Entry<Object, Object> kv : entrySet) {
+      for (Map.Entry<Object, Object> kv : entrySet) {
         String newKey = Joiner.on(".").join(key, kv.getKey().toString());
-        if(kv.getValue() instanceof Map){
-          recursiveUnfold(newKey,(Map)kv.getValue(),obj);
-        }else {
+        if (kv.getValue() instanceof Map) {
+          recursiveUnfold(newKey, (Map) kv.getValue(), obj);
+        } else {
           obj.put(newKey, kv.getValue());
         }
       }
       return obj;
     }
+
     @Override
     public JSONObject handle(String key, Map value, JSONObject obj) {
       return handler.handle(key, value, obj);
     }
 
   }
+
   public static final String MAP_STRATEGY_CONFIG = "mapStrategy";
+  public static final String JSONP_QUERY = "jsonpQuery";
+
   private MapStrategy mapStrategy = MapStrategy.DROP;
+  private TypeRef<List<Map<String, Object>>> typeRef = new TypeRef<List<Map<String, Object>>>() {
+  };
+  private String jsonpQuery = null;
+
 
   @Override
   public void configure(Map<String, Object> config) {
     String strategyStr = (String) config.getOrDefault(MAP_STRATEGY_CONFIG, MapStrategy.DROP.name());
     mapStrategy = MapStrategy.valueOf(strategyStr);
+    if (config.containsKey(JSONP_QUERY)) {
+      jsonpQuery = (String) config.get(JSONP_QUERY);
+      Configuration.setDefaults(new Configuration.Defaults() {
+
+        private final JsonProvider jsonProvider = new JacksonJsonProvider();
+        private final MappingProvider mappingProvider = new JacksonMappingProvider();
+
+        @Override
+        public JsonProvider jsonProvider() {
+          return jsonProvider;
+        }
+
+        @Override
+        public MappingProvider mappingProvider() {
+          return mappingProvider;
+        }
+
+        @Override
+        public Set<Option> options() {
+          return EnumSet.of(Option.SUPPRESS_EXCEPTIONS);
+        }
+      });
+
+      if (CacheProvider.getCache() == null) {
+        CacheProvider.setCache(new LRUCache(100));
+      }
+    }
   }
 
   /**
@@ -87,22 +138,36 @@ public class JSONMapParser extends BasicParser {
   /**
    * Take raw data and convert it to a list of messages.
    *
-   * @param rawMessage
    * @return If null is returned, this is treated as an empty list.
    */
   @Override
+  @SuppressWarnings("unchecked")
   public List<JSONObject> parse(byte[] rawMessage) {
     try {
       String originalString = new String(rawMessage);
-      //convert the JSON blob into a String -> Object map
-      Map<String, Object> rawMap = JSONUtils.INSTANCE.load(originalString, JSONUtils.MAP_SUPPLIER);
-      JSONObject ret = normalizeJSON(rawMap);
-      ret.put("original_string", originalString );
-      if(!ret.containsKey("timestamp")) {
-        //we have to ensure that we have a timestamp.  This is one of the pre-requisites for the parser.
-        ret.put("timestamp", System.currentTimeMillis());
+      List<Map<String, Object>> messages = new ArrayList<>();
+
+      if (!StringUtils.isEmpty(jsonpQuery)) {
+        Object parsedObject = JsonPath.parse(new String(rawMessage)).read(jsonpQuery, typeRef);
+        if(parsedObject != null) {
+          messages.addAll((List<Map<String,Object>>)parsedObject);
+        }
+      } else {
+        messages.add(JSONUtils.INSTANCE.load(originalString, JSONUtils.MAP_SUPPLIER));
+      }
+
+      ArrayList<JSONObject> parsedMessages = new ArrayList<>();
+      for (Map<String, Object> rawMessageMap : messages) {
+        JSONObject originalJsonObject = new JSONObject(rawMessageMap);
+        JSONObject ret = normalizeJson(rawMessageMap);
+        // the original string is the original for THIS sub message
+        ret.put("original_string", originalJsonObject.toJSONString());
+        if (!ret.containsKey("timestamp")) {
+          ret.put("timestamp", System.currentTimeMillis());
+        }
+        parsedMessages.add(ret);
       }
-      return ImmutableList.of(ret);
+      return Collections.unmodifiableList(parsedMessages);
     } catch (Throwable e) {
       String message = "Unable to parse " + new String(rawMessage) + ": " + e.getMessage();
       LOG.error(message, e);
@@ -111,18 +176,16 @@ public class JSONMapParser extends BasicParser {
   }
 
   /**
-   * Process all sub-maps via the MapHandler.  We have standardized on one-dimensional maps as our data model..
-   *
-   * @param map
-   * @return
+   * Process all sub-maps via the MapHandler.
+   * We have standardized on one-dimensional maps as our data model.
    */
-  private JSONObject normalizeJSON(Map<String, Object> map) {
+  @SuppressWarnings("unchecked")
+  private JSONObject normalizeJson(Map<String, Object> map) {
     JSONObject ret = new JSONObject();
-    for(Map.Entry<String, Object> kv : map.entrySet()) {
-      if(kv.getValue() instanceof Map) {
+    for (Map.Entry<String, Object> kv : map.entrySet()) {
+      if (kv.getValue() instanceof Map) {
         mapStrategy.handle(kv.getKey(), (Map) kv.getValue(), ret);
-      }
-      else {
+      } else {
         ret.put(kv.getKey(), kv.getValue());
       }
     }

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/JSONMapQueryIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/JSONMapQueryIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/JSONMapQueryIntegrationTest.java
new file mode 100644
index 0000000..cec6242
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/JSONMapQueryIntegrationTest.java
@@ -0,0 +1,36 @@
+/**
+ * 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.metron.parsers.integration;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.metron.parsers.integration.validation.SampleDataValidation;
+
+public class JSONMapQueryIntegrationTest extends ParserIntegrationTest {
+  @Override
+  String getSensorType() {
+    return "jsonMapQuery";
+  }
+
+  @Override
+  List<ParserValidation> getValidations() {
+    return new ArrayList<ParserValidation>() {{
+      add(new SampleDataValidation());
+    }};
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/validation/SampleDataValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/validation/SampleDataValidation.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/validation/SampleDataValidation.java
index 9ea9b71..1dff22f 100644
--- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/validation/SampleDataValidation.java
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/validation/SampleDataValidation.java
@@ -41,7 +41,7 @@ public class SampleDataValidation implements ParserValidation {
       String expectedMessage = new String(expectedMessages.get(i));
       String actualMessage = new String(actualMessages.get(i));
       try {
-        ValidationUtils.assertJSONEqual(expectedMessage, actualMessage);
+        ValidationUtils.assertJsonEqual(expectedMessage, actualMessage);
       } catch (Throwable t) {
         System.out.println("expected: " + expectedMessage);
         System.out.println("actual: " + actualMessage);

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/json/JSONMapParserQueryTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/json/JSONMapParserQueryTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/json/JSONMapParserQueryTest.java
new file mode 100644
index 0000000..9f8c26b
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/json/JSONMapParserQueryTest.java
@@ -0,0 +1,201 @@
+/**
+ * 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.metron.parsers.json;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.HashMap;
+import java.util.List;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.log4j.Level;
+import org.apache.metron.parsers.BasicParser;
+import org.apache.metron.test.utils.UnitTestHelper;
+import org.json.simple.JSONObject;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class JSONMapParserQueryTest {
+
+  /**
+   * {
+   * "foo" :
+   * [
+   * { "name" : "foo1", "value" : "bar", "number" : 1.0 },
+   * { "name" : "foo2", "value" : "baz", "number" : 2.0 }
+   * ]
+   * }
+   */
+  @Multiline
+  static String JSON_LIST;
+
+  /**
+   * { "name" : "foo1", "value" : "bar", "number" : 1.0 }
+   */
+  @Multiline
+  static String JSON_SINGLE;
+
+  /**
+   * { "name" : "foo2", "value" : "baz", "number" : 2.0 }
+   */
+  @Multiline
+  static String JSON_SINGLE2;
+
+  @Test
+  public void testHappyPath() {
+    JSONMapParser parser = new JSONMapParser();
+    parser.configure(new HashMap<String, Object>() {{
+      put(JSONMapParser.JSONP_QUERY, "$.foo");
+    }});
+    List<JSONObject> output = parser.parse(JSON_LIST.getBytes());
+    Assert.assertEquals(output.size(), 2);
+    //don't forget the timestamp field!
+    Assert.assertEquals(output.get(0).size(), 5);
+    JSONObject message = output.get(0);
+    Assert.assertEquals("foo1", message.get("name"));
+    Assert.assertEquals("bar", message.get("value"));
+    Assert.assertEquals(1.0, message.get("number"));
+    Assert.assertNotNull(message.get("timestamp"));
+    Assert.assertTrue(message.get("timestamp") instanceof Number);
+    Assert.assertNotNull(message.get("number"));
+    Assert.assertTrue(message.get("number") instanceof Number);
+
+    message = output.get(1);
+    Assert.assertEquals("foo2", message.get("name"));
+    Assert.assertEquals("baz", message.get("value"));
+    Assert.assertEquals(2.0, message.get("number"));
+    Assert.assertNotNull(message.get("timestamp"));
+    Assert.assertTrue(message.get("timestamp") instanceof Number);
+    Assert.assertNotNull(message.get("number"));
+    Assert.assertTrue(message.get("number") instanceof Number);
+
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testInvalidJSONPathThrows() {
+    JSONMapParser parser = new JSONMapParser();
+    parser.configure(new HashMap<String, Object>() {{
+      put(JSONMapParser.JSONP_QUERY, "$$..$$SDSE$#$#.");
+    }});
+    List<JSONObject> output = parser.parse(JSON_LIST.getBytes());
+
+  }
+
+  @Test
+  public void testNoMatchesNoExceptions() {
+    JSONMapParser parser = new JSONMapParser();
+    parser.configure(new HashMap<String, Object>() {{
+      put(JSONMapParser.JSONP_QUERY, "$.foo");
+    }});
+    List<JSONObject> output = parser.parse(JSON_SINGLE.getBytes());
+    Assert.assertEquals(0, output.size());
+  }
+
+  /**
+   * {
+   * "foo" :
+   * [
+   * {
+   * "collection" : { "blah" : 7, "blah2" : "foo", "bigblah" : { "innerBlah" : "baz", "reallyInnerBlah" : { "color" : "grey" }}}
+   * },
+   * {
+   * "collection" : { "blah" : 8, "blah2" : "bar", "bigblah" : { "innerBlah" : "baz2", "reallyInnerBlah" : { "color" : "blue" }}}
+   * }
+   * ]
+   * }
+   */
+  @Multiline
+  static String collectionHandlingJSON;
+
+  @Test
+  public void testCollectionHandlingDrop() {
+    JSONMapParser parser = new JSONMapParser();
+    parser.configure(new HashMap<String, Object>() {{
+      put(JSONMapParser.JSONP_QUERY, "$.foo");
+    }});
+    List<JSONObject> output = parser.parse(collectionHandlingJSON.getBytes());
+    Assert.assertEquals(output.size(), 2);
+
+    //don't forget the timestamp field!
+    Assert.assertEquals(output.get(0).size(), 2);
+
+    JSONObject message = output.get(0);
+    Assert.assertNotNull(message.get("timestamp"));
+    Assert.assertTrue(message.get("timestamp") instanceof Number);
+
+    message = output.get(1);
+    Assert.assertNotNull(message.get("timestamp"));
+    Assert.assertTrue(message.get("timestamp") instanceof Number);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testCollectionHandlingError() {
+    JSONMapParser parser = new JSONMapParser();
+    parser.configure(ImmutableMap
+        .of(JSONMapParser.MAP_STRATEGY_CONFIG, JSONMapParser.MapStrategy.ERROR.name(),
+            JSONMapParser.JSONP_QUERY, "$.foo"));
+    UnitTestHelper.setLog4jLevel(BasicParser.class, Level.FATAL);
+    parser.parse(collectionHandlingJSON.getBytes());
+    UnitTestHelper.setLog4jLevel(BasicParser.class, Level.ERROR);
+  }
+
+
+  @Test
+  public void testCollectionHandlingAllow() {
+    JSONMapParser parser = new JSONMapParser();
+    parser.configure(ImmutableMap
+        .of(JSONMapParser.MAP_STRATEGY_CONFIG, JSONMapParser.MapStrategy.ALLOW.name(),
+            JSONMapParser.JSONP_QUERY, "$.foo"));
+    List<JSONObject> output = parser.parse(collectionHandlingJSON.getBytes());
+    Assert.assertEquals(output.size(), 2);
+    Assert.assertEquals(output.get(0).size(), 3);
+    JSONObject message = output.get(0);
+    Assert.assertNotNull(message.get("timestamp"));
+    Assert.assertTrue(message.get("timestamp") instanceof Number);
+
+    Assert.assertEquals(output.get(1).size(), 3);
+    message = output.get(1);
+    Assert.assertNotNull(message.get("timestamp"));
+    Assert.assertTrue(message.get("timestamp") instanceof Number);
+  }
+
+  @Test
+  public void testCollectionHandlingUnfold() {
+    JSONMapParser parser = new JSONMapParser();
+    parser.configure(ImmutableMap
+        .of(JSONMapParser.MAP_STRATEGY_CONFIG, JSONMapParser.MapStrategy.UNFOLD.name(),
+            JSONMapParser.JSONP_QUERY, "$.foo"));
+    List<JSONObject> output = parser.parse(collectionHandlingJSON.getBytes());
+    Assert.assertEquals(output.size(), 2);
+    Assert.assertEquals(output.get(0).size(), 6);
+    JSONObject message = output.get(0);
+    Assert.assertEquals(message.get("collection.blah"), 7);
+    Assert.assertEquals(message.get("collection.blah2"), "foo");
+    Assert.assertEquals(message.get("collection.bigblah.innerBlah"), "baz");
+    Assert.assertEquals(message.get("collection.bigblah.reallyInnerBlah.color"), "grey");
+    Assert.assertNotNull(message.get("timestamp"));
+    Assert.assertTrue(message.get("timestamp") instanceof Number);
+
+    Assert.assertEquals(output.get(1).size(), 6);
+    message = output.get(1);
+    Assert.assertEquals(message.get("collection.blah"), 8);
+    Assert.assertEquals(message.get("collection.blah2"), "bar");
+    Assert.assertEquals(message.get("collection.bigblah.innerBlah"), "baz2");
+    Assert.assertEquals(message.get("collection.bigblah.reallyInnerBlah.color"), "blue");
+    Assert.assertNotNull(message.get("timestamp"));
+    Assert.assertTrue(message.get("timestamp") instanceof Number);
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/9c5d9d76/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/ValidationUtils.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/ValidationUtils.java b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/ValidationUtils.java
index 279caa3..98fd258 100644
--- a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/ValidationUtils.java
+++ b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/utils/ValidationUtils.java
@@ -7,7 +7,7 @@
  * "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
+ * 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,
@@ -15,33 +15,57 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.metron.test.utils;
 
-import org.codehaus.jackson.map.ObjectMapper;
-import org.junit.Assert;
+package org.apache.metron.test.utils;
 
 import java.io.IOException;
 import java.util.Map;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Assert;
 
 public class ValidationUtils {
 
-  public static void assertJSONEqual(String expected, String actual) throws IOException {
+  /**
+   * Validates that two JSON Strings are equal in value.
+   * Since JSON does not guarentee order of fields, we cannot just compare Strings.
+   * <p>
+   * This utility understands that the 'original_string' field may itself hold JSON,
+   * and will attempt to validate that field as json if it fails straight string compare
+   * </p>
+   * @param expected the expected string value
+   * @param actual the actual string value
+   * @throws IOException if there is an issue parsing as json
+   */
+  public static void assertJsonEqual(String expected, String actual) throws IOException {
     ObjectMapper mapper = new ObjectMapper();
     Map m1 = mapper.readValue(expected, Map.class);
     Map m2 = mapper.readValue(actual, Map.class);
-    for(Object k : m1.keySet()) {
+    for (Object k : m1.keySet()) {
       Object v1 = m1.get(k);
       Object v2 = m2.get(k);
 
-      if(v2 == null) {
+      if (v2 == null) {
         Assert.fail("Unable to find key: " + k + " in output");
       }
-      if(k.equals("timestamp") || k.equals("guid")) {
+      if (k.equals("timestamp") || k.equals("guid")) {
         //TODO: Take the ?!?@ timestamps out of the reference file.
         Assert.assertEquals(v1.toString().length(), v2.toString().length());
-      }
-      else if(!v2.equals(v1)) {
-        Assert.assertEquals("value mismatch for " + k ,v1, v2);
+      } else if (!v2.equals(v1)) {
+        boolean goodDeepDown = false;
+        // if this fails, but is the original_string it may be in json format
+        // where the field/value order may be random
+        if (((String) k).equals("original_string")) {
+          try {
+            mapper.readValue((String) v1, Map.class);
+            assertJsonEqual((String) v1, (String) v2);
+            goodDeepDown = true;
+          } catch (Exception e) {
+            // nothing, the original fail stands
+          }
+        }
+        if (!goodDeepDown) {
+          Assert.assertEquals("value mismatch for " + k, v1, v2);
+        }
       }
     }
     Assert.assertEquals(m1.size(), m2.size());