You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ce...@apache.org on 2016/09/24 05:10:24 UTC

incubator-metron git commit: METRON-434: JSON Parser closes apache/incubator-metron#261

Repository: incubator-metron
Updated Branches:
  refs/heads/master c10a53eb8 -> 6834e146d


METRON-434: JSON Parser closes apache/incubator-metron#261


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

Branch: refs/heads/master
Commit: 6834e146de163b5f6a441410a72fefb8db96944e
Parents: c10a53e
Author: cstella <ce...@gmail.com>
Authored: Sat Sep 24 01:09:43 2016 -0400
Committer: cstella <ce...@gmail.com>
Committed: Sat Sep 24 01:09:43 2016 -0400

----------------------------------------------------------------------
 .../data/jsonMap/parsed/jsonMapExampleParsed    |   2 +
 .../data/jsonMap/raw/jsonMapExampleOutput       |   2 +
 .../main/config/zookeeper/parsers/jsonMap.json  |   4 +
 .../metron/parsers/json/JSONMapParser.java      | 129 +++++++++++++++++++
 .../integration/JSONMapIntegrationTest.java     |  37 ++++++
 .../metron/parsers/json/JSONMapParserTest.java  | 112 ++++++++++++++++
 6 files changed, 286 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/6834e146/metron-platform/metron-integration-test/src/main/sample/data/jsonMap/parsed/jsonMapExampleParsed
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/jsonMap/parsed/jsonMapExampleParsed b/metron-platform/metron-integration-test/src/main/sample/data/jsonMap/parsed/jsonMapExampleParsed
new file mode 100644
index 0000000..795ed45
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/jsonMap/parsed/jsonMapExampleParsed
@@ -0,0 +1,2 @@
+{ "string" : "bar", "number" : 2, "ignored" : [ "blah" ], "original_string":"{ \"string\" : \"bar\", \"number\" : 2, \"ignored\" : [ \"blah\" ] }","timestamp":1000000000000, "source.type":"jsonMap" }
+{ "number" : 7 , "original_string" : "{ \"number\" : 7 }", "source.type":"jsonMap","timestamp":1000000000000}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/6834e146/metron-platform/metron-integration-test/src/main/sample/data/jsonMap/raw/jsonMapExampleOutput
----------------------------------------------------------------------
diff --git a/metron-platform/metron-integration-test/src/main/sample/data/jsonMap/raw/jsonMapExampleOutput b/metron-platform/metron-integration-test/src/main/sample/data/jsonMap/raw/jsonMapExampleOutput
new file mode 100644
index 0000000..8064084
--- /dev/null
+++ b/metron-platform/metron-integration-test/src/main/sample/data/jsonMap/raw/jsonMapExampleOutput
@@ -0,0 +1,2 @@
+{ "string" : "bar", "number" : 2, "ignored" : [ "blah" ] }
+{ "number" : 7 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/6834e146/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/jsonMap.json
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/jsonMap.json b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/jsonMap.json
new file mode 100644
index 0000000..39e1da9
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/config/zookeeper/parsers/jsonMap.json
@@ -0,0 +1,4 @@
+{
+  "parserClassName":"org.apache.metron.parsers.json.JSONMapParser",
+  "sensorTopic":"jsonMap"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/6834e146/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
new file mode 100644
index 0000000..0bcf0f5
--- /dev/null
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/json/JSONMapParser.java
@@ -0,0 +1,129 @@
+/**
+ * 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.fasterxml.jackson.core.type.TypeReference;
+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 java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class JSONMapParser extends BasicParser {
+  private static 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) -> {
+      Set<Map.Entry<Object, Object>> entrySet = value.entrySet();
+      for(Map.Entry<Object, Object> kv : entrySet) {
+        String newKey = Joiner.on(".").join(key, kv.getKey().toString());
+        obj.put(newKey, kv.getValue());
+      }
+      return 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.");
+    })
+    ;
+    Handler handler;
+    MapStrategy(Handler handler) {
+      this.handler = handler;
+    }
+
+    @Override
+    public JSONObject handle(String key, Map value, JSONObject obj) {
+      return handler.handle(key, value, obj);
+    }
+
+  }
+  public static final String MAP_STRATEGY_CONFIG = "mapStrategy";
+  private MapStrategy mapStrategy = MapStrategy.DROP;
+
+  @Override
+  public void configure(Map<String, Object> config) {
+    String strategyStr = (String) config.getOrDefault(MAP_STRATEGY_CONFIG, MapStrategy.DROP.name());
+    mapStrategy = MapStrategy.valueOf(strategyStr);
+  }
+
+  /**
+   * Initialize the message parser.  This is done once.
+   */
+  @Override
+  public void init() {
+
+  }
+
+  /**
+   * 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
+  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, new TypeReference<Map<String, Object>>() {
+      });
+      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());
+      }
+      return ImmutableList.of(ret);
+    } catch (Throwable e) {
+      String message = "Unable to parse " + new String(rawMessage) + ": " + e.getMessage();
+      LOG.error(message, e);
+      throw new IllegalStateException(message, e);
+    }
+  }
+
+  /**
+   * Process all sub-maps via the MapHandler.  We have standardized on one-dimensional maps as our data model..
+   *
+   * @param map
+   * @return
+   */
+  private JSONObject normalizeJSON(Map<String, Object> map) {
+    JSONObject ret = new JSONObject();
+    for(Map.Entry<String, Object> kv : map.entrySet()) {
+      if(kv.getValue() instanceof Map) {
+        mapStrategy.handle(kv.getKey(), (Map) kv.getValue(), ret);
+      }
+      else {
+        ret.put(kv.getKey(), kv.getValue());
+      }
+    }
+    return ret;
+  }
+
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/6834e146/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/json/JSONMapParserTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/json/JSONMapParserTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/json/JSONMapParserTest.java
new file mode 100644
index 0000000..1299d97
--- /dev/null
+++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/json/JSONMapParserTest.java
@@ -0,0 +1,112 @@
+/**
+ * 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 org.adrianwalker.multilinestring.Multiline;
+import org.json.simple.JSONObject;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+
+public class JSONMapParserTest {
+
+  /**
+   {
+     "foo" : "bar"
+    ,"blah" : "blah"
+    ,"number" : 2.0
+   }
+   */
+   @Multiline
+   static String happyPathJSON;
+
+  @Test
+  public void testHappyPath() {
+    JSONMapParser parser = new JSONMapParser();
+    List<JSONObject> output = parser.parse(happyPathJSON.getBytes());
+    Assert.assertEquals(output.size(), 1);
+    //don't forget the timestamp field!
+    Assert.assertEquals(output.get(0).size(), 5);
+    JSONObject message = output.get(0);
+    Assert.assertEquals("bar", message.get("foo"));
+    Assert.assertEquals("blah", message.get("blah"));
+    Assert.assertNotNull(message.get("timestamp"));
+    Assert.assertTrue(message.get("timestamp") instanceof Number);
+    Assert.assertNotNull(message.get("number"));
+    Assert.assertTrue(message.get("number") instanceof Number);
+  }
+
+  /**
+   {
+    "collection" : { "blah" : 7, "blah2" : "foo" }
+   }
+   */
+   @Multiline
+   static String collectionHandlingJSON;
+
+  @Test
+  public void testCollectionHandlingDrop() {
+    JSONMapParser parser = new JSONMapParser();
+    List<JSONObject> output = parser.parse(collectionHandlingJSON.getBytes());
+    Assert.assertEquals(output.size(), 1);
+    //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);
+  }
+
+  @Test(expected=IllegalStateException.class)
+  public void testCollectionHandlingError() {
+    JSONMapParser parser = new JSONMapParser();
+    parser.configure(ImmutableMap.of(JSONMapParser.MAP_STRATEGY_CONFIG, JSONMapParser.MapStrategy.ERROR.name()));
+    parser.parse(collectionHandlingJSON.getBytes());
+  }
+
+
+  @Test
+  public void testCollectionHandlingAllow() {
+    JSONMapParser parser = new JSONMapParser();
+    parser.configure(ImmutableMap.of(JSONMapParser.MAP_STRATEGY_CONFIG, JSONMapParser.MapStrategy.ALLOW.name()));
+    List<JSONObject> output = parser.parse(collectionHandlingJSON.getBytes());
+    Assert.assertEquals(output.size(), 1);
+    //don't forget the timestamp field!
+    Assert.assertEquals(output.get(0).size(), 3);
+    JSONObject message = output.get(0);
+    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()));
+    List<JSONObject> output = parser.parse(collectionHandlingJSON.getBytes());
+    Assert.assertEquals(output.size(), 1);
+    //don't forget the timestamp field!
+    Assert.assertEquals(output.get(0).size(), 4);
+    JSONObject message = output.get(0);
+    Assert.assertEquals(message.get("collection.blah"), 7);
+    Assert.assertEquals(message.get("collection.blah2"), "foo");
+    Assert.assertNotNull(message.get("timestamp"));
+    Assert.assertTrue(message.get("timestamp") instanceof Number);
+  }
+}