You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ni...@apache.org on 2017/09/11 15:43:17 UTC

metron git commit: METRON-1167 Define Session Specific Global Configuration Values in the REPL (nickwallen) closes apache/metron#740

Repository: metron
Updated Branches:
  refs/heads/master 382816062 -> 2afe9d7f5


METRON-1167 Define Session Specific Global Configuration Values in the REPL (nickwallen) closes apache/metron#740


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

Branch: refs/heads/master
Commit: 2afe9d7f5d277e52928946e7b53267aa5115ae0a
Parents: 3828160
Author: nickwallen <ni...@nickallen.org>
Authored: Mon Sep 11 11:40:42 2017 -0400
Committer: nickallen <ni...@apache.org>
Committed: Mon Sep 11 11:40:42 2017 -0400

----------------------------------------------------------------------
 metron-stellar/stellar-common/README.md         |  74 +++++++---
 .../stellar/common/shell/StellarExecutor.java   |  33 +++--
 .../stellar/common/shell/StellarShell.java      | 146 ++++++++++++++++---
 3 files changed, 194 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metron/blob/2afe9d7f/metron-stellar/stellar-common/README.md
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-common/README.md b/metron-stellar/stellar-common/README.md
index a070c13..8552138 100644
--- a/metron-stellar/stellar-common/README.md
+++ b/metron-stellar/stellar-common/README.md
@@ -1,24 +1,24 @@
 
+# Stellar Language
 
-# Contents
+For a variety of components (threat intelligence triage and field transformations) we have the need to do simple computation and transformation using the data from messages as variables.  For those purposes, there exists a simple, scaled down DSL created to do simple computation and transformation.
 
-* [Stellar Language](#stellar-language)
-    * [Stellar Language Keywords](#stellar-language-keywords)
-    * [Stellar Core Functions](#stellar-core-functions)
-    * [Stellar Benchmarks](#stellar-benchmarks)
-    * [Stellar Shell](#stellar-shell)
-* [Stellar Configuration](#stellar-configuration)
 
+* [Introduction](#introduction)
+* [Stellar Core Functions](#stellar-core-functions)
+* [Stellar Benchmarks](#stellar-benchmarks)
+* [Stellar Shell](#stellar-shell)
+    * [Getting Started](#getting-started)
+    * [Command Line Options](#command-line-options)
+    * [Variable Assignment](#variable-assignment)
+    * [Magic Commands](#magic-commands)
+    * [Advanced Usage](#advanced-usage)
+* [Stellar Configuration](#stellar-configuration)
 
-# Stellar Language
 
-For a variety of components (threat intelligence triage and field
-transformations) we have the need to do simple computation and
-transformation using the data from messages as variables.  
-For those purposes, there exists a simple, scaled down DSL 
-created to do simple computation and transformation.
+## Introduction
 
-The query language supports the following:
+The Stellar language supports the following:
 * Referencing fields in the enriched JSON
 * String literals are quoted with either `'` or `"`
 * String literals support escaping for `'`, `"`, `\t`, `\r`, `\n`, and backslash 
@@ -35,7 +35,7 @@ The query language supports the following:
 * The ability to have parenthesis to make order of operations explicit
 * User defined functions, including Lambda expressions 
 
-## Stellar Language Keywords
+### Stellar Language Keywords
 The following keywords need to be single quote escaped in order to be used in Stellar expressions:
 
 |               |               |             |             |             |
@@ -47,13 +47,13 @@ The following keywords need to be single quote escaped in order to be used in St
 
 Using parens such as: "foo" : "\<ok\>" requires escaping; "foo": "\'\<ok\>\'"
 
-## Stellar Language Inclusion Checks (`in` and `not in`)
+### Stellar Language Inclusion Checks (`in` and `not in`)
 1. `in` supports string contains. e.g. `'foo' in 'foobar' == true`
 2. `in` supports collection contains. e.g. `'foo' in [ 'foo', 'bar' ] == true`
 3. `in` supports map key contains. e.g. `'foo' in { 'foo' : 5} == true`
 4. `not in` is the negation of the in expression. e.g. `'grok' not in 'foobar' == true`
 
-## Stellar Language Comparisons (`<`, `<=`, `>`, `>=`)
+### Stellar Language Comparisons (`<`, `<=`, `>`, `>=`)
 
 1. If either side of the comparison is null then return false.
 2. If both values being compared implement number then the following:
@@ -64,7 +64,7 @@ Using parens such as: "foo" : "\<ok\>" requires escaping; "foo": "\'\<ok\>\'"
 3. If both sides are of the same type and are comparable then use the compareTo method to compare values.
 4. If none of the above are met then an exception is thrown.
 
-## Stellar Language Equality Check (`==`, `!=`)
+### Stellar Language Equality Check (`==`, `!=`)
 
 Below is how the `==` operator is expected to work:
 
@@ -78,7 +78,7 @@ Below is how the `==` operator is expected to work:
 
 The `!=` operator is the negation of the above.
 
-## Stellar Language Lambda Expressions
+### Stellar Language Lambda Expressions
 
 Stellar provides the capability to pass lambda expressions to functions
 which wish to support that layer of indirection.  The syntax is:
@@ -1206,8 +1206,6 @@ IN_SUBNET
 Lists all variables in the Stellar environment.
 
 ```
-Stellar, Go!
-{es.clustername=metron, es.ip=node1, es.port=9300, es.date.format=yyyy.MM.dd.HH}
 [Stellar]>>> %vars
 [Stellar]>>> foo := 2 + 2
 4.0
@@ -1215,6 +1213,38 @@ Stellar, Go!
 foo = 4.0
 ```
 
+#### `%globals`
+
+Lists all values that are defined in the global configuration.
+
+Most of Metron's functional components have access to what is called the global configuration.  This is a key/value configuration store that can be used to customize Metron.  Many Stellar functions accept configuration values from the global configuration.  The Stellar Shell also leverages the global configuration for customizing the behavior of many Stellar functions.  
+
+```
+[Stellar]>>> %globals
+{es.clustername=metron, es.ip=node1:9300, es.date.format=yyyy.MM.dd.HH, parser.error.topic=indexing, update.hbase.table=metron_update, update.hbase.cf=t}
+```
+
+#### `%define`
+
+Defines a global configuration value in the current shell session.  This value will be forgotten once the session is ended.
+
+```
+[Stellar]>>> %define bootstrap.servers := "node1:6667"
+node1:6667
+[Stellar]>>> %globals
+{bootstrap.servers=node1:6667}
+``` 
+
+#### `%undefine`
+
+Undefine a global configuration value in the current shell session.  This will not modify the persisted global configuration.
+
+```
+[Stellar]>>> %undefine bootstrap.servers
+[Stellar]>>> %globals
+{}
+```
+
 #### `?<function>`
 
 Returns formatted documentation of the Stellar function.  Provides the description of the function along with the expected arguments.
@@ -1264,7 +1294,7 @@ Please note that functions are loading lazily in the background and will be unav
 ABS, APPEND_IF_MISSING, BIN, BLOOM_ADD, BLOOM_EXISTS, BLOOM_INIT, BLOOM_MERGE, CHOMP, CHOP, COUNT_MATCHES, DAY_OF_MONTH, DAY_OF_WEEK, DAY_OF_YEAR, DOMAIN_REMOVE_SUBDOMAINS, DOMAIN_REMOVE_TLD, DOMAIN_TO_TLD, ENDS_WITH, FILL_LEFT, FILL_RIGHT, FILTER, FORMAT, GET, GET_FIRST, GET_LAST, HLLP_ADD, HLLP_CARDINALITY, HLLP_INIT, HLLP_MERGE, IN_SUBNET, IS_DATE, IS_DOMAIN, IS_EMAIL, IS_EMPTY, IS_INTEGER, IS_IP, IS_URL, JOIN, LENGTH, LIST_ADD, MAAS_GET_ENDPOINT, MAAS_MODEL_APPLY, MAP, MAP_EXISTS, MAP_GET, MONTH, OUTLIER_MAD_ADD, OUTLIER_MAD_SCORE, OUTLIER_MAD_STATE_MERGE, PREPEND_IF_MISSING, PROFILE_FIXED, PROFILE_GET, PROFILE_WINDOW, PROTOCOL_TO_NAME, REDUCE, REGEXP_MATCH, SPLIT, STARTS_WITH, STATS_ADD, STATS_BIN, STATS_COUNT, STATS_GEOMETRIC_MEAN, STATS_INIT, STATS_KURTOSIS, STATS_MAX, STATS_MEAN, STATS_MERGE, STATS_MIN, STATS_PERCENTILE, STATS_POPULATION_VARIANCE, STATS_QUADRATIC_MEAN, STATS_SD, STATS_SKEWNESS, STATS_SUM, STATS_SUM_LOGS, STATS_SUM_SQUARES, STATS_VARIANCE, STRING_ENTROPY, SYS
 TEM_ENV_GET, SYSTEM_PROPERTY_GET, TO_DOUBLE, TO_EPOCH_TIMESTAMP, TO_FLOAT, TO_INTEGER, TO_LONG, TO_LOWER, TO_STRING, TO_UPPER, TRIM, URL_TO_HOST, URL_TO_PATH, URL_TO_PORT, URL_TO_PROTOCOL, WEEK_OF_MONTH, WEEK_OF_YEAR, YEAR 
 ```
 
-# Stellar Configuration
+## Stellar Configuration
 
 Stellar can be configured in a variety of ways from the [Global Configuration](../../metron-platform/metron-common/README.md#global-configuration).
 In particular, there are three main configuration parameters around configuring Stellar:

http://git-wip-us.apache.org/repos/asf/metron/blob/2afe9d7f/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/StellarExecutor.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/StellarExecutor.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/StellarExecutor.java
index 1181c6f..febde40 100644
--- a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/StellarExecutor.java
+++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/StellarExecutor.java
@@ -92,9 +92,10 @@ public class StellarExecutor {
   }
 
   /**
-   * prefix tree index of autocompletes
+   * Prefix tree index of auto-completes.
    */
   private PatriciaTrie<AutoCompleteType> autocompleteIndex;
+
   /**
    * The variables known by Stellar.
    */
@@ -118,7 +119,9 @@ public class StellarExecutor {
   private Console console;
 
   public enum OperationType {
-    DOC,MAGIC,NORMAL;
+    DOC
+    , MAGIC
+    , NORMAL
   }
 
   public interface AutoCompleteTransformation {
@@ -138,6 +141,7 @@ public class StellarExecutor {
     , VARIABLE((type, key) -> key )
     , TOKEN((type, key) -> key)
     ;
+
     AutoCompleteTransformation transform;
     AutoCompleteType(AutoCompleteTransformation transform) {
       this.transform = transform;
@@ -147,7 +151,6 @@ public class StellarExecutor {
     public String transform(OperationType type, String key) {
       return transform.transform(type, key);
     }
-
   }
 
   /**
@@ -178,18 +181,17 @@ public class StellarExecutor {
 
     // asynchronously update the index with function names found from a classpath scan.
     new Thread( () -> {
-        Iterable<StellarFunctionInfo> functions = functionResolver.getFunctionInfo();
-        indexLock.writeLock().lock();
-        try {
-          for(StellarFunctionInfo info: functions) {
-            String functionName = info.getName();
-            autocompleteIndex.put(functionName, AutoCompleteType.FUNCTION);
-          }
+      Iterable<StellarFunctionInfo> functions = functionResolver.getFunctionInfo();
+      indexLock.writeLock().lock();
+      try {
+        for(StellarFunctionInfo info: functions) {
+          String functionName = info.getName();
+          autocompleteIndex.put(functionName, AutoCompleteType.FUNCTION);
         }
-          finally {
-            System.out.println("Functions loaded, you may refer to functions now...");
-            indexLock.writeLock().unlock();
-          }
+      } finally {
+        System.out.println("Functions loaded, you may refer to functions now...");
+        indexLock.writeLock().unlock();
+      }
     }).start();
   }
 
@@ -203,6 +205,9 @@ public class StellarExecutor {
     index.put("quit", AutoCompleteType.TOKEN);
     index.put(StellarShell.MAGIC_FUNCTIONS, AutoCompleteType.FUNCTION);
     index.put(StellarShell.MAGIC_VARS, AutoCompleteType.FUNCTION);
+    index.put(StellarShell.MAGIC_GLOBALS, AutoCompleteType.FUNCTION);
+    index.put(StellarShell.MAGIC_DEFINE, AutoCompleteType.FUNCTION);
+    index.put(StellarShell.MAGIC_UNDEFINE, AutoCompleteType.FUNCTION);
     return new PatriciaTrie<>(index);
   }
 

http://git-wip-us.apache.org/repos/asf/metron/blob/2afe9d7f/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/StellarShell.java
----------------------------------------------------------------------
diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/StellarShell.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/StellarShell.java
index a860778..0002c5a 100644
--- a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/StellarShell.java
+++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/shell/StellarShell.java
@@ -30,10 +30,9 @@ import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.PosixParser;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.metron.stellar.dsl.Context;
-import org.apache.metron.stellar.dsl.StellarFunctionInfo;
 import org.apache.metron.stellar.common.StellarAssignment;
 import org.apache.metron.stellar.common.utils.JSONUtils;
+import org.apache.metron.stellar.dsl.StellarFunctionInfo;
 import org.jboss.aesh.complete.CompleteOperation;
 import org.jboss.aesh.complete.Completion;
 import org.jboss.aesh.console.AeshConsoleCallback;
@@ -51,13 +50,18 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Properties;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
+import static org.apache.metron.stellar.dsl.Context.Capabilities.GLOBAL_CONFIG;
+
 /**
  * A REPL environment for Stellar.
  *
@@ -90,6 +94,9 @@ public class StellarShell extends AeshConsoleCallback implements Completion {
   public static final String MAGIC_VARS = MAGIC_PREFIX + "vars";
   public static final String DOC_PREFIX = "?";
   public static final String STELLAR_PROPERTIES_FILENAME = "stellar.properties";
+  public static final String MAGIC_GLOBALS = MAGIC_PREFIX + "globals";
+  public static final String MAGIC_DEFINE = MAGIC_PREFIX + "define";
+  public static final String MAGIC_UNDEFINE = MAGIC_PREFIX + "undefine";
 
   private StellarExecutor executor;
 
@@ -241,7 +248,7 @@ public class StellarShell extends AeshConsoleCallback implements Completion {
     // welcome message and print globals
     writeLine(WELCOME);
     executor.getContext()
-            .getCapability(Context.Capabilities.GLOBAL_CONFIG, false)
+            .getCapability(GLOBAL_CONFIG, false)
             .ifPresent(conf -> writeLine(conf.toString()));
 
     console.start();
@@ -287,32 +294,25 @@ public class StellarShell extends AeshConsoleCallback implements Completion {
    * Executes a magic expression.
    * @param rawExpression The expression to execute.
    */
-  private void handleMagic( String rawExpression) {
-    String[] expression = rawExpression.trim().split(" ");
+  private void handleMagic(String rawExpression) {
 
+    String[] expression = rawExpression.trim().split("\\s+");
     String command = expression[0];
-    if(MAGIC_FUNCTIONS.equals(command)) {
 
-      // if '%functions FOO' then show only functions that contain 'FOO'
-      Predicate<String> nameFilter = (name -> true);
-      if(expression.length > 1) {
-        nameFilter = (name -> name.contains(expression[1]));
-      }
+    if (MAGIC_FUNCTIONS.equals(command)) {
+      handleMagicFunctions(expression);
+
+    } else if (MAGIC_VARS.equals(command)) {
+      handleMagicVars();
 
-      // list available functions
-      String functions = StreamSupport
-              .stream(executor.getFunctionResolver().getFunctionInfo().spliterator(), false)
-              .map(info -> String.format("%s", info.getName()))
-              .filter(nameFilter)
-              .sorted()
-              .collect(Collectors.joining(", "));
-      writeLine(functions);
+    } else if (MAGIC_GLOBALS.equals(command)) {
+      handleMagicGlobals();
 
-    } else if(MAGIC_VARS.equals(command)) {
+    } else if (MAGIC_DEFINE.equals(command)) {
+      handleMagicDefine(rawExpression);
 
-      // list all variables
-      executor.getVariables()
-              .forEach((k,v) -> writeLine(String.format("%s = %s", k, v)));
+    } else if(MAGIC_UNDEFINE.equals(command)) {
+      handleMagicUndefine(expression);
 
     } else {
       writeLine(ERROR_PROMPT + "undefined magic command: " + rawExpression);
@@ -320,6 +320,106 @@ public class StellarShell extends AeshConsoleCallback implements Completion {
   }
 
   /**
+   * Handle a magic '%functions'.  Lists all of the variables in-scope.
+   * @param expression
+   */
+  private void handleMagicFunctions(String[] expression) {
+
+    // if '%functions FOO' then show only functions that contain 'FOO'
+    Predicate<String> nameFilter = (name -> true);
+    if (expression.length > 1) {
+      nameFilter = (name -> name.contains(expression[1]));
+    }
+
+    // '%functions' -> list all functions in scope
+    String functions = StreamSupport
+            .stream(executor.getFunctionResolver().getFunctionInfo().spliterator(), false)
+            .map(info -> String.format("%s", info.getName()))
+            .filter(nameFilter)
+            .sorted()
+            .collect(Collectors.joining(", "));
+    writeLine(functions);
+  }
+
+  /**
+   * Handle a magic '%vars'.  Lists all of the variables in-scope.
+   */
+  private void handleMagicVars() {
+    executor.getVariables()
+            .forEach((k, v) -> writeLine(String.format("%s = %s", k, v)));
+  }
+
+  /**
+   * Handle a magic '%globals'.  List all of the global configuration values.
+   */
+  private void handleMagicGlobals() {
+    Map<String, Object> globals = getOrCreateGlobalConfig(executor);
+    writeLine(globals.toString());
+  }
+
+  /**
+   * Handle a magic '%define var=value'.  Alter the global configuration.
+   * @param expression The expression passed to %define
+   */
+  public void handleMagicDefine(String expression) {
+
+    // grab the expression in '%define <assign-expression>'
+    String assignExpr = StringUtils.trimToEmpty(expression.substring(MAGIC_DEFINE.length()));
+    if (assignExpr.length() > 0) {
+
+      // the expression must be an assignment
+      if(StellarAssignment.isAssignment(assignExpr)) {
+        StellarAssignment expr = StellarAssignment.from(assignExpr);
+
+        // execute the expression
+        Object result = executor.execute(expr.getStatement());
+        if (result != null) {
+          writeLine(result.toString());
+
+          // alter the global configuration
+          getOrCreateGlobalConfig(executor).put(expr.getVariable(), result);
+        }
+
+      } else {
+        // the expression is not an assignment.  boo!
+        writeLine(ERROR_PROMPT + MAGIC_DEFINE + " expected assignment expression");
+      }
+    }
+  }
+
+  /**
+   * Handle a magic '%undefine var'.  Removes a variable from the global configuration.
+   * @param expression
+   */
+  private void handleMagicUndefine(String[] expression) {
+    if(expression.length > 1) {
+      Map<String, Object> globals = getOrCreateGlobalConfig(executor);
+      globals.remove(expression[1]);
+    }
+  }
+
+  /**
+   * Retrieves the GLOBAL_CONFIG, if it exists.  If it does not, it creates the GLOBAL_CONFIG
+   * and adds it to the Stellar execution context.
+   * @param executor The Stellar executor.
+   * @return The global configuration.
+   */
+  private Map<String, Object> getOrCreateGlobalConfig(StellarExecutor executor) {
+    Map<String, Object> globals;
+    Optional<Object> capability = executor.getContext().getCapability(GLOBAL_CONFIG, false);
+    if (capability.isPresent()) {
+      globals = (Map<String, Object>) capability.get();
+
+    } else {
+      // if it does not exist, create it.  this creates the global config for the current stellar executor
+      // session only.  this does not change the global config maintained externally in zookeeper
+      globals = new HashMap<>();
+      executor.getContext().addCapability(GLOBAL_CONFIG, () -> globals);
+    }
+    return globals;
+  }
+
+  /**
    * Executes a doc expression.
    * @param expression The doc expression to execute.
    */