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/01 10:56:19 UTC

incubator-metron git commit: METRON-401 ParseException Thrown by ThreatIntelJoinBolt (nickwallen via cestella) closes apache/incubator-metron#239

Repository: incubator-metron
Updated Branches:
  refs/heads/master b7e16d2f9 -> 839bd2093


METRON-401 ParseException Thrown by ThreatIntelJoinBolt (nickwallen via cestella) closes apache/incubator-metron#239


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

Branch: refs/heads/master
Commit: 839bd20931d62219f4e375199601899ab3f71001
Parents: b7e16d2
Author: nickwallen <ni...@nickallen.org>
Authored: Thu Sep 1 20:56:07 2016 +1000
Committer: cstella <ce...@gmail.com>
Committed: Thu Sep 1 20:56:07 2016 +1000

----------------------------------------------------------------------
 .../enrichment/bolt/ThreatIntelJoinBolt.java    |   5 +-
 .../bolt/ThreatIntelJoinBoltTest.java           | 138 ++++++----
 .../threatintel/triage/ThreatTriageTest.java    | 253 +++++++++++--------
 3 files changed, 230 insertions(+), 166 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/839bd209/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
index 00d0498..9506697 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBolt.java
@@ -40,10 +40,10 @@ import java.util.Map;
 
 public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
 
-  protected static final Logger LOG = LoggerFactory
-          .getLogger(ThreatIntelJoinBolt.class);
+  protected static final Logger LOG = LoggerFactory.getLogger(ThreatIntelJoinBolt.class);
   private FunctionResolver functionResolver;
   private org.apache.metron.common.dsl.Context stellarContext;
+
   public ThreatIntelJoinBolt(String zookeeperUrl) {
     super(zookeeperUrl);
   }
@@ -73,6 +73,7 @@ public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
     this.stellarContext = new Context.Builder()
                                 .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
                                 .build();
+    this.functionResolver = StellarFunctions.FUNCTION_RESOLVER();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/839bd209/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
index b3004bb..bb539f3 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/ThreatIntelJoinBoltTest.java
@@ -19,6 +19,7 @@ package org.apache.metron.enrichment.bolt;
 
 import com.fasterxml.jackson.databind.JsonMappingException;
 import junit.framework.Assert;
+import junit.framework.TestCase;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
@@ -39,33 +40,33 @@ import java.util.Map;
 public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
 
   /**
-   {
-   "field1": "value1",
-   "enrichedField1": "enrichedValue1",
-   "source.type": "test"
-   }
+   * {
+   * "field1": "value1",
+   * "enrichedField1": "enrichedValue1",
+   * "source.type": "test"
+   * }
    */
   @Multiline
   private String messageString;
 
   /**
-   {
-   "field1": "value1",
-   "enrichedField1": "enrichedValue1",
-   "source.type": "test",
-   "threatintels.field.end.ts": "timing"
-   }
+   * {
+   * "field1": "value1",
+   * "enrichedField1": "enrichedValue1",
+   * "source.type": "test",
+   * "threatintels.field.end.ts": "timing"
+   * }
    */
   @Multiline
   private String messageWithTimingString;
 
   /**
-   {
-   "field1": "value1",
-   "enrichedField1": "enrichedValue1",
-   "source.type": "test",
-   "threatintels.field": "threatIntelValue"
-   }
+   * {
+   * "field1": "value1",
+   * "enrichedField1": "enrichedValue1",
+   * "source.type": "test",
+   * "threatintels.field": "threatIntelValue"
+   * }
    */
   @Multiline
   private String alertMessageString;
@@ -83,31 +84,75 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
   }
 
   /**
-    {
-    "riskLevelRules" : {
-        "enrichedField1 == 'enrichedValue1'" : 10
-                          }
-   ,"aggregator" : "MAX"
-   }
+   * {
+   *  "riskLevelRules" : {
+   *    "enrichedField1 == 'enrichedValue1'" : 10
+   *  },
+   *  "aggregator" : "MAX"
+   * }
    */
   @Multiline
-  private static String threatTriageConfigStr;
+  private static String testWithTriageConfig;
+
+  @Test
+  public void testWithTriage() throws IOException {
+    test(testWithTriageConfig, false);
+  }
+
+  /**
+   * {
+   *  "riskLevelRules" : {
+   *    "enrichedField1 == 'enrichedValue1": 10
+   *  },
+   *  "aggregator" : "MAX"
+   * }
+   */
+  @Multiline
+  private static String testWithBadTriageRuleConfig;
+
+  @Test
+  public void testWithBadTriageRule() throws IOException {
+    test(testWithBadTriageRuleConfig, true);
+  }
+
+  @Test
+  public void testWithoutTriage() throws IOException {
+    test(null, false);
+  }
+
+  /**
+   * {
+   *   "riskLevelRules": {
+   *      "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))": 10
+   *   },
+   *   "aggregator": "MAX"
+   * }
+   */
+  @Multiline
+  private static String testWithStellarFunctionConfig;
+
+  @Test
+  public void testWithStellarFunction() throws IOException {
+    test(testWithStellarFunctionConfig, false);
+  }
 
   public void test(String threatTriageConfig, boolean badConfig) throws IOException {
+
     ThreatIntelJoinBolt threatIntelJoinBolt = new ThreatIntelJoinBolt("zookeeperUrl");
     threatIntelJoinBolt.setCuratorFramework(client);
     threatIntelJoinBolt.setTreeCache(cache);
-    SensorEnrichmentConfig enrichmentConfig = JSONUtils.INSTANCE.load(new FileInputStream(sampleSensorEnrichmentConfigPath), SensorEnrichmentConfig.class);
+
+    SensorEnrichmentConfig enrichmentConfig = JSONUtils.INSTANCE.load(
+            new FileInputStream(sampleSensorEnrichmentConfigPath), SensorEnrichmentConfig.class);
     boolean withThreatTriage = threatTriageConfig != null;
-    if(withThreatTriage) {
+    if (withThreatTriage) {
       try {
         enrichmentConfig.getThreatIntel().setTriageConfig(JSONUtils.INSTANCE.load(threatTriageConfig, ThreatTriageConfig.class));
-        if(badConfig) {
+        if (badConfig) {
           Assert.fail(threatTriageConfig + "\nThis should not parse!");
         }
-      }
-      catch(JsonMappingException pe) {
-        if(!badConfig) {
+      } catch (JsonMappingException pe) {
+        if (!badConfig) {
           throw pe;
         }
       }
@@ -116,49 +161,32 @@ public class ThreatIntelJoinBoltTest extends BaseEnrichmentBoltTest {
     threatIntelJoinBolt.withMaxCacheSize(100);
     threatIntelJoinBolt.withMaxTimeRetain(10000);
     threatIntelJoinBolt.prepare(new HashMap<>(), topologyContext, outputCollector);
+
     Map<String, Object> fieldMap = threatIntelJoinBolt.getFieldMap("incorrectSourceType");
     Assert.assertNull(fieldMap);
+
     fieldMap = threatIntelJoinBolt.getFieldMap(sensorType);
     Assert.assertTrue(fieldMap.containsKey("hbaseThreatIntel"));
+
     Map<String, JSONObject> streamMessageMap = new HashMap<>();
     streamMessageMap.put("message", message);
     JSONObject joinedMessage = threatIntelJoinBolt.joinMessages(streamMessageMap);
     Assert.assertFalse(joinedMessage.containsKey("is_alert"));
+
     streamMessageMap.put("message", messageWithTiming);
     joinedMessage = threatIntelJoinBolt.joinMessages(streamMessageMap);
     Assert.assertFalse(joinedMessage.containsKey("is_alert"));
+
     streamMessageMap.put("message", alertMessage);
     joinedMessage = threatIntelJoinBolt.joinMessages(streamMessageMap);
     Assert.assertTrue(joinedMessage.containsKey("is_alert") && "true".equals(joinedMessage.get("is_alert")));
+
     if(withThreatTriage && !badConfig) {
-      Assert.assertTrue(joinedMessage.containsKey("threat.triage.level") && Math.abs(10d - (Double) joinedMessage.get("threat.triage.level")) < 1e-10);
+      Assert.assertTrue(joinedMessage.containsKey("threat.triage.level") &&
+              Math.abs(10d - (Double) joinedMessage.get("threat.triage.level")) < 1e-10);
     }
     else {
       Assert.assertFalse(joinedMessage.containsKey("threat.triage.level"));
     }
   }
-  /**
-    {
-    "riskLevelRules" : {
-        "enrichedField1 == 'enrichedValue1" : 10
-                          }
-   ,"aggregator" : "MAX"
-   }
-   */
-  @Multiline
-  private static String badRuleThreatTriageConfigStr;
-
-
-  @Test
-  public void testWithTriage() throws IOException {
-    test(threatTriageConfigStr, false);
-  }
-  @Test
-  public void testWithBadTriageRule() throws IOException {
-    test(badRuleThreatTriageConfigStr, true);
-  }
-  @Test
-  public void testWithoutTriage() throws IOException {
-    test(null, false);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/839bd209/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java
index 37292e2..92191b6 100644
--- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/threatintel/triage/ThreatTriageTest.java
@@ -20,7 +20,6 @@ package org.apache.metron.threatintel.triage;
 
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
-import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
 import org.apache.metron.common.dsl.Context;
 import org.apache.metron.common.dsl.StellarFunctions;
 import org.apache.metron.common.utils.JSONUtils;
@@ -32,96 +31,99 @@ import java.util.HashMap;
 
 public class ThreatTriageTest {
   /**
-   {
-    "threatIntel" : {
-      "triageConfig" :
-      {
-        "riskLevelRules" : {
-            "user.type in [ 'admin', 'power' ] and asset.type == 'web'" : 10
-           ,"asset.type == 'web'" : 5
-          ,"user.type == 'normal'  and asset.type == 'web'" : 0
-          ,"user.type in whitelist" : -1
-                          }
-        ,"aggregator" : "MAX"
-      },
-      "config" : {
-        "whitelist" : [ "abnormal" ]
-                 }
-    }
-   }
+   * {
+   *  "threatIntel": {
+   *    "triageConfig": {
+   *      "riskLevelRules" : {
+   *        "user.type in [ 'admin', 'power' ] and asset.type == 'web'" : 10,
+   *        "asset.type == 'web'" : 5,
+   *        "user.type == 'normal'  and asset.type == 'web'" : 0,
+   *        "user.type in whitelist" : -1
+   *      },
+   *      "aggregator" : "MAX"
+   *    },
+   *    "config": {
+   *      "whitelist": [ "abnormal" ]
+   *    }
+   *  }
+   * }
    */
   @Multiline
   public static String smokeTestProcessorConfig;
 
-  private static ThreatTriageProcessor getProcessor(String config) throws IOException {
-    SensorEnrichmentConfig c = JSONUtils.INSTANCE.load(config, SensorEnrichmentConfig.class);
-    return new ThreatTriageProcessor(c, StellarFunctions.FUNCTION_RESOLVER(), Context.EMPTY_CONTEXT());
-  }
-
   @Test
   public void smokeTest() throws Exception {
     ThreatTriageProcessor threatTriageProcessor = getProcessor(smokeTestProcessorConfig);
-    Assert.assertEquals("Expected a score of 0"
-                       , 0d
-                       ,new ThreatTriageProcessor(new SensorEnrichmentConfig(), StellarFunctions.FUNCTION_RESOLVER(), Context.EMPTY_CONTEXT()).apply(new HashMap<Object, Object>() {{
-                          put("user.type", "admin");
-                          put("asset.type", "web");
-                                        }}
-                                        )
-                       , 1e-10
-                       );
-    Assert.assertEquals("Expected a score of 10"
-                       , 10d
-                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
-                          put("user.type", "admin");
-                          put("asset.type", "web");
-                                        }}
-                                        )
-                       , 1e-10
-                       );
-    Assert.assertEquals("Expected a score of 5"
-                       , 5d
-                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
-                          put("user.type", "normal");
-                          put("asset.type", "web");
-                                        }}
-                                        )
-                       , 1e-10
-                       );
-    Assert.assertEquals("Expected a score of 0"
-                       , 0d
-                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
-                          put("user.type", "foo");
-                          put("asset.type", "bar");
-                                        }}
-                                        )
-                       , 1e-10
-                       );
-    Assert.assertEquals("Expected a score of -Inf"
-                       , Double.NEGATIVE_INFINITY
-                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
-                          put("user.type", "abnormal");
-                          put("asset.type", "bar");
-                                        }}
-                                        )
-                       , 1e-10
-                       );
+
+    Assert.assertEquals(
+            "Expected a score of 0",
+            0d,
+            new ThreatTriageProcessor(
+                    new SensorEnrichmentConfig(),
+                    StellarFunctions.FUNCTION_RESOLVER(),
+                    Context.EMPTY_CONTEXT()).apply(
+                            new HashMap<Object, Object>() {{
+                              put("user.type", "admin");
+                              put("asset.type", "web");
+                            }}),
+            1e-10);
+
+    Assert.assertEquals(
+            "Expected a score of 10",
+            10d,
+            threatTriageProcessor.apply(
+                    new HashMap<Object, Object>() {{
+                      put("user.type", "admin");
+                      put("asset.type", "web");
+                    }}
+            ),
+            1e-10);
+
+    Assert.assertEquals(
+            "Expected a score of 5",
+            5d,
+            threatTriageProcessor.apply(
+                    new HashMap<Object, Object>() {{
+                      put("user.type", "normal");
+                      put("asset.type", "web");
+                    }}
+            ),
+            1e-10);
+
+    Assert.assertEquals(
+            "Expected a score of 0",
+            0d,
+            threatTriageProcessor.apply(
+                    new HashMap<Object, Object>() {{
+                      put("user.type", "foo");
+                      put("asset.type", "bar");
+                    }}),
+            1e-10);
+
+    Assert.assertEquals(
+            "Expected a score of -Inf",
+            Double.NEGATIVE_INFINITY,
+            threatTriageProcessor.apply(
+                    new HashMap<Object, Object>() {{
+                      put("user.type", "abnormal");
+                      put("asset.type", "bar");
+                    }}),
+            1e-10);
   }
 
   /**
-   {
-    "threatIntel" : {
-      "triageConfig" :
-      {
-        "riskLevelRules" : {
-            "user.type in [ 'admin', 'power' ] and asset.type == 'web'" : 10
-           ,"asset.type == 'web'" : 5
-           ,"user.type == 'normal'  and asset.type == 'web'" : 0
-                          }
-        ,"aggregator" : "POSITIVE_MEAN"
-      }
-                   }
-   }
+   * {
+   *  "threatIntel": {
+   *  "triageConfig": {
+   *    "riskLevelRules" : {
+   *      "user.type in [ 'admin', 'power' ] and asset.type == 'web'" : 10,
+   *      "asset.type == 'web'" : 5,
+   *      "user.type == 'normal' and asset.type == 'web'" : 0
+   *     },
+   *     "aggregator" : "POSITIVE_MEAN"
+   *    }
+   *  }
+   * }
    */
   @Multiline
   public static String positiveMeanProcessorConfig;
@@ -130,33 +132,66 @@ public class ThreatTriageTest {
   public void positiveMeanAggregationTest() throws Exception {
 
     ThreatTriageProcessor threatTriageProcessor = getProcessor(positiveMeanProcessorConfig);
-    Assert.assertEquals("Expected a score of 0"
-                       , 5d
-                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
-                          put("user.type", "normal");
-                          put("asset.type", "web");
-                                        }}
-                                        )
-                       , 1e-10
-                       );
-    Assert.assertEquals("Expected a score of 7.5"
-                       , (10 + 5)/2.0
-                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
-                          put("user.type", "admin");
-                          put("asset.type", "web");
-                                        }}
-                                        )
-                       , 1e-10
-                       );
-
-    Assert.assertEquals("Expected a score of 0"
-                       , 0d
-                       , threatTriageProcessor.apply(new HashMap<Object, Object>() {{
-                          put("user.type", "foo");
-                          put("asset.type", "bar");
-                                        }}
-                                        )
-                       , 1e-10
-                       );
+    Assert.assertEquals(
+            "Expected a score of 0",
+            5d,
+            threatTriageProcessor.apply(
+                    new HashMap<Object, Object>() {{
+                      put("user.type", "normal");
+                      put("asset.type", "web");
+                    }}),
+            1e-10);
+
+    Assert.assertEquals(
+            "Expected a score of 7.5",
+            (10 + 5)/2.0,
+            threatTriageProcessor.apply(
+                    new HashMap<Object, Object>() {{
+                      put("user.type", "admin");
+                      put("asset.type", "web");
+                    }}),
+            1e-10);
+
+    Assert.assertEquals(
+            "Expected a score of 0",
+            0d,
+            threatTriageProcessor.apply(
+                    new HashMap<Object, Object>() {{
+                      put("user.type", "foo");
+                      put("asset.type", "bar");
+                    }}),
+            1e-10);
+  }
+
+  /**
+   * {
+   *    "threatIntel" : {
+   *      "triageConfig": {
+   *        "riskLevelRules": {
+   *          "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24'))" : 10
+   *        },
+   *        "aggregator" : "MAX"
+   *      }
+   *    }
+   * }
+   */
+  @Multiline
+  private static String testWithStellarFunction;
+
+  @Test
+  public void testWithStellarFunction() throws Exception {
+    ThreatTriageProcessor threatTriageProcessor = getProcessor(testWithStellarFunction);
+    Assert.assertEquals(
+            10d,
+            threatTriageProcessor.apply(
+                    new HashMap<Object, Object>() {{
+                      put("ip_dst_addr", "172.2.2.2");
+                    }}),
+            1e-10);
+  }
+
+  private static ThreatTriageProcessor getProcessor(String config) throws IOException {
+    SensorEnrichmentConfig c = JSONUtils.INSTANCE.load(config, SensorEnrichmentConfig.class);
+    return new ThreatTriageProcessor(c, StellarFunctions.FUNCTION_RESOLVER(), Context.EMPTY_CONTEXT());
   }
 }