You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ni...@apache.org on 2016/10/11 14:40:14 UTC

[1/3] incubator-metron git commit: METRON-421 Make Stellar Profiler Client API Accessible in Parser and Enrichment Topologies (nickwallen) closes apache/incubator-metron#290

Repository: incubator-metron
Updated Branches:
  refs/heads/master 0e9316904 -> 070473f5f


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-parsers/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/pom.xml b/metron-platform/metron-parsers/pom.xml
index 5c52734..25fbe8b 100644
--- a/metron-platform/metron-parsers/pom.xml
+++ b/metron-platform/metron-parsers/pom.xml
@@ -54,6 +54,11 @@
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>metron-profiler-client</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>
             <version>${global_hadoop_version}</version>
@@ -144,6 +149,11 @@
             </exclusions>
         </dependency>
         <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+            <version>1.2</version>
+        </dependency>
+        <dependency>
             <groupId>org.apache.metron</groupId>
             <artifactId>metron-test-utilities</artifactId>
             <version>${project.parent.version}</version>
@@ -193,20 +203,36 @@
         <plugins>
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-compiler-plugin</artifactId>
                 <version>3.1</version>
                 <inherited>true</inherited>
                 <configuration>
-                    <source>1.8</source>
-                    <target>1.8</target>
+                    <source>${global_java_version}</source>
+                    <target>${global_java_version}</target>
                 </configuration>
             </plugin>
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-shade-plugin</artifactId>
-                <version>2.3</version>
+                <version>${global_shade_version}</version>
                 <configuration>
                     <createDependencyReducedPom>true</createDependencyReducedPom>
+                    <artifactSet>
+                        <excludes>
+                            <exclude>*slf4j*</exclude>
+                        </excludes>
+                    </artifactSet>
                 </configuration>
                 <executions>
                     <execution>
@@ -215,10 +241,12 @@
                             <goal>shade</goal>
                         </goals>
                         <configuration>
+                            <shadedArtifactAttached>true</shadedArtifactAttached>
+                            <shadedClassifierName>uber</shadedClassifierName>
                             <relocations>
                                 <relocation>
-                                    <pattern>com.fasterxml.jackson.core</pattern>
-                                    <shadedPattern>com.fasterxml.jackson.core.metron.parsers</shadedPattern>
+                                    <pattern>com.fasterxml.jackson</pattern>
+                                    <shadedPattern>org.apache.metron.jackson</shadedPattern>
                                 </relocation>
                             </relocations>
                             <artifactSet>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-parsers/src/main/assembly/assembly.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/assembly/assembly.xml b/metron-platform/metron-parsers/src/main/assembly/assembly.xml
index 6c508ed..e0b3f6d 100644
--- a/metron-platform/metron-parsers/src/main/assembly/assembly.xml
+++ b/metron-platform/metron-parsers/src/main/assembly/assembly.xml
@@ -56,7 +56,7 @@
     <fileSet>
       <directory>${project.basedir}/target</directory>
       <includes>
-        <include>${project.artifactId}-${project.version}.jar</include>
+        <include>${project.artifactId}-${project.version}-uber.jar</include>
       </includes>
       <outputDirectory>/lib</outputDirectory>
       <useDefaultExcludes>true</useDefaultExcludes>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-parsers/src/main/scripts/start_parser_topology.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/src/main/scripts/start_parser_topology.sh b/metron-platform/metron-parsers/src/main/scripts/start_parser_topology.sh
index 05fb9f7..8faf89e 100755
--- a/metron-platform/metron-parsers/src/main/scripts/start_parser_topology.sh
+++ b/metron-platform/metron-parsers/src/main/scripts/start_parser_topology.sh
@@ -18,5 +18,5 @@
 #
 METRON_VERSION=${project.version}
 METRON_HOME=/usr/metron/$METRON_VERSION
-TOPOLOGY_JAR=metron-parsers-$METRON_VERSION.jar
+TOPOLOGY_JAR=metron-parsers-$METRON_VERSION-uber.jar
 storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.metron.parsers.topology.ParserTopologyCLI "$@"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/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 1318ee1..dc3069f 100644
--- a/metron-platform/metron-pcap-backend/pom.xml
+++ b/metron-platform/metron-pcap-backend/pom.xml
@@ -21,8 +21,6 @@
         <version>0.2.1BETA</version>
     </parent>
     <artifactId>metron-pcap-backend</artifactId>
-    <name>Metron PCAP Backend</name>
-    <description>The Metron PCAP Backend Topology</description>
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-solr/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/pom.xml b/metron-platform/metron-solr/pom.xml
index 79a7c60..65077e0 100644
--- a/metron-platform/metron-solr/pom.xml
+++ b/metron-platform/metron-solr/pom.xml
@@ -40,6 +40,12 @@
             <groupId>org.apache.metron</groupId>
             <artifactId>metron-enrichment</artifactId>
             <version>${project.parent.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.metron</groupId>
+                    <artifactId>metron-profiler-client</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
         <dependency>
             <groupId>org.apache.solr</groupId>
@@ -236,7 +242,8 @@
                             <goal>shade</goal>
                         </goals>
                         <configuration>
-
+                            <shadedArtifactAttached>true</shadedArtifactAttached>
+                            <shadedClassifierName>uber</shadedClassifierName>
                             <artifactSet>
                                 <excludes>
                                     <exclude>storm:storm-core:*</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-solr/src/main/assembly/assembly.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/assembly/assembly.xml b/metron-platform/metron-solr/src/main/assembly/assembly.xml
index 7a5dbee..93eeeb1 100644
--- a/metron-platform/metron-solr/src/main/assembly/assembly.xml
+++ b/metron-platform/metron-solr/src/main/assembly/assembly.xml
@@ -45,7 +45,7 @@
     <fileSet>
       <directory>${project.basedir}/target</directory>
       <includes>
-        <include>${project.artifactId}-${project.version}.jar</include>
+        <include>${project.artifactId}-${project.version}-uber.jar</include>
       </includes>
       <outputDirectory>/lib</outputDirectory>
       <useDefaultExcludes>true</useDefaultExcludes>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-solr/src/main/scripts/start_solr_topology.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/scripts/start_solr_topology.sh b/metron-platform/metron-solr/src/main/scripts/start_solr_topology.sh
index 7a98fc7..cae0c3c 100755
--- a/metron-platform/metron-solr/src/main/scripts/start_solr_topology.sh
+++ b/metron-platform/metron-solr/src/main/scripts/start_solr_topology.sh
@@ -18,5 +18,5 @@
 #
 METRON_VERSION=${project.version}
 METRON_HOME=/usr/metron/$METRON_VERSION
-TOPOLOGY_JAR=${project.artifactId}-$METRON_VERSION.jar
+TOPOLOGY_JAR=${project.artifactId}-$METRON_VERSION-uber.jar
 storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.storm.flux.Flux --remote $METRON_HOME/flux/indexing/remote.yaml --filter $METRON_HOME/config/solr.properties


[2/3] incubator-metron git commit: METRON-421 Make Stellar Profiler Client API Accessible in Parser and Enrichment Topologies (nickwallen) closes apache/incubator-metron#290

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/stellar/DefaultStellarExecutorTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/stellar/DefaultStellarExecutorTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/stellar/DefaultStellarExecutorTest.java
new file mode 100644
index 0000000..d90c699
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/stellar/DefaultStellarExecutorTest.java
@@ -0,0 +1,181 @@
+/*
+ *
+ *  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.profiler.stellar;
+
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.common.dsl.Context;
+import org.apache.metron.profiler.stellar.DefaultStellarExecutor;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests the DefaultStellarExecutor.
+ */
+public class DefaultStellarExecutorTest {
+
+  /**
+   * {
+   *   "ip_src_addr": "10.0.0.1",
+   *   "ip_dst_addr": "10.0.0.20"
+   * }
+   */
+  @Multiline
+  private String input;
+
+  private JSONObject message;
+  private DefaultStellarExecutor executor;
+
+  @Before
+  public void setup() throws ParseException {
+
+    // parse the input message
+    JSONParser parser = new JSONParser();
+    message = (JSONObject) parser.parse(input);
+
+    // create the executor to test
+    executor = new DefaultStellarExecutor();
+    executor.setContext(Context.EMPTY_CONTEXT());
+  }
+
+  /**
+   * Ensure that a value can be assigned to a variable.
+   */
+  @Test
+  public void testAssign() {
+    executor.assign("foo", "2", message);
+
+    // verify
+    Object var = executor.getState().get("foo");
+    assertThat(var, instanceOf(Integer.class));
+    assertThat(var, equalTo(2));
+  }
+
+  /**
+   * Ensure that a variable can be resolved from a message field.
+   */
+  @Test
+  public void testAssignWithVariableResolution() {
+    executor.assign("foo", "ip_src_addr", message);
+
+    // verify
+    Object var = executor.getState().get("foo");
+    assertThat(var, instanceOf(String.class));
+    assertThat(var, equalTo("10.0.0.1"));
+  }
+
+  /**
+   * Ensure that state is maintained correctly in the execution environment.
+   */
+  @Test
+  public void testState() {
+    executor.assign("two", "2", message);
+    executor.assign("four", "4", message);
+    executor.assign("sum", "two + four", message);
+
+    // verify
+    Object var = executor.getState().get("sum");
+    assertEquals(6.0, var);
+  }
+
+  /**
+   * Ensure that state is maintained correctly in the execution environment.
+   */
+  @Test
+  public void testClearState() {
+    executor.assign("two", "2", message);
+    executor.clearState();
+
+    // verify
+    assertThat(executor.getState().containsKey("two"), equalTo(false));
+  }
+
+  /**
+   * Ensure that a Transformation function can be executed.
+   *
+   * There are two sets of functions in Stellar currently.  One can be executed with
+   * a PredicateProcessor and the other a TransformationProcessor.  The StellarExecutor
+   * abstracts away that complication.
+   */
+  @Test
+  public void testExecuteTransformation() {
+    String actual = executor.execute("TO_UPPER('lowercase')", message, String.class);
+    assertThat(actual, equalTo("LOWERCASE"));
+  }
+
+  /**
+   * Ensure that a Predicate function can be executed.
+   *
+   * There are two sets of functions in Stellar currently.  One can be executed with
+   * a PredicateProcessor and the other a TransformationProcessor.  The StellarExecutor
+   * abstracts away that complication.
+   */
+  @Test
+  public void testExecutePredicate() {
+    boolean actual = executor.execute("IS_INTEGER(2)", message, Boolean.class);
+    assertThat(actual, equalTo(true));
+  }
+
+  /**
+   * An exception is expected if an expression results in an unexpected type.
+   */
+  @Test(expected = RuntimeException.class)
+  public void testExecuteWithWrongType() {
+    executor.execute("2 + 2", message, Boolean.class);
+  }
+
+  /**
+   * A best effort should be made to do sensible type conversions.
+   */
+  @Test
+  public void testExecuteWithTypeConversion() {
+    executor.execute("2", message, Double.class);
+    executor.execute("2", message, Float.class);
+    executor.execute("2", message, Short.class);
+    executor.execute("2", message, Long.class);
+  }
+
+  /**
+   * The executor must be serializable.
+   */
+  @Test
+  public void testSerializable() throws Exception {
+
+    // serialize
+    ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+    new ObjectOutputStream(bytes).writeObject(executor);
+
+    // deserialize - success when no exceptions
+    new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())).readObject();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/test/resources/log4j.properties b/metron-analytics/metron-profiler-common/src/test/resources/log4j.properties
new file mode 100644
index 0000000..70be8ae
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/test/resources/log4j.properties
@@ -0,0 +1,28 @@
+#
+#
+#  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.
+#
+#
+
+# Root logger option
+log4j.rootLogger=ERROR, stdout
+
+# Direct log messages to stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/README.md
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md
index 2fd83d5..5659090 100644
--- a/metron-analytics/metron-profiler/README.md
+++ b/metron-analytics/metron-profiler/README.md
@@ -247,22 +247,14 @@ This section will describe the steps required to get your first profile running.
     $ /usr/hdp/current/hbase-client/bin/hbase shell
     hbase(main):001:0> create 'profiler', 'P'
     ```
-
-4. Shorten the flush intervals to more immediately see results.  Edit the Profiler topology properties located at `/usr/metron/0.2.1BETA/config/profiler.properties`.  Alter the following two properties.
-    ```
-    profiler.period.duration=30
-    profiler.period.duration.units=SECONDS
-    profiler.hbase.flush.interval.seconds=5
-    ```
-
-5. Create the Profiler definition in a file located at `/usr/metron/0.2.1BETA/config/zookeeper/profiler.json`.  The following JSON will create a profile that simply counts the number of messages.
+    
+4. Create the Profiler definition in a file located at `/usr/metron/0.2.1BETA/config/zookeeper/profiler.json`.  The following JSON will create a profile that simply counts the number of messages.
     ```
     {
       "profiles": [
         {
           "profile": "test",
           "foreach": "ip_src_addr",
-          "onlyif":  "true",
           "init":    { "sum": 0 },
           "update":  { "sum": "sum + 1" },
           "result":  "sum"
@@ -271,19 +263,19 @@ This section will describe the steps required to get your first profile running.
     }
     ```
 
-6. Upload the Profiler definition to Zookeeper.
+5. Upload the Profiler definition to Zookeeper.
     ```
     $ bin/zk_load_configs.sh -m PUSH -i config/zookeeper/ -z node1:2181
     ```
 
-7. Start the Profiler topology.
+6. Start the Profiler topology.
     ```
     bin/start_profiler_topology.sh
     ```
 
-8. Ensure that test messages are being sent to the Profiler's input topic in Kafka.  The Profiler will consume messages from the `inputTopic` in the Profiler definition.
+7. Ensure that test messages are being sent to the Profiler's input topic in Kafka.  The Profiler will consume messages from the `inputTopic` in the Profiler definition.
 
-9. Check the HBase table to validate that the Profiler is working. 
+8. Check the HBase table to validate that the Profiler is working.  Remember that the Profiler is flushing the profile every 15 minutes.  You will need to wait at least this long to start seeing profile data in HBase.
     ```
     $ /usr/hdp/current/hbase-client/bin/hbase shell
     hbase(main):001:0> count 'profiler'

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/pom.xml
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/pom.xml b/metron-analytics/metron-profiler/pom.xml
index 49c0820..6f12d16 100644
--- a/metron-analytics/metron-profiler/pom.xml
+++ b/metron-analytics/metron-profiler/pom.xml
@@ -28,6 +28,11 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.metron</groupId>
+            <artifactId>metron-profiler-common</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
             <artifactId>metron-common</artifactId>
             <version>${project.parent.version}</version>
             <exclusions>
@@ -43,40 +48,6 @@
             <version>${project.parent.version}</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.metron</groupId>
-            <artifactId>metron-test-utilities</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>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-client</artifactId>
-            <version>${global_hadoop_version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-common</artifactId>
-            <classifier>tests</classifier>
-            <version>${global_hadoop_version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-hdfs</artifactId>
-            <classifier>tests</classifier>
-            <version>${global_hadoop_version}</version>
-        </dependency>
-        <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-client</artifactId>
             <version>${global_hbase_version}</version>
@@ -109,11 +80,7 @@
                     <groupId>org.slf4j</groupId>
                 </exclusion>
             </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-hdfs</artifactId>
-            <version>${global_storm_version}</version>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.kafka</groupId>
@@ -138,6 +105,18 @@
             <version>1.2-20140129.191141-5</version>
             <scope>test</scope>
         </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.metron</groupId>
+            <artifactId>metron-test-utilities</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
     <reporting>
         <plugins>
@@ -225,12 +204,10 @@
                             <goal>shade</goal>
                         </goals>
                         <configuration>
+                            <shadedArtifactAttached>true</shadedArtifactAttached>
+                            <shadedClassifierName>uber</shadedClassifierName>
                             <relocations>
                                 <relocation>
-                                    <pattern>com.lmax</pattern>
-                                    <shadedPattern>org.apache.metron.lmax.metron-profiler</shadedPattern>
-                                </relocation>
-                                <relocation>
                                     <pattern>com.google.common</pattern>
                                     <shadedPattern>org.apache.metron.guava.metron-profiler</shadedPattern>
                                 </relocation>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/main/assembly/assembly.xml
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/assembly/assembly.xml b/metron-analytics/metron-profiler/src/main/assembly/assembly.xml
index a5f1643..8f28ba1 100644
--- a/metron-analytics/metron-profiler/src/main/assembly/assembly.xml
+++ b/metron-analytics/metron-profiler/src/main/assembly/assembly.xml
@@ -63,7 +63,7 @@
         <fileSet>
             <directory>${project.basedir}/target</directory>
             <includes>
-                <include>${project.artifactId}-${project.version}.jar</include>
+                <include>${project.artifactId}-${project.version}-uber.jar</include>
             </includes>
             <outputDirectory>/lib</outputDirectory>
             <useDefaultExcludes>true</useDefaultExcludes>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java
deleted file mode 100644
index 210b92f..0000000
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- *
- *  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.profiler;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Represents a single data point within a Profile.
- *
- * A Profile is effectively a time series.  To this end a Profile is composed
- * of many ProfileMeasurement values which in aggregate form a time series.
- */
-public class ProfileMeasurement {
-
-  /**
-   * The name of the profile that this measurement is associated with.
-   */
-  private String profileName;
-
-  /**
-   * The name of the entity being profiled.
-   */
-  private String entity;
-
-  /**
-   * The actual measurement itself.
-   */
-  private Object value;
-
-  /**
-   * A set of expressions used to group the profile measurements when persisted.
-   */
-  private List<String> groupBy;
-
-  /**
-   * The period in which the ProfileMeasurement was taken.
-   */
-  private ProfilePeriod period;
-
-  /**
-   * @param profileName The name of the profile.
-   * @param entity The name of the entity being profiled.
-   * @param whenMillis When the measurement was taken in epoch milliseconds.
-   * @param periodDuration The duration of each profile period.
-   * @param periodUnits The units of the duration of each profile period.
-   */
-  public ProfileMeasurement(String profileName, String entity, long whenMillis, long periodDuration, TimeUnit periodUnits) {
-    this.profileName = profileName;
-    this.entity = entity;
-    this.period = new ProfilePeriod(whenMillis, periodDuration, periodUnits);
-  }
-
-  public String getProfileName() {
-    return profileName;
-  }
-
-  public String getEntity() {
-    return entity;
-  }
-
-  public Object getValue() {
-    return value;
-  }
-
-  public ProfilePeriod getPeriod() {
-    return period;
-  }
-
-  public List<String> getGroupBy() {
-    return groupBy;
-  }
-
-  public void setValue(Object value) {
-    this.value = value;
-  }
-
-  public void setGroupBy(List<String> groupBy) {
-    this.groupBy = groupBy;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    ProfileMeasurement that = (ProfileMeasurement) o;
-    if (profileName != null ? !profileName.equals(that.profileName) : that.profileName != null) return false;
-    if (entity != null ? !entity.equals(that.entity) : that.entity != null) return false;
-    if (value != null ? !value.equals(that.value) : that.value != null) return false;
-    if (groupBy != null ? !groupBy.equals(that.groupBy) : that.groupBy != null) return false;
-    return period != null ? period.equals(that.period) : that.period == null;
-
-  }
-
-  @Override
-  public int hashCode() {
-    int result = profileName != null ? profileName.hashCode() : 0;
-    result = 31 * result + (entity != null ? entity.hashCode() : 0);
-    result = 31 * result + (value != null ? value.hashCode() : 0);
-    result = 31 * result + (groupBy != null ? groupBy.hashCode() : 0);
-    result = 31 * result + (period != null ? period.hashCode() : 0);
-    return result;
-  }
-
-  @Override
-  public String toString() {
-    return "ProfileMeasurement{" +
-            "profileName='" + profileName + '\'' +
-            ", entity='" + entity + '\'' +
-            ", value=" + value +
-            ", groupBy=" + groupBy +
-            ", period=" + period +
-            '}';
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java
deleted file mode 100644
index 1b8efc8..0000000
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- *
- *  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.profiler;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * The Profiler captures a ProfileMeasurement once every ProfilePeriod.  There can be
- * multiple ProfilePeriods every hour.
- */
-public class ProfilePeriod {
-
-  /**
-   * A monotonically increasing number identifying the period.  The first period is 0
-   * and began at the epoch.
-   */
-  private long period;
-
-  /**
-   * The duration of each period in milliseconds.
-   */
-  private long durationMillis;
-
-  /**
-   * @param epochMillis A timestamp contained somewhere within the profile period.
-   * @param duration The duration of each profile period.
-   * @param units The units of the duration; hours, minutes, etc.
-   */
-  public ProfilePeriod(long epochMillis, long duration, TimeUnit units) {
-    this.durationMillis = units.toMillis(duration);
-    this.period = epochMillis / durationMillis;
-  }
-
-  /**
-   * When this period started in milliseconds since the epoch.
-   */
-  public long getStartTimeMillis() {
-    return period * durationMillis;
-  }
-
-  /**
-   * Returns the next ProfilePeriod in time.
-   */
-  public ProfilePeriod next() {
-    long nextStart = getStartTimeMillis() + durationMillis;
-    return new ProfilePeriod(nextStart, durationMillis, TimeUnit.MILLISECONDS);
-  }
-
-  public long getPeriod() {
-    return period;
-  }
-
-  public long getDurationMillis() {
-    return durationMillis;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    ProfilePeriod that = (ProfilePeriod) o;
-    if (period != that.period) return false;
-    return durationMillis == that.durationMillis;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = (int) (period ^ (period >>> 32));
-    result = 31 * result + (int) (durationMillis ^ (durationMillis >>> 32));
-    return result;
-  }
-
-  @Override
-  public String toString() {
-    return "ProfilePeriod{" +
-            "period=" + period +
-            ", durationMillis=" + durationMillis +
-            '}';
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java
deleted file mode 100644
index c645822..0000000
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- *
- *  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.profiler.hbase;
-
-import org.apache.metron.hbase.bolt.mapper.ColumnList;
-import org.apache.metron.profiler.ProfileMeasurement;
-
-import java.io.Serializable;
-
-/**
- * Defines how fields in a ProfileMeasurement will be mapped to columns in HBase.
- */
-public interface ColumnBuilder extends Serializable {
-
-  /**
-   * Generate the columns used to store a ProfileMeasurement.
-   * @param measurement The profile measurement.
-   */
-  ColumnList columns(ProfileMeasurement measurement);
-
-  /**
-   * Returns the column family used to store the ProfileMeasurement values.
-   * @return
-   */
-  String getColumnFamily();
-
-  /**
-   * Returns the column qualifiers for the given field of a ProfileMeasurement.
-   * @return The column qualifier used to store a ProfileMeasurement's field in HBase.
-   */
-  byte[] getColumnQualifier(String fieldName);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java
deleted file mode 100644
index 1ce4906..0000000
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- *
- *  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.profiler.hbase;
-
-import org.apache.metron.profiler.ProfileMeasurement;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Builds a row key that can be used to read or write ProfileMeasurement data
- * to HBase.
- */
-public interface RowKeyBuilder extends Serializable {
-
-  /**
-   * Build a row key for a given ProfileMeasurement.
-   * 
-   * This method is useful when writing ProfileMeasurements to HBase.
-   *
-   * @param measurement The profile measurement.
-   * @param groups      The groups used to sort the profile data.
-   * @return The HBase row key.
-   */
-  byte[] rowKey(ProfileMeasurement measurement, List<Object> groups);
-
-  /**
-   * Builds a list of row keys necessary to retrieve a profile's measurements over
-   * a time horizon.
-   *
-   * This method is useful when attempting to read ProfileMeasurements stored in HBase.
-   *
-   * @param profile The name of the profile.
-   * @param entity The name of the entity.
-   * @param groups The group(s) used to sort the profile data.
-   * @param start When the time horizon starts in epoch milliseconds.
-   * @param end When the time horizon ends in epoch milliseconds.
-   * @return All of the row keys necessary to retrieve the profile measurements.
-   */
-  List<byte[]> rowKeys(String profile, String entity, List<Object> groups, long start, long end);
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java
deleted file mode 100644
index 3b12472..0000000
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- *
- *  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.profiler.hbase;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.profiler.ProfileMeasurement;
-import org.apache.metron.profiler.ProfilePeriod;
-
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-/**
- * A RowKeyBuilder that uses a salt to prevent hot-spotting.
- *
- * Responsible for building the row keys used to store profile data in HBase.  The row key is composed of the following
- * fields in the given order.
- * <ul>
- * <li>salt - A salt that helps prevent hot-spotting.
- * <li>profile - The name of the profile.
- * <li>entity - The name of the entity being profiled.
- * <li>group(s) - The group(s) used to sort the data in HBase. For example, a group may distinguish between weekends and weekdays.
- * <li>year - The year based on UTC.
- * <li>day of year - The current day within the year based on UTC; [1, 366]
- * <li>hour - The hour within the day based on UTC; [0, 23]
- * </ul>period - The period within the hour.  The number of periods per hour can be defined by the user; defaults to 4.
- */
-public class SaltyRowKeyBuilder implements RowKeyBuilder {
-
-  /**
-   * A salt can be prepended to the row key to help prevent hot-spotting.  The salt
-   * divisor is used to generate the salt.  The salt divisor should be roughly equal
-   * to the number of nodes in the Hbase cluster.
-   */
-  private int saltDivisor;
-
-  /**
-   * The duration of each profile period in milliseconds.
-   */
-  private long periodDurationMillis;
-
-  public SaltyRowKeyBuilder() {
-    this.saltDivisor = 1000;
-    this.periodDurationMillis = TimeUnit.MINUTES.toMillis(15);
-  }
-
-  public SaltyRowKeyBuilder(int saltDivisor, long duration, TimeUnit units) {
-    this.saltDivisor = saltDivisor;
-    this.periodDurationMillis = units.toMillis(duration);
-  }
-
-  /**
-   * Builds a list of row keys necessary to retrieve profile measurements over
-   * a time horizon.
-   *
-   * @param profile The name of the profile.
-   * @param entity The name of the entity.
-   * @param groups The group(s) used to sort the profile data.
-   * @param start When the time horizon starts in epoch milliseconds.
-   * @param end When the time horizon ends in epoch milliseconds.
-   * @return All of the row keys necessary to retrieve the profile measurements.
-   */
-  @Override
-  public List<byte[]> rowKeys(String profile, String entity, List<Object> groups, long start, long end) {
-    List<byte[]> rowKeys = new ArrayList<>();
-
-    // be forgiving of out-of-order start and end times; order is critical to this algorithm
-    end = Math.max(start, end);
-    start = Math.min(start, end);
-
-    // find the starting period and advance until the end time is reached
-    ProfilePeriod period = new ProfilePeriod(start, periodDurationMillis, TimeUnit.MILLISECONDS);
-    while(period.getStartTimeMillis() <= end) {
-
-      byte[] k = rowKey(profile, entity, period, groups);
-      rowKeys.add(k);
-
-      // advance to the next period
-      period = period.next();
-    }
-
-    return rowKeys;
-  }
-
-  /**
-   * Builds the row key for a given profile measurement.
-   * @param m The profile measurement.
-   * @param groups The groups used to sort the profile data.
-   * @return The HBase row key.
-   */
-  @Override
-  public byte[] rowKey(ProfileMeasurement m, List<Object> groups) {
-    return rowKey(m.getProfileName(), m.getEntity(), m.getPeriod(), groups);
-  }
-
-  public void withPeriodDuration(long duration, TimeUnit units) {
-    periodDurationMillis = units.toMillis(duration);
-  }
-
-  public void setSaltDivisor(int saltDivisor) {
-    this.saltDivisor = saltDivisor;
-  }
-
-  /**
-   * Build the row key.
-   * @param profile The name of the profile.
-   * @param entity The name of the entity.
-   * @param period The period in which the measurement was taken.
-   * @param groups The groups.
-   * @return The HBase row key.
-   */
-  public byte[] rowKey(String profile, String entity, ProfilePeriod period, List<Object> groups) {
-
-    // row key = salt + prefix + group(s) + time
-    byte[] salt = getSalt(period, saltDivisor);
-    byte[] prefixKey = prefixKey(profile, entity);
-    byte[] groupKey = groupKey(groups);
-    byte[] timeKey = timeKey(period);
-
-    int capacity = salt.length + prefixKey.length + groupKey.length + timeKey.length;
-    return ByteBuffer
-            .allocate(capacity)
-            .put(salt)
-            .put(prefixKey)
-            .put(groupKey)
-            .put(timeKey)
-            .array();
-  }
-
-  /**
-   * Builds the 'prefix' component of the row key.
-   * @param profile The name of the profile.
-   * @param entity The name of the entity.
-   */
-  private static byte[] prefixKey(String profile, String entity) {
-    return ByteBuffer
-            .allocate(profile.length() + entity.length())
-            .put(profile.getBytes())
-            .put(entity.getBytes())
-            .array();
-  }
-
-  /**
-   * Builds the 'group' component of the row key.
-   * @param groups The groups to include in the row key.
-   */
-  private static byte[] groupKey(List<Object> groups) {
-
-    StringBuilder builder = new StringBuilder();
-    groups.forEach(g -> builder.append(g));
-    String groupStr = builder.toString();
-
-    return ByteBuffer
-            .allocate(groupStr.length())
-            .put(groupStr.getBytes())
-            .array();
-  }
-
-  /**
-   * Builds the 'time' portion of the row key
-   * @param period The ProfilePeriod in which the ProfileMeasurement was taken.
-   */
-  private static byte[] timeKey(ProfilePeriod period) {
-    return ByteBuffer
-            .allocate(Long.BYTES)
-            .putLong(period.getPeriod())
-            .array();
-  }
-
-  /**
-   * Calculates a salt value that is used as part of the row key.
-   *
-   * The salt is calculated as 'md5(timestamp) % N' where N is a configurable value that ideally
-   * is close to the number of nodes in the Hbase cluster.
-   *
-   * @param period The period in which a profile measurement is taken.
-   */
-  public static byte[] getSalt(ProfilePeriod period, int saltDivisor) {
-    try {
-      MessageDigest digest = MessageDigest.getInstance("MD5");
-      byte[] hash = digest.digest(timeKey(period));
-      int salt = Bytes.toInt(hash) % saltDivisor;
-      return ByteBuffer
-              .allocate(Integer.BYTES)
-              .putInt(salt)
-              .array();
-
-    } catch(NoSuchAlgorithmException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java
deleted file mode 100644
index bb1baf6..0000000
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- *
- *  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.profiler.hbase;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.common.utils.SerDeUtils;
-import org.apache.metron.profiler.ProfileMeasurement;
-import org.apache.metron.hbase.bolt.mapper.ColumnList;
-
-/**
- * A ColumnBuilder that writes only the value of a ProfileMeasurement.
- */
-public class ValueOnlyColumnBuilder implements ColumnBuilder {
-
-  /**
-   * The column family storing the profile data.
-   */
-  private String columnFamily;
-
-  private byte[] columnFamilyBytes;
-
-  public ValueOnlyColumnBuilder() {
-    setColumnFamily("P");
-  }
-
-  public ValueOnlyColumnBuilder(String columnFamily) {
-    setColumnFamily(columnFamily);
-  }
-
-  @Override
-  public ColumnList columns(ProfileMeasurement measurement) {
-
-    ColumnList cols = new ColumnList();
-    cols.addColumn(columnFamilyBytes, getColumnQualifier("value"), SerDeUtils.toBytes(measurement.getValue()));
-
-    return cols;
-  }
-
-  @Override
-  public String getColumnFamily() {
-    return this.columnFamily;
-  }
-
-  public void setColumnFamily(String columnFamily) {
-    this.columnFamily = columnFamily;
-    this.columnFamilyBytes = Bytes.toBytes(columnFamily);
-  }
-
-  @Override
-  public byte[] getColumnQualifier(String fieldName) {
-
-    if("value".equals(fieldName)) {
-      return Bytes.toBytes("value");
-    }
-
-    throw new IllegalArgumentException(("unexpected field name: " + fieldName));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java
deleted file mode 100644
index 474ac73..0000000
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- *
- *  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.profiler.stellar;
-
-import org.apache.metron.common.dsl.Context;
-import org.apache.metron.common.dsl.FunctionResolver;
-import org.apache.metron.common.dsl.MapVariableResolver;
-import org.apache.metron.common.dsl.ParseException;
-import org.apache.metron.common.dsl.StellarFunctions;
-import org.apache.metron.common.dsl.VariableResolver;
-import org.apache.metron.common.stellar.StellarProcessor;
-import org.apache.metron.common.utils.ConversionUtils;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The default implementation of a StellarExecutor.
- */
-public class DefaultStellarExecutor implements StellarExecutor, Serializable {
-
-  /**
-   * The current state of the Stellar execution environment.
-   */
-  private Map<String, Object> state;
-
-  /**
-   * Provides additional context for initializing certain Stellar functions.  For
-   * example, references to Zookeeper or HBase.
-   */
-  private Context context;
-
-  public DefaultStellarExecutor() {
-    clearState();
-    context = Context.EMPTY_CONTEXT();
-  }
-
-  /**
-   * @param initialState Initial state loaded into the execution environment.
-   */
-  public DefaultStellarExecutor(Map<String, Object> initialState) {
-    this();
-    this.state = new HashMap<>(initialState);
-  }
-
-  /**
-   * The current state of the Stellar execution environment.
-   */
-  @Override
-  public Map<String, Object> getState() {
-    return new HashMap<>(state);
-  }
-
-  /**
-   * Execute an expression and assign the result to a variable.  The variable is maintained
-   * in the context of this executor and is available to all subsequent expressions.
-   *
-   * @param variable       The name of the variable to assign to.
-   * @param expression     The expression to execute.
-   * @param transientState Additional state available to the expression.  This most often represents
-   *                       the values available to the expression from an individual message. The state
-   *                       maps a variable name to a variable's value.
-   */
-  @Override
-  public void assign(String variable, String expression, Map<String, Object> transientState) {
-    Object result = execute(expression, transientState);
-    state.put(variable, result);
-  }
-
-  /**
-   * Execute a Stellar expression and return the result.  The internal state of the executor
-   * is not modified.
-   *
-   * @param expression The expression to execute.
-   * @param state      Additional state available to the expression.  This most often represents
-   *                   the values available to the expression from an individual message. The state
-   *                   maps a variable name to a variable's value.
-   * @param clazz      The expected type of the expression's result.
-   * @param <T>        The expected type of the expression's result.
-   */
-  @Override
-  public <T> T execute(String expression, Map<String, Object> state, Class<T> clazz) {
-    Object resultObject = execute(expression, state);
-
-    // perform type conversion, if necessary
-    T result = ConversionUtils.convert(resultObject, clazz);
-    if (result == null) {
-      throw new IllegalArgumentException(String.format("Unexpected type: expected=%s, actual=%s, expression=%s",
-              clazz.getSimpleName(), resultObject.getClass().getSimpleName(), expression));
-    }
-
-    return result;
-  }
-
-  @Override
-  public void clearState() {
-    this.state = new HashMap<>();
-  }
-
-  /**
-   * Sets the Context for the Stellar execution environment.  This provides global data used
-   * to initialize Stellar functions.
-   *
-   * @param context The Stellar context.
-   */
-  @Override
-  public void setContext(Context context) {
-    this.context = context;
-  }
-
-  /**
-   * Execute a Stellar expression.
-   *
-   * @param expression     The expression to execute.
-   * @param transientState Additional state available to the expression.  This most often represents
-   *                       the values available to the expression from an individual message. The state
-   *                       maps a variable name to a variable's value.
-   */
-  private Object execute(String expression, Map<String, Object> transientState) {
-    FunctionResolver functionResolver = StellarFunctions.FUNCTION_RESOLVER();
-    VariableResolver variableResolver = new MapVariableResolver(state, transientState);
-    StellarProcessor processor = new StellarProcessor();
-    return processor.parse(expression, variableResolver, functionResolver, context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java
deleted file mode 100644
index 869db42..0000000
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- *
- *  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.profiler.stellar;
-
-import org.apache.metron.common.dsl.Context;
-
-import java.util.Map;
-
-/**
- * Executes Stellar expressions and maintains state across multiple invocations.
- */
-public interface StellarExecutor {
-
-  /**
-   * Execute an expression and assign the result to a variable.  The variable is maintained
-   * in the context of this executor and is available to all subsequent expressions.
-   *
-   * @param variable   The name of the variable to assign to.
-   * @param expression The expression to execute.
-   * @param state      Additional state available to the expression.  This most often represents
-   *                   the values available to the expression from an individual message. The state
-   *                   maps a variable name to a variable's value.
-   */
-  void assign(String variable, String expression, Map<String, Object> state);
-
-  /**
-   * Execute a Stellar expression and return the result.  The internal state of the executor
-   * is not modified.
-   *
-   * @param expression The expression to execute.
-   * @param state      Additional state available to the expression.  This most often represents
-   *                   the values available to the expression from an individual message. The state
-   *                   maps a variable name to a variable's value.
-   * @param clazz      The expected type of the expression's result.
-   * @param <T>        The expected type of the expression's result.
-   */
-  <T> T execute(String expression, Map<String, Object> state, Class<T> clazz);
-
-  /**
-   * The current state of the Stellar execution environment.
-   */
-  Map<String, Object> getState();
-
-  /**
-   * Removes all state from the execution environment.
-   */
-  void clearState();
-
-  /**
-   * Sets the Context for the Stellar execution environment.  This provides global data used
-   * to initialize Stellar functions.
-   *
-   * @param context The Stellar context.
-   */
-  void setContext(Context context);
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/main/scripts/start_profiler_topology.sh
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/scripts/start_profiler_topology.sh b/metron-analytics/metron-profiler/src/main/scripts/start_profiler_topology.sh
index 6f78471..6ec78f5 100644
--- a/metron-analytics/metron-profiler/src/main/scripts/start_profiler_topology.sh
+++ b/metron-analytics/metron-profiler/src/main/scripts/start_profiler_topology.sh
@@ -18,5 +18,5 @@
 #
 METRON_VERSION=${project.version}
 METRON_HOME=/usr/metron/$METRON_VERSION
-TOPOLOGY_JAR=${project.artifactId}-$METRON_VERSION.jar
+TOPOLOGY_JAR=${project.artifactId}-$METRON_VERSION-uber.jar
 storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.storm.flux.Flux --remote $METRON_HOME/flux/profiler/remote.yaml --filter $METRON_HOME/config/profiler.properties

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java
deleted file mode 100644
index 7e05890..0000000
--- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- *
- *  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.profiler;
-
-import org.junit.Test;
-
-import java.util.concurrent.TimeUnit;
-import java.util.stream.IntStream;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests the ProfilePeriod class.
- */
-public class ProfilePeriodTest {
-
-  /**
-   * Thu, Aug 25 2016 13:27:10 GMT
-   */
-  private long AUG2016 = 1472131630748L;
-
-  @Test
-  public void testFirstPeriodAtEpoch() {
-    long duration = 1;
-    TimeUnit units = TimeUnit.HOURS;
-
-    ProfilePeriod period = new ProfilePeriod(0, duration, units);
-    assertEquals(0, period.getPeriod());
-    assertEquals(0, period.getStartTimeMillis());
-    assertEquals(units.toMillis(duration), period.getDurationMillis());
-  }
-
-  @Test
-  public void testOneMinutePeriods() {
-    long duration = 1;
-    TimeUnit units = TimeUnit.MINUTES;
-
-    ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units);
-    assertEquals(24535527, period.getPeriod());
-    assertEquals(1472131620000L, period.getStartTimeMillis());  // Thu, 25 Aug 2016 13:27:00 GMT
-    assertEquals(units.toMillis(duration), period.getDurationMillis());
-  }
-
-  @Test
-  public void testFifteenMinutePeriods() {
-    long duration = 15;
-    TimeUnit units = TimeUnit.MINUTES;
-
-    ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units);
-    assertEquals(1635701, period.getPeriod());
-    assertEquals(1472130900000L, period.getStartTimeMillis());  // Thu, 25 Aug 2016 13:15:00 GMT
-    assertEquals(units.toMillis(duration), period.getDurationMillis());
-  }
-
-  @Test
-  public void testOneHourPeriods() {
-    long duration = 1;
-    TimeUnit units = TimeUnit.HOURS;
-
-    ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units);
-    assertEquals(408925, period.getPeriod());
-    assertEquals(1472130000000L, period.getStartTimeMillis());  // Thu, 25 Aug 2016 13:00:00 GMT
-    assertEquals(units.toMillis(duration), period.getDurationMillis());
-  }
-
-  @Test
-  public void testTwoHourPeriods() {
-    long duration = 2;
-    TimeUnit units = TimeUnit.HOURS;
-
-    ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units);
-    assertEquals(204462, period.getPeriod());
-    assertEquals(1472126400000L, period.getStartTimeMillis());  //  Thu, 25 Aug 2016 12:00:00 GMT
-    assertEquals(units.toMillis(duration), period.getDurationMillis());
-  }
-
-  @Test
-  public void testEightHourPeriods() {
-    long duration = 8;
-    TimeUnit units = TimeUnit.HOURS;
-
-    ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units);
-    assertEquals(51115, period.getPeriod());
-    assertEquals(1472112000000L, period.getStartTimeMillis());  // Thu, 25 Aug 2016 08:00:00 GMT
-    assertEquals(units.toMillis(duration), period.getDurationMillis());
-  }
-
-  @Test
-  public void testNextWithFifteenMinutePeriods() {
-    long duration = 15;
-    TimeUnit units = TimeUnit.MINUTES;
-
-    ProfilePeriod previous = new ProfilePeriod(AUG2016, duration, units);
-    IntStream.range(0, 100).forEach(i -> {
-
-      ProfilePeriod next = previous.next();
-      assertEquals(previous.getPeriod() + 1, next.getPeriod());
-      assertEquals(previous.getStartTimeMillis() + previous.getDurationMillis(), next.getStartTimeMillis());
-      assertEquals(previous.getDurationMillis(), next.getDurationMillis());
-    });
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java
index 415d89f..caeddee 100644
--- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java
+++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java
@@ -31,7 +31,6 @@ import org.apache.metron.profiler.stellar.DefaultStellarExecutor;
 import org.apache.metron.test.bolt.BaseBoltTest;
 import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
-import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java
deleted file mode 100644
index 56f1d51..0000000
--- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/*
- *
- *  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.profiler.hbase;
-
-import backtype.storm.tuple.Tuple;
-import org.apache.metron.profiler.ProfileMeasurement;
-import org.apache.metron.profiler.ProfilePeriod;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Formatter;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static org.hamcrest.core.IsEqual.equalTo;
-import static org.junit.Assert.assertThat;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests the SaltyRowKeyBuilder.
- */
-public class SaltyRowKeyBuilderTest {
-
-  private static final int saltDivisor = 1000;
-  private static final long periodDuration = 15;
-  private static final TimeUnit periodUnits = TimeUnit.MINUTES;
-
-  private SaltyRowKeyBuilder rowKeyBuilder;
-  private ProfileMeasurement measurement;
-  private Tuple tuple;
-
-  /**
-   * Thu, Aug 25 2016 13:27:10 GMT
-   */
-  private long AUG2016 = 1472131630748L;
-
-  @Before
-  public void setup() throws Exception {
-
-    // a profile measurement
-    measurement = new ProfileMeasurement("profile", "entity", AUG2016, periodDuration, periodUnits);
-    measurement.setValue(22);
-
-    // the tuple will contain the original message
-    tuple = mock(Tuple.class);
-    when(tuple.getValueByField(eq("measurement"))).thenReturn(measurement);
-  }
-
-  /**
-   * Build a row key that includes only one group.
-   */
-  @Test
-  public void testRowKeyWithOneGroup() throws Exception {
-    // setup
-    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
-    List<Object> groups = Arrays.asList("group1");
-
-    // the expected row key
-    ByteBuffer buffer = ByteBuffer
-            .allocate(100)
-            .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor))
-            .put(measurement.getProfileName().getBytes())
-            .put(measurement.getEntity().getBytes())
-            .put("group1".getBytes())
-            .putLong(1635701L);
-
-    buffer.flip();
-    final byte[] expected = new byte[buffer.limit()];
-    buffer.get(expected, 0, buffer.limit());
-
-    // validate
-    byte[] actual = rowKeyBuilder.rowKey(measurement, groups);
-    Assert.assertTrue(Arrays.equals(expected, actual));
-  }
-
-  /**
-   * Build a row key that includes two groups.
-   */
-  @Test
-  public void testRowKeyWithTwoGroups() throws Exception {
-    // setup
-    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
-    List<Object> groups = Arrays.asList("group1","group2");
-
-    // the expected row key
-    ByteBuffer buffer = ByteBuffer
-            .allocate(100)
-            .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor))
-            .put(measurement.getProfileName().getBytes())
-            .put(measurement.getEntity().getBytes())
-            .put("group1".getBytes())
-            .put("group2".getBytes())
-            .putLong(1635701L);
-
-    buffer.flip();
-    final byte[] expected = new byte[buffer.limit()];
-    buffer.get(expected, 0, buffer.limit());
-
-    // validate
-    byte[] actual = rowKeyBuilder.rowKey(measurement, groups);
-    Assert.assertTrue(Arrays.equals(expected, actual));
-  }
-
-  /**
-   * Build a row key that includes a single group that is an integer.
-   */
-  @Test
-  public void testRowKeyWithOneIntegerGroup() throws Exception {
-    // setup
-    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
-    List<Object> groups = Arrays.asList(200);
-
-    // the expected row key
-    ByteBuffer buffer = ByteBuffer
-            .allocate(100)
-            .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor))
-            .put(measurement.getProfileName().getBytes())
-            .put(measurement.getEntity().getBytes())
-            .put("200".getBytes())
-            .putLong(1635701L);
-
-    buffer.flip();
-    final byte[] expected = new byte[buffer.limit()];
-    buffer.get(expected, 0, buffer.limit());
-
-    // validate
-    byte[] actual = rowKeyBuilder.rowKey(measurement, groups);
-    Assert.assertTrue(Arrays.equals(expected, actual));
-  }
-
-  /**
-   * Build a row key that includes a single group that is an integer.
-   */
-  @Test
-  public void testRowKeyWithMixedGroups() throws Exception {
-    // setup
-    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
-    List<Object> groups = Arrays.asList(200, "group1");
-
-    // the expected row key
-    ByteBuffer buffer = ByteBuffer
-            .allocate(100)
-            .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor))
-            .put(measurement.getProfileName().getBytes())
-            .put(measurement.getEntity().getBytes())
-            .put("200".getBytes())
-            .put("group1".getBytes())
-            .putLong(1635701L);
-
-    buffer.flip();
-    final byte[] expected = new byte[buffer.limit()];
-    buffer.get(expected, 0, buffer.limit());
-
-    // validate
-    byte[] actual = rowKeyBuilder.rowKey(measurement, groups);
-    Assert.assertTrue(Arrays.equals(expected, actual));
-  }
-
-  /**
-   * Build a row key that does not include any groups.
-   */
-  @Test
-  public void testRowKeyWithNoGroup() throws Exception {
-    // setup
-    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
-    List<Object> groups = Collections.emptyList();
-
-    // the expected row key
-    ByteBuffer buffer = ByteBuffer
-            .allocate(100)
-            .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor))
-            .put(measurement.getProfileName().getBytes())
-            .put(measurement.getEntity().getBytes())
-            .putLong(1635701L);
-
-    buffer.flip();
-    final byte[] expected = new byte[buffer.limit()];
-    buffer.get(expected, 0, buffer.limit());
-
-    // validate
-    byte[] actual = rowKeyBuilder.rowKey(measurement, groups);
-    Assert.assertTrue(Arrays.equals(expected, actual));
-  }
-
-  /**
-   * `rowKeys` should return all of the row keys needed to retrieve the profile values over a given time horizon.
-   */
-  @Test
-  public void testRowKeys() throws Exception {
-    int hoursAgo = 1;
-
-    // setup
-    List<Object> groups = Collections.emptyList();
-    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
-
-    // a dummy profile measurement
-    long now = System.currentTimeMillis();
-    long oldest = now - TimeUnit.HOURS.toMillis(hoursAgo);
-    ProfileMeasurement m = new ProfileMeasurement("profile", "entity", oldest, periodDuration, periodUnits);
-    m.setValue(22);
-
-    // generate a list of expected keys
-    List<byte[]> expectedKeys = new ArrayList<>();
-    for  (int i=0; i<(hoursAgo * 4)+1; i++) {
-
-      // generate the expected key
-      byte[] rk = rowKeyBuilder.rowKey(m, groups);
-      expectedKeys.add(rk);
-
-      // advance to the next period
-      ProfilePeriod next = m.getPeriod().next();
-      m = new ProfileMeasurement("profile", "entity", next.getStartTimeMillis(), periodDuration, periodUnits);
-    }
-
-    // execute
-    List<byte[]> actualKeys = rowKeyBuilder.rowKeys(measurement.getProfileName(), measurement.getEntity(), groups, oldest, now);
-
-    // validate - expectedKeys == actualKeys
-    for(int i=0; i<actualKeys.size(); i++) {
-      byte[] actual = actualKeys.get(i);
-      byte[] expected = expectedKeys.get(i);
-      assertThat(actual, equalTo(expected));
-    }
-  }
-
-  private void printBytes(byte[] bytes) {
-    StringBuilder sb = new StringBuilder(bytes.length * 2);
-    Formatter formatter = new Formatter(sb);
-    for (byte b : bytes) {
-      formatter.format("%02x ", b);
-    }
-    System.out.println(sb.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/stellar/DefaultStellarExecutorTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/stellar/DefaultStellarExecutorTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/stellar/DefaultStellarExecutorTest.java
deleted file mode 100644
index 9e2a66a..0000000
--- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/stellar/DefaultStellarExecutorTest.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- *
- *  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.profiler.stellar;
-
-import org.adrianwalker.multilinestring.Multiline;
-import org.apache.metron.common.dsl.Context;
-import org.json.simple.JSONObject;
-import org.json.simple.parser.JSONParser;
-import org.json.simple.parser.ParseException;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-/**
- * Tests the DefaultStellarExecutor.
- */
-public class DefaultStellarExecutorTest {
-
-  /**
-   * {
-   *   "ip_src_addr": "10.0.0.1",
-   *   "ip_dst_addr": "10.0.0.20"
-   * }
-   */
-  @Multiline
-  private String input;
-
-  private JSONObject message;
-  private DefaultStellarExecutor executor;
-
-  @Before
-  public void setup() throws ParseException {
-
-    // parse the input message
-    JSONParser parser = new JSONParser();
-    message = (JSONObject) parser.parse(input);
-
-    // create the executor to test
-    executor = new DefaultStellarExecutor();
-    executor.setContext(Context.EMPTY_CONTEXT());
-  }
-
-  /**
-   * Ensure that a value can be assigned to a variable.
-   */
-  @Test
-  public void testAssign() {
-    executor.assign("foo", "2", message);
-
-    // verify
-    Object var = executor.getState().get("foo");
-    assertThat(var, instanceOf(Integer.class));
-    assertThat(var, equalTo(2));
-  }
-
-  /**
-   * Ensure that a variable can be resolved from a message field.
-   */
-  @Test
-  public void testAssignWithVariableResolution() {
-    executor.assign("foo", "ip_src_addr", message);
-
-    // verify
-    Object var = executor.getState().get("foo");
-    assertThat(var, instanceOf(String.class));
-    assertThat(var, equalTo("10.0.0.1"));
-  }
-
-  /**
-   * Ensure that state is maintained correctly in the execution environment.
-   */
-  @Test
-  public void testState() {
-    executor.assign("two", "2", message);
-    executor.assign("four", "4", message);
-    executor.assign("sum", "two + four", message);
-
-    // verify
-    Object var = executor.getState().get("sum");
-    assertEquals(6.0, var);
-  }
-
-  /**
-   * Ensure that state is maintained correctly in the execution environment.
-   */
-  @Test
-  public void testClearState() {
-    executor.assign("two", "2", message);
-    executor.clearState();
-
-    // verify
-    assertThat(executor.getState().containsKey("two"), equalTo(false));
-  }
-
-  /**
-   * Ensure that a Transformation function can be executed.
-   *
-   * There are two sets of functions in Stellar currently.  One can be executed with
-   * a PredicateProcessor and the other a TransformationProcessor.  The StellarExecutor
-   * abstracts away that complication.
-   */
-  @Test
-  public void testExecuteTransformation() {
-    String actual = executor.execute("TO_UPPER('lowercase')", message, String.class);
-    assertThat(actual, equalTo("LOWERCASE"));
-  }
-
-  /**
-   * Ensure that a Predicate function can be executed.
-   *
-   * There are two sets of functions in Stellar currently.  One can be executed with
-   * a PredicateProcessor and the other a TransformationProcessor.  The StellarExecutor
-   * abstracts away that complication.
-   */
-  @Test
-  public void testExecutePredicate() {
-    boolean actual = executor.execute("IS_INTEGER(2)", message, Boolean.class);
-    assertThat(actual, equalTo(true));
-  }
-
-  /**
-   * An exception is expected if an expression results in an unexpected type.
-   */
-  @Test(expected = RuntimeException.class)
-  public void testExecuteWithWrongType() {
-    executor.execute("2 + 2", message, Boolean.class);
-  }
-
-  /**
-   * A best effort should be made to do sensible type conversions.
-   */
-  @Test
-  public void testExecuteWithTypeConversion() {
-    executor.execute("2", message, Double.class);
-    executor.execute("2", message, Float.class);
-    executor.execute("2", message, Short.class);
-    executor.execute("2", message, Long.class);
-  }
-
-  /**
-   * The executor must be serializable.
-   */
-  @Test
-  public void testSerializable() throws Exception {
-
-    // serialize
-    ByteArrayOutputStream bytes = new ByteArrayOutputStream();
-    new ObjectOutputStream(bytes).writeObject(executor);
-
-    // deserialize - success when no exceptions
-    new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())).readObject();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/pom.xml
----------------------------------------------------------------------
diff --git a/metron-analytics/pom.xml b/metron-analytics/pom.xml
index 299efd6..a788a99 100644
--- a/metron-analytics/pom.xml
+++ b/metron-analytics/pom.xml
@@ -44,7 +44,8 @@
 		<module>metron-maas-common</module>
 		<module>metron-profiler</module>
 		<module>metron-profiler-client</module>
-	</modules>
+		<module>metron-profiler-common</module>
+    </modules>
 	<dependencies>
 		<dependency>
 			<groupId>junit</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec
----------------------------------------------------------------------
diff --git a/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec b/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec
index b0ef974..c2f4cc1 100644
--- a/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec
+++ b/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec
@@ -134,7 +134,7 @@ This package installs the Metron Parser files
 %{metron_home}/patterns/squid
 %{metron_home}/patterns/websphere
 %{metron_home}/patterns/yaf
-%attr(0644,root,root) %{metron_home}/lib/metron-parsers-%{full_version}.jar
+%attr(0644,root,root) %{metron_home}/lib/metron-parsers-%{full_version}-uber.jar
 
 # ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
@@ -155,7 +155,7 @@ This package installs the Metron Elasticsearch files
 %dir %{metron_home}/lib
 %{metron_home}/bin/start_elasticsearch_topology.sh
 %{metron_home}/config/elasticsearch.properties
-%attr(0644,root,root) %{metron_home}/lib/metron-elasticsearch-%{full_version}.jar
+%attr(0644,root,root) %{metron_home}/lib/metron-elasticsearch-%{full_version}-uber.jar
 
 # ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
@@ -201,7 +201,7 @@ This package installs the Metron Solr files
 %dir %{metron_home}/lib
 %{metron_home}/bin/start_solr_topology.sh
 %{metron_home}/config/solr.properties
-%attr(0644,root,root) %{metron_home}/lib/metron-solr-%{full_version}.jar
+%attr(0644,root,root) %{metron_home}/lib/metron-solr-%{full_version}-uber.jar
 
 # ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~\u2028\u2028
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-deployment/roles/metron_streaming/defaults/main.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/defaults/main.yml b/metron-deployment/roles/metron_streaming/defaults/main.yml
index b048bf0..549fae8 100644
--- a/metron-deployment/roles/metron_streaming/defaults/main.yml
+++ b/metron-deployment/roles/metron_streaming/defaults/main.yml
@@ -34,6 +34,7 @@ metron_elasticsearch_bundle_name: metron-elasticsearch-{{ metron_version }}-arch
 metron_parsers_bundle_name: metron-parsers-{{ metron_version }}-archive.tar.gz
 metron_maas_bundle_name: metron-maas-service-{{ metron_version }}-archive.tar.gz
 metron_profiler_bundle_name: metron-profiler-{{ metron_version }}-archive.tar.gz
+metron_profiler_client_bundle_name: metron-profiler-client-{{ metron_version }}-archive.tar.gz
 
 # bundle paths
 metron_pcap_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-pcap-backend/target/{{ metron_pcap_bundle_name }}"
@@ -46,6 +47,8 @@ metron_elasticsearch_bundle_path: "{{ playbook_dir }}/../../metron-platform/metr
 metron_parsers_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-parsers/target/{{ metron_parsers_bundle_name }}"
 metron_maas_bundle_path: "{{ playbook_dir }}/../../metron-analytics/metron-maas-service/target/{{ metron_maas_bundle_name }}"
 metron_profiler_bundle_path: "{{ playbook_dir }}/../../metron-analytics/metron-profiler/target/{{ metron_profiler_bundle_name }}"
+metron_profiler_client_bundle_path: "{{ playbook_dir }}/../../metron-analytics/metron-profiler-client/target/{{ metron_profiler_client_bundle_name }}"
+
 
 # configuration paths
 config_path: "{{ metron_directory }}/config"

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml
----------------------------------------------------------------------
diff --git a/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml b/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml
index ca972cb..be9b1d3 100644
--- a/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml
+++ b/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml
@@ -15,69 +15,38 @@
 #  limitations under the License.
 #
 ---
-- name: Copy Metron Solr bundle
+- name: Copy Metron bundles
   copy:
-    src: "{{ metron_solr_bundle_path }}"
-    dest: "{{ metron_directory }}"
-
-- name: Copy Metron Elasticsearch bundle
-  copy:
-    src: "{{ metron_elasticsearch_bundle_path }}"
-    dest: "{{ metron_directory }}"
-
-- name: Copy Metron Enrichment bundle
-  copy:
-    src: "{{ metron_enrichment_bundle_path }}"
-    dest: "{{ metron_directory }}"
-
-- name: Copy Metron Indexing bundle
-  copy:
-    src: "{{ metron_indexing_bundle_path }}"
-    dest: "{{ metron_directory }}"
-
-- name: Copy Metron Parsers bundle
-  copy:
-    src: "{{ metron_parsers_bundle_path }}"
-    dest: "{{ metron_directory }}"
-
-- name: Copy Metron DataLoads bundle
-  copy:
-    src: "{{ metron_data_management_bundle_path }}"
-    dest: "{{ metron_directory }}"
-
-- name: Copy Metron Common bundle
-  copy:
-    src: "{{ metron_common_bundle_path }}"
-    dest: "{{ metron_directory }}"
-
-- name: Copy Metron Pcap bundle
-  copy:
-    src: "{{ metron_pcap_bundle_path }}"
-    dest: "{{ metron_directory }}"
-
-- name: Copy Metron MaaS bundle
-  copy:
-    src: "{{ metron_maas_bundle_path }}"
-    dest: "{{ metron_directory }}"
-
-- name: Copy Metron Profiler bundle
-  copy:
-    src: "{{ metron_profiler_bundle_path }}"
+    src: "{{ item }}"
     dest: "{{ metron_directory }}"
+  with_items:
+    - "{{ metron_solr_bundle_path }}"
+    - "{{ metron_elasticsearch_bundle_path }}"
+    - "{{ metron_enrichment_bundle_path }}"
+    - "{{ metron_indexing_bundle_path }}"
+    - "{{ metron_parsers_bundle_path }}"
+    - "{{ metron_data_management_bundle_path }}"
+    - "{{ metron_common_bundle_path }}"
+    - "{{ metron_pcap_bundle_path }}"
+    - "{{ metron_maas_bundle_path }}"
+    - "{{ metron_profiler_bundle_path }}"
+    - "{{ metron_profiler_client_bundle_path }}"
 
 - name: Unbundle Metron bundles
   shell: "{{ item }}"
   args:
     chdir: "{{ metron_directory }}"
   with_items:
-    - tar xzvf metron-solr*.tar.gz
-    - tar xzvf metron-elasticsearch*.tar.gz
-    - tar xzvf metron-enrichment*.tar.gz
-    - tar xzvf metron-indexing*.tar.gz
-    - tar xzvf metron-parsers*.tar.gz
-    - tar xzvf metron-data-management*.tar.gz
-    - tar xzvf metron-common*.tar.gz
-    - tar xzvf metron-pcap-backend*.tar.gz
-    - tar xzvf metron-maas-service*.tar.gz
-    - tar xzvf metron-profiler*.tar.gz
-    - rm *.tar.gz
+    - "tar xzvf {{ metron_solr_bundle_name }}"
+    - "tar xzvf {{ metron_elasticsearch_bundle_name }}"
+    - "tar xzvf {{ metron_enrichment_bundle_name }}"
+    - "tar xzvf {{ metron_indexing_bundle_name }}"
+    - "tar xzvf {{ metron_parsers_bundle_name }}"
+    - "tar xzvf {{ metron_data_management_bundle_name }}"
+    - "tar xzvf {{ metron_common_bundle_name }}"
+    - "tar xzvf {{ metron_pcap_bundle_name }}"
+    - "tar xzvf {{ metron_maas_bundle_name }}"
+    - "tar xzvf {{ metron_profiler_bundle_name }}"
+    - "tar xvzf {{ metron_profiler_bundle_name }} "
+    - "tar xvzf {{ metron_profiler_client_bundle_name }} "
+    - rm *.tar.gz
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/pom.xml b/metron-platform/metron-elasticsearch/pom.xml
index f080516..4a3ce7a 100644
--- a/metron-platform/metron-elasticsearch/pom.xml
+++ b/metron-platform/metron-elasticsearch/pom.xml
@@ -249,6 +249,8 @@
                             <goal>shade</goal>
                         </goals>
                         <configuration>
+                            <shadedArtifactAttached>true</shadedArtifactAttached>
+                            <shadedClassifierName>uber</shadedClassifierName>
                             <relocations>
                                 <relocation>
                                     <pattern>com.google.common</pattern>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-elasticsearch/src/main/assembly/assembly.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/assembly/assembly.xml b/metron-platform/metron-elasticsearch/src/main/assembly/assembly.xml
index 1692f2e..27d5d0b 100644
--- a/metron-platform/metron-elasticsearch/src/main/assembly/assembly.xml
+++ b/metron-platform/metron-elasticsearch/src/main/assembly/assembly.xml
@@ -44,7 +44,7 @@
     <fileSet>
       <directory>${project.basedir}/target</directory>
       <includes>
-        <include>${project.artifactId}-${project.version}.jar</include>
+        <include>${project.artifactId}-${project.version}-uber.jar</include>
       </includes>
       <outputDirectory>/lib</outputDirectory>
       <useDefaultExcludes>true</useDefaultExcludes>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_topology.sh
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_topology.sh b/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_topology.sh
index 16f83d3..8ee7518 100755
--- a/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_topology.sh
+++ b/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_topology.sh
@@ -18,5 +18,5 @@
 #
 METRON_VERSION=${project.version}
 METRON_HOME=/usr/metron/$METRON_VERSION
-TOPOLOGY_JAR=${project.artifactId}-$METRON_VERSION.jar
+TOPOLOGY_JAR=${project.artifactId}-$METRON_VERSION-uber.jar
 storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.storm.flux.Flux --remote $METRON_HOME/flux/indexing/remote.yaml --filter $METRON_HOME/config/elasticsearch.properties

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-enrichment/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/pom.xml b/metron-platform/metron-enrichment/pom.xml
index a577e7a..b2c3a8c 100644
--- a/metron-platform/metron-enrichment/pom.xml
+++ b/metron-platform/metron-enrichment/pom.xml
@@ -46,6 +46,11 @@
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>metron-profiler-client</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
             <groupId>com.fasterxml.jackson.core</groupId>
             <artifactId>jackson-databind</artifactId>
             <version>${global_jackson_version}</version>
@@ -254,6 +259,14 @@
                                     <shadedPattern>org.apache.metron.jackson</shadedPattern>
                                 </relocation>
                             </relocations>
+                            <artifactSet>
+                                <excludes>
+                                    <exclude>storm:storm-core:*</exclude>
+                                    <exclude>storm:storm-lib:*</exclude>
+                                    <exclude>org.slf4j.impl*</exclude>
+                                    <exclude>org.slf4j:slf4j-log4j*</exclude>
+                                </excludes>
+                            </artifactSet>
                             <transformers>
                                 <transformer
                                         implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-indexing/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/pom.xml b/metron-platform/metron-indexing/pom.xml
index e5306e4..74aff80 100644
--- a/metron-platform/metron-indexing/pom.xml
+++ b/metron-platform/metron-indexing/pom.xml
@@ -211,6 +211,8 @@
                             <goal>shade</goal>
                         </goals>
                         <configuration>
+                            <shadedArtifactAttached>true</shadedArtifactAttached>
+                            <shadedClassifierName>uber</shadedClassifierName>
                             <relocations>
                                 <relocation>
                                     <pattern>com.google.common</pattern>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-platform/metron-management/src/main/java/org/apache/metron/management/GrokFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-management/src/main/java/org/apache/metron/management/GrokFunctions.java b/metron-platform/metron-management/src/main/java/org/apache/metron/management/GrokFunctions.java
index 2100ae5..d97832c 100644
--- a/metron-platform/metron-management/src/main/java/org/apache/metron/management/GrokFunctions.java
+++ b/metron-platform/metron-management/src/main/java/org/apache/metron/management/GrokFunctions.java
@@ -27,6 +27,7 @@ import org.apache.metron.common.dsl.ParseException;
 import org.apache.metron.common.dsl.Stellar;
 import org.apache.metron.common.dsl.StellarFunction;
 
+import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.StringReader;
 import java.util.*;
@@ -36,11 +37,17 @@ public class GrokFunctions {
   private static final Logger LOG = Logger.getLogger(GrokFunctions.class);
   private static Grok getGrok(String grokExpr) throws GrokException {
     Grok grok = new Grok();
-    grok.addPatternFromReader(new InputStreamReader(GrokFunctions.class.getResourceAsStream("/patterns/common")));
+
+    InputStream input = GrokFunctions.class.getResourceAsStream("/patterns/common");
+    if(input != null) {
+      grok.addPatternFromReader(new InputStreamReader(input));
+    }
+
     if(grokExpr != null) {
       grok.addPatternFromReader(new StringReader("pattern " + grokExpr));
       grok.compile("%{pattern}");
     }
+
     return grok;
   }
 



[3/3] incubator-metron git commit: METRON-421 Make Stellar Profiler Client API Accessible in Parser and Enrichment Topologies (nickwallen) closes apache/incubator-metron#290

Posted by ni...@apache.org.
METRON-421 Make Stellar Profiler Client API Accessible in Parser and Enrichment Topologies (nickwallen) closes apache/incubator-metron#290


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

Branch: refs/heads/master
Commit: 070473f5fc6ce723ccdb037e6dc6d172c579a0f7
Parents: 0e93169
Author: nickwallen <ni...@nickallen.org>
Authored: Tue Oct 11 10:39:45 2016 -0400
Committer: Nick Allen <ni...@nickallen.org>
Committed: Tue Oct 11 10:39:45 2016 -0400

----------------------------------------------------------------------
 .../metron-profiler-client/README.md            |  82 ++++++
 metron-analytics/metron-profiler-client/pom.xml | 159 +++---------
 .../profiler/client/stellar/GetProfile.java     |   8 +-
 metron-analytics/metron-profiler-common/pom.xml |  53 ++++
 .../metron/profiler/ProfileMeasurement.java     | 134 ++++++++++
 .../apache/metron/profiler/ProfilePeriod.java   |  99 +++++++
 .../metron/profiler/hbase/ColumnBuilder.java    |  50 ++++
 .../metron/profiler/hbase/RowKeyBuilder.java    |  60 +++++
 .../profiler/hbase/SaltyRowKeyBuilder.java      | 204 +++++++++++++++
 .../profiler/hbase/ValueOnlyColumnBuilder.java  |  76 ++++++
 .../stellar/DefaultStellarExecutor.java         | 144 ++++++++++
 .../profiler/stellar/StellarExecutor.java       |  74 ++++++
 .../src/main/resources/META-INF/LICENSE         | 234 +++++++++++++++++
 .../metron/profiler/ProfilePeriodTest.java      | 121 +++++++++
 .../profiler/hbase/SaltyRowKeyBuilderTest.java  | 260 +++++++++++++++++++
 .../stellar/DefaultStellarExecutorTest.java     | 181 +++++++++++++
 .../src/test/resources/log4j.properties         |  28 ++
 metron-analytics/metron-profiler/README.md      |  20 +-
 metron-analytics/metron-profiler/pom.xml        |  63 ++---
 .../src/main/assembly/assembly.xml              |   2 +-
 .../metron/profiler/ProfileMeasurement.java     | 134 ----------
 .../apache/metron/profiler/ProfilePeriod.java   |  99 -------
 .../metron/profiler/hbase/ColumnBuilder.java    |  50 ----
 .../metron/profiler/hbase/RowKeyBuilder.java    |  60 -----
 .../profiler/hbase/SaltyRowKeyBuilder.java      | 214 ---------------
 .../profiler/hbase/ValueOnlyColumnBuilder.java  |  76 ------
 .../stellar/DefaultStellarExecutor.java         | 144 ----------
 .../profiler/stellar/StellarExecutor.java       |  74 ------
 .../src/main/scripts/start_profiler_topology.sh |   2 +-
 .../metron/profiler/ProfilePeriodTest.java      | 120 ---------
 .../profiler/bolt/ProfileBuilderBoltTest.java   |   1 -
 .../profiler/hbase/SaltyRowKeyBuilderTest.java  | 259 ------------------
 .../stellar/DefaultStellarExecutorTest.java     | 180 -------------
 metron-analytics/pom.xml                        |   3 +-
 .../docker/rpm-docker/SPECS/metron.spec         |   6 +-
 .../roles/metron_streaming/defaults/main.yml    |   3 +
 .../metron_streaming/tasks/copy_bundles.yml     |  85 ++----
 metron-platform/metron-elasticsearch/pom.xml    |   2 +
 .../src/main/assembly/assembly.xml              |   2 +-
 .../scripts/start_elasticsearch_topology.sh     |   2 +-
 metron-platform/metron-enrichment/pom.xml       |  13 +
 metron-platform/metron-indexing/pom.xml         |   2 +
 .../apache/metron/management/GrokFunctions.java |   9 +-
 metron-platform/metron-parsers/pom.xml          |  38 ++-
 .../src/main/assembly/assembly.xml              |   2 +-
 .../src/main/scripts/start_parser_topology.sh   |   2 +-
 metron-platform/metron-pcap-backend/pom.xml     |   2 -
 metron-platform/metron-solr/pom.xml             |   9 +-
 .../metron-solr/src/main/assembly/assembly.xml  |   2 +-
 .../src/main/scripts/start_solr_topology.sh     |   2 +-
 50 files changed, 1975 insertions(+), 1674 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-client/README.md
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/README.md b/metron-analytics/metron-profiler-client/README.md
new file mode 100644
index 0000000..ff2ba0c
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/README.md
@@ -0,0 +1,82 @@
+# Metron Profiler Client
+
+This project provides a client API for accessing the profiles generated by the [Metron Profiler](..//metron-profiler).  This includes both a Java API and Stellar API for accessing the profile data.  The primary use case is to extract profile data for use during model scoring.
+
+## Stellar Client API
+
+The following are usage examples that show how the Stellar API can be used to read profiles generated by the [Metron Profiler](../metron-profiler).  This API would be used in conjunction with other Stellar functions like [`MAAS_MODEL_APPLY`](../../metron-platform/metron-common) to perform model scoring on streaming data.
+
+These examples assume a profile has been defined called 'snort-alerts' that tracks the number of Snort alerts associated with an IP address over time.  The profile definition might look similar to the following.
+
+```
+{
+  "profiles": [
+    {
+      "profile": "snort-alerts",
+      "foreach": "ip_src_addr",
+      "onlyif":  "source.type == 'snort'",
+      "update":  { "s": "STATS_ADD(s, 1)" },
+      "result":  "STATS_MEAN(s)"
+    }
+  ]
+}
+```
+
+During model scoring the entity being scored, in this case a particular IP address, will be known.  The following examples highlight how this profile data might be retrieved.
+
+Retrieve all values of 'snort-alerts' from '10.0.0.1' over the past 4 hours.
+```
+PROFILE_GET('snort-alerts', '10.0.0.1', 4, 'HOURS')
+```
+
+Retrieve all values of 'snort-alerts' from '10.0.0.1' over the past 2 days.
+```
+PROFILE_GET('snort-alerts', '10.0.0.1', 2, 'DAYS')
+```
+
+If the profile had been defined to group the data by weekday versus weekend, then the following example would apply.
+
+Retrieve all values of 'snort-alerts' from '10.0.0.1' that occurred on 'weekdays' over the past month.
+```
+PROFILE_GET('snort-alerts', '10.0.0.1', 1, 'MONTHS', 'weekdays')
+```
+
+### Period Duration
+
+*WARNING*
+
+By default, the Profiler creates Profiles with a period duration of 15 minutes.  This means that data is accumulated, summarized and flushed every 15 minutes.  The Client API must also have knowledge of this duration to correctly retrieve the profile data.  If the client API is expected 15 minute periods, it will not be able to read data generated by a Profiler that has been configured with a 1 hour period.
+
+The period duration can be configured in the Profiler by altering the Profiler topology's static properties file.  The Stellar Client API currently provides no means to configure the period duration and defaults also to 15 minutes.  This means that the Stellar Client API can only read profiles with a period duration of 15 minutes.  This is a known limitation that has not yet been addressed.
+
+## Getting Started
+
+These instructions step through the process of using the Stellar Client API on a live cluster.  These instructions assume that the 'Getting Started' instructions included with the [Metron Profiler](../metron-profiler) have been followed.  This will create a Profile called 'test' whose data will be retrieved with the Stellar Client API.
+
+To validate that everything is working, login to the server hosting Metron.  We will use the Stellar Shell to replicate the execution environment of Stellar running in a Storm topology, like Metron's Parser or Enrichment topology.  Replace 'node1:2181' with the URL to a Zookeeper Broker.  
+
+```
+[root@node1 0.2.1BETA]# bin/stellar -z node1:2181
+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]>>> ?PROFILE_GET
+Functions loaded, you may refer to functions now...
+PROFILE_GET
+Description: Retrieves a series of values from a stored profile.
+
+Arguments:
+	profile - The name of the profile.
+	entity - The name of the entity.
+	durationAgo - How long ago should values be retrieved from?
+	units - The units of 'durationAgo'.
+	groups - Optional - The groups used to sort the profile.
+
+Returns: The profile measurements.
+
+[Stellar]>>> PROFILE_GET('test','192.168.138.158', 1, 'HOURS')
+[12078.0, 8921.0, 12131.0]
+```
+
+The client API call above has retrieved the past hour of the 'test' profile for the entity '192.168.138.158'.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-client/pom.xml
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/pom.xml b/metron-analytics/metron-profiler-client/pom.xml
index eec190d..47989db 100644
--- a/metron-analytics/metron-profiler-client/pom.xml
+++ b/metron-analytics/metron-profiler-client/pom.xml
@@ -28,14 +28,8 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.metron</groupId>
-            <artifactId>metron-profiler</artifactId>
+            <artifactId>metron-profiler-common</artifactId>
             <version>${project.parent.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>com.google.guava</groupId>
-                    <artifactId>guava</artifactId>
-                </exclusion>
-            </exclusions>
         </dependency>
         <dependency>
             <groupId>org.apache.metron</groupId>
@@ -43,58 +37,10 @@
             <version>${project.parent.version}</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.metron</groupId>
-            <artifactId>metron-test-utilities</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>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-client</artifactId>
-            <version>${global_hadoop_version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-common</artifactId>
-            <classifier>tests</classifier>
-            <version>${global_hadoop_version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-hdfs</artifactId>
-            <classifier>tests</classifier>
-            <version>${global_hadoop_version}</version>
-        </dependency>
-        <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-client</artifactId>
             <version>${global_hbase_version}</version>
             <scope>provided</scope>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-log4j12</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.google.guava</groupId>
-                    <artifactId>guava</artifactId>
-                </exclusion>
-            </exclusions>
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
@@ -134,6 +80,12 @@
             <version>${global_mockito_version}</version>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>metron-integration-test</artifactId>
+            <version>${project.parent.version}</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
     <reporting>
         <plugins>
@@ -147,67 +99,31 @@
                     <dependencyLocationsEnabled>false</dependencyLocationsEnabled>
                 </configuration>
             </plugin>
-
             <plugin>
                 <groupId>org.codehaus.mojo</groupId>
                 <artifactId>emma-maven-plugin</artifactId>
                 <version>1.0-alpha-3</version>
-                <inherited>true</inherited>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-pmd-plugin</artifactId>
+                <configuration>
+                    <targetJdk>${global_java_version}</targetJdk>
+                </configuration>
             </plugin>
         </plugins>
     </reporting>
-
     <build>
         <plugins>
             <plugin>
-                <!-- Separates the unit tests from the integration tests. -->
                 <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-surefire-plugin</artifactId>
-                <version>2.12.4</version>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>3.1</version>
                 <configuration>
-                    <!-- Skip the default running of this plug-in (or everything is run twice...see below) -->
-                    <argLine>-Xmx2048m</argLine>
-                    <skip>true</skip>
-                    <!-- Show 100% of the lines from the stack trace (doesn't work) -->
-                    <trimStackTrace>false</trimStackTrace>
+                    <source>${global_java_version}</source>
+                    <compilerArgument>-Xlint:unchecked</compilerArgument>
+                    <target>${global_java_version}</target>
                 </configuration>
-                <executions>
-                    <execution>
-                        <id>unit-tests</id>
-                        <phase>test</phase>
-                        <goals>
-                            <goal>test</goal>
-                        </goals>
-                        <configuration>
-                            <!-- Never skip running the tests when the test phase is invoked -->
-                            <skip>false</skip>
-                            <includes>
-                                <!-- Include unit tests within integration-test phase. -->
-                                <include>**/*Test.java</include>
-                            </includes>
-                            <excludes>
-                                <!-- Exclude integration tests within (unit) test phase. -->
-                                <exclude>**/*IntegrationTest.java</exclude>
-                            </excludes>
-
-                        </configuration>
-                    </execution>
-                    <execution>
-                        <id>integration-tests</id>
-                        <phase>integration-test</phase>
-                        <goals>
-                            <goal>test</goal>
-                        </goals>
-                        <configuration>
-                            <!-- Never skip running the tests when the integration-test phase is invoked -->
-                            <skip>false</skip>
-                            <includes>
-                                <!-- Include integration tests within integration-test phase. -->
-                                <include>**/*IntegrationTest.java</include>
-                            </includes>
-                        </configuration>
-                    </execution>
-                </executions>
             </plugin>
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
@@ -215,6 +131,11 @@
                 <version>${global_shade_version}</version>
                 <configuration>
                     <createDependencyReducedPom>true</createDependencyReducedPom>
+                    <artifactSet>
+                        <excludes>
+                            <exclude>*slf4j*</exclude>
+                        </excludes>
+                    </artifactSet>
                 </configuration>
                 <executions>
                     <execution>
@@ -225,35 +146,16 @@
                         <configuration>
                             <relocations>
                                 <relocation>
-                                    <pattern>com.lmax</pattern>
-                                    <shadedPattern>org.apache.metron.lmax.metron-profiler</shadedPattern>
-                                </relocation>
-                                <relocation>
                                     <pattern>com.google.common</pattern>
-                                    <shadedPattern>org.apache.metron.guava.metron-profiler</shadedPattern>
-                                </relocation>
-                                <relocation>
-                                    <pattern>com.fasterxml.jackson.core</pattern>
-                                    <shadedPattern>com.fasterxml.jackson.core.metron.elasticsearch</shadedPattern>
+                                    <shadedPattern>org.apache.metron.guava</shadedPattern>
                                 </relocation>
                             </relocations>
-                            <artifactSet>
-                                <excludes>
-                                    <exclude>storm:storm-core:*</exclude>
-                                    <exclude>storm:storm-lib:*</exclude>
-                                    <exclude>org.slf4j.impl*</exclude>
-                                    <exclude>org.slf4j:slf4j-log4j*</exclude>
-                                </excludes>
-                            </artifactSet>
                             <transformers>
-                                <transformer
-                                        implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
                                     <resource>.yaml</resource>
                                 </transformer>
-                                <transformer
-                                        implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
-                                <transformer
-                                        implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
                                     <mainClass></mainClass>
                                 </transformer>
                             </transformers>
@@ -277,5 +179,10 @@
                 </executions>
             </plugin>
         </plugins>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+            </resource>
+        </resources>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/GetProfile.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/GetProfile.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/GetProfile.java
index 20546a2..f4b3a8d 100644
--- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/GetProfile.java
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/GetProfile.java
@@ -226,7 +226,13 @@ public class GetProfile implements StellarFunction {
    * @param global The global configuration.
    */
   private RowKeyBuilder getRowKeyBuilder(Map<String, Object> global) {
-    // the builder is not currently configurable - but should be made so
+    /*
+     * WARNING: the row key builder is not currently configurable.  by invoking
+     * the default constructor below, this defaults to generating keys using a
+     * period duration of 15 minutes. this function will NOT be able to read
+     * profiles created by a profiler running with any other period duration, but
+     * 15 minutes.
+     */
     return new SaltyRowKeyBuilder();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/pom.xml
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/pom.xml b/metron-analytics/metron-profiler-common/pom.xml
new file mode 100644
index 0000000..cb54b81
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/pom.xml
@@ -0,0 +1,53 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+<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>
+    <parent>
+        <artifactId>metron-analytics</artifactId>
+        <groupId>org.apache.metron</groupId>
+        <version>0.2.1BETA</version>
+    </parent>
+    <artifactId>metron-profiler-common</artifactId>
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+    </properties>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>metron-hbase</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <version>${global_hbase_version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-core</artifactId>
+            <version>${global_storm_version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <version>${global_mockito_version}</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java
new file mode 100644
index 0000000..210b92f
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java
@@ -0,0 +1,134 @@
+/*
+ *
+ *  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.profiler;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Represents a single data point within a Profile.
+ *
+ * A Profile is effectively a time series.  To this end a Profile is composed
+ * of many ProfileMeasurement values which in aggregate form a time series.
+ */
+public class ProfileMeasurement {
+
+  /**
+   * The name of the profile that this measurement is associated with.
+   */
+  private String profileName;
+
+  /**
+   * The name of the entity being profiled.
+   */
+  private String entity;
+
+  /**
+   * The actual measurement itself.
+   */
+  private Object value;
+
+  /**
+   * A set of expressions used to group the profile measurements when persisted.
+   */
+  private List<String> groupBy;
+
+  /**
+   * The period in which the ProfileMeasurement was taken.
+   */
+  private ProfilePeriod period;
+
+  /**
+   * @param profileName The name of the profile.
+   * @param entity The name of the entity being profiled.
+   * @param whenMillis When the measurement was taken in epoch milliseconds.
+   * @param periodDuration The duration of each profile period.
+   * @param periodUnits The units of the duration of each profile period.
+   */
+  public ProfileMeasurement(String profileName, String entity, long whenMillis, long periodDuration, TimeUnit periodUnits) {
+    this.profileName = profileName;
+    this.entity = entity;
+    this.period = new ProfilePeriod(whenMillis, periodDuration, periodUnits);
+  }
+
+  public String getProfileName() {
+    return profileName;
+  }
+
+  public String getEntity() {
+    return entity;
+  }
+
+  public Object getValue() {
+    return value;
+  }
+
+  public ProfilePeriod getPeriod() {
+    return period;
+  }
+
+  public List<String> getGroupBy() {
+    return groupBy;
+  }
+
+  public void setValue(Object value) {
+    this.value = value;
+  }
+
+  public void setGroupBy(List<String> groupBy) {
+    this.groupBy = groupBy;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    ProfileMeasurement that = (ProfileMeasurement) o;
+    if (profileName != null ? !profileName.equals(that.profileName) : that.profileName != null) return false;
+    if (entity != null ? !entity.equals(that.entity) : that.entity != null) return false;
+    if (value != null ? !value.equals(that.value) : that.value != null) return false;
+    if (groupBy != null ? !groupBy.equals(that.groupBy) : that.groupBy != null) return false;
+    return period != null ? period.equals(that.period) : that.period == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = profileName != null ? profileName.hashCode() : 0;
+    result = 31 * result + (entity != null ? entity.hashCode() : 0);
+    result = 31 * result + (value != null ? value.hashCode() : 0);
+    result = 31 * result + (groupBy != null ? groupBy.hashCode() : 0);
+    result = 31 * result + (period != null ? period.hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "ProfileMeasurement{" +
+            "profileName='" + profileName + '\'' +
+            ", entity='" + entity + '\'' +
+            ", value=" + value +
+            ", groupBy=" + groupBy +
+            ", period=" + period +
+            '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java
new file mode 100644
index 0000000..1b8efc8
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java
@@ -0,0 +1,99 @@
+/*
+ *
+ *  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.profiler;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * The Profiler captures a ProfileMeasurement once every ProfilePeriod.  There can be
+ * multiple ProfilePeriods every hour.
+ */
+public class ProfilePeriod {
+
+  /**
+   * A monotonically increasing number identifying the period.  The first period is 0
+   * and began at the epoch.
+   */
+  private long period;
+
+  /**
+   * The duration of each period in milliseconds.
+   */
+  private long durationMillis;
+
+  /**
+   * @param epochMillis A timestamp contained somewhere within the profile period.
+   * @param duration The duration of each profile period.
+   * @param units The units of the duration; hours, minutes, etc.
+   */
+  public ProfilePeriod(long epochMillis, long duration, TimeUnit units) {
+    this.durationMillis = units.toMillis(duration);
+    this.period = epochMillis / durationMillis;
+  }
+
+  /**
+   * When this period started in milliseconds since the epoch.
+   */
+  public long getStartTimeMillis() {
+    return period * durationMillis;
+  }
+
+  /**
+   * Returns the next ProfilePeriod in time.
+   */
+  public ProfilePeriod next() {
+    long nextStart = getStartTimeMillis() + durationMillis;
+    return new ProfilePeriod(nextStart, durationMillis, TimeUnit.MILLISECONDS);
+  }
+
+  public long getPeriod() {
+    return period;
+  }
+
+  public long getDurationMillis() {
+    return durationMillis;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    ProfilePeriod that = (ProfilePeriod) o;
+    if (period != that.period) return false;
+    return durationMillis == that.durationMillis;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = (int) (period ^ (period >>> 32));
+    result = 31 * result + (int) (durationMillis ^ (durationMillis >>> 32));
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "ProfilePeriod{" +
+            "period=" + period +
+            ", durationMillis=" + durationMillis +
+            '}';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java
new file mode 100644
index 0000000..c645822
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java
@@ -0,0 +1,50 @@
+/*
+ *
+ *  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.profiler.hbase;
+
+import org.apache.metron.hbase.bolt.mapper.ColumnList;
+import org.apache.metron.profiler.ProfileMeasurement;
+
+import java.io.Serializable;
+
+/**
+ * Defines how fields in a ProfileMeasurement will be mapped to columns in HBase.
+ */
+public interface ColumnBuilder extends Serializable {
+
+  /**
+   * Generate the columns used to store a ProfileMeasurement.
+   * @param measurement The profile measurement.
+   */
+  ColumnList columns(ProfileMeasurement measurement);
+
+  /**
+   * Returns the column family used to store the ProfileMeasurement values.
+   * @return
+   */
+  String getColumnFamily();
+
+  /**
+   * Returns the column qualifiers for the given field of a ProfileMeasurement.
+   * @return The column qualifier used to store a ProfileMeasurement's field in HBase.
+   */
+  byte[] getColumnQualifier(String fieldName);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java
new file mode 100644
index 0000000..1ce4906
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java
@@ -0,0 +1,60 @@
+/*
+ *
+ *  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.profiler.hbase;
+
+import org.apache.metron.profiler.ProfileMeasurement;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Builds a row key that can be used to read or write ProfileMeasurement data
+ * to HBase.
+ */
+public interface RowKeyBuilder extends Serializable {
+
+  /**
+   * Build a row key for a given ProfileMeasurement.
+   * 
+   * This method is useful when writing ProfileMeasurements to HBase.
+   *
+   * @param measurement The profile measurement.
+   * @param groups      The groups used to sort the profile data.
+   * @return The HBase row key.
+   */
+  byte[] rowKey(ProfileMeasurement measurement, List<Object> groups);
+
+  /**
+   * Builds a list of row keys necessary to retrieve a profile's measurements over
+   * a time horizon.
+   *
+   * This method is useful when attempting to read ProfileMeasurements stored in HBase.
+   *
+   * @param profile The name of the profile.
+   * @param entity The name of the entity.
+   * @param groups The group(s) used to sort the profile data.
+   * @param start When the time horizon starts in epoch milliseconds.
+   * @param end When the time horizon ends in epoch milliseconds.
+   * @return All of the row keys necessary to retrieve the profile measurements.
+   */
+  List<byte[]> rowKeys(String profile, String entity, List<Object> groups, long start, long end);
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java
new file mode 100644
index 0000000..ee54a1a
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java
@@ -0,0 +1,204 @@
+/*
+ *
+ *  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.profiler.hbase;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.profiler.ProfileMeasurement;
+import org.apache.metron.profiler.ProfilePeriod;
+
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A RowKeyBuilder that uses a salt to prevent hot-spotting.
+ *
+ * Responsible for building the row keys used to store profile data in HBase.  The row key is composed of the following
+ * fields in the given order.
+ * <ul>
+ * <li>salt - A salt that helps prevent hot-spotting.
+ * <li>profile - The name of the profile.
+ * <li>entity - The name of the entity being profiled.
+ * <li>group(s) - The group(s) used to sort the data in HBase. For example, a group may distinguish between weekends and weekdays.
+ * <li>year - The year based on UTC.
+ * <li>day of year - The current day within the year based on UTC; [1, 366]
+ * <li>hour - The hour within the day based on UTC; [0, 23]
+ * </ul>period - The period within the hour.  The number of periods per hour can be defined by the user; defaults to 4.
+ */
+public class SaltyRowKeyBuilder implements RowKeyBuilder {
+
+  /**
+   * A salt can be prepended to the row key to help prevent hot-spotting.  The salt
+   * divisor is used to generate the salt.  The salt divisor should be roughly equal
+   * to the number of nodes in the Hbase cluster.
+   */
+  private int saltDivisor;
+
+  /**
+   * The duration of each profile period in milliseconds.
+   */
+  private long periodDurationMillis;
+
+  public SaltyRowKeyBuilder() {
+    this.saltDivisor = 1000;
+    this.periodDurationMillis = TimeUnit.MINUTES.toMillis(15);
+  }
+
+  public SaltyRowKeyBuilder(int saltDivisor, long duration, TimeUnit units) {
+    this.saltDivisor = saltDivisor;
+    this.periodDurationMillis = units.toMillis(duration);
+  }
+
+  /**
+   * Builds a list of row keys necessary to retrieve profile measurements over
+   * a time horizon.
+   *
+   * @param profile The name of the profile.
+   * @param entity The name of the entity.
+   * @param groups The group(s) used to sort the profile data.
+   * @param start When the time horizon starts in epoch milliseconds.
+   * @param end When the time horizon ends in epoch milliseconds.
+   * @return All of the row keys necessary to retrieve the profile measurements.
+   */
+  @Override
+  public List<byte[]> rowKeys(String profile, String entity, List<Object> groups, long start, long end) {
+    List<byte[]> rowKeys = new ArrayList<>();
+
+    // be forgiving of out-of-order start and end times; order is critical to this algorithm
+    end = Math.max(start, end);
+    start = Math.min(start, end);
+
+    // find the starting period and advance until the end time is reached
+    ProfilePeriod period = new ProfilePeriod(start, periodDurationMillis, TimeUnit.MILLISECONDS);
+    while(period.getStartTimeMillis() <= end) {
+
+      byte[] k = rowKey(profile, entity, period, groups);
+      rowKeys.add(k);
+
+      // advance to the next period
+      period = period.next();
+    }
+
+    return rowKeys;
+  }
+
+  /**
+   * Builds the row key for a given profile measurement.
+   * @param m The profile measurement.
+   * @param groups The groups used to sort the profile data.
+   * @return The HBase row key.
+   */
+  @Override
+  public byte[] rowKey(ProfileMeasurement m, List<Object> groups) {
+    return rowKey(m.getProfileName(), m.getEntity(), m.getPeriod(), groups);
+  }
+
+  public void withPeriodDuration(long duration, TimeUnit units) {
+    periodDurationMillis = units.toMillis(duration);
+  }
+
+  public void setSaltDivisor(int saltDivisor) {
+    this.saltDivisor = saltDivisor;
+  }
+
+  /**
+   * Build the row key.
+   * @param profile The name of the profile.
+   * @param entity The name of the entity.
+   * @param period The period in which the measurement was taken.
+   * @param groups The groups.
+   * @return The HBase row key.
+   */
+  public byte[] rowKey(String profile, String entity, ProfilePeriod period, List<Object> groups) {
+
+    // row key = salt + prefix + group(s) + time
+    byte[] salt = getSalt(period, saltDivisor);
+    byte[] prefixKey = prefixKey(profile, entity);
+    byte[] groupKey = groupKey(groups);
+    byte[] timeKey = timeKey(period);
+
+    int capacity = salt.length + prefixKey.length + groupKey.length + timeKey.length;
+    return ByteBuffer
+            .allocate(capacity)
+            .put(salt)
+            .put(prefixKey)
+            .put(groupKey)
+            .put(timeKey)
+            .array();
+  }
+
+  /**
+   * Builds the 'prefix' component of the row key.
+   * @param profile The name of the profile.
+   * @param entity The name of the entity.
+   */
+  private static byte[] prefixKey(String profile, String entity) {
+    byte[] profileBytes = Bytes.toBytes(profile);
+    byte[] entityBytes = Bytes.toBytes(entity);
+    return ByteBuffer
+            .allocate(profileBytes.length + entityBytes.length)
+            .put(profileBytes)
+            .put(entityBytes)
+            .array();
+  }
+
+  /**
+   * Builds the 'group' component of the row key.
+   * @param groups The groups to include in the row key.
+   */
+  private static byte[] groupKey(List<Object> groups) {
+    StringBuilder builder = new StringBuilder();
+    groups.forEach(g -> builder.append(g));
+    return Bytes.toBytes(builder.toString());
+  }
+
+  /**
+   * Builds the 'time' portion of the row key
+   * @param period The ProfilePeriod in which the ProfileMeasurement was taken.
+   */
+  private static byte[] timeKey(ProfilePeriod period) {
+    return Bytes.toBytes(period.getPeriod());
+  }
+
+  /**
+   * Calculates a salt value that is used as part of the row key.
+   *
+   * The salt is calculated as 'md5(timestamp) % N' where N is a configurable value that ideally
+   * is close to the number of nodes in the Hbase cluster.
+   *
+   * @param period The period in which a profile measurement is taken.
+   */
+  public static byte[] getSalt(ProfilePeriod period, int saltDivisor) {
+    try {
+      // an MD5 is 16 bytes aka 128 bits
+      MessageDigest digest = MessageDigest.getInstance("MD5");
+      byte[] hash = digest.digest(timeKey(period));
+      int salt = Bytes.toShort(hash) % saltDivisor;
+      return Bytes.toBytes(salt);
+
+    } catch(NoSuchAlgorithmException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java
new file mode 100644
index 0000000..bb1baf6
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java
@@ -0,0 +1,76 @@
+/*
+ *
+ *  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.profiler.hbase;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.common.utils.SerDeUtils;
+import org.apache.metron.profiler.ProfileMeasurement;
+import org.apache.metron.hbase.bolt.mapper.ColumnList;
+
+/**
+ * A ColumnBuilder that writes only the value of a ProfileMeasurement.
+ */
+public class ValueOnlyColumnBuilder implements ColumnBuilder {
+
+  /**
+   * The column family storing the profile data.
+   */
+  private String columnFamily;
+
+  private byte[] columnFamilyBytes;
+
+  public ValueOnlyColumnBuilder() {
+    setColumnFamily("P");
+  }
+
+  public ValueOnlyColumnBuilder(String columnFamily) {
+    setColumnFamily(columnFamily);
+  }
+
+  @Override
+  public ColumnList columns(ProfileMeasurement measurement) {
+
+    ColumnList cols = new ColumnList();
+    cols.addColumn(columnFamilyBytes, getColumnQualifier("value"), SerDeUtils.toBytes(measurement.getValue()));
+
+    return cols;
+  }
+
+  @Override
+  public String getColumnFamily() {
+    return this.columnFamily;
+  }
+
+  public void setColumnFamily(String columnFamily) {
+    this.columnFamily = columnFamily;
+    this.columnFamilyBytes = Bytes.toBytes(columnFamily);
+  }
+
+  @Override
+  public byte[] getColumnQualifier(String fieldName) {
+
+    if("value".equals(fieldName)) {
+      return Bytes.toBytes("value");
+    }
+
+    throw new IllegalArgumentException(("unexpected field name: " + fieldName));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java
new file mode 100644
index 0000000..474ac73
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java
@@ -0,0 +1,144 @@
+/*
+ *
+ *  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.profiler.stellar;
+
+import org.apache.metron.common.dsl.Context;
+import org.apache.metron.common.dsl.FunctionResolver;
+import org.apache.metron.common.dsl.MapVariableResolver;
+import org.apache.metron.common.dsl.ParseException;
+import org.apache.metron.common.dsl.StellarFunctions;
+import org.apache.metron.common.dsl.VariableResolver;
+import org.apache.metron.common.stellar.StellarProcessor;
+import org.apache.metron.common.utils.ConversionUtils;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * The default implementation of a StellarExecutor.
+ */
+public class DefaultStellarExecutor implements StellarExecutor, Serializable {
+
+  /**
+   * The current state of the Stellar execution environment.
+   */
+  private Map<String, Object> state;
+
+  /**
+   * Provides additional context for initializing certain Stellar functions.  For
+   * example, references to Zookeeper or HBase.
+   */
+  private Context context;
+
+  public DefaultStellarExecutor() {
+    clearState();
+    context = Context.EMPTY_CONTEXT();
+  }
+
+  /**
+   * @param initialState Initial state loaded into the execution environment.
+   */
+  public DefaultStellarExecutor(Map<String, Object> initialState) {
+    this();
+    this.state = new HashMap<>(initialState);
+  }
+
+  /**
+   * The current state of the Stellar execution environment.
+   */
+  @Override
+  public Map<String, Object> getState() {
+    return new HashMap<>(state);
+  }
+
+  /**
+   * Execute an expression and assign the result to a variable.  The variable is maintained
+   * in the context of this executor and is available to all subsequent expressions.
+   *
+   * @param variable       The name of the variable to assign to.
+   * @param expression     The expression to execute.
+   * @param transientState Additional state available to the expression.  This most often represents
+   *                       the values available to the expression from an individual message. The state
+   *                       maps a variable name to a variable's value.
+   */
+  @Override
+  public void assign(String variable, String expression, Map<String, Object> transientState) {
+    Object result = execute(expression, transientState);
+    state.put(variable, result);
+  }
+
+  /**
+   * Execute a Stellar expression and return the result.  The internal state of the executor
+   * is not modified.
+   *
+   * @param expression The expression to execute.
+   * @param state      Additional state available to the expression.  This most often represents
+   *                   the values available to the expression from an individual message. The state
+   *                   maps a variable name to a variable's value.
+   * @param clazz      The expected type of the expression's result.
+   * @param <T>        The expected type of the expression's result.
+   */
+  @Override
+  public <T> T execute(String expression, Map<String, Object> state, Class<T> clazz) {
+    Object resultObject = execute(expression, state);
+
+    // perform type conversion, if necessary
+    T result = ConversionUtils.convert(resultObject, clazz);
+    if (result == null) {
+      throw new IllegalArgumentException(String.format("Unexpected type: expected=%s, actual=%s, expression=%s",
+              clazz.getSimpleName(), resultObject.getClass().getSimpleName(), expression));
+    }
+
+    return result;
+  }
+
+  @Override
+  public void clearState() {
+    this.state = new HashMap<>();
+  }
+
+  /**
+   * Sets the Context for the Stellar execution environment.  This provides global data used
+   * to initialize Stellar functions.
+   *
+   * @param context The Stellar context.
+   */
+  @Override
+  public void setContext(Context context) {
+    this.context = context;
+  }
+
+  /**
+   * Execute a Stellar expression.
+   *
+   * @param expression     The expression to execute.
+   * @param transientState Additional state available to the expression.  This most often represents
+   *                       the values available to the expression from an individual message. The state
+   *                       maps a variable name to a variable's value.
+   */
+  private Object execute(String expression, Map<String, Object> transientState) {
+    FunctionResolver functionResolver = StellarFunctions.FUNCTION_RESOLVER();
+    VariableResolver variableResolver = new MapVariableResolver(state, transientState);
+    StellarProcessor processor = new StellarProcessor();
+    return processor.parse(expression, variableResolver, functionResolver, context);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java
new file mode 100644
index 0000000..869db42
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java
@@ -0,0 +1,74 @@
+/*
+ *
+ *  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.profiler.stellar;
+
+import org.apache.metron.common.dsl.Context;
+
+import java.util.Map;
+
+/**
+ * Executes Stellar expressions and maintains state across multiple invocations.
+ */
+public interface StellarExecutor {
+
+  /**
+   * Execute an expression and assign the result to a variable.  The variable is maintained
+   * in the context of this executor and is available to all subsequent expressions.
+   *
+   * @param variable   The name of the variable to assign to.
+   * @param expression The expression to execute.
+   * @param state      Additional state available to the expression.  This most often represents
+   *                   the values available to the expression from an individual message. The state
+   *                   maps a variable name to a variable's value.
+   */
+  void assign(String variable, String expression, Map<String, Object> state);
+
+  /**
+   * Execute a Stellar expression and return the result.  The internal state of the executor
+   * is not modified.
+   *
+   * @param expression The expression to execute.
+   * @param state      Additional state available to the expression.  This most often represents
+   *                   the values available to the expression from an individual message. The state
+   *                   maps a variable name to a variable's value.
+   * @param clazz      The expected type of the expression's result.
+   * @param <T>        The expected type of the expression's result.
+   */
+  <T> T execute(String expression, Map<String, Object> state, Class<T> clazz);
+
+  /**
+   * The current state of the Stellar execution environment.
+   */
+  Map<String, Object> getState();
+
+  /**
+   * Removes all state from the execution environment.
+   */
+  void clearState();
+
+  /**
+   * Sets the Context for the Stellar execution environment.  This provides global data used
+   * to initialize Stellar functions.
+   *
+   * @param context The Stellar context.
+   */
+  void setContext(Context context);
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/main/resources/META-INF/LICENSE b/metron-analytics/metron-profiler-common/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000..e5616e7
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,234 @@
+Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "{}"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright {yyyy} {name of copyright owner}
+
+   Licensed 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.
+
+------------------------------------------------------------------------------------
+
+
+
+This product bundles asm 3.1, which is available under a "BSD Software License" license.  For details, see http://asm.ow2.org/
+This product bundles kryo 2.21, which is available under a "BSD Software License" license.  For details, see http://code.google.com/p/kryo/
+This product bundles minlog 1.2, which is available under a "BSD Software License" license.  For details, see http://code.google.com/p/minlog/
+This product bundles reflectasm shaded, which is available under a "BSD Software License" license.  For details, see https://github.com/EsotericSoftware/reflectasm
+This product bundles protobuf-java 2.5.0, which is available under a "BSD Software License" license.  For details, see http://code.google.com/p/protobuf
+This product bundles jsch 0.1.42, which is available under a "BSD Software License" license.  For details, see http://www.jcraft.com/jsch/
+This product bundles jersey-guice 1.9, which is available under a "Common Development and Distribution License v1.1" license.  For details, see https://jersey.java.net/
+This product bundles jersey-client 1.9, which is available under a "Common Development and Distribution License v1.1" license.  For details, see https://jersey.java.net/
+This product bundles jersey-core 1.9, which is available under a "Common Development and Distribution License v1.1" license.  For details, see https://jersey.java.net/
+This product bundles jersey-json 1.9, which is available under a "Common Development and Distribution License v1.1" license.  For details, see https://jersey.java.net/
+This product bundles jersey-server 1.9, which is available under a "Common Development and Distribution License v1.1" license.  For details, see https://jersey.java.net/
+This product bundles jaxb-impl 2.2.3-1, which is available under a "Common Development and Distribution License" license.  For details, see http://jaxb.java.net/
+This product bundles paranamer 2.3, which is available under a "BSD Software License" license.  For details, see https://github.com/paul-hammant/paranamer
+This product bundles activation 1.1, which is available under a "Common Development and Distribution License v1.0" license.  For details, see http://java.sun.com/products/javabeans/jaf/index.jsp
+This product bundles servlet-api 2.5, which is available under a "Common Development and Distribution License" license.  For details, see http://oracle.com
+This product bundles jaxb-api 2.2.2, which is available under a "Common Development and Distribution License" license.  For details, see https://jaxb.dev.java.net/
+This product bundles stax-api 1.0-2, which is available under a "Common Development and Distribution License v1.0" license.  For details, see https://docs.oracle.com/javase/7/docs/api/javax/xml/stream/package-summary.html
+This product bundles jline 0.9.94, which is available under a "BSD Software License" license.  For details, see http://jline.sourceforge.net
+This product bundles jopt-simple 3.2, which is available under a "MIT Software License" license.  For details, see http://jopt-simple.sourceforge.net
+This product bundles clojure 1.6.0, which is available under a "Eclipse Public License v1.0" license.  For details, see http://clojure.org/
+This product bundles leveldbjni-all 1.8, which is available under a "BSD Software License" license.  For details, see https://github.com/fusesource/leveldbjni
+This product bundles jcodings 1.0.8, which is available under a "MIT Software License" license.  For details, see https://github.com/jruby/jcodings
+This product bundles joni 2.1.2, which is available under a "MIT Software License" license.  For details, see https://github.com/jruby/joni
+This product bundles asm 4.0, which is available under a "BSD Software License" license.  For details, see http://asm.ow2.org/
+This product bundles scala-library 2.9.2, which is available under a "BSD Software License" license.  For details, see http://www.scala-lang.org/
+This product bundles slf4j-api 1.7.10, which is available under a "MIT Software License" license.  For details, see http://www.slf4j.org
+This product bundles slf4j-log4j12 1.7.10, which is available under a "MIT Software License" license.  For details, see http://www.slf4j.org
+This product bundles xmlenc 0.52, which is available under a "BSD Software License" license.  For details, see http://xmlenc.sourceforge.net

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java
new file mode 100644
index 0000000..0173511
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java
@@ -0,0 +1,121 @@
+/*
+ *
+ *  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.profiler;
+
+import org.apache.metron.profiler.ProfilePeriod;
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+import java.util.stream.IntStream;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests the ProfilePeriod class.
+ */
+public class ProfilePeriodTest {
+
+  /**
+   * Thu, Aug 25 2016 13:27:10 GMT
+   */
+  private long AUG2016 = 1472131630748L;
+
+  @Test
+  public void testFirstPeriodAtEpoch() {
+    long duration = 1;
+    TimeUnit units = TimeUnit.HOURS;
+
+    ProfilePeriod period = new ProfilePeriod(0, duration, units);
+    assertEquals(0, period.getPeriod());
+    assertEquals(0, period.getStartTimeMillis());
+    assertEquals(units.toMillis(duration), period.getDurationMillis());
+  }
+
+  @Test
+  public void testOneMinutePeriods() {
+    long duration = 1;
+    TimeUnit units = TimeUnit.MINUTES;
+
+    ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units);
+    assertEquals(24535527, period.getPeriod());
+    assertEquals(1472131620000L, period.getStartTimeMillis());  // Thu, 25 Aug 2016 13:27:00 GMT
+    assertEquals(units.toMillis(duration), period.getDurationMillis());
+  }
+
+  @Test
+  public void testFifteenMinutePeriods() {
+    long duration = 15;
+    TimeUnit units = TimeUnit.MINUTES;
+
+    ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units);
+    assertEquals(1635701, period.getPeriod());
+    assertEquals(1472130900000L, period.getStartTimeMillis());  // Thu, 25 Aug 2016 13:15:00 GMT
+    assertEquals(units.toMillis(duration), period.getDurationMillis());
+  }
+
+  @Test
+  public void testOneHourPeriods() {
+    long duration = 1;
+    TimeUnit units = TimeUnit.HOURS;
+
+    ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units);
+    assertEquals(408925, period.getPeriod());
+    assertEquals(1472130000000L, period.getStartTimeMillis());  // Thu, 25 Aug 2016 13:00:00 GMT
+    assertEquals(units.toMillis(duration), period.getDurationMillis());
+  }
+
+  @Test
+  public void testTwoHourPeriods() {
+    long duration = 2;
+    TimeUnit units = TimeUnit.HOURS;
+
+    ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units);
+    assertEquals(204462, period.getPeriod());
+    assertEquals(1472126400000L, period.getStartTimeMillis());  //  Thu, 25 Aug 2016 12:00:00 GMT
+    assertEquals(units.toMillis(duration), period.getDurationMillis());
+  }
+
+  @Test
+  public void testEightHourPeriods() {
+    long duration = 8;
+    TimeUnit units = TimeUnit.HOURS;
+
+    ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units);
+    assertEquals(51115, period.getPeriod());
+    assertEquals(1472112000000L, period.getStartTimeMillis());  // Thu, 25 Aug 2016 08:00:00 GMT
+    assertEquals(units.toMillis(duration), period.getDurationMillis());
+  }
+
+  @Test
+  public void testNextWithFifteenMinutePeriods() {
+    long duration = 15;
+    TimeUnit units = TimeUnit.MINUTES;
+
+    ProfilePeriod previous = new ProfilePeriod(AUG2016, duration, units);
+    IntStream.range(0, 100).forEach(i -> {
+
+      ProfilePeriod next = previous.next();
+      assertEquals(previous.getPeriod() + 1, next.getPeriod());
+      assertEquals(previous.getStartTimeMillis() + previous.getDurationMillis(), next.getStartTimeMillis());
+      assertEquals(previous.getDurationMillis(), next.getDurationMillis());
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/070473f5/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java
new file mode 100644
index 0000000..d59f38d
--- /dev/null
+++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java
@@ -0,0 +1,260 @@
+/*
+ *
+ *  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.profiler.hbase;
+
+import backtype.storm.tuple.Tuple;
+import org.apache.metron.profiler.ProfileMeasurement;
+import org.apache.metron.profiler.ProfilePeriod;
+import org.apache.metron.profiler.hbase.SaltyRowKeyBuilder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Formatter;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.core.IsEqual.equalTo;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests the SaltyRowKeyBuilder.
+ */
+public class SaltyRowKeyBuilderTest {
+
+  private static final int saltDivisor = 1000;
+  private static final long periodDuration = 15;
+  private static final TimeUnit periodUnits = TimeUnit.MINUTES;
+
+  private SaltyRowKeyBuilder rowKeyBuilder;
+  private ProfileMeasurement measurement;
+  private Tuple tuple;
+
+  /**
+   * Thu, Aug 25 2016 13:27:10 GMT
+   */
+  private long AUG2016 = 1472131630748L;
+
+  @Before
+  public void setup() throws Exception {
+
+    // a profile measurement
+    measurement = new ProfileMeasurement("profile", "entity", AUG2016, periodDuration, periodUnits);
+    measurement.setValue(22);
+
+    // the tuple will contain the original message
+    tuple = mock(Tuple.class);
+    when(tuple.getValueByField(eq("measurement"))).thenReturn(measurement);
+  }
+
+  /**
+   * Build a row key that includes only one group.
+   */
+  @Test
+  public void testRowKeyWithOneGroup() throws Exception {
+    // setup
+    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
+    List<Object> groups = Arrays.asList("group1");
+
+    // the expected row key
+    ByteBuffer buffer = ByteBuffer
+            .allocate(100)
+            .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor))
+            .put(measurement.getProfileName().getBytes())
+            .put(measurement.getEntity().getBytes())
+            .put("group1".getBytes())
+            .putLong(1635701L);
+
+    buffer.flip();
+    final byte[] expected = new byte[buffer.limit()];
+    buffer.get(expected, 0, buffer.limit());
+
+    // validate
+    byte[] actual = rowKeyBuilder.rowKey(measurement, groups);
+    Assert.assertTrue(Arrays.equals(expected, actual));
+  }
+
+  /**
+   * Build a row key that includes two groups.
+   */
+  @Test
+  public void testRowKeyWithTwoGroups() throws Exception {
+    // setup
+    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
+    List<Object> groups = Arrays.asList("group1","group2");
+
+    // the expected row key
+    ByteBuffer buffer = ByteBuffer
+            .allocate(100)
+            .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor))
+            .put(measurement.getProfileName().getBytes())
+            .put(measurement.getEntity().getBytes())
+            .put("group1".getBytes())
+            .put("group2".getBytes())
+            .putLong(1635701L);
+
+    buffer.flip();
+    final byte[] expected = new byte[buffer.limit()];
+    buffer.get(expected, 0, buffer.limit());
+
+    // validate
+    byte[] actual = rowKeyBuilder.rowKey(measurement, groups);
+    Assert.assertTrue(Arrays.equals(expected, actual));
+  }
+
+  /**
+   * Build a row key that includes a single group that is an integer.
+   */
+  @Test
+  public void testRowKeyWithOneIntegerGroup() throws Exception {
+    // setup
+    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
+    List<Object> groups = Arrays.asList(200);
+
+    // the expected row key
+    ByteBuffer buffer = ByteBuffer
+            .allocate(100)
+            .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor))
+            .put(measurement.getProfileName().getBytes())
+            .put(measurement.getEntity().getBytes())
+            .put("200".getBytes())
+            .putLong(1635701L);
+
+    buffer.flip();
+    final byte[] expected = new byte[buffer.limit()];
+    buffer.get(expected, 0, buffer.limit());
+
+    // validate
+    byte[] actual = rowKeyBuilder.rowKey(measurement, groups);
+    Assert.assertTrue(Arrays.equals(expected, actual));
+  }
+
+  /**
+   * Build a row key that includes a single group that is an integer.
+   */
+  @Test
+  public void testRowKeyWithMixedGroups() throws Exception {
+    // setup
+    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
+    List<Object> groups = Arrays.asList(200, "group1");
+
+    // the expected row key
+    ByteBuffer buffer = ByteBuffer
+            .allocate(100)
+            .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor))
+            .put(measurement.getProfileName().getBytes())
+            .put(measurement.getEntity().getBytes())
+            .put("200".getBytes())
+            .put("group1".getBytes())
+            .putLong(1635701L);
+
+    buffer.flip();
+    final byte[] expected = new byte[buffer.limit()];
+    buffer.get(expected, 0, buffer.limit());
+
+    // validate
+    byte[] actual = rowKeyBuilder.rowKey(measurement, groups);
+    Assert.assertTrue(Arrays.equals(expected, actual));
+  }
+
+  /**
+   * Build a row key that does not include any groups.
+   */
+  @Test
+  public void testRowKeyWithNoGroup() throws Exception {
+    // setup
+    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
+    List<Object> groups = Collections.emptyList();
+
+    // the expected row key
+    ByteBuffer buffer = ByteBuffer
+            .allocate(100)
+            .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor))
+            .put(measurement.getProfileName().getBytes())
+            .put(measurement.getEntity().getBytes())
+            .putLong(1635701L);
+
+    buffer.flip();
+    final byte[] expected = new byte[buffer.limit()];
+    buffer.get(expected, 0, buffer.limit());
+
+    // validate
+    byte[] actual = rowKeyBuilder.rowKey(measurement, groups);
+    Assert.assertTrue(Arrays.equals(expected, actual));
+  }
+
+  /**
+   * `rowKeys` should return all of the row keys needed to retrieve the profile values over a given time horizon.
+   */
+  @Test
+  public void testRowKeys() throws Exception {
+    int hoursAgo = 1;
+
+    // setup
+    List<Object> groups = Collections.emptyList();
+    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodUnits);
+
+    // a dummy profile measurement
+    long now = System.currentTimeMillis();
+    long oldest = now - TimeUnit.HOURS.toMillis(hoursAgo);
+    ProfileMeasurement m = new ProfileMeasurement("profile", "entity", oldest, periodDuration, periodUnits);
+    m.setValue(22);
+
+    // generate a list of expected keys
+    List<byte[]> expectedKeys = new ArrayList<>();
+    for  (int i=0; i<(hoursAgo * 4)+1; i++) {
+
+      // generate the expected key
+      byte[] rk = rowKeyBuilder.rowKey(m, groups);
+      expectedKeys.add(rk);
+
+      // advance to the next period
+      ProfilePeriod next = m.getPeriod().next();
+      m = new ProfileMeasurement("profile", "entity", next.getStartTimeMillis(), periodDuration, periodUnits);
+    }
+
+    // execute
+    List<byte[]> actualKeys = rowKeyBuilder.rowKeys(measurement.getProfileName(), measurement.getEntity(), groups, oldest, now);
+
+    // validate - expectedKeys == actualKeys
+    for(int i=0; i<actualKeys.size(); i++) {
+      byte[] actual = actualKeys.get(i);
+      byte[] expected = expectedKeys.get(i);
+      assertThat(actual, equalTo(expected));
+    }
+  }
+
+  private void printBytes(byte[] bytes) {
+    StringBuilder sb = new StringBuilder(bytes.length * 2);
+    Formatter formatter = new Formatter(sb);
+    for (byte b : bytes) {
+      formatter.format("%02x ", b);
+    }
+    System.out.println(sb.toString());
+  }
+}