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 2017/03/05 18:30:14 UTC

[1/2] incubator-metron git commit: METRON-744: Allow Stellar functions to be loaded from HDFS this closes apache/incubator-metron#468

Repository: incubator-metron
Updated Branches:
  refs/heads/master 818b0b17b -> 27ee49096


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/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 abbd9d8..20c15ce 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)
                                 .with(Context.Capabilities.GLOBAL_CONFIG, () -> getConfigurations().getGlobalConfig())
+                                .with(Context.Capabilities.STELLAR_CONFIG, () -> getConfigurations().getGlobalConfig())
                                 .build();
     StellarFunctions.initialize(stellarContext);
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-pcap-backend/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-pcap-backend/pom.xml b/metron-platform/metron-pcap-backend/pom.xml
index 9d6a9eb..819016f 100644
--- a/metron-platform/metron-pcap-backend/pom.xml
+++ b/metron-platform/metron-pcap-backend/pom.xml
@@ -257,6 +257,7 @@
                                 </relocation>
                             </relocations>
                             <transformers>
+                                <transformer implementation="org.atteo.classindex.ClassIndexTransformer"/>
                                 <transformer
                                   implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
                                      <resources>
@@ -281,6 +282,13 @@
                         </configuration>
                     </execution>
                 </executions>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.atteo.classindex</groupId>
+                        <artifactId>classindex-transformer</artifactId>
+                        <version>${global_classindex_version}</version>
+                    </dependency>
+                </dependencies>
              </plugin>
              <plugin>
                 <artifactId>maven-assembly-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-solr/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/pom.xml b/metron-platform/metron-solr/pom.xml
index 1a59cc4..8c1f784 100644
--- a/metron-platform/metron-solr/pom.xml
+++ b/metron-platform/metron-solr/pom.xml
@@ -288,6 +288,7 @@
                                 </excludes>
                             </artifactSet>
                             <transformers>
+                                <transformer implementation="org.atteo.classindex.ClassIndexTransformer"/>
                                 <transformer
                                         implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
                                     <resource>.yaml</resource>
@@ -302,6 +303,13 @@
                         </configuration>
                     </execution>
                 </executions>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.atteo.classindex</groupId>
+                        <artifactId>classindex-transformer</artifactId>
+                        <version>${global_classindex_version}</version>
+                    </dependency>
+                </dependencies>
             </plugin>
             <plugin>
                 <artifactId>maven-assembly-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3a8eb9b..df674af 100644
--- a/pom.xml
+++ b/pom.xml
@@ -70,11 +70,13 @@
         <base_kafka_version>0.10.0.1</base_kafka_version>
         <base_hadoop_version>2.7.1</base_hadoop_version>
         <base_hbase_version>1.1.1</base_hbase_version>
+        <global_accumulo_version>1.8.0</global_accumulo_version>
         <base_flume_version>1.5.2</base_flume_version>
         <!-- full dependency versions -->
         <global_antlr_version>4.5</global_antlr_version>
         <global_opencsv_version>3.7</global_opencsv_version>
         <global_curator_version>2.7.1</global_curator_version>
+        <global_classindex_version>3.3</global_classindex_version>
         <global_storm_version>${base_storm_version}</global_storm_version>
         <global_flux_version>${base_flux_version}</global_flux_version>
         <global_pcap_version>1.7.1</global_pcap_version>


[2/2] incubator-metron git commit: METRON-744: Allow Stellar functions to be loaded from HDFS this closes apache/incubator-metron#468

Posted by ce...@apache.org.
METRON-744: Allow Stellar functions to be loaded from HDFS this closes apache/incubator-metron#468


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

Branch: refs/heads/master
Commit: 27ee490960614dfd890acdab443c6b242ce2cdb8
Parents: 818b0b1
Author: cstella <ce...@gmail.com>
Authored: Sun Mar 5 18:29:38 2017 +0000
Committer: cstella <ce...@gmail.com>
Committed: Sun Mar 5 18:29:38 2017 +0000

----------------------------------------------------------------------
 dependencies_with_url.csv                       |   1 +
 .../apache/metron/profiler/ProfileBuilder.java  |   1 +
 metron-analytics/metron-profiler/pom.xml        |  19 +++
 .../profiler/bolt/ProfileSplitterBolt.java      |   1 +
 .../metron/rest/config/ZookeeperConfig.java     |   4 +-
 .../rest/service/impl/StellarServiceImpl.java   | 104 +++++++-----
 .../service/impl/StellarServiceImplTest.java    |   6 +-
 .../metron-common/3rdPartyStellar.md            | 160 +++++++++++++++++
 metron-platform/metron-common/README.md         |  53 +++++-
 metron-platform/metron-common/pom.xml           |  34 ++++
 .../metron/common/bolt/ConfiguredBolt.java      |   5 +
 .../common/bolt/ConfiguredEnrichmentBolt.java   |   3 +
 .../common/configuration/ConfigurationType.java |   2 +-
 .../configuration/ConfigurationsUtils.java      |  67 +++++++-
 .../org/apache/metron/common/dsl/Stellar.java   |   3 +
 .../resolver/BaseFunctionResolver.java          |   5 +-
 .../resolver/ClasspathFunctionResolver.java     | 170 ++++++++++++++-----
 .../resolver/SimpleFunctionResolver.java        |   2 +-
 .../common/stellar/shell/StellarExecutor.java   |  14 +-
 .../common/stellar/shell/StellarShell.java      |  16 +-
 .../metron/common/utils/VFSClassloaderUtil.java | 168 ++++++++++++++++++
 .../src/main/scripts/zk_load_configs.sh         |  11 +-
 .../classpath-resources/custom-1.0-SNAPSHOT.jar | Bin 0 -> 3570 bytes
 ...lasspathFunctionResolverIntegrationTest.java |  87 ++++++++++
 .../resolver/ClasspathFunctionResolverTest.java |  42 ++++-
 .../metron/common/stellar/StellarTest.java      |   9 +-
 metron-platform/metron-data-management/pom.xml  |   8 +
 .../TransformFilterExtractorDecorator.java      |   1 +
 metron-platform/metron-elasticsearch/pom.xml    |   8 +
 metron-platform/metron-enrichment/pom.xml       |   8 +
 .../enrichment/bolt/GenericEnrichmentBolt.java  |   1 +
 .../enrichment/bolt/ThreatIntelJoinBolt.java    |   1 +
 metron-platform/metron-parsers/pom.xml          |   8 +
 .../apache/metron/parsers/bolt/ParserBolt.java  |   1 +
 metron-platform/metron-pcap-backend/pom.xml     |   8 +
 metron-platform/metron-solr/pom.xml             |   8 +
 pom.xml                                         |   2 +
 37 files changed, 903 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/dependencies_with_url.csv
----------------------------------------------------------------------
diff --git a/dependencies_with_url.csv b/dependencies_with_url.csv
index 819ab84..21f0cb5 100644
--- a/dependencies_with_url.csv
+++ b/dependencies_with_url.csv
@@ -287,3 +287,4 @@ antlr:antlr:jar:2.7.7:compile,BSD 3-Clause License,http://www.antlr2.org
 com.h2database:h2:jar:1.4.192:compile,EPL 1.0,http://www.h2database.com/html/license.html
 de.jollyday:jollyday:jar:0.5.2:compile,ASLv2,http://jollyday.sourceforge.net/license.html
 org.threeten:threeten-extra:jar:1.0:compile,BSD,http://www.threeten.org/threeten-extra/license.html
+org.atteo.classindex:classindex:jar:3.3:compile,ASLv2,https://github.com/atteo/classindex

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileBuilder.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileBuilder.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileBuilder.java
index b444ba1..88d6e1b 100644
--- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileBuilder.java
+++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileBuilder.java
@@ -118,6 +118,7 @@ public class ProfileBuilder implements Serializable {
     Context context = new Context.Builder()
             .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
             .with(Context.Capabilities.GLOBAL_CONFIG, () -> global)
+            .with(Context.Capabilities.STELLAR_CONFIG, () -> global)
             .build();
     StellarFunctions.initialize(context);
     this.executor.setContext(context);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-analytics/metron-profiler/pom.xml
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/pom.xml b/metron-analytics/metron-profiler/pom.xml
index 3c8baef..d62f025 100644
--- a/metron-analytics/metron-profiler/pom.xml
+++ b/metron-analytics/metron-profiler/pom.xml
@@ -61,6 +61,17 @@
             </exclusions>
         </dependency>
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <version>${global_hadoop_version}</version>
+            <exclusions>
+                <exclusion>
+                    <artifactId>servlet-api</artifactId>
+                    <groupId>javax.servlet</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
             <groupId>org.apache.metron</groupId>
             <artifactId>metron-writer</artifactId>
             <version>${project.parent.version}</version>
@@ -363,6 +374,7 @@
                                 </excludes>
                             </artifactSet>
                             <transformers>
+                                <transformer implementation="org.atteo.classindex.ClassIndexTransformer"/>
                                 <transformer
                                   implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
                                      <resources>
@@ -387,6 +399,13 @@
                         </configuration>
                     </execution>
                 </executions>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.atteo.classindex</groupId>
+                        <artifactId>classindex-transformer</artifactId>
+                        <version>${global_classindex_version}</version>
+                    </dependency>
+                </dependencies>
             </plugin>
             <plugin>
                 <artifactId>maven-assembly-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/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 0fb1fd2..9ad145a 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
@@ -86,6 +86,7 @@ public class ProfileSplitterBolt extends ConfiguredProfilerBolt {
     Context context = new Context.Builder()
             .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
             .with(Context.Capabilities.GLOBAL_CONFIG, () -> getConfigurations().getGlobalConfig())
+            .with(Context.Capabilities.STELLAR_CONFIG, () -> getConfigurations().getGlobalConfig())
             .build();
     StellarFunctions.initialize(context);
     executor.setContext(context);

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/ZookeeperConfig.java
----------------------------------------------------------------------
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/ZookeeperConfig.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/ZookeeperConfig.java
index 5f58193..1f72afb 100644
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/ZookeeperConfig.java
+++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/ZookeeperConfig.java
@@ -23,6 +23,7 @@ import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.apache.metron.rest.MetronRestConstants;
 import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Configuration;
@@ -40,7 +41,8 @@ public class ZookeeperConfig {
     int sleepTime = Integer.parseInt(environment.getProperty(MetronRestConstants.CURATOR_SLEEP_TIME));
     int maxRetries = Integer.parseInt(environment.getProperty(MetronRestConstants.CURATOR_MAX_RETRIES));
     RetryPolicy retryPolicy = new ExponentialBackoffRetry(sleepTime, maxRetries);
-    return CuratorFrameworkFactory.newClient(environment.getProperty(MetronRestConstants.ZK_URL_SPRING_PROPERTY), retryPolicy);
+    CuratorFramework ret = CuratorFrameworkFactory.newClient(environment.getProperty(MetronRestConstants.ZK_URL_SPRING_PROPERTY), retryPolicy);
+    return ret;
   }
 
   @Bean(destroyMethod="close")

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/StellarServiceImpl.java
----------------------------------------------------------------------
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/StellarServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/StellarServiceImpl.java
index f5392be..321dc48 100644
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/StellarServiceImpl.java
+++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/StellarServiceImpl.java
@@ -17,9 +17,12 @@
  */
 package org.apache.metron.rest.service.impl;
 
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.apache.metron.common.dsl.Context;
 import org.apache.metron.common.dsl.ParseException;
 import org.apache.metron.common.dsl.StellarFunctionInfo;
+import org.apache.metron.common.dsl.StellarFunctions;
 import org.apache.metron.common.dsl.functions.resolver.SingletonFunctionResolver;
 import org.apache.metron.common.field.transformation.FieldTransformations;
 import org.apache.metron.common.stellar.StellarProcessor;
@@ -27,6 +30,7 @@ import org.apache.metron.rest.model.StellarFunctionDescription;
 import org.apache.metron.rest.model.SensorParserContext;
 import org.apache.metron.rest.service.StellarService;
 import org.json.simple.JSONObject;
+import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Service;
 
 import java.util.ArrayList;
@@ -38,55 +42,67 @@ import java.util.stream.Collectors;
 @Service
 public class StellarServiceImpl implements StellarService {
 
-    @Override
-    public Map<String, Boolean> validateRules(List<String> rules) {
-        Map<String, Boolean> results = new HashMap<>();
-        StellarProcessor stellarProcessor = new StellarProcessor();
-        for(String rule: rules) {
-            try {
-                boolean result = stellarProcessor.validate(rule, Context.EMPTY_CONTEXT());
-                results.put(rule, result);
-            } catch (ParseException e) {
-                results.put(rule, false);
-            }
-        }
-        return results;
-    }
+  private CuratorFramework client;
 
-    @Override
-    public Map<String, Object> applyTransformations(SensorParserContext sensorParserContext) {
-        JSONObject sampleJson = new JSONObject(sensorParserContext.getSampleData());
-        sensorParserContext.getSensorParserConfig().getFieldTransformations().forEach(fieldTransformer -> {
-                    fieldTransformer.transformAndUpdate(sampleJson, sensorParserContext.getSensorParserConfig().getParserConfig(), Context.EMPTY_CONTEXT());
-                }
-        );
-        return sampleJson;
+  @Autowired
+  public StellarServiceImpl(CuratorFramework client) {
+    this.client = client;
+    try {
+      ConfigurationsUtils.setupStellarStatically(this.client);
+    } catch (Exception e) {
+      throw new IllegalStateException("Unable to setup stellar statically: " + e.getMessage(), e);
     }
+  }
 
-    @Override
-    public FieldTransformations[] getTransformations() {
-        return FieldTransformations.values();
+  @Override
+  public Map<String, Boolean> validateRules(List<String> rules) {
+    Map<String, Boolean> results = new HashMap<>();
+    StellarProcessor stellarProcessor = new StellarProcessor();
+    for(String rule: rules) {
+      try {
+        boolean result = stellarProcessor.validate(rule, Context.EMPTY_CONTEXT());
+        results.put(rule, result);
+      } catch (ParseException e) {
+        results.put(rule, false);
+      }
     }
+    return results;
+  }
 
-    @Override
-    public List<StellarFunctionDescription> getStellarFunctions() {
-        List<StellarFunctionDescription> stellarFunctionDescriptions = new ArrayList<>();
-        Iterable<StellarFunctionInfo> stellarFunctionsInfo = SingletonFunctionResolver.getInstance().getFunctionInfo();
-        stellarFunctionsInfo.forEach(stellarFunctionInfo -> {
-            stellarFunctionDescriptions.add(new StellarFunctionDescription(
-                    stellarFunctionInfo.getName(),
-                    stellarFunctionInfo.getDescription(),
-                    stellarFunctionInfo.getParams(),
-                    stellarFunctionInfo.getReturns()));
-        });
-        return stellarFunctionDescriptions;
-    }
+  @Override
+  public Map<String, Object> applyTransformations(SensorParserContext sensorParserContext) {
+    JSONObject sampleJson = new JSONObject(sensorParserContext.getSampleData());
+    sensorParserContext.getSensorParserConfig().getFieldTransformations().forEach(fieldTransformer -> {
+              fieldTransformer.transformAndUpdate(sampleJson, sensorParserContext.getSensorParserConfig().getParserConfig(), Context.EMPTY_CONTEXT());
+            }
+    );
+    return sampleJson;
+  }
 
-    @Override
-    public List<StellarFunctionDescription> getSimpleStellarFunctions() {
-      List<StellarFunctionDescription> stellarFunctionDescriptions = getStellarFunctions();
-      return stellarFunctionDescriptions.stream().filter(stellarFunctionDescription ->
-              stellarFunctionDescription.getParams().length == 1).sorted((o1, o2) -> o1.getName().compareTo(o2.getName())).collect(Collectors.toList());
-    }
+  @Override
+  public FieldTransformations[] getTransformations() {
+    return FieldTransformations.values();
+  }
+
+  @Override
+  public List<StellarFunctionDescription> getStellarFunctions() {
+    List<StellarFunctionDescription> stellarFunctionDescriptions = new ArrayList<>();
+    Iterable<StellarFunctionInfo> stellarFunctionsInfo = StellarFunctions.FUNCTION_RESOLVER().getFunctionInfo();
+    stellarFunctionsInfo.forEach(stellarFunctionInfo -> {
+      stellarFunctionDescriptions.add(new StellarFunctionDescription(
+              stellarFunctionInfo.getName(),
+              stellarFunctionInfo.getDescription(),
+              stellarFunctionInfo.getParams(),
+              stellarFunctionInfo.getReturns()));
+    });
+    return stellarFunctionDescriptions;
+  }
+
+  @Override
+  public List<StellarFunctionDescription> getSimpleStellarFunctions() {
+    List<StellarFunctionDescription> stellarFunctionDescriptions = getStellarFunctions();
+    return stellarFunctionDescriptions.stream().filter(stellarFunctionDescription ->
+            stellarFunctionDescription.getParams().length == 1).sorted((o1, o2) -> o1.getName().compareTo(o2.getName())).collect(Collectors.toList());
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/StellarServiceImplTest.java
----------------------------------------------------------------------
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/StellarServiceImplTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/StellarServiceImplTest.java
index 3ddbf56..629dfc3 100644
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/StellarServiceImplTest.java
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/StellarServiceImplTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.metron.rest.service.impl;
 
+import org.apache.curator.framework.CuratorFramework;
 import org.apache.metron.common.configuration.FieldTransformer;
 import org.apache.metron.common.configuration.SensorParserConfig;
 import org.apache.metron.rest.model.SensorParserContext;
@@ -33,14 +34,17 @@ import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 public class StellarServiceImplTest {
 
   private StellarService stellarService;
+  CuratorFramework curatorFramework;
 
   @Before
   public void setUp() throws Exception {
-    stellarService = new StellarServiceImpl();
+    curatorFramework = mock(CuratorFramework.class);
+    stellarService = new StellarServiceImpl(curatorFramework);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/3rdPartyStellar.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/3rdPartyStellar.md b/metron-platform/metron-common/3rdPartyStellar.md
new file mode 100644
index 0000000..a107a72
--- /dev/null
+++ b/metron-platform/metron-common/3rdPartyStellar.md
@@ -0,0 +1,160 @@
+# Custom Stellar Functions
+
+Metron is fundamentally a programmable, extensible system
+and Stellar is the extension language.  We have some great Stellar functions
+available out of the box and we'll be adding more over time, but they may
+not quite scratch quite your particular itch.  
+
+Of course, we'd love to have your contribution inside of Metron if you think it
+general purpose enough, but not every function is general-purpose or it may rely
+on libraries those licenses aren't acceptable for an Apache project.  In that case, then you will
+be wondering how to add your custom function to a running instance of Metron.
+
+## Building Your Own Function
+
+Let's say that I need a function that returns the current time in milliseconds
+since the epoch.  I notice that there's nothing like that currently in Metron,
+so I embark on the adventure of adding it for my cluster.
+
+I will presume that you have an installed Metron into your local maven repo via `mvn install` .  In the future, when we publish to a maven repo,
+you will not need this.  I will depend on 0.3.1 for the
+purpose of this demonstration
+
+### Hack, Hack, Hack
+
+I like to use Maven, so we'll use that for this demonstration, but you can use whatever
+build system that you like.  Here's my favorite way to build a project with groupId `com.mycompany.stellar`
+and artifactId of `tempus`
+`mvn archetype:create -DgroupId=com.mycompany.stellar -DartifactId=tempus -DarchetypeArtifactId=maven-archetype-quickstart`
+
+First, we should depend on `metron-common` and we can do that by adjusting the `pom.xml` just created:
+```
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  
+  <groupId>com.mycompany.stellar</groupId>
+  <artifactId>tempus</artifactId>
+  <version>1.0-SNAPSHOT</version>
+  <packaging>jar</packaging>
+  
+  <name>Stellar Time Functions</name>
+  <url>http://mycompany.com</url>
+  
+  <properties>
+    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+  </properties>
+  
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.metron</groupId>
+      <artifactId>metron-common</artifactId>
+      <version>0.3.1</version>
+      <!-- NOTE: We will want to depend on the deployed common on the classpath. -->
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+       <groupId>junit</groupId>
+       <artifactId>junit</artifactId>
+       <version>3.8.1</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>
+```
+
+Let's add our implementation in `src/main/java/com/mycompany/stellar/TimeFunctions.java` with the following content:
+```
+package com.notmetron.stellar;
+    
+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 java.util.List;
+    
+public class TimeFunction {
+  @Stellar( name="NOW",
+            description = "Right now!",
+            params = {},
+            returns="Timestamp"
+          )
+  public static class Now implements StellarFunction {
+    
+    public Object apply(List<Object> list, Context context) throws ParseException {
+      return System.currentTimeMillis();
+    }
+    
+    public void initialize(Context context) { }
+    
+    public boolean isInitialized() {
+      return true;
+    }
+  }
+}
+```
+
+Now we can build the project via `mvn package` which will create a `target/tempus-1.0-SNAPSHOT.jar` file.
+
+## Install the Function
+
+Now that we have a jar with our custom function, we must make Metron aware of it.
+
+### Deploy the Jar
+
+First you need to place the jar in HDFS, if we have it on an access node, one way to do that is:
+* `hadoop fs -put tempus-1.0-SNAPSHOT.jar /apps/metron/stellar`
+This presumes that:
+* you've standardized on `/apps/metron/stellar` as the location for custom jars
+* you are running the command from an access node with the `hadoop` command installed
+* you are running from a user that has write access to `/apps/metron/stellar`
+
+### Set Global Config
+
+You may not need this if your Metron administrator already has this setup.
+
+With that dispensed with, we need to ensure that Metron knows to look at that location.
+We need to ensure that the `stellar.function.paths` property in the `global.json` is in place that makes Metron aware
+to look for Stellar functions in `/apps/metron/stellar` on HDFS.  
+
+This property looks like, the following for a vagrant install
+```
+{
+  "es.clustername": "metron",
+  "es.ip": "node1",
+  "es.port": "9300",
+  "es.date.format": "yyyy.MM.dd.HH",
+  "stellar.function.paths" : "hdfs://node1:8020/apps/metron/stellar/.*.jar",
+}
+```
+
+The `stellar.function.paths` property takes a comma separated list of URIs or URIs with regex expressions at the end.
+Also, note path is prefaced by the HDFS default name, which, if you do not know, can be found by executing,
+`hdfs getconf -confKey fs.default.name`, such as
+```
+[root@node1 ~]# hdfs getconf -confKey fs.default.name
+hdfs://node1:8020
+```
+### Use the Function
+
+Now that we have deployed the function, if we want to use it,
+any running topologies that use Stellar will need to be restarted.
+
+Beyond that, let's take a look at it in the REPL:
+```
+Stellar, Go!
+Please note that functions are loading lazily in the background and will be unavailable until loaded fully.
+{es.clustername=metron, es.ip=node1, es.port=9300, es.date.format=yyyy.MM.dd.HH, stellar.function.paths=hdfs://node1:8020/apps/metron/stellar/.*.jar, profiler.client.period.duration=1, profiler.client.period.duration.units=MINUTES}
+[Stellar]>>> # Get the help for NOW
+[Stellar]>>> ?NOW
+Functions loaded, you may refer to functions now...
+NOW
+Description: Right now!
+     
+Returns: Timestamp
+[Stellar]>>> # Try to run the NOW function, which we added:
+[Stellar]>>> NOW()
+1488400515655
+[Stellar]>>> # Looks like I got a timestamp, success!
+```

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/README.md b/metron-platform/metron-common/README.md
index 4ffb8f1..97ecd58 100644
--- a/metron-platform/metron-common/README.md
+++ b/metron-platform/metron-common/README.md
@@ -640,6 +640,8 @@ Shell-like operations are supported such as
     happens in the background.  Until that happens, autocomplete will not include function names. 
 * emacs or vi keybindings for edit mode
 
+Note: Stellar classpath configuration from the global config is honored here if the REPL knows about zookeeper.
+
 ### Getting Started
 
 ```
@@ -688,6 +690,7 @@ work on it via the REPL.
 *Optional*
 
 Attempts to connect to Zookeeper and read the Metron global configuration.  Stellar functions may require the global configuration to work properly.  If found, the global configuration values are printed to the console.
+If specified, then the classpath may be augmented by the paths specified in the stellar config in the global config.
 
 ```
 $ $METRON_HOME/bin/stellar -z node1:2181
@@ -793,7 +796,55 @@ This configuration is stored in zookeeper, but looks something like
 }
 ```
 
-## Validation Framework
+# Stellar Configuration
+
+Stellar can be configured in a variety of ways from the global config.
+In particular, there are three main configuration parameters around configuring Stellar:
+* `stellar.function.paths`
+* `stellar.function.resolver.includes`
+* `stellar.function.resolver.excludes`
+
+## `stellar.function.paths`
+
+If specified, Stellar will use a custom classloader which will wrap the
+context classloader and allow for the resolution of classes stored in jars
+not shipped with Metron and stored in a variety of mediums:
+* On HDFS
+* In tar.gz files
+* At http/s locations
+* At ftp locations
+
+This path is a comma separated list of 
+* URIs
+* URIs with a regex pattern ending it for matching within a directory
+
+```json
+{
+ ...
+  "stellar.function.paths" : "hdfs://node1:8020/apps/metron/stellar/metron-management-0.3.1.jar, hdfs://node1:8020/apps/metron/3rdparty/.*.jar"
+}
+```
+
+Please be aware that this classloader does not reload functions dynamically
+and the classpath specified here in the global config is read on topology start.
+  A change in classpath, to be picked up, would necessitate a topology restart
+at the moment
+
+## `stellar.function.resolver.{includes,excludes}`
+
+If specified, this defines one or more regular expressions applied to the classes implementing the Stellar function
+that specify what should be included when searching for Stellar functions.
+* `stellar.function.resolver.includes` defines the list of classes to include.
+* `stellar.function.resolver.excludes` defines the list of classes to exclude.
+
+```json
+{
+ ...
+  "stellar.function.resolver.includes" : "org.apache.metron.*,com.myorg.stellar.*"
+}
+```
+
+# Validation Framework
 
 Inside of the global configuration, there is a validation framework in
 place that enables the validation that messages coming from all parsers

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/pom.xml b/metron-platform/metron-common/pom.xml
index 5d4c2fd..d51762a 100644
--- a/metron-platform/metron-common/pom.xml
+++ b/metron-platform/metron-common/pom.xml
@@ -47,6 +47,18 @@
             <version>3.6.1</version>
         </dependency>
         <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>metron-integration-test</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-auth</artifactId>
+            <version>${global_hadoop_version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>commons-validator</groupId>
             <artifactId>commons-validator</artifactId>
             <version>1.5.1</version>
@@ -146,6 +158,26 @@
             <version>${global_curator_version}</version>
         </dependency>
         <dependency>
+            <groupId>org.atteo.classindex</groupId>
+            <artifactId>classindex</artifactId>
+            <version>${global_classindex_version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.accumulo</groupId>
+            <artifactId>accumulo-start</artifactId>
+            <version>${global_accumulo_version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-client</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-io</groupId>
+                    <artifactId>commons-io</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
             <groupId>com.google.guava</groupId>
             <artifactId>guava</artifactId>
             <version>${global_guava_version}</version>
@@ -276,6 +308,7 @@
                 </exclusion>
             </exclusions>
         </dependency>
+
         <dependency>
             <groupId>org.apache.curator</groupId>
             <artifactId>curator-test</artifactId>
@@ -330,6 +363,7 @@
             <artifactId>stream</artifactId>
             <version>2.9.5</version>
         </dependency>
+
     </dependencies>
 
     <reporting>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
index cef4aff..45463bf 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java
@@ -17,6 +17,7 @@
  */
 package org.apache.metron.common.bolt;
 
+import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.base.BaseRichBolt;
@@ -75,6 +76,10 @@ public abstract class ConfiguredBolt<CONFIG_T extends Configurations> extends Ba
       }
       client.start();
 
+      //this is temporary to ensure that any validation passes.
+      //The individual bolt will reinitialize stellar to dynamically pull from
+      //zookeeper.
+      ConfigurationsUtils.setupStellarStatically(client);
       if (cache == null) {
         cache = new TreeCache(client, Constants.ZOOKEEPER_TOPOLOGY_ROOT);
         TreeCacheListener listener = new TreeCacheListener() {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java
index dedf681..d5f0db4 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java
@@ -21,6 +21,8 @@ import org.apache.log4j.Logger;
 import org.apache.metron.common.configuration.ConfigurationType;
 import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.apache.metron.common.configuration.EnrichmentConfigurations;
+import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.StellarFunctions;
 
 import java.io.IOException;
 
@@ -40,6 +42,7 @@ public abstract class ConfiguredEnrichmentBolt extends ConfiguredBolt<Enrichment
   @Override
   public void loadConfig() {
     try {
+
       ConfigurationsUtils.updateEnrichmentConfigsFromZookeeper(getConfigurations(), client);
     } catch (Exception e) {
       LOG.warn("Unable to load configs from zookeeper, but the cache should load lazily...");

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
index e57ca2e..23e91c8 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java
@@ -61,7 +61,7 @@ public enum ConfigurationType implements Function<String, Object> {
       throw new RuntimeException("Unable to load " + s, e);
     }
   }),
-  PROFILER("profiler","profiler", s -> {
+  PROFILER("profiler",".", s -> {
     try {
       return JSONUtils.INSTANCE.load(s, ProfilerConfig.class);
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
index 2a51273..cb888c8 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java
@@ -18,12 +18,15 @@
 package org.apache.metron.common.configuration;
 
 import org.apache.commons.io.FilenameUtils;
+import org.apache.curator.CuratorZookeeperClient;
 import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.metron.common.Constants;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.StellarFunctions;
 import org.apache.metron.common.utils.JSONUtils;
 import org.apache.zookeeper.KeeperException;
 
@@ -35,6 +38,7 @@ import java.nio.file.Files;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 
 import static org.apache.metron.common.configuration.ConfigurationType.*;
 
@@ -201,7 +205,10 @@ public class ConfigurationsUtils {
   }
 
   public static byte[] readFromZookeeper(String path, CuratorFramework client) throws Exception {
-    return client.getData().forPath(path);
+    if(client != null && client.getData() != null && path != null) {
+      return client.getData().forPath(path);
+    }
+    return new byte[]{};
   }
 
   public static void uploadConfigsToZookeeper(String globalConfigPath,
@@ -229,8 +236,9 @@ public class ConfigurationsUtils {
 
     // global
     if (globalConfigPath != null) {
-      byte[] globalConfig = readGlobalConfigFromFile(globalConfigPath);
+      final byte[] globalConfig = readGlobalConfigFromFile(globalConfigPath);
       if (globalConfig.length > 0) {
+        setupStellarStatically(client, Optional.of(new String(globalConfig)));
         ConfigurationsUtils.writeGlobalConfigToZookeeper(readGlobalConfigFromFile(globalConfigPath), client);
       }
     }
@@ -261,13 +269,47 @@ public class ConfigurationsUtils {
 
     // profiler
     if (profilerConfigPath != null) {
-      byte[] globalConfig = readProfilerConfigFromFile(profilerConfigPath);
-      if (globalConfig.length > 0) {
-        ConfigurationsUtils.writeProfilerConfigToZookeeper(readProfilerConfigFromFile(profilerConfigPath), client);
+      byte[] profilerConfig = readProfilerConfigFromFile(profilerConfigPath);
+      if (profilerConfig.length > 0) {
+        ConfigurationsUtils.writeProfilerConfigToZookeeper(profilerConfig, client);
       }
     }
   }
 
+  public static void setupStellarStatically(CuratorFramework client) throws Exception {
+    byte[] ret = null;
+    try {
+      ret = readGlobalConfigBytesFromZookeeper(client);
+    }
+    catch(KeeperException.NoNodeException nne) {
+      //can't find the node
+    }
+    if(ret == null || ret.length == 0) {
+      setupStellarStatically(client, Optional.empty());
+    }
+    else {
+      setupStellarStatically(client, Optional.of(new String(ret)));
+    }
+  }
+
+  public static void setupStellarStatically(CuratorFramework client, Optional<String> globalConfig) {
+    /*
+      In order to validate stellar functions, the function resolver must be initialized.  Otherwise,
+      those utilities that require validation cannot validate the stellar expressions necessarily.
+    */
+    Context.Builder builder = new Context.Builder().with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
+            ;
+    if(globalConfig.isPresent()) {
+      builder = builder.with(Context.Capabilities.GLOBAL_CONFIG, () -> GLOBAL.deserialize(globalConfig.get()))
+              .with(Context.Capabilities.STELLAR_CONFIG, () -> GLOBAL.deserialize(globalConfig.get()));
+    }
+    else {
+      builder = builder.with(Context.Capabilities.STELLAR_CONFIG, () -> new HashMap<>());
+    }
+    Context stellarContext = builder.build();
+    StellarFunctions.FUNCTION_RESOLVER().initialize(stellarContext);
+  }
+
   public static byte[] readGlobalConfigFromFile(String rootPath) throws IOException {
     byte[] globalConfig = new byte[0];
     File configPath = new File(rootPath, GLOBAL.getName() + ".json");
@@ -323,8 +365,11 @@ public class ConfigurationsUtils {
     void visit(ConfigurationType configurationType, String name, String data);
   }
 
-  public static void visitConfigs(CuratorFramework client, ConfigurationVisitor callback) throws Exception {
-    visitConfigs(client, callback, GLOBAL);
+  public static void visitConfigs(CuratorFramework client, final ConfigurationVisitor callback) throws Exception {
+    visitConfigs(client, (type, name, data) -> {
+      setupStellarStatically(client, Optional.ofNullable(data));
+      callback.visit(type, name, data);
+    }, GLOBAL);
     visitConfigs(client, callback, PARSER);
     visitConfigs(client, callback, INDEXING);
     visitConfigs(client, callback, ENRICHMENT);
@@ -338,8 +383,12 @@ public class ConfigurationsUtils {
       if (configType.equals(GLOBAL)) {
         byte[] globalConfigData = client.getData().forPath(configType.getZookeeperRoot());
         callback.visit(configType, "global", new String(globalConfigData));
-
-      } else if (configType.equals(PARSER) || configType.equals(ENRICHMENT) || configType.equals(PROFILER) || configType.equals(INDEXING)) {
+      }
+      else if(configType.equals(PROFILER)) {
+        byte[] profilerConfigData = client.getData().forPath(configType.getZookeeperRoot());
+        callback.visit(configType, "profiler", new String(profilerConfigData));
+      }
+      else if (configType.equals(PARSER) || configType.equals(ENRICHMENT) || configType.equals(INDEXING)) {
         List<String> children = client.getChildren().forPath(configType.getZookeeperRoot());
         for (String child : children) {
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/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
index a4e895b..d55d36b 100644
--- 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
@@ -17,12 +17,15 @@
  */
 package org.apache.metron.common.dsl;
 
+import org.atteo.classindex.IndexAnnotated;
+
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
 @Retention(RetentionPolicy.RUNTIME)
+@IndexAnnotated
 public @interface Stellar {
   String namespace() default "";
   String name();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/BaseFunctionResolver.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/BaseFunctionResolver.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/BaseFunctionResolver.java
index ccb0ac2..ba4f442 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/BaseFunctionResolver.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/BaseFunctionResolver.java
@@ -67,7 +67,7 @@ public abstract class BaseFunctionResolver implements FunctionResolver, Serializ
    * Returns a set of classes that should undergo further interrogation for resolution
    * (aka discovery) of Stellar functions.
    */
-  protected abstract Set<Class<? extends StellarFunction>> resolvables();
+  public abstract Set<Class<? extends StellarFunction>> resolvables();
 
   /**
    * Provides metadata about each Stellar function that is resolvable.
@@ -119,11 +119,10 @@ public abstract class BaseFunctionResolver implements FunctionResolver, Serializ
     for(Class<? extends StellarFunction> clazz : resolvables()) {
       StellarFunctionInfo fn = resolveFunction(clazz);
       if(fn != null) {
-
         // check for duplicate function names
         StellarFunctionInfo fnSameName = functions.get(fn.getName());
         if (fnSameName != null && ObjectUtils.notEqual(fnSameName, fn)) {
-          throw new IllegalStateException(format(
+          LOG.warn(format(
                   "Namespace conflict: duplicate function names; `%s` implemented by [%s, %s]",
                   fn.getName(), fnSameName.getFunction(), fn.getFunction()));
         }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolver.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolver.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolver.java
index 95cabbb..4589b61 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolver.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolver.java
@@ -18,23 +18,25 @@
 
 package org.apache.metron.common.dsl.functions.resolver;
 
-import org.apache.commons.lang.StringUtils;
+import org.apache.commons.vfs2.FileObject;
+import org.apache.commons.vfs2.FileSystemException;
+import org.apache.commons.vfs2.impl.VFSClassLoader;
 import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.dsl.StellarFunction;
-import org.reflections.Reflections;
+import org.apache.metron.common.utils.VFSClassloaderUtil;
+import org.apache.metron.common.utils.ConversionUtils;
+import org.atteo.classindex.ClassIndex;
 import org.reflections.util.ClasspathHelper;
-import org.reflections.util.ConfigurationBuilder;
 import org.reflections.util.FilterBuilder;
 
 import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
+import java.util.*;
 
 import static org.apache.metron.common.dsl.Context.Capabilities.STELLAR_CONFIG;
+import static org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver.Config.STELLAR_SEARCH_EXCLUDES_KEY;
+import static org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver.Config.STELLAR_SEARCH_INCLUDES_KEY;
+import static org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver.Config.STELLAR_VFS_PATHS;
 
 /**
  * Performs function resolution for Stellar by searching the classpath.
@@ -65,24 +67,53 @@ import static org.apache.metron.common.dsl.Context.Capabilities.STELLAR_CONFIG;
  *
  */
 public class ClasspathFunctionResolver extends BaseFunctionResolver {
+  public enum Config {
+    /**
+     * The set of paths.  These paths are comma separated URLs with optional regex patterns at the end.
+     * e.g. hdfs://node1:8020/apps/metron/stellar/.*.jar,hdfs://node1:8020/apps/metron/my_org/.*.jar
+     * would signify all the jars under /apps/metron/stellar and /apps/metron/my_org in HDFS.
+     */
+    STELLAR_VFS_PATHS("stellar.function.paths", ""),
+    /**
+     * The key for a global property that defines one or more regular expressions
+     * that specify what should be included when searching for Stellar functions.
+     */
+    STELLAR_SEARCH_INCLUDES_KEY("stellar.function.resolver.includes", ""),
+    /**
+     * The key for a global property that defines one or more regular expressions
+     * that specify what should be excluded when searching for Stellar functions.
+     */
+    STELLAR_SEARCH_EXCLUDES_KEY("stellar.function.resolver.excludes", ""),
 
-  /**
-   * The key for a global property that defines one or more regular expressions
-   * that specify what should be included when searching for Stellar functions.
-   */
-  public static final String STELLAR_SEARCH_INCLUDES_KEY = "stellar.function.resolver.includes";
 
-  /**
-   * The key for a global property that defines one or more regular expressions
-   * that specify what should be excluded when searching for Stellar functions.
-   */
-  public static final String STELLAR_SEARCH_EXCLUDES_KEY = "stellar.function.resolver.excludes";
+    ;
+    String param;
+    Object defaultValue;
+    Config(String param, String defaultValue) {
+      this.param = param;
+      this.defaultValue = defaultValue;
+    }
+
+    public String param() {
+      return param;
+    }
+
+    public Object get(Map<String, Object> config) {
+      return config.getOrDefault(param, defaultValue);
+    }
+
+    public <T> T get(Map<String, Object> config, Class<T> clazz) {
+      return ConversionUtils.convert(get(config), clazz);
+    }
+  }
+
 
   /**
    * The includes and excludes can include a list of multiple includes or excludes that
    * are delimited by these values.
    */
-  protected static final String STELLAR_SEARCH_DELIMS = "[,:]";
+  private static final String STELLAR_SEARCH_DELIMS = "[,:]";
+
 
   /**
    * Regular expressions defining packages that should be included in the Stellar function resolution
@@ -96,9 +127,24 @@ public class ClasspathFunctionResolver extends BaseFunctionResolver {
    */
   private List<String> excludes;
 
+  /**
+   * Classloaders to try to load from
+   */
+  private List<ClassLoader> classLoaders;
+
   public ClasspathFunctionResolver() {
     this.includes = new ArrayList<>();
     this.excludes = new ArrayList<>();
+    this.classLoaders = new ArrayList<>();
+  }
+
+  /**
+   * Use one or more classloaders
+   * @param classloaders
+   */
+  public void classLoaders(ClassLoader... classloaders) {
+    classLoaders.clear();
+    Arrays.stream(classloaders).forEach(c -> classLoaders.add(c));
   }
 
   /**
@@ -131,19 +177,41 @@ public class ClasspathFunctionResolver extends BaseFunctionResolver {
       Optional<Object> optional = context.getCapability(STELLAR_CONFIG, false);
       if (optional.isPresent()) {
         Map<String, Object> stellarConfig = (Map<String, Object>) optional.get();
-
-        // handle any includes
-        String includes = (String) stellarConfig.getOrDefault(STELLAR_SEARCH_INCLUDES_KEY, "");
-        if(StringUtils.isNotBlank(includes)) {
-          include(includes.split(STELLAR_SEARCH_DELIMS));
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Setting up classloader using the following config: " + stellarConfig);
         }
 
-        // handle any excludes
-        String excludes = (String) stellarConfig.getOrDefault(STELLAR_SEARCH_EXCLUDES_KEY, "");
-        if(StringUtils.isNotBlank(excludes)) {
-          exclude(excludes.split(STELLAR_SEARCH_DELIMS));
+        include(STELLAR_SEARCH_INCLUDES_KEY.get(stellarConfig, String.class).split(STELLAR_SEARCH_DELIMS));
+        exclude(STELLAR_SEARCH_EXCLUDES_KEY.get(stellarConfig, String.class).split(STELLAR_SEARCH_DELIMS));
+        Optional<ClassLoader> vfsLoader = null;
+        try {
+          vfsLoader = VFSClassloaderUtil.configureClassloader(STELLAR_VFS_PATHS.get(stellarConfig, String.class));
+          if(vfsLoader.isPresent()) {
+            LOG.debug("CLASSLOADER LOADED WITH: " + STELLAR_VFS_PATHS.get(stellarConfig, String.class));
+            if(LOG.isDebugEnabled()) {
+              for (FileObject fo : ((VFSClassLoader) vfsLoader.get()).getFileObjects()) {
+                LOG.error(fo.getURL() + " - " + fo.exists());
+              }
+            }
+            classLoaders(vfsLoader.get());
+          }
+        } catch (FileSystemException e) {
+          LOG.error("Unable to process filesystem: " + e.getMessage(), e);
         }
       }
+      else {
+        LOG.info("No stellar config set; I'm reverting to the context classpath with no restrictions.");
+        if(LOG.isDebugEnabled()) {
+          try {
+            throw new IllegalStateException("No config set, stacktrace follows.");
+          } catch (IllegalStateException ise) {
+            LOG.error(ise.getMessage(), ise);
+          }
+        }
+      }
+    }
+    else {
+      throw new IllegalStateException("CONTEXT IS NULL!");
     }
   }
 
@@ -152,29 +220,37 @@ public class ClasspathFunctionResolver extends BaseFunctionResolver {
    * (aka discovery) of Stellar functions.
    */
   @Override
-  protected Set<Class<? extends StellarFunction>> resolvables() {
+  public Set<Class<? extends StellarFunction>> resolvables() {
 
-    ClassLoader classLoader = getClass().getClassLoader();
-    Collection<URL> searchPath = effectiveClassPathUrls(classLoader);
+    ClassLoader[] cls = null;
+    if(this.classLoaders.size() == 0) {
+      LOG.warn("Using System classloader");
+      cls = new ClassLoader[] { getClass().getClassLoader() };
+    }
+    else {
+      cls = new ClassLoader[this.classLoaders.size()];
+      for (int i = 0; i < this.classLoaders.size(); ++i) {
+        cls[i] = this.classLoaders.get(i);
+      }
+    }
 
     FilterBuilder filterBuilder = new FilterBuilder();
     excludes.forEach(excl -> filterBuilder.exclude(excl));
     includes.forEach(incl -> filterBuilder.include(incl));
-
-    Reflections reflections = new Reflections(
-            new ConfigurationBuilder()
-                    .setUrls(searchPath)
-                    .filterInputsBy(filterBuilder));
-    return reflections.getSubTypesOf(StellarFunction.class);
+    Set<String> classes = new HashSet<>();
+    Set<Class<? extends StellarFunction>> ret = new HashSet<>();
+    for(ClassLoader cl : cls) {
+      for(Class<?> c : ClassIndex.getAnnotated(Stellar.class, cl)) {
+        if(StellarFunction.class.isAssignableFrom(c) && filterBuilder.apply(c.getCanonicalName())) {
+          String className = c.getName();
+          if(!classes.contains(className)) {
+            ret.add((Class<? extends StellarFunction>) c);
+            classes.add(className);
+          }
+        }
+      }
+    }
+    return ret;
   }
 
-  /**
-   * 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
-   */
-  public static Collection<URL> effectiveClassPathUrls(ClassLoader... classLoaders) {
-    return ClasspathHelper.forManifest(ClasspathHelper.forClassLoader(classLoaders));
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/SimpleFunctionResolver.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/SimpleFunctionResolver.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/SimpleFunctionResolver.java
index a6ec43a..027574f 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/SimpleFunctionResolver.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/resolver/SimpleFunctionResolver.java
@@ -44,7 +44,7 @@ public class SimpleFunctionResolver extends BaseFunctionResolver {
   Set<Class<? extends StellarFunction>> classesToResolve = new HashSet<>();
 
   @Override
-  protected Set<Class<? extends StellarFunction>> resolvables() {
+  public Set<Class<? extends StellarFunction>> resolvables() {
     return classesToResolve;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/shell/StellarExecutor.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/shell/StellarExecutor.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/shell/StellarExecutor.java
index 7d78b70..2a13a0b 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/shell/StellarExecutor.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/shell/StellarExecutor.java
@@ -257,12 +257,24 @@ public class StellarExecutor {
 
       contextBuilder
               .with(GLOBAL_CONFIG, () -> global)
-              .with(ZOOKEEPER_CLIENT, () -> client.get());
+              .with(ZOOKEEPER_CLIENT, () -> client.get())
+              .with(STELLAR_CONFIG, () -> getStellarConfig(global, properties));
     }
 
     return contextBuilder.build();
   }
 
+  private Map<String, Object> getStellarConfig(Map<String, Object> globalConfig, Properties props) {
+    Map<String, Object> ret = new HashMap<>();
+    ret.putAll(globalConfig);
+    if(props != null) {
+      for (Map.Entry<Object, Object> kv : props.entrySet()) {
+        ret.put(kv.getKey().toString(), kv.getValue());
+      }
+    }
+    return ret;
+  }
+
   /**
    * Executes the Stellar expression and returns the result.
    * @param expression The Stellar expression to execute.

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/shell/StellarShell.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/shell/StellarShell.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/shell/StellarShell.java
index 9b817f1..4e24212 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/shell/StellarShell.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/shell/StellarShell.java
@@ -57,6 +57,8 @@ import java.util.Properties;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
+import static org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver.Config.STELLAR_SEARCH_INCLUDES_KEY;
+
 /**
  * A REPL environment for Stellar.
  *
@@ -222,11 +224,6 @@ public class StellarShell extends AeshConsoleCallback implements Completion {
       }
     }
 
-    // if still no properties, use the default set
-    if(properties.size() == 0) {
-      addDefaultStellarProperties(properties);
-    }
-
     return properties;
   }
 
@@ -245,15 +242,6 @@ public class StellarShell extends AeshConsoleCallback implements Completion {
   }
 
   /**
-   * Adds default Stellar properties.  Only used if no properties file can be found.
-   */
-  private void addDefaultStellarProperties(Properties props) {
-
-    // only functions defined by metron are resolved, by default
-    props.put(ClasspathFunctionResolver.STELLAR_SEARCH_INCLUDES_KEY, "org.apache.metron.*");
-  }
-
-  /**
    * Handles user interaction when executing a Stellar expression.
    * @param expression The expression to execute.
    */

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/VFSClassloaderUtil.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/VFSClassloaderUtil.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/VFSClassloaderUtil.java
new file mode 100644
index 0000000..1a02fef
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/VFSClassloaderUtil.java
@@ -0,0 +1,168 @@
+/**
+ * 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.utils;
+
+import org.apache.accumulo.start.classloader.vfs.UniqueFileReplicator;
+import org.apache.commons.vfs2.*;
+import org.apache.commons.vfs2.cache.SoftRefFilesCache;
+import org.apache.commons.vfs2.impl.DefaultFileSystemManager;
+import org.apache.commons.vfs2.impl.FileContentInfoFilenameFactory;
+import org.apache.commons.vfs2.impl.VFSClassLoader;
+import org.apache.commons.vfs2.provider.hdfs.HdfsFileProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.*;
+
+public class VFSClassloaderUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(VFSClassloaderUtil.class);
+
+  /**
+   * Create a FileSystem manager suitable for our purposes.
+   * This manager supports files of the following types:
+   * * res - resource files
+   * * jar
+   * * tar
+   * * bz2
+   * * tgz
+   * * zip
+   * * HDFS
+   * * FTP
+   * * HTTP/S
+   * * file
+   * @return
+   * @throws FileSystemException
+   */
+  public static FileSystemManager generateVfs() throws FileSystemException {
+    DefaultFileSystemManager vfs = new DefaultFileSystemManager();
+    vfs.addProvider("res", new org.apache.commons.vfs2.provider.res.ResourceFileProvider());
+    vfs.addProvider("zip", new org.apache.commons.vfs2.provider.zip.ZipFileProvider());
+    vfs.addProvider("gz", new org.apache.commons.vfs2.provider.gzip.GzipFileProvider());
+    vfs.addProvider("ram", new org.apache.commons.vfs2.provider.ram.RamFileProvider());
+    vfs.addProvider("file", new org.apache.commons.vfs2.provider.local.DefaultLocalFileProvider());
+    vfs.addProvider("jar", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
+    vfs.addProvider("http", new org.apache.commons.vfs2.provider.http.HttpFileProvider());
+    vfs.addProvider("https", new org.apache.commons.vfs2.provider.https.HttpsFileProvider());
+    vfs.addProvider("ftp", new org.apache.commons.vfs2.provider.ftp.FtpFileProvider());
+    vfs.addProvider("ftps", new org.apache.commons.vfs2.provider.ftps.FtpsFileProvider());
+    vfs.addProvider("war", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
+    vfs.addProvider("par", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
+    vfs.addProvider("ear", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
+    vfs.addProvider("sar", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
+    vfs.addProvider("ejb3", new org.apache.commons.vfs2.provider.jar.JarFileProvider());
+    vfs.addProvider("tmp", new org.apache.commons.vfs2.provider.temp.TemporaryFileProvider());
+    vfs.addProvider("tar", new org.apache.commons.vfs2.provider.tar.TarFileProvider());
+    vfs.addProvider("tbz2", new org.apache.commons.vfs2.provider.tar.TarFileProvider());
+    vfs.addProvider("tgz", new org.apache.commons.vfs2.provider.tar.TarFileProvider());
+    vfs.addProvider("bz2", new org.apache.commons.vfs2.provider.bzip2.Bzip2FileProvider());
+    vfs.addProvider("hdfs", new HdfsFileProvider());
+    vfs.addExtensionMap("jar", "jar");
+    vfs.addExtensionMap("zip", "zip");
+    vfs.addExtensionMap("gz", "gz");
+    vfs.addExtensionMap("tar", "tar");
+    vfs.addExtensionMap("tbz2", "tar");
+    vfs.addExtensionMap("tgz", "tar");
+    vfs.addExtensionMap("bz2", "bz2");
+    vfs.addMimeTypeMap("application/x-tar", "tar");
+    vfs.addMimeTypeMap("application/x-gzip", "gz");
+    vfs.addMimeTypeMap("application/zip", "zip");
+    vfs.setFileContentInfoFactory(new FileContentInfoFilenameFactory());
+    vfs.setFilesCache(new SoftRefFilesCache());
+    vfs.setReplicator(new UniqueFileReplicator(new File(System.getProperty("java.io.tmpdir"))));
+    vfs.setCacheStrategy(CacheStrategy.ON_RESOLVE);
+    vfs.init();
+    return vfs;
+  }
+
+  /**
+   * Create a classloader backed by a virtual filesystem which can handle the following URI types:
+   * * res - resource files
+   * * jar
+   * * tar
+   * * bz2
+   * * tgz
+   * * zip
+   * * HDFS
+   * * FTP
+   * * HTTP/S
+   * * file
+   * @param paths A set of comma separated paths.  The paths are URIs or URIs with a regex pattern at the end.
+   * @return A classloader object if it can create it
+   * @throws FileSystemException
+   */
+  public static Optional<ClassLoader> configureClassloader(String paths) throws FileSystemException {
+    if(paths.trim().isEmpty()) {
+      return Optional.empty();
+    }
+    FileSystemManager vfs = generateVfs();
+    FileObject[] objects = resolve(vfs, paths);
+    if(objects == null || objects.length == 0) {
+      return Optional.empty();
+    }
+    return Optional.of(new VFSClassLoader(objects, vfs, vfs.getClass().getClassLoader()));
+  }
+
+  /**
+   * Resolve a set of URIs into FileObject objects.
+   * This is not recursive. The URIs can refer directly to a file or directory or an optional regex at the end.
+   * (NOTE: This is NOT a glob).
+   * @param vfs The file system manager to use to resolve URIs
+   * @param uris comma separated URIs and URI + globs
+   * @return
+   * @throws FileSystemException
+   */
+  static FileObject[] resolve(FileSystemManager vfs, String uris) throws FileSystemException {
+    if (uris == null) {
+      return new FileObject[0];
+    }
+
+    ArrayList<FileObject> classpath = new ArrayList<>();
+    for (String path : uris.split(",")) {
+      path = path.trim();
+      if (path.equals("")) {
+        continue;
+      }
+      FileObject fo = vfs.resolveFile(path);
+      switch (fo.getType()) {
+        case FILE:
+        case FOLDER:
+          classpath.add(fo);
+          break;
+        case IMAGINARY:
+          // assume its a pattern
+          String pattern = fo.getName().getBaseName();
+          if (fo.getParent() != null && fo.getParent().getType() == FileType.FOLDER) {
+            FileObject[] children = fo.getParent().getChildren();
+            for (FileObject child : children) {
+              if (child.getType() == FileType.FILE && child.getName().getBaseName().matches(pattern)) {
+                classpath.add(child);
+              }
+            }
+          } else {
+            LOG.warn("ignoring classpath entry " + fo);
+          }
+          break;
+        default:
+          LOG.warn("ignoring classpath entry " + fo);
+          break;
+      }
+    }
+    return classpath.toArray(new FileObject[classpath.size()]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/main/scripts/zk_load_configs.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/scripts/zk_load_configs.sh b/metron-platform/metron-common/src/main/scripts/zk_load_configs.sh
index e28ca23..39e5e3e 100755
--- a/metron-platform/metron-common/src/main/scripts/zk_load_configs.sh
+++ b/metron-platform/metron-common/src/main/scripts/zk_load_configs.sh
@@ -28,7 +28,10 @@ elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
 fi
 export METRON_VERSION=${project.version}
 export METRON_HOME=/usr/metron/$METRON_VERSION
-export PARSERS_JAR=${project.artifactId}-$METRON_VERSION.jar
-export ZK_HOME=${ZK_HOME:-/usr/hdp/current/zookeeper-client}
-export ZK_CLIENT_JARS=${ZK_CLIENT_JARS:-$ZK_HOME/lib}
-java -cp $METRON_HOME/lib/$PARSERS_JAR:$ZK_CLIENT_JARS/* org.apache.metron.common.cli.ConfigurationManager "$@"
+export CLASSNAME="org.apache.metron.common.cli.ConfigurationManager"
+export JAR=metron-parsers-$METRON_VERSION-uber.jar
+export HBASE_HOME=${HBASE_HOME:-/usr/hdp/current/hbase-client}
+
+CP=$METRON_HOME/lib/$JAR:${HBASE_HOME}/lib/hbase-server.jar:`${HBASE_HOME}/bin/hbase classpath`
+java -cp $CP $CLASSNAME "$@"
+

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/test/classpath-resources/custom-1.0-SNAPSHOT.jar
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/classpath-resources/custom-1.0-SNAPSHOT.jar b/metron-platform/metron-common/src/test/classpath-resources/custom-1.0-SNAPSHOT.jar
new file mode 100644
index 0000000..7007eda
Binary files /dev/null and b/metron-platform/metron-common/src/test/classpath-resources/custom-1.0-SNAPSHOT.jar differ

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/test/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolverIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolverIntegrationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolverIntegrationTest.java
new file mode 100644
index 0000000..ac9ea7e
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolverIntegrationTest.java
@@ -0,0 +1,87 @@
+/**
+ * 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.functions.resolver;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver;
+import org.apache.metron.integration.components.MRComponent;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Properties;
+
+import static org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver.Config.STELLAR_VFS_PATHS;
+
+public class ClasspathFunctionResolverIntegrationTest {
+  static MRComponent component;
+  static Configuration configuration;
+  @BeforeClass
+  public static void setup() {
+    component = new MRComponent().withBasePath("target");
+    component.start();
+    configuration = component.getConfiguration();
+
+    try {
+      FileSystem fs = FileSystem.newInstance(configuration);
+      fs.mkdirs(new Path("/classpath-resources"));
+      fs.copyFromLocalFile(new Path("src/test/classpath-resources/custom-1.0-SNAPSHOT.jar"), new Path("/classpath-resources"));
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to start cluster", e);
+    }
+  }
+
+  @AfterClass
+  public static void teardown() {
+    component.stop();
+  }
+
+
+  public static ClasspathFunctionResolver create(Properties config) {
+    ClasspathFunctionResolver resolver = new ClasspathFunctionResolver();
+
+    Context context = new Context.Builder()
+            .with(Context.Capabilities.STELLAR_CONFIG, () -> config)
+            .build();
+    resolver.initialize(context);
+
+    return resolver;
+  }
+
+  @Test
+  public void test() throws Exception {
+    Properties config = new Properties();
+    config.put(STELLAR_VFS_PATHS.param(), configuration.get("fs.defaultFS") + "/classpath-resources/.*.jar");
+    ClasspathFunctionResolver resolver = create(config);
+    HashSet<String> functions = new HashSet<>(Lists.newArrayList(resolver.getFunctions()));
+    Assert.assertTrue(functions.contains("NOW"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-common/src/test/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolverTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolverTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolverTest.java
index 15a1e8e..d460edf 100644
--- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolverTest.java
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/dsl/functions/resolver/ClasspathFunctionResolverTest.java
@@ -19,13 +19,18 @@
 package org.apache.metron.common.dsl.functions.resolver;
 
 import com.google.common.collect.Lists;
+import org.apache.commons.vfs2.*;
 import org.apache.metron.common.dsl.Context;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.util.List;
-import java.util.Properties;
+import java.io.File;
+import java.util.*;
+
+import static org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver.Config.STELLAR_SEARCH_EXCLUDES_KEY;
+import static org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver.Config.STELLAR_SEARCH_INCLUDES_KEY;
+import static org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver.Config.STELLAR_VFS_PATHS;
 
 public class ClasspathFunctionResolverTest {
 
@@ -36,9 +41,6 @@ public class ClasspathFunctionResolverTest {
 
     // search the entire classpath for functions - provides a baseline to test against
     Properties config = new Properties();
-    config.put(ClasspathFunctionResolver.STELLAR_SEARCH_INCLUDES_KEY, "");
-    config.put(ClasspathFunctionResolver.STELLAR_SEARCH_EXCLUDES_KEY, "");
-
     // use a permissive regex that should not filter anything
     ClasspathFunctionResolver resolver = create(config);
 
@@ -65,7 +67,7 @@ public class ClasspathFunctionResolverTest {
 
     // setup - include all `org.apache.metron.*` functions
     Properties config = new Properties();
-    config.put(ClasspathFunctionResolver.STELLAR_SEARCH_INCLUDES_KEY, "org.apache.metron.*");
+    config.put(STELLAR_SEARCH_INCLUDES_KEY.param(), "org.apache.metron.*");
 
     // execute
     ClasspathFunctionResolver resolver = create(config);
@@ -80,7 +82,7 @@ public class ClasspathFunctionResolverTest {
 
     // setup - include all of the common and management functions, which is most of them
     Properties config = new Properties();
-    config.put(ClasspathFunctionResolver.STELLAR_SEARCH_INCLUDES_KEY, "org.apache.metron.common.*, org.apache.metron.management.*");
+    config.put(STELLAR_SEARCH_INCLUDES_KEY.param(), "org.apache.metron.common.*, org.apache.metron.management.*");
 
     // execute
     ClasspathFunctionResolver resolver = create(config);
@@ -96,7 +98,7 @@ public class ClasspathFunctionResolverTest {
 
     // setup - exclude all `org.apache.metron.*` functions
     Properties config = new Properties();
-    config.put(ClasspathFunctionResolver.STELLAR_SEARCH_EXCLUDES_KEY, "org.apache.metron.*");
+    config.put(STELLAR_SEARCH_EXCLUDES_KEY.param(), "org.apache.metron.*");
 
     // use a permissive regex that should not filter anything
     ClasspathFunctionResolver resolver = create(config);
@@ -106,4 +108,28 @@ public class ClasspathFunctionResolverTest {
     Assert.assertEquals(0, actual.size());
   }
 
+  @Test
+  public void testExternalLocal() throws FileSystemException, ClassNotFoundException {
+    File jar = new File("src/test/classpath-resources");
+    Assert.assertTrue(jar.exists());
+    Properties config = new Properties();
+    config.put(STELLAR_VFS_PATHS.param(), jar.toURI() + "/.*.jar");
+
+    ClasspathFunctionResolver resolver = create(config);
+    HashSet<String> functions = new HashSet<>(Lists.newArrayList(resolver.getFunctions()));
+    Assert.assertTrue(functions.contains("NOW"));
+  }
+
+
+  @Test
+  public void testExternalHDFS() throws FileSystemException, ClassNotFoundException {
+    File jar = new File("src/test/classpath-resources");
+    Assert.assertTrue(jar.exists());
+    Properties config = new Properties();
+    config.put(STELLAR_VFS_PATHS.param(), jar.toURI() + "/.*.jar");
+
+    ClasspathFunctionResolver resolver = create(config);
+    HashSet<String> functions = new HashSet<>(Lists.newArrayList(resolver.getFunctions()));
+    Assert.assertTrue(functions.contains("NOW"));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/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 6ff3d8a..681bb3d 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
@@ -24,6 +24,8 @@ import org.apache.commons.lang3.StringUtils;
 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.dsl.StellarFunctions;
+import org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -38,7 +40,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 
-import static org.apache.metron.common.dsl.functions.resolver.ClasspathFunctionResolver.effectiveClassPathUrls;
 import static org.apache.metron.common.utils.StellarProcessorUtils.run;
 import static org.apache.metron.common.utils.StellarProcessorUtils.runPredicate;
 
@@ -48,9 +49,10 @@ 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)) {
+    int numFound = 0;
+    for (Class<?> clazz : new ClasspathFunctionResolver().resolvables()) {
       if (clazz.isAnnotationPresent(Stellar.class)) {
+        numFound++;
         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()));
@@ -59,6 +61,7 @@ public class StellarTest {
         Assert.assertFalse("Must specify a returns annotation for " + clazz.getName(), StringUtils.isEmpty(annotation.returns()));
       }
     }
+    Assert.assertTrue(numFound > 0);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-data-management/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/pom.xml b/metron-platform/metron-data-management/pom.xml
index 3962e98..211f973 100644
--- a/metron-platform/metron-data-management/pom.xml
+++ b/metron-platform/metron-data-management/pom.xml
@@ -396,6 +396,7 @@
                                 </relocation>
                              </relocations>
                              <transformers>
+                                 <transformer implementation="org.atteo.classindex.ClassIndexTransformer"/>
                                 <transformer
                                   implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
                                      <resources>
@@ -427,6 +428,13 @@
                         </configuration>
                     </execution>
                 </executions>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.atteo.classindex</groupId>
+                        <artifactId>classindex-transformer</artifactId>
+                        <version>${global_classindex_version}</version>
+                    </dependency>
+                </dependencies>
             </plugin>
             <plugin>
                 <artifactId>maven-assembly-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/extractor/TransformFilterExtractorDecorator.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/extractor/TransformFilterExtractorDecorator.java b/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/extractor/TransformFilterExtractorDecorator.java
index a1448d9..a99db87 100644
--- a/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/extractor/TransformFilterExtractorDecorator.java
+++ b/metron-platform/metron-data-management/src/main/java/org/apache/metron/dataloads/extractor/TransformFilterExtractorDecorator.java
@@ -172,6 +172,7 @@ public class TransformFilterExtractorDecorator extends ExtractorDecorator {
       builder.with(Context.Capabilities.ZOOKEEPER_CLIENT, zkClient::get);
     }
     builder.with(Context.Capabilities.GLOBAL_CONFIG, () -> globalConfig);
+    builder.with(Context.Capabilities.STELLAR_CONFIG, () -> globalConfig);
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/pom.xml b/metron-platform/metron-elasticsearch/pom.xml
index 11cbd66..eacbbda 100644
--- a/metron-platform/metron-elasticsearch/pom.xml
+++ b/metron-platform/metron-elasticsearch/pom.xml
@@ -305,6 +305,7 @@
                                 </excludes>
                             </artifactSet>
                             <transformers>
+                                <transformer implementation="org.atteo.classindex.ClassIndexTransformer"/>
                                 <transformer
                                   implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
                                      <resources>
@@ -329,6 +330,13 @@
                         </configuration>
                     </execution>
                 </executions>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.atteo.classindex</groupId>
+                        <artifactId>classindex-transformer</artifactId>
+                        <version>${global_classindex_version}</version>
+                    </dependency>
+                </dependencies>
             </plugin>
             <plugin>
                 <artifactId>maven-assembly-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-enrichment/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/pom.xml b/metron-platform/metron-enrichment/pom.xml
index 7daf297..39673fc 100644
--- a/metron-platform/metron-enrichment/pom.xml
+++ b/metron-platform/metron-enrichment/pom.xml
@@ -352,6 +352,7 @@
                                 </excludes>
                             </artifactSet>
                             <transformers>
+                                <transformer implementation="org.atteo.classindex.ClassIndexTransformer"/>
                                 <transformer
                                   implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
                                      <resources>
@@ -376,6 +377,13 @@
                         </configuration>
                     </execution>
                 </executions>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.atteo.classindex</groupId>
+                        <artifactId>classindex-transformer</artifactId>
+                        <version>${global_classindex_version}</version>
+                    </dependency>
+                </dependencies>
             </plugin>
             <plugin>
                 <artifactId>maven-assembly-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/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 f3ed5b4..bb77b84 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
@@ -163,6 +163,7 @@ public class GenericEnrichmentBolt extends ConfiguredEnrichmentBolt {
     stellarContext = new Context.Builder()
                          .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
                          .with(Context.Capabilities.GLOBAL_CONFIG, () -> getConfigurations().getGlobalConfig())
+                         .with(Context.Capabilities.STELLAR_CONFIG, () -> getConfigurations().getGlobalConfig())
                          .build();
     StellarFunctions.initialize(stellarContext);
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/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 4fd5e02..4d924c3 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
@@ -113,6 +113,7 @@ public class ThreatIntelJoinBolt extends EnrichmentJoinBolt {
     this.stellarContext = new Context.Builder()
                                 .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client)
                                 .with(Context.Capabilities.GLOBAL_CONFIG, () -> getConfigurations().getGlobalConfig())
+                                .with(Context.Capabilities.STELLAR_CONFIG, () -> getConfigurations().getGlobalConfig())
                                 .build();
     StellarFunctions.initialize(stellarContext);
     this.functionResolver = StellarFunctions.FUNCTION_RESOLVER();

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/27ee4909/metron-platform/metron-parsers/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/pom.xml b/metron-platform/metron-parsers/pom.xml
index 3049a71..fd279ae 100644
--- a/metron-platform/metron-parsers/pom.xml
+++ b/metron-platform/metron-parsers/pom.xml
@@ -295,6 +295,7 @@
                                 </excludes>
                             </artifactSet>
                             <transformers>
+                                <transformer implementation="org.atteo.classindex.ClassIndexTransformer"/>
                                 <transformer
                                   implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
                                      <resources>
@@ -319,6 +320,13 @@
                         </configuration>
                     </execution>
                 </executions>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.atteo.classindex</groupId>
+                        <artifactId>classindex-transformer</artifactId>
+                        <version>${global_classindex_version}</version>
+                    </dependency>
+                </dependencies>
             </plugin>
             <plugin>
                 <artifactId>maven-assembly-plugin</artifactId>