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/09/16 21:18:08 UTC

incubator-metron git commit: METRON-420 Add Expiration to a Profile Definition (nickwallen) closes apache/incubator-metron#254

Repository: incubator-metron
Updated Branches:
  refs/heads/master 32d5ff64a -> 095313255


METRON-420 Add Expiration to a Profile Definition (nickwallen) closes apache/incubator-metron#254


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

Branch: refs/heads/master
Commit: 095313255620355132b885fae24cf4669db4392d
Parents: 32d5ff6
Author: nickwallen <ni...@nickallen.org>
Authored: Fri Sep 16 17:16:23 2016 -0400
Committer: Nick Allen <ni...@nickallen.org>
Committed: Fri Sep 16 17:16:23 2016 -0400

----------------------------------------------------------------------
 metron-analytics/metron-profiler/README.md      | 11 +++-
 .../profiler/bolt/ProfileBuilderBolt.java       |  8 +--
 .../profiler/bolt/ProfileHBaseMapper.java       | 36 ++++++++-----
 .../zookeeper/readme-example-1/profiler.json    |  3 +-
 .../profiler/bolt/ProfileHBaseMapperTest.java   | 50 +++++++++++++++---
 .../integration/ProfilerIntegrationTest.java    | 48 +++++++----------
 .../configuration/profiler/ProfileConfig.java   | 55 +++++++++++++++-----
 .../metron/hbase/bolt/mapper/HBaseMapper.java   | 18 ++++---
 8 files changed, 157 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/09531325/metron-analytics/metron-profiler/README.md
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md
index 6d8fea9..44538b3 100644
--- a/metron-analytics/metron-profiler/README.md
+++ b/metron-analytics/metron-profiler/README.md
@@ -21,6 +21,7 @@ The Profiler configuration requires a JSON-formatted set of elements, many of wh
 | [init](#init)  	    | Optional  	| One or more expressions executed at the start of a window period.
 | [update](#update)  	| Required  	| One or more expressions executed when a message is applied to the profile.
 | [result](#result)   	| Required  	| A Stellar expression that is executed when the window period expires.
+| [expires](#expires)   | Optional      | Profile data is purged after this period of time, specified in milliseconds.
 
 #### `profile` 
 
@@ -86,6 +87,12 @@ One or more expressions executed when a message is applied to the profile.  A ma
 
 A Stellar expression that is executed when the window period expires.  The expression is expected to summarize the messages that were applied to the profile over the window period.  The expression must result in a numeric value such as a Double, Long, Float, Short, or Integer.  	   
 
+#### `expires`
+
+*Optional*
+
+A numeric value that defines how many days the profile data is retained.  After this time, the data expires and is no longer accessible.  If no value is defined, the data does not expire.
+
 ### Examples
 
 Examples of the types of profiles that can be built include the following.  Each shows the configuration that would be required to produce the profile.  These examples assume a fictitious input messages that looks something like the following.
@@ -130,7 +137,8 @@ The total number of bytes of HTTP data for each host. The following configuratio
       "update": {
         "total_bytes": "total_bytes + bytes_in"
       },
-      "result": "total_bytes"
+      "result": "total_bytes",
+      "expires": 30
     }
   ]
 }
@@ -143,6 +151,7 @@ This creates a profile...
  * Initializes a counter \u2018total_bytes\u2019 to zero
  * Adds to \u2018total_bytes\u2019 the value of the message's \u2018bytes_in\u2019 field
  * Returns \u2018total_bytes\u2019 as the result
+ * The profile data will expire in 30 days
 
 #### Example 2
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/09531325/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java
index de8561b..15c803c 100644
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java
+++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java
@@ -130,7 +130,7 @@ public class ProfileBuilderBolt extends ConfiguredProfilerBolt {
   @Override
   public void declareOutputFields(OutputFieldsDeclarer declarer) {
     // once the time window expires, a complete ProfileMeasurement is emitted
-    declarer.declare(new Fields("measurement"));
+    declarer.declare(new Fields("measurement", "profile"));
   }
 
   @Override
@@ -234,7 +234,7 @@ public class ProfileBuilderBolt extends ConfiguredProfilerBolt {
     try {
       String resultExpr = profileConfig.getResult();
       result = executor.execute(resultExpr, new JSONObject(), Object.class);
-    
+
     } catch(ParseException e) {
       throw new ParseException("Bad 'result' expression", e);
     }
@@ -255,7 +255,7 @@ public class ProfileBuilderBolt extends ConfiguredProfilerBolt {
    * @param measurement The completed ProfileMeasurement.
    */
   private void emit(ProfileMeasurement measurement, Tuple anchor) {
-    collector.emit(anchor, new Values(measurement));
+    collector.emit(anchor, new Values(measurement, profileConfig));
     collector.ack(anchor);
   }
 
@@ -281,7 +281,7 @@ public class ProfileBuilderBolt extends ConfiguredProfilerBolt {
    */
   protected static boolean isTickTuple(Tuple tuple) {
     return Constants.SYSTEM_COMPONENT_ID.equals(tuple.getSourceComponent()) &&
-      Constants.SYSTEM_TICK_STREAM_ID.equals(tuple.getSourceStreamId());
+            Constants.SYSTEM_TICK_STREAM_ID.equals(tuple.getSourceStreamId());
   }
 
   public StellarExecutor getExecutor() {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/09531325/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java
index 2c8cb67..d6377e4 100644
--- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java
+++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java
@@ -22,17 +22,16 @@ package org.apache.metron.profiler.bolt;
 
 import backtype.storm.tuple.Tuple;
 import org.apache.commons.beanutils.BeanMap;
-import org.apache.commons.lang.StringUtils;
 import org.apache.metron.common.configuration.profiler.ProfileConfig;
 import org.apache.metron.common.dsl.ParseException;
+import org.apache.metron.hbase.bolt.mapper.ColumnList;
 import org.apache.metron.hbase.bolt.mapper.HBaseMapper;
 import org.apache.metron.profiler.ProfileMeasurement;
 import org.apache.metron.profiler.hbase.ColumnBuilder;
+import org.apache.metron.profiler.hbase.RowKeyBuilder;
 import org.apache.metron.profiler.hbase.SaltyRowKeyBuilder;
 import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder;
-import org.apache.metron.profiler.hbase.RowKeyBuilder;
 import org.apache.metron.profiler.stellar.StellarExecutor;
-import org.apache.metron.hbase.bolt.mapper.ColumnList;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -72,9 +71,10 @@ public class ProfileHBaseMapper implements HBaseMapper {
   }
 
   /**
-   * Define the row key for a ProfileMeasurement.
-   * @param tuple The tuple to map to Hbase.
-   * @return The Hbase row key.
+   * Defines the HBase row key that will be used when writing the data from a
+   * tuple to HBase.
+   *
+   * @param tuple The tuple to map to HBase.
    */
   @Override
   public byte[] rowKey(Tuple tuple) {
@@ -84,8 +84,10 @@ public class ProfileHBaseMapper implements HBaseMapper {
   }
 
   /**
-   * Defines how the fields within a ProfileMeasurement are mapped to HBase.
-   * @param tuple The tuple to map to Hbase.
+   * Defines the columnar structure that will be used when writing the data
+   * from a tuple to HBase.
+   *
+   * @param tuple The tuple to map to HBase.
    */
   @Override
   public ColumnList columns(Tuple tuple) {
@@ -94,14 +96,22 @@ public class ProfileHBaseMapper implements HBaseMapper {
   }
 
   /**
-   * The time-to-live can be defined differently for each profile.
-   * @param tuple The tuple to map to Hbase.
-   * @return
+   * Defines the TTL (time-to-live) that will be used when writing the data
+   * from a tuple to HBase.  After the TTL, the data will expire and will be
+   * purged.
+   *
+   * @param tuple The tuple to map to HBase.
+   * @return The TTL in milliseconds.
    */
   @Override
   public Optional<Long> getTTL(Tuple tuple) {
-    // TTL not yet supported for profiles
     Optional result = Optional.empty();
+
+    ProfileConfig profileConfig = (ProfileConfig) tuple.getValueByField("profile");
+    if(profileConfig.getExpires() != null) {
+      result = result.of(profileConfig.getExpires());
+    }
+
     return result;
   }
 
@@ -146,4 +156,4 @@ public class ProfileHBaseMapper implements HBaseMapper {
   public void setColumnBuilder(ColumnBuilder columnBuilder) {
     this.columnBuilder = columnBuilder;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/09531325/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json
index f811df7..ebc56fd 100644
--- a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json
+++ b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json
@@ -11,7 +11,8 @@
       "update": {
         "total_bytes": "total_bytes + bytes_in"
       },
-      "result": "total_bytes"
+      "result": "total_bytes",
+      "expires": 30
     }
   ]
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/09531325/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java
index 4834072..97caff9 100644
--- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java
+++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java
@@ -21,6 +21,7 @@
 package org.apache.metron.profiler.bolt;
 
 import backtype.storm.tuple.Tuple;
+import org.apache.metron.common.configuration.profiler.ProfileConfig;
 import org.apache.metron.profiler.ProfileMeasurement;
 import org.apache.metron.profiler.hbase.RowKeyBuilder;
 import org.apache.metron.profiler.stellar.DefaultStellarExecutor;
@@ -31,6 +32,7 @@ import org.mockito.ArgumentCaptor;
 
 import java.util.Arrays;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
 import static org.mockito.Matchers.any;
@@ -44,11 +46,12 @@ import static org.mockito.Mockito.when;
  */
 public class ProfileHBaseMapperTest {
 
-  Tuple tuple;
-  ProfileHBaseMapper mapper;
-  ProfileMeasurement measurement;
-  DefaultStellarExecutor executor;
-  RowKeyBuilder rowKeyBuilder;
+  private Tuple tuple;
+  private ProfileHBaseMapper mapper;
+  private ProfileMeasurement measurement;
+  private DefaultStellarExecutor executor;
+  private RowKeyBuilder rowKeyBuilder;
+  private ProfileConfig profile;
 
   @Before
   public void setup() {
@@ -63,9 +66,13 @@ public class ProfileHBaseMapperTest {
     measurement = new ProfileMeasurement("profile", "entity", 20000, 15, TimeUnit.MINUTES);
     measurement.setValue(22);
 
+    profile = new ProfileConfig();
+
+
     // the tuple will contain the original message
     tuple = mock(Tuple.class);
     when(tuple.getValueByField(eq("measurement"))).thenReturn(measurement);
+    when(tuple.getValueByField(eq("profile"))).thenReturn(profile);
   }
 
   /**
@@ -92,7 +99,7 @@ public class ProfileHBaseMapperTest {
 
   /**
    * The mapper should execute each 'groupBy' Stellar expression and use that to generate
-   * a row key.  There can be multiple.
+   * a row key.  There can be multiple groups.
    */
   @Test
   public void testExecuteMultipleGroupBys() throws Exception {
@@ -112,4 +119,35 @@ public class ProfileHBaseMapperTest {
     Assert.assertEquals(4.0, actual.get(0));
     Assert.assertEquals(8.0, actual.get(1));
   }
+
+  /**
+   * The mapper should return the expiration for a tuple based on the Profile definition.
+   */
+  @Test
+  public void testExpires() throws Exception {
+    final Long expiresDays = 30L;
+    profile.setExpires(expiresDays);
+
+    Optional<Long> actual = mapper.getTTL(tuple);
+    Assert.assertTrue(actual.isPresent());
+    Assert.assertEquals(expiresDays, (Long) TimeUnit.MILLISECONDS.toDays(actual.get()));
+  }
+
+  /**
+   * The expiration field is optional within a Profile definition.
+   */
+  @Test
+  public void testExpiresUndefined() throws Exception {
+
+    // do not set the TTL on the profile
+    ProfileConfig profileNoTTL = new ProfileConfig();
+
+    // the tuple references the profile with the missing TTL
+    Tuple tupleNoTTL = mock(Tuple.class);
+    when(tupleNoTTL.getValueByField(eq("profile"))).thenReturn(profileNoTTL);
+
+    // the TTL should not be defined
+    Optional<Long> actual = mapper.getTTL(tupleNoTTL);
+    Assert.assertFalse(actual.isPresent());
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/09531325/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java
index 80b4399..04a9971 100644
--- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java
+++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java
@@ -136,7 +136,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
             timeout(seconds(90)));
 
     // verify - there are 5 'HTTP' each with 390 bytes
-    double actual = readDouble(columnBuilder.getColumnQualifier("value"));
+    double actual = read(columnBuilder.getColumnQualifier("value"), Double.class);
     Assert.assertEquals(390.0 * 5, actual, 0.01);
   }
 
@@ -157,7 +157,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
             timeout(seconds(90)));
 
     // verify - there are 5 'HTTP' and 5 'DNS' messages thus 5/5 = 1
-    double actual = readDouble(columnBuilder.getColumnQualifier("value"));
+    double actual = read(columnBuilder.getColumnQualifier("value"), Double.class);
     Assert.assertEquals(5.0 / 5.0, actual, 0.01);
   }
 
@@ -178,7 +178,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
             timeout(seconds(90)));
 
     // verify - there are 5 'HTTP' messages each with a length of 20, thus the average should be 20
-    double actual = readDouble(columnBuilder.getColumnQualifier("value"));
+    double actual = read(columnBuilder.getColumnQualifier("value"), Double.class);
     Assert.assertEquals(20.0, actual, 0.01);
   }
 
@@ -195,9 +195,9 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
     waitOrTimeout(() -> profilerTable.getPutLog().size() > 0,
             timeout(seconds(90)));
 
-    // verify - there are 5 'HTTP' messages each with a length of 20, thus the average should be 20
-    double actual = readInteger(columnBuilder.getColumnQualifier("value"));
-    Assert.assertEquals(10.0, actual, 0.01);
+    // verify - the profile literally writes 10 as an integer
+    int actual = read(columnBuilder.getColumnQualifier("value"), Integer.class);
+    Assert.assertEquals(10, actual);
   }
 
   @Test
@@ -214,37 +214,27 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
             timeout(seconds(90)));
 
     // verify - the 70th percentile of 5 x 20s = 20.0
-    double actual = readDouble(columnBuilder.getColumnQualifier("value"));
+    double actual = read(columnBuilder.getColumnQualifier("value"), Double.class);
     Assert.assertEquals(20.0, actual, 0.01);
   }
 
   /**
-   * Reads a Double value written by the Profiler.
-   * @param columnQual The column qualifier.
-   */
-  private Double readDouble(byte[] columnQual) throws IOException {
-    final byte[] cf = Bytes.toBytes(columnFamily);
-    ResultScanner scanner = profilerTable.getScanner(cf, columnQual);
-
-    for (Result result : scanner) {
-      byte[] raw = result.getValue(cf, columnQual);
-      return SerDeUtils.fromBytes(raw, Double.class);
-    }
-
-    throw new IllegalStateException("No results found");
-  }
-
-  /**
-   * Reads an Integer value written by the Profiler.
-   * @param columnQual The column qualifier.
+   * Reads a value written by the Profiler.
+   *
+   * @param column The column qualifier.
+   * @param clazz The expected type of the result.
+   * @param <T> The expected type of the result.
+   * @return The value contained within the column.
    */
-  private Integer readInteger(byte[] columnQual) throws IOException {
+  private <T> T read(byte[] column, Class<T> clazz) throws IOException {
     final byte[] cf = Bytes.toBytes(columnFamily);
-    ResultScanner scanner = profilerTable.getScanner(cf, columnQual);
+    ResultScanner scanner = profilerTable.getScanner(cf, column);
 
     for (Result result : scanner) {
-      byte[] raw = result.getValue(cf, columnQual);
-      return SerDeUtils.fromBytes(raw, Integer.class);
+      if(result.containsColumn(cf, column)) {
+        byte[] raw = result.getValue(cf, column);
+        return SerDeUtils.fromBytes(raw, clazz);
+      }
     }
 
     throw new IllegalStateException("No results found");

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/09531325/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java
index 34dab13..4cf3ac2 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 /**
  * The user defined configuration values required to generate a Profile.
@@ -80,6 +81,12 @@ public class ProfileConfig implements Serializable {
    */
   private String result;
 
+  /**
+   * How long the data created by this Profile will be retained.  After this period of time the
+   * profile data will be purged and no longer accessible.
+   */
+  private Long expires;
+
   public String getProfile() {
     return profile;
   }
@@ -104,14 +111,6 @@ public class ProfileConfig implements Serializable {
     this.onlyif = onlyif;
   }
 
-  public String getResult() {
-    return result;
-  }
-
-  public void setResult(String result) {
-    this.result = result;
-  }
-
   public Map<String, String> getInit() {
     return init;
   }
@@ -128,18 +127,46 @@ public class ProfileConfig implements Serializable {
     this.update = update;
   }
 
+  public List<String> getGroupBy() {
+    return groupBy;
+  }
+
+  public void setGroupBy(List<String> groupBy) {
+    this.groupBy = groupBy;
+  }
+
+  public String getResult() {
+    return result;
+  }
+
+  public void setResult(String result) {
+    this.result = result;
+  }
+
+  public Long getExpires() {
+    return expires;
+  }
+
+  public void setExpires(Long expiresDays) {
+    this.expires = TimeUnit.DAYS.toMillis(expiresDays);
+  }
+
   @Override
   public boolean equals(Object o) {
     if (this == o) return true;
     if (o == null || getClass() != o.getClass()) return false;
 
     ProfileConfig that = (ProfileConfig) o;
+
     if (profile != null ? !profile.equals(that.profile) : that.profile != null) return false;
     if (foreach != null ? !foreach.equals(that.foreach) : that.foreach != null) return false;
     if (onlyif != null ? !onlyif.equals(that.onlyif) : that.onlyif != null) return false;
-    if (result != null ? !result.equals(that.result) : that.result != null) return false;
     if (init != null ? !init.equals(that.init) : that.init != null) return false;
-    return update != null ? update.equals(that.update) : that.update == null;
+    if (update != null ? !update.equals(that.update) : that.update != null) return false;
+    if (groupBy != null ? !groupBy.equals(that.groupBy) : that.groupBy != null) return false;
+    if (result != null ? !result.equals(that.result) : that.result != null) return false;
+    return expires != null ? expires.equals(that.expires) : that.expires == null;
+
   }
 
   @Override
@@ -147,9 +174,11 @@ public class ProfileConfig implements Serializable {
     int result1 = profile != null ? profile.hashCode() : 0;
     result1 = 31 * result1 + (foreach != null ? foreach.hashCode() : 0);
     result1 = 31 * result1 + (onlyif != null ? onlyif.hashCode() : 0);
-    result1 = 31 * result1 + (result != null ? result.hashCode() : 0);
     result1 = 31 * result1 + (init != null ? init.hashCode() : 0);
     result1 = 31 * result1 + (update != null ? update.hashCode() : 0);
+    result1 = 31 * result1 + (groupBy != null ? groupBy.hashCode() : 0);
+    result1 = 31 * result1 + (result != null ? result.hashCode() : 0);
+    result1 = 31 * result1 + (expires != null ? expires.hashCode() : 0);
     return result1;
   }
 
@@ -159,9 +188,11 @@ public class ProfileConfig implements Serializable {
             "profile='" + profile + '\'' +
             ", foreach='" + foreach + '\'' +
             ", onlyif='" + onlyif + '\'' +
-            ", result='" + result + '\'' +
             ", init=" + init +
             ", update=" + update +
+            ", groupBy=" + groupBy +
+            ", result='" + result + '\'' +
+            ", expires=" + expires +
             '}';
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/09531325/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/mapper/HBaseMapper.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/mapper/HBaseMapper.java b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/mapper/HBaseMapper.java
index e662c76..6cae1b4 100644
--- a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/mapper/HBaseMapper.java
+++ b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/mapper/HBaseMapper.java
@@ -34,23 +34,29 @@ import java.util.Optional;
 public interface HBaseMapper extends Serializable {
 
   /**
-   * Given a tuple, return the HBase rowkey.
+   * Defines the HBase row key that will be used when writing the data from a
+   * tuple to HBase.
    *
-   * @param tuple The tuple to map to Hbase
+   * @param tuple The tuple to map to HBase.
    */
   byte[] rowKey(Tuple tuple);
 
   /**
-   * Given a tuple, return a list of HBase columns to insert.
+   * Defines the columnar structure that will be used when writing the data
+   * from a tuple to HBase.
    *
-   * @param tuple The tuple to map to Hbase
+   * @param tuple The tuple to map to HBase.
    */
   ColumnList columns(Tuple tuple);
 
   /**
-   * Given a tuple, return the time to live.
    *
-   * @param tuple The tuple to map to Hbase
+   * Defines the TTL (time-to-live) that will be used when writing the data
+   * from a tuple to HBase.  After the TTL, the data will expire and will be
+   * purged.
+   *
+   * @param tuple The tuple to map to HBase.
+   * @return The TTL in milliseconds.
    */
   Optional<Long> getTTL(Tuple tuple);
 }