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/06 19:13:46 UTC

[1/2] incubator-metron git commit: METRON-396: Make Stellar function resolution happen via an annotation and classpath search closes apache/incubator-metron#233

Repository: incubator-metron
Updated Branches:
  refs/heads/master 8e8f9e05c -> fc2dc3d6b


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/maas/StellarMaaSIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/maas/StellarMaaSIntegrationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/maas/StellarMaaSIntegrationTest.java
index f5618f5..e3d3f3e 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/maas/StellarMaaSIntegrationTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/maas/StellarMaaSIntegrationTest.java
@@ -139,17 +139,17 @@ public class StellarMaaSIntegrationTest {
   @Test
   public void testModelApply() throws Exception {
     {
-      String stellar = "MAP_GET('is_malicious', MODEL_APPLY(MAAS_GET_ENDPOINT('dga'), {'host': host}))";
+      String stellar = "MAP_GET('is_malicious', MAAS_MODEL_APPLY(MAAS_GET_ENDPOINT('dga'), {'host': host}))";
       Object result = StellarTest.run(stellar, ImmutableMap.of("host", "badguy.com"), context);
       Assert.assertTrue((Boolean) result);
     }
     {
-      String stellar = "MAP_GET('is_malicious', MODEL_APPLY(MAAS_GET_ENDPOINT('dga'), {'host': host}))";
+      String stellar = "MAP_GET('is_malicious', MAAS_MODEL_APPLY(MAAS_GET_ENDPOINT('dga'), {'host': host}))";
       Object result = StellarTest.run(stellar, ImmutableMap.of("host", "youtube.com"), context);
       Assert.assertFalse((Boolean) result);
     }
     {
-      String stellar = "MAP_GET('is_malicious', MODEL_APPLY(MAAS_GET_ENDPOINT('dga'), 'apply', {'host': host}))";
+      String stellar = "MAP_GET('is_malicious', MAAS_MODEL_APPLY(MAAS_GET_ENDPOINT('dga'), 'apply', {'host': host}))";
       Object result = StellarTest.run(stellar, ImmutableMap.of("host", "youtube.com"), context);
       Assert.assertFalse((Boolean) result);
     }
@@ -159,7 +159,7 @@ public class StellarMaaSIntegrationTest {
   @Test
   public void testModelApplyNegative() {
     {
-      String stellar = "MAP_GET('is_malicious', MODEL_APPLY(MAAS_GET_ENDPOINT('dga', '2.0'), {'host': host}))";
+      String stellar = "MAP_GET('is_malicious', MAAS_MODEL_APPLY(MAAS_GET_ENDPOINT('dga', '2.0'), {'host': host}))";
       Object result = StellarTest.run(stellar, ImmutableMap.of("host", "youtube.com"), context);
       Assert.assertNull( result);
     }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
index 5d4943d..6fad0f8 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java
@@ -159,6 +159,7 @@ public class GenericEnrichmentBolt extends ConfiguredEnrichmentBolt {
     stellarContext = new Context.Builder()
                          .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
                          .build();
+    StellarFunctions.initialize(stellarContext);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/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 9506697..6635dd6 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
@@ -73,6 +73,7 @@ public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
     this.stellarContext = new Context.Builder()
                                 .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
                                 .build();
+    StellarFunctions.initialize(stellarContext);
     this.functionResolver = StellarFunctions.FUNCTION_RESOLVER();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
index 3ee0424..f2156a7 100644
--- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
+++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
@@ -101,6 +101,7 @@ public class ParserBolt extends ConfiguredParserBolt implements Serializable {
     this.stellarContext = new Context.Builder()
                                 .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
                                 .build();
+    StellarFunctions.initialize(stellarContext);
   }
 
   @SuppressWarnings("unchecked")


[2/2] incubator-metron git commit: METRON-396: Make Stellar function resolution happen via an annotation and classpath search closes apache/incubator-metron#233

Posted by ce...@apache.org.
METRON-396: Make Stellar function resolution happen via an annotation and classpath search closes apache/incubator-metron#233


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

Branch: refs/heads/master
Commit: fc2dc3d6b9b958b85dee4dc06ac5f274a7c86af7
Parents: 8e8f9e0
Author: cstella <ce...@gmail.com>
Authored: Tue Sep 6 12:13:14 2016 -0700
Committer: cstella <ce...@gmail.com>
Committed: Tue Sep 6 12:13:14 2016 -0700

----------------------------------------------------------------------
 .../profiler/bolt/ProfileBuilderBolt.java       |   1 +
 .../profiler/bolt/ProfileSplitterBolt.java      |   1 +
 metron-platform/metron-common/pom.xml           |   5 +
 .../metron/common/dsl/FunctionResolver.java     |   3 +
 .../common/dsl/FunctionResolverSingleton.java   | 206 +++++++++++++++++++
 .../common/dsl/Predicate2StellarFunction.java   |  33 +++
 .../org/apache/metron/common/dsl/Stellar.java   |  32 +++
 .../metron/common/dsl/StellarFunctionInfo.java  |  79 +++++++
 .../metron/common/dsl/StellarFunctions.java     | 147 +------------
 .../dsl/functions/ConversionFunctions.java      |  26 +++
 .../dsl/functions/DataStructureFunctions.java   |   6 +
 .../common/dsl/functions/DateFunctions.java     |  47 +++++
 .../common/dsl/functions/MaaSFunctions.java     |  21 ++
 .../common/dsl/functions/MapFunctions.java      |  23 +++
 .../common/dsl/functions/NetworkFunctions.java  |  71 +++++++
 .../functions/StellarStatisticsFunctions.java   | 122 +++++++++++
 .../common/dsl/functions/StringFunctions.java   |  77 +++++++
 .../IPProtocolTransformation.java               |   8 +
 .../transformation/RemoveTransformation.java    |   2 +-
 .../validation/network/DomainValidation.java    |  16 ++
 .../validation/network/EmailValidation.java     |  18 +-
 .../field/validation/network/IPValidation.java  |  15 ++
 .../field/validation/network/URLValidation.java |  17 ++
 .../validation/primitive/DateValidation.java    |  17 ++
 .../validation/primitive/IntegerValidation.java |  15 ++
 .../common/stellar/BaseStellarProcessor.java    |   2 +-
 .../metron/common/stellar/StellarCompiler.java  |   6 +-
 .../stellar/StellarPredicateProcessor.java      |   3 +-
 .../metron/common/stellar/StellarTest.java      |  21 ++
 .../maas/StellarMaaSIntegrationTest.java        |   8 +-
 .../enrichment/bolt/GenericEnrichmentBolt.java  |   1 +
 .../enrichment/bolt/ThreatIntelJoinBolt.java    |   1 +
 .../apache/metron/parsers/bolt/ParserBolt.java  |   1 +
 33 files changed, 898 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java
index 6cf5053..698e5d0 100644
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java
+++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java
@@ -114,6 +114,7 @@ public class ProfileBuilderBolt extends ConfiguredProfilerBolt {
     stellarContext = new Context.Builder()
                          .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
                          .build();
+    StellarFunctions.initialize(stellarContext);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java
index c9af519..6008dab 100644
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java
+++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java
@@ -90,6 +90,7 @@ public class ProfileSplitterBolt extends ConfiguredProfilerBolt {
     stellarContext = new Context.Builder()
                          .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
                          .build();
+    StellarFunctions.initialize(stellarContext);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/pom.xml b/metron-platform/metron-common/pom.xml
index 139546a..7a1ea25 100644
--- a/metron-platform/metron-common/pom.xml
+++ b/metron-platform/metron-common/pom.xml
@@ -224,6 +224,11 @@
             <artifactId>commons-beanutils</artifactId>
             <version>1.8.3</version>
         </dependency>
+        <dependency>
+            <groupId>org.reflections</groupId>
+            <artifactId>reflections</artifactId>
+            <version>0.9.10</version>
+        </dependency>
     </dependencies>
 
     <reporting>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/FunctionResolver.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/FunctionResolver.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/FunctionResolver.java
index 4535b3b..0e90b84 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/FunctionResolver.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/FunctionResolver.java
@@ -20,4 +20,7 @@ package org.apache.metron.common.dsl;
 import java.util.function.Function;
 
 public interface FunctionResolver extends Function<String, StellarFunction> {
+  Iterable<StellarFunctionInfo> getFunctionInfo();
+  Iterable<String> getFunctions();
+  void initialize(Context context);
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/FunctionResolverSingleton.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/FunctionResolverSingleton.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/FunctionResolverSingleton.java
new file mode 100644
index 0000000..c631878
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/FunctionResolverSingleton.java
@@ -0,0 +1,206 @@
+/**
+ * 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.common.dsl;
+
+import com.google.common.base.Joiner;
+import org.reflections.Reflections;
+import org.reflections.util.ClasspathHelper;
+import org.reflections.util.ConfigurationBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URL;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class FunctionResolverSingleton implements FunctionResolver {
+  protected static final Logger LOG = LoggerFactory.getLogger(FunctionResolverSingleton.class);
+  private final Map<String, StellarFunctionInfo> functions = new HashMap<>();
+  private final AtomicBoolean isInitialized = new AtomicBoolean(false);
+  private static final ReadWriteLock lock = new ReentrantReadWriteLock();
+  private static FunctionResolverSingleton INSTANCE = new FunctionResolverSingleton();
+
+  private FunctionResolverSingleton() {}
+
+  public static FunctionResolver getInstance() {
+    return INSTANCE;
+  }
+
+
+
+  @Override
+  public Iterable<StellarFunctionInfo> getFunctionInfo() {
+    return _getFunctions().values();
+  }
+
+  @Override
+  public Iterable<String> getFunctions() {
+    return _getFunctions().keySet();
+  }
+
+  @Override
+  public void initialize(Context context) {
+    //forces a load of the stellar functions.
+    _getFunctions();
+  }
+
+  /**
+   * This allows the lazy loading of the functions.  We do not want to take a multi-second hit to analyze the full classpath
+   * every time a unit test starts up.  That would cause the runtime of things to blow right up.  Instead, we only want
+   * to take the hit if a function is actually called from a stellar expression.
+   *
+   *
+   * @return The map of the stellar functions that we found on the classpath indexed by fully qualified name
+   */
+  private Map<String, StellarFunctionInfo> _getFunctions() {
+    /*
+     * Because we are not doing this in a static block and because this object is a singleton we have to concern ourselves with
+     * the possiblity that two threads are calling this function at the same time.  Normally, I would consider just making the
+     * function synchronized, but since every stellar statement which uses a function will be here, I wanted to distinguish
+     * between read locks (that happen often and are quickly resolved) and write locks (which should happen at initialization time).
+     */
+    lock.readLock().lock();
+    try {
+      if (isInitialized.get()) {
+        return functions;
+      }
+    }
+    finally {
+      lock.readLock().unlock();
+    }
+    //we should VERY rarely get here.
+    lock.writeLock().lock();
+    try {
+      //I have to check again because two threads or more could be waiting at the lock statement.  The loser threads
+      //shouldn't reinitialize.
+      if(!isInitialized.get()) {
+        loadFunctions(functions);
+        isInitialized.set(true);
+      }
+      return functions;
+    }
+    finally {
+      lock.writeLock().unlock();
+    }
+  }
+  /**
+   * Applies this function to the given argument.
+   *
+   * @param s the function argument
+   * @return the function result
+   */
+  @Override
+  public StellarFunction apply(String s) {
+    StellarFunctionInfo ret = _getFunctions().get(s);
+    if(ret == null) {
+      throw new IllegalStateException("Unable to resolve function " + s);
+    }
+    return ret.getFunction();
+  }
+
+  private void loadFunctions(final Map<String, StellarFunctionInfo> ret) {
+    try {
+      ClassLoader classLoader = getClass().getClassLoader();
+      Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(effectiveClassPathUrls(classLoader)));
+      for (Class<?> clazz : reflections.getSubTypesOf(StellarFunction.class)) {
+        if (clazz.isAnnotationPresent(Stellar.class)) {
+          Map.Entry<String, StellarFunctionInfo> instance = create((Class<? extends StellarFunction>) clazz);
+          if (instance != null) {
+            if(ret.containsKey(instance.getKey()) && !ret.get(instance.getKey()).equals(instance.getValue())) {
+              throw new IllegalStateException("You have a namespace conflict: two functions named " + instance.getKey());
+            }
+            ret.put(instance.getKey(), instance.getValue());
+          }
+        }
+      }
+      LOG.info("Found " + ret.size() + " Stellar Functions...");
+    }
+    catch(Throwable ex) {
+      LOG.error("Unable to initialize FunctionResolverImpl: " + ex.getMessage(), ex);
+      throw ex;
+    }
+  }
+
+  /**
+   * To handle the situation where classpath is specified in the manifest of the jar, we have to augment the URLs.
+   * This happens as part of the surefire plugin as well as elsewhere in the wild.
+   * @param classLoaders
+   * @return
+   */
+  public static Collection<URL> effectiveClassPathUrls(ClassLoader... classLoaders) {
+    return ClasspathHelper.forManifest(ClasspathHelper.forClassLoader(classLoaders));
+  }
+
+
+
+  private static Map.Entry<String, StellarFunctionInfo> create(Class<? extends StellarFunction> stellarClazz) {
+    String fqn = getNameFromAnnotation(stellarClazz);
+    if(fqn == null) {
+      LOG.error("Unable to resolve fully qualified stellar name for " + stellarClazz.getName());
+    }
+    StellarFunction f = createFunction(stellarClazz);
+    if(fqn != null && f != null) {
+      Stellar stellarAnnotation = stellarClazz.getAnnotation(Stellar.class);
+      StellarFunctionInfo info = new StellarFunctionInfo(stellarAnnotation.description()
+                                                        , fqn
+                                                        , stellarAnnotation.params()
+                                                        , stellarAnnotation.returns()
+                                                        , f
+                                                        );
+      return new AbstractMap.SimpleEntry<>(fqn, info);
+    }
+    else {
+      LOG.error("Unable to create instance for StellarFunction " + stellarClazz.getName() + " name: " + fqn);
+    }
+    return null;
+  }
+
+  private static String getNameFromAnnotation(Class<? extends StellarFunction> stellarClazz) {
+    if(stellarClazz.isAnnotationPresent(Stellar.class)) {
+      Stellar stellarAnnotation = stellarClazz.getAnnotation(Stellar.class);
+      String namespace = stellarAnnotation.namespace();
+      String name = stellarAnnotation.name();
+      if(name == null || name.trim().length() == 0) {
+        return null;
+      }
+      else {
+        name = name.trim();
+      }
+      if(namespace == null || namespace.length() == 0) {
+        namespace = null;
+      }
+      else {
+        namespace = namespace.trim();
+      }
+      return Joiner.on("_").skipNulls().join(Arrays.asList(namespace, name));
+    }
+    return null;
+
+  }
+
+  private static StellarFunction createFunction(Class<? extends StellarFunction> stellarClazz) {
+    try {
+      return stellarClazz.newInstance();
+    } catch (Exception e) {
+      LOG.error("Unable to load " + stellarClazz.getName() + " because " + e.getMessage(), e);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Predicate2StellarFunction.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Predicate2StellarFunction.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Predicate2StellarFunction.java
new file mode 100644
index 0000000..2f08285
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Predicate2StellarFunction.java
@@ -0,0 +1,33 @@
+/**
+ * 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.common.dsl;
+
+import java.util.List;
+import java.util.function.Predicate;
+
+public class Predicate2StellarFunction extends BaseStellarFunction {
+  Predicate<List<Object>> pred;
+  public Predicate2StellarFunction(Predicate<List<Object>> pred) {
+    this.pred = pred;
+  }
+
+  @Override
+  public Object apply(List<Object> objects) {
+    return pred.test(objects);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Stellar.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Stellar.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Stellar.java
new file mode 100644
index 0000000..a4e895b
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Stellar.java
@@ -0,0 +1,32 @@
+/**
+ * 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.common.dsl;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(RetentionPolicy.RUNTIME)
+public @interface Stellar {
+  String namespace() default "";
+  String name();
+  String description() default "";
+  String returns() default "";
+  String[] params() default {};
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctionInfo.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctionInfo.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctionInfo.java
new file mode 100644
index 0000000..8c9745f
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctionInfo.java
@@ -0,0 +1,79 @@
+/**
+ * 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.common.dsl;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class StellarFunctionInfo {
+  String description;
+  String name;
+  String[] params;
+  StellarFunction function;
+  String returns;
+  public StellarFunctionInfo(String description, String name, String[] params, String returns, StellarFunction function) {
+    this.description = description;
+    this.name = name;
+    this.params = params;
+    this.function = function;
+    this.returns = returns;
+  }
+
+  public String getReturns() { return returns;}
+
+  public String getDescription() {
+    return description;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String[] getParams() {
+    return params;
+  }
+
+  public StellarFunction getFunction() {
+    return function;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    StellarFunctionInfo that = (StellarFunctionInfo) o;
+
+    if (getDescription() != null ? !getDescription().equals(that.getDescription()) : that.getDescription() != null)
+      return false;
+    if (getName() != null ? !getName().equals(that.getName()) : that.getName() != null) return false;
+    // Probably incorrect - comparing Object[] arrays with Arrays.equals
+    if (!Arrays.equals(getParams(), that.getParams())) return false;
+    return getReturns() != null ? getReturns().equals(that.getReturns()) : that.getReturns() == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = getDescription() != null ? getDescription().hashCode() : 0;
+    result = 31 * result + (getName() != null ? getName().hashCode() : 0);
+    result = 31 * result + Arrays.hashCode(getParams());
+    result = 31 * result + (getReturns() != null ? getReturns().hashCode() : 0);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctions.java
index 8ca90a8..37915d4 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctions.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctions.java
@@ -18,150 +18,13 @@
 
 package org.apache.metron.common.dsl;
 
-import org.apache.metron.common.dsl.functions.ConversionFunctions;
-import org.apache.metron.common.dsl.functions.DataStructureFunctions;
-import org.apache.metron.common.dsl.functions.DateFunctions;
-import org.apache.metron.common.dsl.functions.MaaSFunctions;
-import org.apache.metron.common.dsl.functions.MapFunctions;
-import org.apache.metron.common.dsl.functions.NetworkFunctions;
-import org.apache.metron.common.dsl.functions.StringFunctions;
-import org.apache.metron.common.dsl.functions.StellarStatisticsFunctions;
-import org.apache.metron.common.field.transformation.IPProtocolTransformation;
-import org.apache.metron.common.field.validation.network.DomainValidation;
-import org.apache.metron.common.field.validation.network.EmailValidation;
-import org.apache.metron.common.field.validation.network.IPValidation;
-import org.apache.metron.common.field.validation.network.URLValidation;
-import org.apache.metron.common.field.validation.primitive.DateValidation;
-import org.apache.metron.common.field.validation.primitive.IntegerValidation;
-
-import java.util.List;
-import java.util.function.Predicate;
-
-public enum StellarFunctions implements StellarFunction {
-
-  // string functions
-  TO_LOWER(new StringFunctions.ToLower()),
-  TO_UPPER(new StringFunctions.ToUpper()),
-  TRIM(new StringFunctions.Trim()),
-  JOIN(new StringFunctions.JoinFunction()),
-  SPLIT(new StringFunctions.SplitFunction()),
-  GET_FIRST(new StringFunctions.GetFirst()),
-  GET_LAST(new StringFunctions.GetLast()),
-  GET(new StringFunctions.Get()),
-  STARTS_WITH( new StringFunctions.StartsWith()),
-  ENDS_WITH( new StringFunctions.EndsWith()),
-  REGEXP_MATCH( new StringFunctions.RegexpMatch()),
-
-  // conversion functions
-  TO_STRING(new StringFunctions.ToString()),
-  TO_INTEGER(new ConversionFunctions.Cast<>(Integer.class)),
-  TO_DOUBLE(new ConversionFunctions.Cast<>(Double.class)),
-  TO_LONG(new ConversionFunctions.Cast<>(Long.class)),
-
-  // map functions
-  MAP_GET(new MapFunctions.MapGet()),
-  MAP_EXISTS( new MapFunctions.MapExists()),
-
-  // network functions
-  DOMAIN_TO_TLD(new NetworkFunctions.ExtractTLD()),
-  DOMAIN_REMOVE_TLD(new NetworkFunctions.RemoveTLD()),
-  DOMAIN_REMOVE_SUBDOMAINS(new NetworkFunctions.RemoveSubdomains()),
-  URL_TO_HOST(new NetworkFunctions.URLToHost()),
-  URL_TO_PORT(new NetworkFunctions.URLToPort()),
-  URL_TO_PATH(new NetworkFunctions.URLToPath()),
-  URL_TO_PROTOCOL(new NetworkFunctions.URLToProtocol()),
-  IN_SUBNET( new NetworkFunctions.InSubnet()),
-  PROTOCOL_TO_NAME(new IPProtocolTransformation()),
-
-  // date functions
-  TO_EPOCH_TIMESTAMP(new DateFunctions.ToTimestamp()),
-  YEAR(new DateFunctions.Year()),
-  MONTH(new DateFunctions.MonthOfYear()),
-  DAY_OF_MONTH(new DateFunctions.DayOfMonth()),
-  DAY_OF_WEEK(new DateFunctions.DayOfWeek()),
-  WEEK_OF_MONTH(new DateFunctions.WeekOfMonth()),
-  WEEK_OF_YEAR(new DateFunctions.WeekOfYear()),
-  DAY_OF_YEAR(new DateFunctions.DayOfYear()),
-
-  // validation functions
-  IS_EMPTY ( new DataStructureFunctions.IsEmpty()),
-  IS_IP(new Predicate2Transformation(new IPValidation())),
-  IS_DOMAIN(new Predicate2Transformation(new DomainValidation())),
-  IS_EMAIL(new Predicate2Transformation(new EmailValidation())),
-  IS_URL(new Predicate2Transformation(new URLValidation())),
-  IS_DATE(new Predicate2Transformation(new DateValidation())),
-  IS_INTEGER(new Predicate2Transformation(new IntegerValidation())),
-
-  // model-as-a-service functions
-  MAAS_GET_ENDPOINT( new MaaSFunctions.GetEndpoint()),
-  MODEL_APPLY(new MaaSFunctions.ModelApply()),
-
-  // summary statistics
-  STATS_INIT(new StellarStatisticsFunctions.Init()),
-  STATS_ADD(new StellarStatisticsFunctions.Add()),
-  STATS_COUNT(new StellarStatisticsFunctions.Count()),
-  STATS_MEAN(new StellarStatisticsFunctions.Mean()),
-  STATS_GEOMETRIC_MEAN(new StellarStatisticsFunctions.GeometricMean()),
-  STATS_MAX(new StellarStatisticsFunctions.Max()),
-  STATS_MIN(new StellarStatisticsFunctions.Min()),
-  STATS_SUM(new StellarStatisticsFunctions.Sum()),
-  STATS_POPULATION_VARIANCE(new StellarStatisticsFunctions.PopulationVariance()),
-  STATS_VARIANCE(new StellarStatisticsFunctions.Variance()),
-  STATS_QUADRATIC_MEAN(new StellarStatisticsFunctions.QuadraticMean()),
-  STATS_SD(new StellarStatisticsFunctions.StandardDeviation()),
-  STATS_SUM_LOGS(new StellarStatisticsFunctions.SumLogs()),
-  STATS_SUM_SQUARES(new StellarStatisticsFunctions.SumSquares()),
-  STATS_KURTOSIS(new StellarStatisticsFunctions.Kurtosis()),
-  STATS_SKEWNESS(new StellarStatisticsFunctions.Skewness()),
-  STATS_PERCENTILE(new StellarStatisticsFunctions.Percentile());
-
-  private static class Predicate2Transformation extends BaseStellarFunction {
-    Predicate<List<Object>> pred;
-    public Predicate2Transformation(Predicate<List<Object>> pred) {
-      this.pred = pred;
-    }
-
-    @Override
-    public Object apply(List<Object> objects) {
-      return pred.test(objects);
-    }
-  }
-
-  StellarFunction func;
-
-  StellarFunctions(StellarFunction func) {
-    this.func = func;
-  }
-
-  @Override
-  public Object apply(List<Object> input, Context context) {
-    return func.apply(input, context);
-  }
-  @Override
-  public void initialize(Context context) {
-    func.initialize(context);
-  }
-
-  @Override
-  public boolean isInitialized() {
-    return func.isInitialized();
-  }
+public class StellarFunctions {
 
   public static FunctionResolver FUNCTION_RESOLVER() {
-    return new FunctionResolver() {
-
-      @Override
-      public StellarFunction apply(String s) {
-        StellarFunctions func  = null;
-        try {
-          func = StellarFunctions.valueOf(s);
-          return func;
-        }
-        catch(Exception e) {
-          throw new IllegalStateException("Unable to resolve function " + s);
-        }
-      }
-    };
+    return FunctionResolverSingleton.getInstance();
+  }
 
+  public static void initialize(Context context) {
+    FunctionResolverSingleton.getInstance().initialize(context);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ConversionFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ConversionFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ConversionFunctions.java
index 0a712ce..c5069f1 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ConversionFunctions.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ConversionFunctions.java
@@ -18,11 +18,13 @@
 package org.apache.metron.common.dsl.functions;
 
 import org.apache.metron.common.dsl.BaseStellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.utils.ConversionUtils;
 
 import java.util.List;
 
 public class ConversionFunctions {
+
   public static class Cast<T> extends BaseStellarFunction {
     Class<T> clazz;
     public Cast(Class<T> clazz) {
@@ -34,4 +36,28 @@ public class ConversionFunctions {
       return strings.get(0) == null?null: ConversionUtils.convert(strings.get(0), clazz);
     }
   }
+
+  @Stellar(name="TO_INTEGER"
+          , description="Transforms the first argument to an integer"
+          , params = { "input - Object of string or numeric type"}
+          , returns = "Integer"
+          )
+  public static class TO_INTEGER extends Cast<Integer> {
+
+    public TO_INTEGER() {
+      super(Integer.class);
+    }
+  }
+
+  @Stellar(name="TO_DOUBLE"
+          , description="Transforms the first argument to a double precision number"
+          , params = { "input - Object of string or numeric type"}
+          , returns = "Double"
+          )
+  public static class TO_DOUBLE extends Cast<Double> {
+
+    public TO_DOUBLE() {
+      super(Double.class);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DataStructureFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DataStructureFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DataStructureFunctions.java
index d7764ca..dcfb6e8 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DataStructureFunctions.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DataStructureFunctions.java
@@ -18,11 +18,17 @@
 package org.apache.metron.common.dsl.functions;
 
 import org.apache.metron.common.dsl.BaseStellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 
 import java.util.Collection;
 import java.util.List;
 
 public class DataStructureFunctions {
+  @Stellar(name="IS_EMPTY"
+          , description="Returns true if string or collection is empty and false otherwise"
+          , params = { "input - Object of string or collection type (e.g. list)"}
+          , returns = "Boolean"
+          )
   public static class IsEmpty extends BaseStellarFunction {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DateFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DateFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DateFunctions.java
index 41196c0..709a523 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DateFunctions.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DateFunctions.java
@@ -22,6 +22,7 @@ import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import org.apache.metron.common.dsl.BaseStellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.utils.ConversionUtils;
 
 import java.text.ParseException;
@@ -111,6 +112,15 @@ public class DateFunctions {
   /**
    * Stellar Function: TO_EPOCH_TIMESTAMP
    */
+  @Stellar( name="TO_EPOCH_TIMESTAMP"
+          , description="Returns the epoch timestamp of the dateTime given the format. " +
+                        "If the format does not have a timestamp and you wish to assume a " +
+                        "given timestamp, you may specify the timezone optionally."
+          , params = { "dateTime - DateTime in String format"
+                     , "format - DateTime format as a String"
+                     , "timezone - Optional timezone in String format"
+                     }
+          , returns = "Boolean")
   public static class ToTimestamp extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> objects) {
@@ -138,6 +148,11 @@ public class DateFunctions {
    *
    * The numbered day within the week.  The first day of the week, Sunday, has a value of 1.
    */
+  @Stellar( name="DAY_OF_WEEK"
+          , description="The numbered day within the week.  The first day of the week, Sunday, has a value of 1."
+          , params = { "dateTime - The datetime as a long representing the milliseconds since unix epoch"
+                     }
+          , returns = "The numbered day within the week.")
   public static class DayOfWeek extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -161,6 +176,11 @@ public class DateFunctions {
    *
    * The day within the month.  The first day within the month has a value of 1.
    */
+  @Stellar( name="DAY_OF_MONTH"
+          , description="The numbered day within the month.  The first day within the month has a value of 1."
+          , params = { "dateTime - The datetime as a long representing the milliseconds since unix epoch"
+                     }
+          , returns = "The numbered day within the month.")
   public static class DayOfMonth extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -184,6 +204,11 @@ public class DateFunctions {
    *
    * The numbered week within the month.  The first week has a value of 1.
    */
+  @Stellar( name="WEEK_OF_MONTH"
+          , description="The numbered week within the month.  The first week within the month has a value of 1."
+          , params = { "dateTime - The datetime as a long representing the milliseconds since unix epoch"
+                     }
+          , returns = "The numbered week within the month.")
   public static class WeekOfMonth extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -207,6 +232,11 @@ public class DateFunctions {
    *
    * The numbered week within the year.  The first week in the year has a value of 1.
    */
+  @Stellar( name="WEEK_OF_YEAR"
+          , description="The numbered week within the year.  The first week in the year has a value of 1."
+          , params = { "dateTime - The datetime as a long representing the milliseconds since unix epoch"
+                     }
+          , returns = "The numbered week within the year.")
   public static class WeekOfYear extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -230,6 +260,11 @@ public class DateFunctions {
    *
    * A number representing the month.  The first month, January, has a value of 0.
    */
+  @Stellar( name="MONTH"
+          , description="The number representing the month.  The first month, January, has a value of 0."
+          , params = { "dateTime - The datetime as a long representing the milliseconds since unix epoch"
+                     }
+          , returns = "The current month (0-based).")
   public static class MonthOfYear extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -253,6 +288,12 @@ public class DateFunctions {
    *
    * The calendar year.
    */
+  @Stellar( name="YEAR"
+          , description="The number representing the year. "
+          , params = { "dateTime - The datetime as a long representing the milliseconds since unix epoch"
+                     }
+          , returns = "The current year"
+          )
   public static class Year extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -276,6 +317,12 @@ public class DateFunctions {
    *
    * The day number within the year.  The first day of the year has value of 1.
    */
+  @Stellar( name="DAY_OF_YEAR"
+          , description="The day number within the year.  The first day of the year has value of 1."
+          , params = { "dateTime - The datetime as a long representing the milliseconds since unix epoch"
+                     }
+          , returns = "The day number within the year."
+          )
   public static class DayOfYear extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MaaSFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MaaSFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MaaSFunctions.java
index 8dfac68..d0821fd 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MaaSFunctions.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MaaSFunctions.java
@@ -24,6 +24,7 @@ import org.apache.curator.framework.CuratorFramework;
 import org.apache.hadoop.security.authorize.Service;
 import org.apache.metron.common.dsl.Context;
 import org.apache.metron.common.dsl.ParseException;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.dsl.StellarFunction;
 import org.apache.metron.common.utils.JSONUtils;
 import org.apache.metron.maas.config.Endpoint;
@@ -80,6 +81,16 @@ public class MaaSFunctions {
       return result;
     }
   }
+
+  @Stellar(name="MODEL_APPLY"
+          , namespace="MAAS"
+          , description = "Returns the output of a model deployed via model which is deployed at endpoint. NOTE: Results are cached at the client for 10 minutes."
+          , params = { "endpoint - a map containing name, version, url for the REST endpoint"
+                     , "function - the optional endpoint path, default is 'apply'"
+                     , "model_args - dictionary of arguments for the model (these become request params)."
+                     }
+          , returns = "The output of the model deployed as a REST endpoint in Map form.  Assumes REST endpoint returns a JSON Map."
+          )
   public static class ModelApply implements StellarFunction {
     private boolean isInitialized = false;
     private ServiceDiscoverer discoverer;
@@ -223,6 +234,16 @@ public class MaaSFunctions {
     return discoverer;
   }
 
+  @Stellar(name="GET_ENDPOINT"
+          , namespace="MAAS"
+          , description="Inspects zookeeper and returns a map containing the name, version and url for the model referred to by the input params"
+          , params = {
+                      "model_name - the name of the model"
+                     ,"model_version - the optional version of the model.  If it is not specified, the most current version is used."
+                     }
+          , returns = "A map containing the name, version, url for the REST endpoint (fields named name, version and url).  " +
+                      "Note that the output of this function is suitable for input into the first argument of MAAS_MODEL_APPLY."
+          )
   public static class GetEndpoint implements StellarFunction {
     ServiceDiscoverer discoverer;
     private boolean isInitialized = false;

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MapFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MapFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MapFunctions.java
index 0e322b7..c236cda 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MapFunctions.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MapFunctions.java
@@ -19,12 +19,23 @@
 package org.apache.metron.common.dsl.functions;
 
 import org.apache.metron.common.dsl.BaseStellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 
 import java.util.List;
 import java.util.Map;
 import java.util.function.Function;
 
 public class MapFunctions {
+
+  @Stellar(name="EXISTS"
+          ,namespace="MAP"
+          , description="Checks for existence of a key in a map."
+          , params = {
+                      "key - The key to check for existence"
+                     ,"map - The map to check for existence of the key"
+                     }
+          , returns = "True if the key is found in the map and false otherwise."
+          )
   public static class MapExists extends BaseStellarFunction {
 
     @Override
@@ -40,6 +51,18 @@ public class MapFunctions {
       return false;
     }
   }
+
+  @Stellar(name="GET"
+          ,namespace="MAP"
+          , description="Gets the value associated with a key from a map"
+          , params = {
+                      "key - The key"
+                     ,"map - The map"
+                     ,"default - Optionally the default value to return if the key is not in the map."
+                     }
+          , returns = "The object associated with key in the map.  " +
+                      "If there is no value associated, then default if specified and null if a default is not specified."
+          )
   public static class MapGet extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> objects) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/NetworkFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/NetworkFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/NetworkFunctions.java
index e6a1818..b481782 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/NetworkFunctions.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/NetworkFunctions.java
@@ -23,6 +23,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.net.InternetDomainName;
 import org.apache.commons.net.util.SubnetUtils;
 import org.apache.metron.common.dsl.BaseStellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 
 import java.net.MalformedURLException;
 import java.net.URL;
@@ -30,6 +31,14 @@ import java.util.List;
 import java.util.function.Function;
 
 public class NetworkFunctions {
+  @Stellar(name="IN_SUBNET"
+          ,description = "Returns if an IP is within a subnet range."
+          ,params = {
+                     "ip - the IP address in String form"
+                    ,"cidr+ - one or more IP ranges specified in CIDR notation (e.g. 192.168.0.0/24)"
+                    }
+          ,returns = "True if the IP address is within at least one of the network ranges and false otherwise"
+          )
   public static class InSubnet extends BaseStellarFunction {
 
     @Override
@@ -55,6 +64,16 @@ public class NetworkFunctions {
       return inSubnet;
     }
   }
+
+  @Stellar(name="REMOVE_SUBDOMAINS"
+          ,namespace = "DOMAIN"
+          ,description = "Remove subdomains from a domain."
+          , params = {
+                      "domain - fully qualified domain name"
+                     }
+          , returns = "The domain without the subdomains.  " +
+                      "e.g. DOMAIN_REMOVE_SUBDOMAINS('mail.yahoo.com') yields 'yahoo.com'"
+          )
   public static class RemoveSubdomains extends BaseStellarFunction {
 
     @Override
@@ -81,6 +100,16 @@ public class NetworkFunctions {
       return null;
     }
   }
+
+  @Stellar(name="REMOVE_TLD"
+          ,namespace = "DOMAIN"
+          ,description = "Remove top level domain suffix from a domain."
+          , params = {
+                      "domain - fully qualified domain name"
+                     }
+          , returns = "The domain without the TLD.  " +
+                      "e.g. DOMAIN_REMOVE_TLD('mail.yahoo.co.uk') yields 'mail.yahoo'"
+          )
   public static class RemoveTLD extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> objects) {
@@ -102,6 +131,15 @@ public class NetworkFunctions {
     }
   }
 
+  @Stellar(name="TO_TLD"
+          ,namespace = "DOMAIN"
+          ,description = "Extract the top level domain from a domain"
+          , params = {
+                      "domain - fully qualified domain name"
+                     }
+          , returns = "The TLD of the domain.  " +
+                      "e.g. DOMAIN_TO_TLD('mail.yahoo.co.uk') yields 'co.uk'"
+          )
   public static class ExtractTLD extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> objects) {
@@ -114,6 +152,15 @@ public class NetworkFunctions {
     }
   }
 
+  @Stellar(name="TO_PORT"
+          ,namespace="URL"
+          ,description = "Extract the port from a URL.  " +
+                          "If the port is not explicitly stated in the URL, then an implicit port is inferred based on the protocol."
+          , params = {
+                      "url - URL in String form"
+                     }
+          , returns = "The port used in the URL as an Integer.  e.g. URL_TO_PORT('http://www.yahoo.com/foo') would yield 80"
+          )
   public static class URLToPort extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> objects) {
@@ -126,6 +173,13 @@ public class NetworkFunctions {
     }
   }
 
+  @Stellar(name="TO_PATH"
+          ,namespace="URL"
+          ,description = "Extract the path from a URL."
+          , params = {
+                      "url - URL in String form"
+                     }
+          , returns = "The path from the URL as a String.  e.g. URL_TO_PATH('http://www.yahoo.com/foo') would yield 'foo'")
   public static class URLToPath extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> objects) {
@@ -133,6 +187,15 @@ public class NetworkFunctions {
       return url == null?null:url.getPath();
     }
   }
+
+  @Stellar(name="TO_HOST"
+          ,namespace="URL"
+          ,description = "Extract the hostname from a URL."
+          , params = {
+                      "url - URL in String form"
+                     }
+          , returns = "The hostname from the URL as a String.  e.g. URL_TO_HOST('http://www.yahoo.com/foo') would yield 'www.yahoo.com'"
+          )
   public static class URLToHost extends BaseStellarFunction {
 
     @Override
@@ -142,6 +205,13 @@ public class NetworkFunctions {
     }
   }
 
+  @Stellar(name="TO_PROTOCOL"
+          ,namespace="URL"
+          ,description = "Extract the protocol from a URL."
+          , params = {
+                      "url - URL in String form"
+                     }
+          , returns = "The protocol from the URL as a String. e.g. URL_TO_PROTOCOL('http://www.yahoo.com/foo') would yield 'http'")
   public static class URLToProtocol extends BaseStellarFunction {
 
     @Override
@@ -150,6 +220,7 @@ public class NetworkFunctions {
       return url == null?null:url.getProtocol();
     }
   }
+
   private static InternetDomainName toDomainName(Object dnObj) {
     if(dnObj != null) {
       String dn = dnObj.toString();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java
index adf439b..75e86aa 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java
@@ -21,6 +21,7 @@
 package org.apache.metron.common.dsl.functions;
 
 import org.apache.metron.common.dsl.BaseStellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 
 import java.util.Collections;
 import java.util.List;
@@ -57,6 +58,17 @@ public class StellarStatisticsFunctions {
    *             Using no rolling window is less memory intensive, but cannot
    *             calculate certain statistics like percentiles and kurtosis.
    */
+  @Stellar( namespace="STATS"
+          , name="INIT"
+          , description = "Initialize a Statistics object"
+          , params = {
+                      "window_size - The number of input data values to maintain in a rolling window " +
+                      "in memory.  If equal to 0, then no rolling window is maintained. " +
+                      "Using no rolling window is less memory intensive, but cannot " +
+                      "calculate certain statistics like percentiles and kurtosis."
+                      }
+          , returns = "A StellarStatistics object"
+          )
   public static class Init extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -69,6 +81,15 @@ public class StellarStatisticsFunctions {
    *
    *  STATS_ADD (stats, value [, value2, value3, ...])
    */
+  @Stellar(namespace="STATS"
+          , name="ADD"
+          , description = "Add one or more input values to those that are used to calculate the summary statistics."
+          , params = {
+                      "stats - The Stellar statistics object.  If null, then a new one is initialized."
+                     , "value+ - one or more numbers to add "
+                     }
+          , returns = "A StellarStatistics object"
+          )
   public static class Add extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -95,6 +116,14 @@ public class StellarStatisticsFunctions {
    *
    *  STATS_MEAN (stats)
    */
+  @Stellar( namespace="STATS"
+          , name="MEAN"
+          , description = "Calculates the mean of the values accumulated (or in the window if a window is used)."
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The mean of the values in the window or NaN if the statistics object is null."
+          )
   public static class Mean extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -106,6 +135,14 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the geometric mean.
    */
+  @Stellar( namespace="STATS"
+          , name="GEOMETRIC_MEAN"
+          , description = "Calculates the geometric mean of the values accumulated (or in the window if a window is used). See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics "
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The geometric mean of the values in the window or NaN if the statistics object is null."
+          )
   public static class GeometricMean extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -117,6 +154,14 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the sum.
    */
+  @Stellar(namespace="STATS"
+          , name="SUM"
+          , description = "Calculates the sum of the values accumulated (or in the window if a window is used)."
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The sum of the values in the window or NaN if the statistics object is null."
+          )
   public static class Sum extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -128,6 +173,13 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the max.
    */
+  @Stellar(namespace="STATS", name="MAX"
+          , description = "Calculates the max of the values accumulated (or in the window if a window is used)."
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The max of the values in the window or NaN if the statistics object is null."
+          )
   public static class Max extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -139,6 +191,13 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the min.
    */
+  @Stellar(namespace="STATS", name="MIN"
+          , description = "Calculates the min of the values accumulated (or in the window if a window is used)."
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The min of the values in the window or NaN if the statistics object is null."
+          )
   public static class Min extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -150,6 +209,12 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the count of elements
    */
+  @Stellar(namespace="STATS", name="COUNT"
+          , description = "Calculates the count of the values accumulated (or in the window if a window is used)."
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The count of the values in the window or NaN if the statistics object is null.")
   public static class Count extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -161,6 +226,12 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the population variance.
    */
+  @Stellar(namespace="STATS", name="POPULATION_VARIANCE"
+          , description = "Calculates the population variance of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics "
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The population variance of the values in the window or NaN if the statistics object is null.")
   public static class PopulationVariance extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -172,6 +243,12 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the variance.
    */
+  @Stellar(namespace="STATS", name="VARIANCE"
+          , description = "Calculates the variance of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics "
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The variance of the values in the window or NaN if the statistics object is null.")
   public static class Variance extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -183,6 +260,12 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the quadratic mean.
    */
+  @Stellar(namespace="STATS", name="QUADRATIC_MEAN"
+          , description = "Calculates the quadratic mean of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics "
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The quadratic mean of the values in the window or NaN if the statistics object is null.")
   public static class QuadraticMean extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -194,6 +277,12 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the standard deviation.
    */
+  @Stellar(namespace="STATS", name="SD"
+          , description = "Calculates the standard deviation of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics "
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The standard deviation of the values in the window or NaN if the statistics object is null.")
   public static class StandardDeviation extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -205,6 +294,12 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the sum of logs.
    */
+  @Stellar(namespace="STATS", name="SUM_LOGS"
+          , description = "Calculates the sum of the (natural) log of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics "
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The sum of the (natural) log of the values in the window or NaN if the statistics object is null.")
   public static class SumLogs extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -216,6 +311,12 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the sum of squares.
    */
+  @Stellar(namespace="STATS", name="SUM_SQUARES"
+          , description = "Calculates the sum of the squares of the values accumulated (or in the window if a window is used)."
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The sum of the squares of the values in the window or NaN if the statistics object is null.")
   public static class SumSquares extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -227,6 +328,12 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the kurtosis.
    */
+  @Stellar(namespace="STATS", name="KURTOSIS"
+          , description = "Calculates the kurtosis of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics "
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The kurtosis of the values in the window or NaN if the statistics object is null.")
   public static class Kurtosis extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -238,6 +345,12 @@ public class StellarStatisticsFunctions {
   /**
    * Calculates the skewness.
    */
+  @Stellar(namespace="STATS", name="SKEWNESS"
+          , description = "Calculates the skewness of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics "
+          , params = {
+            "stats - The Stellar statistics object."
+                     }
+          , returns = "The skewness of the values in the window or NaN if the statistics object is null.")
   public static class Skewness extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -251,6 +364,15 @@ public class StellarStatisticsFunctions {
    *
    * STATS_PERCENTILE(stats, 0.90)
    */
+  @Stellar(namespace="STATS", name="PERCENTILE"
+          , description = "Computes the p'th percentile of the values accumulated (or in the window if a window is used)."
+          , params = {
+          "stats - The Stellar statistics object."
+          ,"p - a double where 0 <= p < 1 representing the percentile"
+
+                     }
+          , returns = "The p'th percentile of the data or NaN if the statistics object is null"
+          )
   public static class Percentile extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StringFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StringFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StringFunctions.java
index beca43f..6dcc8b6 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StringFunctions.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StringFunctions.java
@@ -22,12 +22,21 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Splitter;
 import com.google.common.collect.Iterables;
 import org.apache.metron.common.dsl.BaseStellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.function.Function;
 
 public class StringFunctions {
+
+  @Stellar(name="REGEXP_MATCH"
+          ,description = "Determines whether a regex matches a string"
+          , params = {
+             "string - The string to test"
+            ,"pattern - The proposed regex pattern"
+            }
+          , returns = "True if the regex pattern matches the string and false otherwise.")
   public static class RegexpMatch extends BaseStellarFunction {
 
     @Override
@@ -44,6 +53,13 @@ public class StringFunctions {
     }
   }
 
+  @Stellar(name="ENDS_WITH"
+          ,description = "Determines whether a string ends with a prefix"
+          , params = {
+             "string - The string to test"
+            ,"suffix - The proposed suffix"
+            }
+          , returns = "True if the string ends with the specified suffix and false otherwise.")
   public static class EndsWith extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> list) {
@@ -58,6 +74,15 @@ public class StringFunctions {
       return str.endsWith(prefix);
     }
   }
+
+  @Stellar(name="STARTS_WITH"
+          ,description = "Determines whether a string starts with a prefix"
+          , params = {
+             "string - The string to test"
+            ,"prefix - The proposed prefix"
+            }
+          , returns = "True if the string starts with the specified prefix and false otherwise."
+          )
   public static class StartsWith extends BaseStellarFunction {
 
     @Override
@@ -73,30 +98,60 @@ public class StringFunctions {
       return str.startsWith(prefix);
     }
   }
+
+  @Stellar( name="TO_LOWER"
+          , description = "Transforms the first argument to a lowercase string"
+          , params = { "input - String" }
+          , returns = "String"
+          )
   public static class ToLower extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> strings) {
       return strings.get(0)==null?null:strings.get(0).toString().toLowerCase();
     }
   }
+
+  @Stellar( name="TO_UPPER"
+          , description = "Transforms the first argument to an uppercase string"
+          , params = { "input - String" }
+          , returns = "String"
+          )
   public static class ToUpper extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> strings) {
       return strings.get(0)==null?null:strings.get(0).toString().toUpperCase();
     }
   }
+
+  @Stellar(name="TO_STRING"
+          , description = "Transforms the first argument to a string"
+          , params = { "input - Object" }
+          , returns = "String"
+          )
   public static class ToString extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> strings) {
       return strings.get(0)==null?null:strings.get(0).toString();
     }
   }
+
+  @Stellar(name="TRIM"
+          , description = "Trims whitespace from both sides of a string."
+          , params = { "input - String" }
+          , returns = "String"
+          )
   public static class Trim extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> strings) {
       return strings.get(0)==null?null:strings.get(0).toString().trim();
     }
   }
+
+  @Stellar( name="JOIN"
+          , description="Joins the components of the list with the specified delimiter."
+          , params = { "list - List of Strings", "delim - String delimiter"}
+          , returns = "String"
+          )
   public static class JoinFunction extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -105,6 +160,12 @@ public class StringFunctions {
       return Joiner.on(delim).join(Iterables.filter(arg1, x -> x != null));
     }
   }
+
+  @Stellar(name="SPLIT"
+          , description="Splits the string by the delimiter."
+          , params = { "input - String to split", "delim - String delimiter"}
+          , returns = "List of Strings"
+          )
   public static class SplitFunction extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -119,6 +180,11 @@ public class StringFunctions {
     }
   }
 
+  @Stellar(name="GET_LAST"
+          , description="Returns the last element of the list"
+          , params = { "input - List"}
+          , returns = "Last element of the list"
+          )
   public static class GetLast extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -126,6 +192,12 @@ public class StringFunctions {
       return Iterables.getLast(arg1, null);
     }
   }
+
+  @Stellar(name="GET_FIRST"
+          , description="Returns the first element of the list"
+          , params = { "input - List"}
+          , returns = "First element of the list"
+          )
   public static class GetFirst extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {
@@ -134,6 +206,11 @@ public class StringFunctions {
     }
   }
 
+  @Stellar(name="GET"
+          , description="Returns the i'th element of the list "
+          , params = { "input - List", "i - the index (0-based)"}
+          , returns = "First element of the list"
+          )
   public static class Get extends BaseStellarFunction {
     @Override
     public Object apply(List<Object> args) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/IPProtocolTransformation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/IPProtocolTransformation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/IPProtocolTransformation.java
index 4cd2201..37b9b1b 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/IPProtocolTransformation.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/IPProtocolTransformation.java
@@ -22,6 +22,7 @@ package org.apache.metron.common.field.transformation;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.metron.common.dsl.Context;
 import org.apache.metron.common.dsl.ParseException;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.dsl.StellarFunction;
 import org.apache.metron.common.utils.ConversionUtils;
 
@@ -30,6 +31,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.function.Function;
 
+@Stellar(name="PROTOCOL_TO_NAME"
+        , description="Convert the IANA protocol number to the protocol name"
+        , params = {
+                    "IANA Number"
+                   }
+        , returns = "The protocol name associated with the IANA number."
+        )
 public class IPProtocolTransformation extends SimpleFieldTransformation implements StellarFunction {
 
   private final static Map<Integer, String> PROTOCOLS = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/RemoveTransformation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/RemoveTransformation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/RemoveTransformation.java
index 3b443a7..6513982 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/RemoveTransformation.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/RemoveTransformation.java
@@ -30,7 +30,7 @@ public class RemoveTransformation implements FieldTransformation {
   public static final String CONDITION_CONF = "condition";
   public static final StellarPredicateProcessor PASSTHROUGH_PROCESSOR = new StellarPredicateProcessor() {
     @Override
-    public Boolean parse(String rule, VariableResolver resolver, Function<String, StellarFunction> functionResolver, Context context) {
+    public Boolean parse(String rule, VariableResolver resolver, FunctionResolver functionResolver, Context context) {
       return true;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/DomainValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/DomainValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/DomainValidation.java
index 8b1fc36..7606860 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/DomainValidation.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/DomainValidation.java
@@ -19,6 +19,8 @@
 package org.apache.metron.common.field.validation.network;
 
 import org.apache.commons.validator.routines.DomainValidator;
+import org.apache.metron.common.dsl.Predicate2StellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.field.validation.FieldValidation;
 import org.apache.metron.common.field.validation.SimpleValidation;
 
@@ -27,6 +29,20 @@ import java.util.function.Predicate;
 
 public class DomainValidation extends SimpleValidation {
 
+  @Stellar(name="IS_DOMAIN"
+          ,description = "Tests if a string is a valid domain.  Domain names are evaluated according" +
+          " to the standards RFC1034 section 3, and RFC1123 section 2.1."
+          ,params = {
+              "address - The String to test"
+                    }
+          , returns = "True if the string is a valid domain and false otherwise.")
+  public static class IS_DOMAIN extends Predicate2StellarFunction {
+
+    public IS_DOMAIN() {
+      super(new DomainValidation());
+    }
+  }
+
   @Override
   public Predicate<Object> getPredicate() {
     return domain -> DomainValidator.getInstance().isValid(domain == null?null:domain.toString());

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/EmailValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/EmailValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/EmailValidation.java
index 44d835c..2c9aa69 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/EmailValidation.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/EmailValidation.java
@@ -19,12 +19,28 @@
 package org.apache.metron.common.field.validation.network;
 
 import org.apache.commons.validator.routines.EmailValidator;
+import org.apache.metron.common.dsl.Predicate2StellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.field.validation.SimpleValidation;
 
 import java.util.Map;
 import java.util.function.Predicate;
 
-public class EmailValidation extends SimpleValidation{
+public class EmailValidation extends SimpleValidation {
+
+  @Stellar(name="IS_EMAIL"
+          ,description = "Tests if a string is a valid email address"
+          ,params = {
+              "address - The String to test"
+                    }
+          , returns = "True if the string is a valid email address and false otherwise.")
+  public static class IS_EMAIL extends Predicate2StellarFunction {
+
+    public IS_EMAIL() {
+      super(new EmailValidation());
+    }
+  }
+
   @Override
   public Predicate<Object> getPredicate() {
     return email -> EmailValidator.getInstance().isValid(email == null?null:email.toString());

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/IPValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/IPValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/IPValidation.java
index 94c32a5..3f27d19 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/IPValidation.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/IPValidation.java
@@ -20,6 +20,8 @@ package org.apache.metron.common.field.validation.network;
 
 import org.apache.commons.validator.routines.InetAddressValidator;
 import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.Predicate2StellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.field.validation.FieldValidation;
 
 import java.util.List;
@@ -28,6 +30,19 @@ import java.util.function.Predicate;
 
 public class IPValidation implements FieldValidation, Predicate<List<Object>> {
 
+  @Stellar(name="IS_IP"
+          , description = "Determine if an string is an IP or not."
+          , params = {
+              "ip - An object which we wish to test is an ip"
+             ,"type (optional) - one of IPV4 or IPV6.  The default is IPV4."
+                     }
+          , returns = "True if the string is an IP and false otherwise.")
+  public static class IS_IP extends Predicate2StellarFunction {
+
+    public IS_IP() {
+      super(new IPValidation());
+    }
+  }
 
   private enum IPType {
      IPV4(ip -> InetAddressValidator.getInstance().isValidInet4Address(ip))

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/URLValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/URLValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/URLValidation.java
index 5ef053b..ff44ce7 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/URLValidation.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/URLValidation.java
@@ -19,11 +19,28 @@
 package org.apache.metron.common.field.validation.network;
 
 import org.apache.commons.validator.routines.UrlValidator;
+import org.apache.metron.common.dsl.Predicate2StellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.field.validation.SimpleValidation;
 
 import java.util.function.Predicate;
 
 public class URLValidation extends SimpleValidation {
+
+  @Stellar(name="IS_URL"
+          ,description = "Tests if a string is a valid URL"
+          ,params = {
+              "url - The String to test"
+                    }
+          , returns = "True if the string is a valid URL and false otherwise."
+          )
+  public static class IS_URL extends Predicate2StellarFunction {
+
+    public IS_URL() {
+      super(new URLValidation());
+    }
+  }
+
   @Override
   public Predicate<Object> getPredicate() {
     return url -> UrlValidator.getInstance().isValid(url == null?null:url.toString());

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/DateValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/DateValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/DateValidation.java
index d0dc43e..c587f09 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/DateValidation.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/DateValidation.java
@@ -19,6 +19,8 @@
 package org.apache.metron.common.field.validation.primitive;
 
 import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.Predicate2StellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.field.validation.FieldValidation;
 
 import java.text.ParseException;
@@ -30,6 +32,21 @@ import java.util.function.Predicate;
 
 public class DateValidation implements FieldValidation, Predicate<List<Object>> {
 
+  @Stellar(name="IS_DATE"
+          ,description = "Determines if a string passed is a date of a given format."
+          ,params = {
+            "date - The date in string form."
+          , "format - The format of the date."
+                    }
+          ,returns = "True if the date is of the specified format and false otherwise."
+          )
+  public static class IS_DATE extends Predicate2StellarFunction {
+
+    public IS_DATE() {
+      super(new DateValidation());
+    }
+  }
+
   /**
    * Evaluates this predicate on the given argument.
    *

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/IntegerValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/IntegerValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/IntegerValidation.java
index d803e37..a469bc5 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/IntegerValidation.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/IntegerValidation.java
@@ -20,11 +20,26 @@ package org.apache.metron.common.field.validation.primitive;
 
 import org.apache.commons.validator.routines.DoubleValidator;
 import org.apache.commons.validator.routines.LongValidator;
+import org.apache.metron.common.dsl.Predicate2StellarFunction;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.field.validation.SimpleValidation;
 
 import java.util.function.Predicate;
 
 public class IntegerValidation extends SimpleValidation{
+  @Stellar(name="IS_INTEGER"
+          , description = "Determine if an object is an integer or not."
+          , params = {
+              "x - An object which we wish to test is an integer"
+                     }
+          , returns = "True if the object can be converted to an integer and false otherwise."
+          )
+  public static class IS_INTEGER extends Predicate2StellarFunction {
+
+    public IS_INTEGER() {
+      super(new IntegerValidation());
+    }
+  }
   @Override
   public Predicate<Object> getPredicate() {
     return x -> LongValidator.getInstance().isValid(x == null?null:x.toString());

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/BaseStellarProcessor.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/BaseStellarProcessor.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/BaseStellarProcessor.java
index e4cdcfa..d1d45e8 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/BaseStellarProcessor.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/BaseStellarProcessor.java
@@ -71,7 +71,7 @@ public class BaseStellarProcessor<T> {
 
   public T parse( String rule
                 , VariableResolver variableResolver
-                , Function<String, StellarFunction> functionResolver
+                , FunctionResolver functionResolver
                 , Context context
                 )
   {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarCompiler.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarCompiler.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarCompiler.java
index b818834..9239159 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarCompiler.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarCompiler.java
@@ -31,10 +31,10 @@ import java.util.function.Function;
 public class StellarCompiler extends StellarBaseListener {
   private Context context = null;
   private Stack<Token> tokenStack = new Stack<>();
-  private Function<String, StellarFunction> functionResolver;
+  private FunctionResolver functionResolver;
   private VariableResolver variableResolver;
   public StellarCompiler( VariableResolver variableResolver
-                        , Function<String, StellarFunction> functionResolver
+                        , FunctionResolver functionResolver
                         , Context context
                         )
   {
@@ -246,7 +246,7 @@ public class StellarCompiler extends StellarBaseListener {
     }
     catch(Exception iae) {
       throw new ParseException("Unable to find string function " + funcName + ".  Valid functions are "
-              + Joiner.on(',').join(StellarFunctions.values())
+              + Joiner.on(',').join(functionResolver.getFunctions())
       );
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarPredicateProcessor.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarPredicateProcessor.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarPredicateProcessor.java
index 093a8d8..bac6b37 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarPredicateProcessor.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarPredicateProcessor.java
@@ -20,6 +20,7 @@ package org.apache.metron.common.stellar;
 
 
 import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.FunctionResolver;
 import org.apache.metron.common.dsl.StellarFunction;
 import org.apache.metron.common.dsl.VariableResolver;
 
@@ -43,7 +44,7 @@ public class StellarPredicateProcessor extends BaseStellarProcessor<Boolean> {
   @Override
   public Boolean parse( String rule
                       , VariableResolver variableResolver
-                      , Function<String, StellarFunction> functionResolver
+                      , FunctionResolver functionResolver
                       , Context context
                       )
   {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/fc2dc3d6/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarTest.java
index efd7d18..21ae7bb 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarTest.java
@@ -21,17 +21,38 @@ package org.apache.metron.common.stellar;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.metron.common.dsl.*;
 import org.junit.Assert;
 import org.junit.Test;
+import org.reflections.Reflections;
+import org.reflections.util.ConfigurationBuilder;
 
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 
+import static org.apache.metron.common.dsl.FunctionResolverSingleton.effectiveClassPathUrls;
+
 public class StellarTest {
 
   @Test
+  public void ensureDocumentation() {
+    ClassLoader classLoader = getClass().getClassLoader();
+    Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(effectiveClassPathUrls(classLoader)));
+    for (Class<?> clazz : reflections.getSubTypesOf(StellarFunction.class)) {
+      if (clazz.isAnnotationPresent(Stellar.class)) {
+        Stellar annotation = clazz.getAnnotation(Stellar.class);
+        Assert.assertFalse("Must specify a name for " + clazz.getName(),StringUtils.isEmpty(annotation.name()));
+        Assert.assertFalse("Must specify a description annotation for " + clazz.getName(),StringUtils.isEmpty(annotation.description()));
+        Assert.assertTrue("Must specify a non-empty params for " + clazz.getName(), annotation.params().length > 0);
+        Assert.assertTrue("Must specify a non-empty params for " + clazz.getName(), StringUtils.isNoneEmpty(annotation.params()));
+        Assert.assertFalse("Must specify a returns annotation for " + clazz.getName(), StringUtils.isEmpty(annotation.returns()));
+      }
+    }
+  }
+
+  @Test
   public void testIfThenElseBug1() {
     String query = "50 + (true == true ? 10 : 20)";
     Assert.assertEquals(60.0, run(query, new HashMap<>()));