You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by mm...@apache.org on 2018/07/11 01:32:53 UTC

[37/50] [abbrv] metron git commit: METRON-1619: Stellar empty collections should be considered false in boolean expressions closes apache/incubator-metron#1064

METRON-1619: Stellar empty collections should be considered false in boolean expressions closes apache/incubator-metron#1064


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

Branch: refs/heads/feature/METRON-1554-pcap-query-panel
Commit: c4c790dbe4c481c8f19293ef9d6134539674d253
Parents: 81282de
Author: cstella <ce...@gmail.com>
Authored: Thu Jun 28 15:57:08 2018 -0400
Committer: cstella <ce...@gmail.com>
Committed: Thu Jun 28 15:57:08 2018 -0400

----------------------------------------------------------------------
 metron-stellar/stellar-common/README.md         | 14 ++++
 .../metron/stellar/common/StellarCompiler.java  | 82 +++++++++++++++++---
 .../org/apache/metron/stellar/dsl/Token.java    |  1 +
 .../stellar/dsl/functions/BasicStellarTest.java | 25 ++++++
 .../metron/stellar/dsl/functions/MatchTest.java | 35 +++++++++
 use-cases/forensic_clustering/README.md         |  2 +-
 use-cases/geographic_login_outliers/README.md   |  5 +-
 use-cases/typosquat_detection/README.md         |  2 +-
 8 files changed, 149 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metron/blob/c4c790db/metron-stellar/stellar-common/README.md
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-common/README.md b/metron-stellar/stellar-common/README.md
index d8d09b4..0dd8e16 100644
--- a/metron-stellar/stellar-common/README.md
+++ b/metron-stellar/stellar-common/README.md
@@ -54,6 +54,20 @@ The Stellar language supports the following:
 * The ability to have parenthesis to make order of operations explicit
 * User defined functions, including Lambda expressions 
 
+### Boolean Expressions
+
+Variables may be used in boolean expressions and variables which are not
+explicitly boolean may be interpreted as booleans subject to the
+following rules:
+* Similar to python and javascript, empty collections (e.g. `[]`) will be
+  interpreted as `false`
+* Similar to python and javascript, missing variables will be
+  interpreted as `false`
+* Variables set to `null` will be interpreted as `false`
+
+Otherwise, boolean variables will be interpreted as their values
+reflect. 
+
 ### Stellar Language Keywords
 The following keywords need to be single quote escaped in order to be used in Stellar expressions:
 

http://git-wip-us.apache.org/repos/asf/metron/blob/c4c790db/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/StellarCompiler.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/StellarCompiler.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/StellarCompiler.java
index 72f0d0a..8a328a2 100644
--- a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/StellarCompiler.java
+++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/StellarCompiler.java
@@ -29,7 +29,10 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
+
+import com.google.common.collect.Iterables;
 import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.metron.stellar.common.evaluators.ArithmeticEvaluator;
@@ -104,6 +107,52 @@ public class StellarCompiler extends StellarBaseListener {
       return tokenDeque;
     }
 
+    /**
+     * When treating empty or missing values as false, we need to ensure we ONLY do so in a conditional context.
+     * @param tokenValueType
+     * @return
+     */
+    private boolean isConditionalContext(Class<?> tokenValueType) {
+      return tokenValueType != null && (
+               tokenValueType == BooleanArg.class
+            || tokenValueType == IfExpr.class
+            || tokenValueType == MatchClauseCheckExpr.class
+      );
+    }
+
+    /**
+     * Determine if a token and value is an empty list in the appropriate conditional context
+     * @param token
+     * @param value
+     * @return
+     */
+    private boolean isEmptyList(Token<?> token, Object value) {
+      if(value != null && isConditionalContext(token.getUnderlyingType())) {
+        if (value instanceof Iterable) {
+          return Iterables.isEmpty((Iterable) value);
+        } else if (value instanceof Map) {
+          return ((Map) value).isEmpty();
+        }
+        else {
+          return false;
+        }
+      }else {
+        return false;
+      }
+    }
+
+    /**
+     * Determine if a token is missing in a conditional context.
+     * @param token
+     * @return
+     */
+    private boolean isBoolean(Token<?> token, Object value) {
+      if(token == null || token.getValue() == null) {
+        return false;
+      }
+      return value == null && isConditionalContext(token.getValue().getClass());
+    }
+
     public Object apply(ExpressionState state) {
       Deque<Token<?>> instanceDeque = new ArrayDeque<>();
       {
@@ -137,6 +186,17 @@ public class StellarCompiler extends StellarBaseListener {
           and with the current context.
            */
           Token<?> curr = instanceDeque.peek();
+          boolean isFalsey = curr != null &&
+                  (isBoolean(token, curr.getValue()) || isEmptyList(token, curr.getValue()));
+          if(isFalsey){
+            //If we're in a situation where the token is a boolean token and the current value is one of the implicitly falsey scenarios
+            //* null or missing variable
+            //* empty list
+            // then we want to treat it as explicitly false by replacing the current token.
+            curr = new Token<>(false, Boolean.class, curr.getMultiArgContext());
+            instanceDeque.removeFirst();
+            instanceDeque.addFirst(curr);
+          }
           if (curr != null && curr.getValue() != null && curr.getValue() instanceof Boolean
               && ShortCircuitOp.class.isAssignableFrom(token.getUnderlyingType())) {
             //if we have a boolean as the current value and the next non-contextual token is a short circuit op
@@ -212,6 +272,8 @@ public class StellarCompiler extends StellarBaseListener {
       }
     }
 
+
+
     public void shortCircuit(Iterator<Token<?>> it, FrameContext.Context context) {
       while (it.hasNext()) {
         Token<?> token = it.next();
@@ -379,7 +441,8 @@ public class StellarCompiler extends StellarBaseListener {
     final FrameContext.Context context = getArgContext();
     expression.tokenDeque.push(new Token<>( (tokenDeque, state) -> {
     Token<Boolean> arg = (Token<Boolean>) popDeque(tokenDeque);
-    tokenDeque.push(new Token<>(!arg.getValue(), Boolean.class, context));
+    Boolean v = Optional.ofNullable(ConversionUtils.convert(arg.getValue(), Boolean.class)).orElse(false);
+    tokenDeque.push(new Token<>(!v, Boolean.class, context));
     }, DeferredFunction.class, context));
   }
 
@@ -393,7 +456,8 @@ public class StellarCompiler extends StellarBaseListener {
         // when parsing, missing variables are an error!
         throw new ParseException(String.format("variable: %s is not defined",varName));
       }
-      tokenDeque.push(new Token<>(state.variableResolver.resolve(varName), Object.class, context));
+      Object resolved = state.variableResolver.resolve(varName);
+      tokenDeque.push(new Token<>(resolved, Object.class, context));
     }, DeferredFunction.class, context));
     expression.variablesUsed.add(ctx.getText());
   }
@@ -492,11 +556,8 @@ public class StellarCompiler extends StellarBaseListener {
   }
 
   private boolean booleanOp(final Token<?> left, final Token<?> right, final BooleanOp op, final String opName) {
-    Boolean l = ConversionUtils.convert(left.getValue(), Boolean.class);
-    Boolean r = ConversionUtils.convert(right.getValue(), Boolean.class);
-    if (l == null || r == null) {
-      throw new ParseException("Unable to operate on " + left.getValue() + " " + opName + " " + right.getValue() + ", null value");
-    }
+    Boolean l = Optional.ofNullable(ConversionUtils.convert(left.getValue(), Boolean.class)).orElse(false);
+    Boolean r = Optional.ofNullable(ConversionUtils.convert(right.getValue(), Boolean.class)).orElse(false);
     return op.op(l, r);
   }
 
@@ -730,14 +791,11 @@ public class StellarCompiler extends StellarBaseListener {
     // a null and we need to protect against that
     if(ctx.getStart() == ctx.getStop()) {
       expression.tokenDeque.push(new Token<>((tokenDeque, state) -> {
-        if (state.context.getActivityType().equals(ActivityType.VALIDATION_ACTIVITY)) {
           if (tokenDeque.size() == 1 && (tokenDeque.peek().getValue() == null
-              || tokenDeque.peek().getUnderlyingType() == Boolean.class)) {
+                  || tokenDeque.peek().getUnderlyingType() == Boolean.class)) {
             tokenDeque.pop();
-            tokenDeque.add(new Token<>(true, Boolean.class, getArgContext()));
+            tokenDeque.add(new Token<>(false, Boolean.class, getArgContext()));
           }
-
-        }
       }, DeferredFunction.class, context));
     }
     expression.tokenDeque.push(new Token<>(new MatchClauseCheckExpr(), MatchClauseCheckExpr.class, getArgContext()));

http://git-wip-us.apache.org/repos/asf/metron/blob/c4c790db/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/Token.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/Token.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/Token.java
index f852544..d6c8458 100644
--- a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/Token.java
+++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/Token.java
@@ -42,6 +42,7 @@ public class Token<T> {
   public T getValue() {
     return value;
   }
+
   public Class<T> getUnderlyingType() {
     return underlyingType;
   }

http://git-wip-us.apache.org/repos/asf/metron/blob/c4c790db/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/dsl/functions/BasicStellarTest.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/dsl/functions/BasicStellarTest.java b/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/dsl/functions/BasicStellarTest.java
index ea859c5..dec05a8 100644
--- a/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/dsl/functions/BasicStellarTest.java
+++ b/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/dsl/functions/BasicStellarTest.java
@@ -975,4 +975,29 @@ public class BasicStellarTest {
     Assert.assertEquals("val1", ret.get("field1"));
     Assert.assertEquals("val2", ret.get("field2"));
   }
+
+  @Test
+  public void nullAsFalse() {
+    checkFalsey("is_alert");
+  }
+
+  private void checkFalsey(String falseyExpr) {
+    VariableResolver resolver = new MapVariableResolver(new HashMap<>());
+    Assert.assertTrue(runPredicate(String.format(" %s || true", falseyExpr), resolver));
+    Assert.assertFalse(runPredicate(String.format("%s && EXCEPTION('blah')", falseyExpr), resolver));
+    Assert.assertTrue(runPredicate(String.format("NOT(%s)", falseyExpr), resolver));
+    Assert.assertFalse(runPredicate(String.format("if %s then true else false", falseyExpr), resolver));
+    Assert.assertFalse(runPredicate(String.format("if %s then true || %s else false", falseyExpr, falseyExpr), resolver));
+    Assert.assertFalse(runPredicate(String.format("if %s then true || %s else false && %s", falseyExpr, falseyExpr, falseyExpr), resolver));
+    Assert.assertFalse(runPredicate(String.format("if %s then true || %s else false && (%s || true)", falseyExpr, falseyExpr, falseyExpr), resolver));
+    //make sure that nulls aren't replaced by false everywhere, only in boolean expressions.
+    Assert.assertNull(run(String.format("MAP_GET(%s, {false : 'blah'})", falseyExpr), resolver));
+  }
+
+  @Test
+  public void emptyAsFalse() {
+    checkFalsey("[]");
+    checkFalsey("{}");
+    checkFalsey("LIST_ADD([])");
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/c4c790db/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/dsl/functions/MatchTest.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/dsl/functions/MatchTest.java b/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/dsl/functions/MatchTest.java
index 54e99a3..d6986cc 100644
--- a/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/dsl/functions/MatchTest.java
+++ b/metron-stellar/stellar-common/src/test/java/org/apache/metron/stellar/dsl/functions/MatchTest.java
@@ -36,6 +36,41 @@ public class MatchTest {
 
   @Test
   @SuppressWarnings("unchecked")
+  public void testMissingVariableFalsey() {
+    Assert.assertTrue(runPredicate(
+        "match{NOT(is_alert) => true, foo > 5 => false, foo > 10 => false, default => false}",
+        new HashMap() {{
+          put("foo", 100);
+        }}));
+    Assert.assertFalse(runPredicate(
+        "match{is_alert => true, foo > 5 => false, foo > 10 => false, default => false}",
+        new HashMap() {{
+          put("foo", 100);
+        }}));
+    Assert.assertFalse(runPredicate(
+        "match{foo > 5 => false, is_alert => true, foo > 10 => false, default => false}",
+        new HashMap() {{
+          put("foo", 100);
+        }}));
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testEmptyListFalsey() {
+    Assert.assertTrue(runPredicate(
+        "match{NOT([]) => true, foo > 5 => false, foo > 10 => false, default => false}",
+        new HashMap() {{
+          put("foo", 100);
+        }}));
+    Assert.assertFalse(runPredicate(
+        "match{[] => true, foo > 5 => false, foo > 10 => false, default => false}",
+        new HashMap() {{
+          put("foo", 100);
+        }}));
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
   public void testThreeTrueClausesFirstOnlyFires() {
     Assert.assertTrue(runPredicate(
         "match{foo > 0 => true, foo > 5 => false, foo > 10 => false, default => false}",

http://git-wip-us.apache.org/repos/asf/metron/blob/c4c790db/use-cases/forensic_clustering/README.md
----------------------------------------------------------------------
diff --git a/use-cases/forensic_clustering/README.md b/use-cases/forensic_clustering/README.md
index f9c1333..dac116c 100644
--- a/use-cases/forensic_clustering/README.md
+++ b/use-cases/forensic_clustering/README.md
@@ -371,7 +371,7 @@ Now, we can create the enrichments thusly by creating `$METRON_HOME/config/zooke
       "stellar" : {
         "config" : [
           "blacklisted := ENRICHMENT_EXISTS( 'blacklist', src_ip, 'threatintel', 't')",
-          "is_alert := (exists(is_alert) && is_alert) || blacklisted"
+          "is_alert := is_alert || blacklisted"
         ]
       }
 

http://git-wip-us.apache.org/repos/asf/metron/blob/c4c790db/use-cases/geographic_login_outliers/README.md
----------------------------------------------------------------------
diff --git a/use-cases/geographic_login_outliers/README.md b/use-cases/geographic_login_outliers/README.md
index 1e092a1..f23bb8c 100644
--- a/use-cases/geographic_login_outliers/README.md
+++ b/use-cases/geographic_login_outliers/README.md
@@ -149,7 +149,7 @@ We can represent these in the `$METRON_HOME/config/zookeeper/profiler.json` via
     {
       "profile": "geo_distribution_from_centroid",
       "foreach": "'global'",
-      "onlyif": "exists(geo_distance) && geo_distance != null",
+      "onlyif": "geo_distance != null",
       "init" : {
         "s": "STATS_INIT()"
                },
@@ -161,7 +161,7 @@ We can represent these in the `$METRON_HOME/config/zookeeper/profiler.json` via
     {
       "profile": "locations_by_user",
       "foreach": "user",
-      "onlyif": "exists(hash) && hash != null && LENGTH(hash) > 0",
+      "onlyif": "hash != null && LENGTH(hash) > 0",
       "init" : {
         "s": "MULTISET_INIT()"
                },
@@ -211,7 +211,6 @@ We also want to set up a triage rule associating a score and setting an alert if
           "dist_median := STATS_PERCENTILE(geo_distance_distr, 50.0)",
           "dist_sd := STATS_SD(geo_distance_distr)",
           "geo_outlier := ABS(dist_median - geo_distance) >= 5*dist_sd",
-          "is_alert := exists(is_alert) && is_alert",
           "is_alert := is_alert || (geo_outlier != null && geo_outlier == true)",
           "geo_distance_distr := null"
         ]

http://git-wip-us.apache.org/repos/asf/metron/blob/c4c790db/use-cases/typosquat_detection/README.md
----------------------------------------------------------------------
diff --git a/use-cases/typosquat_detection/README.md b/use-cases/typosquat_detection/README.md
index 4e4de71..63624c7 100644
--- a/use-cases/typosquat_detection/README.md
+++ b/use-cases/typosquat_detection/README.md
@@ -295,7 +295,7 @@ We can do this by creating `$METRON_HOME/config/zookeeper/enrichments/squid.json
     "fieldMap": {
       "stellar" : {
         "config" : [
-          "is_alert := (exists(is_alert) && is_alert) || is_potential_typosquat"
+          "is_alert := is_alert || is_potential_typosquat"
         ]
       }