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 2019/09/04 20:16:13 UTC

[metron] branch feature/METRON-2088-support-hdp-3.1 updated: METRON-2231 Revert METRON-2175, METRON-2176, METRON-2177 in HDP 3.1 upgrade feature branch (nickwallen) closes apache/metron#1497

This is an automated email from the ASF dual-hosted git repository.

nickallen pushed a commit to branch feature/METRON-2088-support-hdp-3.1
in repository https://gitbox.apache.org/repos/asf/metron.git


The following commit(s) were added to refs/heads/feature/METRON-2088-support-hdp-3.1 by this push:
     new f77946d  METRON-2231 Revert METRON-2175, METRON-2176, METRON-2177 in HDP 3.1 upgrade feature branch (nickwallen) closes apache/metron#1497
f77946d is described below

commit f77946d6eb421ad89ac0221048113111b34502a1
Author: nickwallen <ni...@nickallen.org>
AuthorDate: Wed Sep 4 16:15:34 2019 -0400

    METRON-2231 Revert METRON-2175, METRON-2176, METRON-2177 in HDP 3.1 upgrade feature branch (nickwallen) closes apache/metron#1497
---
 .../profiler/client/HBaseProfilerClient.java       | 117 ++---
 .../client/HBaseProfilerClientFactory.java         | 101 ----
 .../metron/profiler/client/ProfilerClient.java     |   3 +-
 .../profiler/client/ProfilerClientFactories.java   |  45 --
 .../profiler/client/ProfilerClientFactory.java     |  36 --
 .../metron/profiler/client/stellar/GetProfile.java | 255 +++++-----
 .../client/stellar/ProfilerClientConfig.java       |   8 +-
 .../metron/profiler/client/stellar/Util.java       |   2 +-
 .../profiler/client/stellar/VerboseProfile.java    | 165 ++++---
 .../client/HBaseProfilerClientFactoryTest.java     | 108 -----
 .../profiler/client/HBaseProfilerClientTest.java   | 287 ++++++------
 .../metron/profiler/client/ProfileWriter.java      | 125 +++++
 .../profiler/client/stellar/GetProfileTest.java    | 512 +++++++++++++++------
 .../client/stellar/VerboseProfileTest.java         | 294 ++++++------
 .../metron/profiler/hbase/SaltyRowKeyBuilder.java  |   8 -
 .../profiler/hbase/ValueOnlyColumnBuilder.java     |   3 +
 .../profiler/DefaultMessageDistributorTest.java    |   2 +
 metron-analytics/metron-profiler-spark/pom.xml     |   6 -
 .../metron/profiler/spark/BatchProfiler.java       |   6 +-
 .../metron/profiler/spark/BatchProfilerConfig.java |   7 +-
 .../spark/function/HBaseWriterFunction.java        | 157 +++----
 .../spark/BatchProfilerIntegrationTest.java        |  44 +-
 .../spark/function/HBaseWriterFunctionTest.java    |  96 ++--
 metron-analytics/metron-profiler-storm/pom.xml     |  29 ++
 .../src/main/config/profiler.properties            |   2 -
 .../src/main/flux/profiler/remote.yaml             |   9 +-
 .../org/apache/metron/hbase/bolt/HBaseBolt.java    |  99 ++--
 .../apache/metron/hbase/bolt/HBaseBoltTest.java    |  15 +-
 .../java/org/apache/metron/hbase/bolt/Widget.java  |  84 ----
 .../org/apache/metron/hbase/bolt/WidgetMapper.java |  71 ---
 .../storm/integration/ProfilerIntegrationTest.java |  50 +-
 .../package/templates/profiler.properties.j2       |   2 -
 metron-interface/metron-rest/pom.xml               |  38 +-
 .../apache/metron/rest/MetronRestConstants.java    |   1 +
 .../org/apache/metron/rest/config/HBaseConfig.java | 111 ++---
 .../org/apache/metron/rest/config/IndexConfig.java |  79 ++--
 .../metron/rest/service/AlertsUIService.java       |   2 +-
 .../rest/service/impl/AlertsUIServiceImpl.java     |  13 +-
 .../impl/SensorEnrichmentConfigServiceImpl.java    |   6 +-
 .../metron/rest/user/HBaseUserSettingsClient.java  | 218 ---------
 .../metron/rest/user/UserSettingsClient.java       | 229 +++++----
 .../src/main/resources/application-test.yml        |   4 +-
 .../apache/metron/rest/config/HBaseConfigTest.java |  86 ++--
 .../org/apache/metron/rest/config/TestConfig.java  |  86 +---
 .../AlertsUIControllerIntegrationTest.java         |   4 -
 .../metron/rest/controller/DaoControllerTest.java  |   3 +-
 ...rEnrichmentConfigControllerIntegrationTest.java | 155 ++++---
 .../UpdateControllerIntegrationTest.java           |  16 +
 .../rest/service/impl/AlertsUIServiceImplTest.java | 167 ++++---
 .../SensorEnrichmentConfigServiceImplTest.java     |   6 +-
 .../rest/user/HBaseUserSettingsClientTest.java     | 194 --------
 .../metron/rest/user/UserSettingsClientTest.java   |  99 ++++
 metron-platform/elasticsearch-shaded/pom.xml       |  10 +
 .../metron-elasticsearch-common/pom.xml            |  14 -
 .../metron/elasticsearch/dao/ElasticsearchDao.java |   5 -
 .../dao/ElasticsearchMetaAlertDao.java             |   5 -
 .../dao/ElasticsearchMetaAlertDaoTest.java         |   4 -
 .../ElasticsearchUpdateIntegrationTest.java        |   5 +
 .../metron/hbase/coprocessor/HBaseCacheWriter.java |   4 +-
 .../java/org/apache/metron/hbase/ColumnList.java   |  97 +---
 .../metron/hbase/HBaseProjectionCriteria.java      |  10 +-
 .../java/org/apache/metron/hbase/TableConfig.java  |   5 +-
 .../apache/metron/hbase/client/HBaseClient.java    | 317 ++++++++++---
 .../metron/hbase/client/HBaseClientFactory.java    |  70 ---
 .../hbase/client/HBaseConnectionFactory.java       |  59 ---
 .../metron/hbase/client/HBaseTableClient.java      | 285 ------------
 .../hbase/client/HBaseTableClientFactory.java      |  54 ---
 .../metron/hbase/client/HBaseWriterParams.java     |  51 --
 .../metron/hbase/client/LegacyHBaseClient.java     | 337 --------------
 .../metron/hbase/client/FakeHBaseClient.java       | 335 --------------
 .../hbase/client/FakeHBaseClientFactory.java       |  43 --
 .../metron/hbase/client/FakeHBaseClientTest.java   | 268 -----------
 .../hbase/client/FakeHBaseConnectionFactory.java   |  89 ----
 .../metron/hbase/client/HBaseClientTest.java       |  10 +-
 .../HBaseTableClientIntegrationTest.java           | 286 ------------
 .../metron-indexing/metron-indexing-common/pom.xml |  28 --
 .../apache/metron/indexing/dao/AccessConfig.java   |  32 +-
 .../org/apache/metron/indexing/dao/HBaseDao.java   | 126 ++---
 .../org/apache/metron/indexing/dao/IndexDao.java   |   4 +-
 .../apache/metron/indexing/dao/MultiIndexDao.java  |   7 -
 .../apache/metron/indexing/dao/InMemoryDao.java    |   5 -
 .../metron/indexing/dao/InMemoryMetaAlertDao.java  |   6 -
 .../metron/indexing/dao/UpdateIntegrationTest.java |  84 +---
 .../integration/HBaseDaoIntegrationTest.java       |  89 ++--
 metron-platform/metron-pcap/pom.xml                |  33 +-
 .../java/org/apache/metron/solr/dao/SolrDao.java   |   5 -
 .../apache/metron/solr/dao/SolrMetaAlertDao.java   |  10 -
 .../org/apache/metron/solr/dao/SolrUtilities.java  |   5 +-
 .../metron/solr/dao/SolrMetaAlertDaoTest.java      |   4 -
 .../apache/metron/solr/dao/SolrSearchDaoTest.java  |  22 +-
 .../apache/metron/solr/dao/SolrUtilitiesTest.java  |   5 +-
 .../SolrRetrieveLatestIntegrationTest.java         |  22 +-
 .../integration/SolrUpdateIntegrationTest.java     |  54 ++-
 .../metron/stellar/dsl/StellarFunctionInfo.java    |   6 +-
 .../functions/resolver/BaseFunctionResolver.java   |  18 +-
 .../dsl/functions/resolver/FunctionResolver.java   |  17 +-
 .../functions/resolver/SimpleFunctionResolver.java |  13 +-
 97 files changed, 2417 insertions(+), 4806 deletions(-)

diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/HBaseProfilerClient.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/HBaseProfilerClient.java
index b9d0384..2e537da 100644
--- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/HBaseProfilerClient.java
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/HBaseProfilerClient.java
@@ -20,11 +20,11 @@
 
 package org.apache.metron.profiler.client;
 
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.metron.common.utils.SerDeUtils;
-import org.apache.metron.hbase.HBaseProjectionCriteria;
-import org.apache.metron.hbase.client.HBaseClient;
 import org.apache.metron.profiler.ProfileMeasurement;
 import org.apache.metron.profiler.ProfilePeriod;
 import org.apache.metron.profiler.hbase.ColumnBuilder;
@@ -44,7 +44,7 @@ public class HBaseProfilerClient implements ProfilerClient {
   /**
    * Used to access the profile data stored in HBase.
    */
-  private HBaseClient hbaseClient;
+  private HTableInterface table;
 
   /**
    * Generates the row keys necessary to scan HBase.
@@ -52,22 +52,19 @@ public class HBaseProfilerClient implements ProfilerClient {
   private RowKeyBuilder rowKeyBuilder;
 
   /**
-   * Knows how profiles are organized by columns in HBase.
+   * Knows how profiles are organized in HBase.
    */
   private ColumnBuilder columnBuilder;
 
-  /**
-   * The period duration in milliseconds for the profiles that will be read by this client.
-   */
   private long periodDurationMillis;
 
-  public HBaseProfilerClient(HBaseClient hbaseClient,
+  public HBaseProfilerClient(HTableInterface table,
                              RowKeyBuilder rowKeyBuilder,
                              ColumnBuilder columnBuilder,
                              long periodDurationMillis) {
-    this.rowKeyBuilder = rowKeyBuilder;
-    this.columnBuilder = columnBuilder;
-    this.hbaseClient = hbaseClient;
+    setTable(table);
+    setRowKeyBuilder(rowKeyBuilder);
+    setColumnBuilder(columnBuilder);
     this.periodDurationMillis = periodDurationMillis;
   }
 
@@ -85,13 +82,7 @@ public class HBaseProfilerClient implements ProfilerClient {
    * @return A list of values.
    */
   @Override
-  public <T> List<ProfileMeasurement> fetch(Class<T> clazz,
-                                            String profile,
-                                            String entity,
-                                            List<Object> groups,
-                                            long start,
-                                            long end,
-                                            Optional<T> defaultValue) {
+  public <T> List<ProfileMeasurement> fetch(Class<T> clazz, String profile, String entity, List<Object> groups, long start, long end, Optional<T> defaultValue) {
     List<ProfilePeriod> periods = ProfilePeriod.visitPeriods(
             start,
             end,
@@ -102,8 +93,6 @@ public class HBaseProfilerClient implements ProfilerClient {
     return fetch(clazz, profile, entity, groups, periods, defaultValue);
   }
 
-
-
   /**
    * Fetch the values stored in a profile based on a set of timestamps.
    *
@@ -116,12 +105,7 @@ public class HBaseProfilerClient implements ProfilerClient {
    * @return A list of values.
    */
   @Override
-  public <T> List<ProfileMeasurement> fetch(Class<T> clazz,
-                                            String profile,
-                                            String entity,
-                                            List<Object> groups,
-                                            Iterable<ProfilePeriod> periods,
-                                            Optional<T> defaultValue) {
+  public <T> List<ProfileMeasurement> fetch(Class<T> clazz, String profile, String entity, List<Object> groups, Iterable<ProfilePeriod> periods, Optional<T> defaultValue) {
     // create a list of profile measurements that need fetched
     List<ProfileMeasurement> toFetch = new ArrayList<>();
     for(ProfilePeriod period: periods) {
@@ -136,76 +120,59 @@ public class HBaseProfilerClient implements ProfilerClient {
     return doFetch(toFetch, clazz, defaultValue);
   }
 
-  @Override
-  public void close() throws IOException {
-    if(hbaseClient != null) {
-      hbaseClient.close();
-    }
-  }
-
-  private <T> List<ProfileMeasurement> doFetch(List<ProfileMeasurement> measurements,
-                                               Class<T> clazz,
-                                               Optional<T> defaultValue) {
+  private <T> List<ProfileMeasurement> doFetch(List<ProfileMeasurement> measurements, Class<T> clazz, Optional<T> defaultValue) {
     List<ProfileMeasurement> values = new ArrayList<>();
 
-    // define which columns need fetched
+    // build the gets for HBase
     byte[] columnFamily = Bytes.toBytes(columnBuilder.getColumnFamily());
     byte[] columnQualifier = columnBuilder.getColumnQualifier("value");
-    HBaseProjectionCriteria.ColumnMetaData column = new HBaseProjectionCriteria.ColumnMetaData(columnFamily, columnQualifier);
-    HBaseProjectionCriteria criteria = new HBaseProjectionCriteria().addColumn(column);
-
+    List<Get> gets = new ArrayList<>();
     for(ProfileMeasurement measurement: measurements) {
       byte[] rowKey = rowKeyBuilder.rowKey(measurement);
-      hbaseClient.addGet(rowKey, criteria);
+      Get get = new Get(rowKey).addColumn(columnFamily, columnQualifier);
+      gets.add(get);
     }
 
     // query HBase
-    Result[] results = hbaseClient.getAll();
-    for(int i = 0; i < results.length; ++i) {
-      Result result = results[i];
-      ProfileMeasurement measurement = measurements.get(i);
-
-      boolean exists = result.containsColumn(columnFamily, columnQualifier);
-      if(exists) {
-        // value found
-        byte[] value = result.getValue(columnFamily, columnQualifier);
-        measurement.withProfileValue(SerDeUtils.fromBytes(value, clazz));
-        values.add(measurement);
-
-      } else if(defaultValue.isPresent()) {
-        // no value found, use default value provided
-        measurement.withProfileValue(defaultValue.get());
-        values.add(measurement);
-
-      } else {
-        // no value found and no default provided. nothing to do
+    try {
+      Result[] results = table.get(gets);
+      for(int i = 0; i < results.length; ++i) {
+        Result result = results[i];
+        ProfileMeasurement measurement = measurements.get(i);
+
+        boolean exists = result.containsColumn(columnFamily, columnQualifier);
+        if(exists) {
+          // value found
+          byte[] value = result.getValue(columnFamily, columnQualifier);
+          measurement.withProfileValue(SerDeUtils.fromBytes(value, clazz));
+          values.add(measurement);
+
+        } else if(defaultValue.isPresent()) {
+          // no value found, use default value provided
+          measurement.withProfileValue(defaultValue.get());
+          values.add(measurement);
+
+        } else {
+          // no value found and no default provided. nothing to do
+        }
       }
+    } catch(IOException e) {
+      throw new RuntimeException(e);
     }
 
     return values;
   }
 
-  protected HBaseClient getHbaseClient() {
-    return hbaseClient;
-  }
 
-  protected RowKeyBuilder getRowKeyBuilder() {
-    return rowKeyBuilder;
+  public void setTable(HTableInterface table) {
+    this.table = table;
   }
 
-  protected void setRowKeyBuilder(RowKeyBuilder rowKeyBuilder) {
+  public void setRowKeyBuilder(RowKeyBuilder rowKeyBuilder) {
     this.rowKeyBuilder = rowKeyBuilder;
   }
 
-  protected ColumnBuilder getColumnBuilder() {
-    return columnBuilder;
-  }
-
-  protected void setColumnBuilder(ColumnBuilder columnBuilder) {
+  public void setColumnBuilder(ColumnBuilder columnBuilder) {
     this.columnBuilder = columnBuilder;
   }
-
-  protected long getPeriodDurationMillis() {
-    return periodDurationMillis;
-  }
 }
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/HBaseProfilerClientFactory.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/HBaseProfilerClientFactory.java
deleted file mode 100644
index 9d83100..0000000
--- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/HBaseProfilerClientFactory.java
+++ /dev/null
@@ -1,101 +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.client;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.metron.hbase.client.HBaseClient;
-import org.apache.metron.hbase.client.HBaseClientFactory;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
-import org.apache.metron.hbase.client.HBaseTableClientFactory;
-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 java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_COLUMN_FAMILY;
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE;
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_CONNECTION_FACTORY;
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_SALT_DIVISOR;
-import static org.apache.metron.profiler.client.stellar.Util.getPeriodDurationInMillis;
-
-/**
- * Creates an {@link HBaseProfilerClient}.
- */
-public class HBaseProfilerClientFactory implements ProfilerClientFactory {
-
-  /**
-   * The factory that provides the {@link HBaseClient} that is used to interact with HBase.
-   */
-  private HBaseClientFactory hBaseClientFactory;
-
-  public HBaseProfilerClientFactory() {
-    this(new HBaseTableClientFactory());
-  }
-
-  public HBaseProfilerClientFactory(HBaseClientFactory hBaseClientFactory) {
-    this.hBaseClientFactory = hBaseClientFactory;
-  }
-
-  @Override
-  public HBaseProfilerClient create(Map<String, Object> globals) {
-    // create the hbase client
-    String tableName = PROFILER_HBASE_TABLE.get(globals, String.class);
-    HBaseConnectionFactory connFactory = getConnectionFactory(globals);
-    Configuration config = HBaseConfiguration.create();
-    HBaseClient hbaseClient = hBaseClientFactory.create(connFactory, config, tableName);
-
-    // create the profiler client
-    RowKeyBuilder rowKeyBuilder = getRowKeyBuilder(globals);
-    ColumnBuilder columnBuilder = getColumnBuilder(globals);
-    long periodDuration = getPeriodDurationInMillis(globals);
-    return new HBaseProfilerClient(hbaseClient, rowKeyBuilder, columnBuilder, periodDuration);
-  }
-
-  /**
-   * Creates the ColumnBuilder to use in accessing the profile data.
-   * @param global The global configuration.
-   */
-  private static ColumnBuilder getColumnBuilder(Map<String, Object> global) {
-    String columnFamily = PROFILER_COLUMN_FAMILY.get(global, String.class);
-    return new ValueOnlyColumnBuilder(columnFamily);
-  }
-
-  /**
-   * Creates the ColumnBuilder to use in accessing the profile data.
-   * @param global The global configuration.
-   */
-  private static RowKeyBuilder getRowKeyBuilder(Map<String, Object> global) {
-    Integer saltDivisor = PROFILER_SALT_DIVISOR.get(global, Integer.class);
-    return new SaltyRowKeyBuilder(saltDivisor, getPeriodDurationInMillis(global), TimeUnit.MILLISECONDS);
-  }
-
-  /**
-   * Create the {@link HBaseConnectionFactory} to use when accessing HBase.
-   * @param global The global configuration.
-   */
-  private static HBaseConnectionFactory getConnectionFactory(Map<String, Object> global) {
-    String clazzName = PROFILER_HBASE_CONNECTION_FACTORY.get(global, String.class);
-    return HBaseConnectionFactory.byName(clazzName);
-  }
-}
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClient.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClient.java
index 7283d21..161575f 100644
--- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClient.java
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClient.java
@@ -23,14 +23,13 @@ package org.apache.metron.profiler.client;
 import org.apache.metron.profiler.ProfileMeasurement;
 import org.apache.metron.profiler.ProfilePeriod;
 
-import java.io.Closeable;
 import java.util.List;
 import java.util.Optional;
 
 /**
  * An interface for a client capable of retrieving the profile data that has been persisted by the Profiler.
  */
-public interface ProfilerClient extends Closeable {
+public interface ProfilerClient {
 
   /**
    * Fetch the values stored in a profile based on a start and end timestamp.
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClientFactories.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClientFactories.java
deleted file mode 100644
index 5fb4145..0000000
--- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClientFactories.java
+++ /dev/null
@@ -1,45 +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.client;
-
-import java.util.Map;
-
-/**
- * Enumerates the available {@link ProfilerClientFactory} implementations.
- */
-public enum ProfilerClientFactories implements ProfilerClientFactory {
-
-  /**
-   * The default factory that returns a {@link ProfilerClient} that interacts
-   * with profiles stored in HBase.
-   */
-  DEFAULT(new HBaseProfilerClientFactory());
-
-  private ProfilerClientFactory factory;
-
-  ProfilerClientFactories(ProfilerClientFactory factory) {
-    this.factory = factory;
-  }
-
-  @Override
-  public ProfilerClient create(Map<String, Object> globals) {
-    return factory.create(globals);
-  }
-}
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClientFactory.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClientFactory.java
deleted file mode 100644
index 7f659a7..0000000
--- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClientFactory.java
+++ /dev/null
@@ -1,36 +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.client;
-
-import java.util.Map;
-
-/**
- * Responsible for creating a {@link ProfilerClient}.
- */
-public interface ProfilerClientFactory {
-
-  /**
-   * Create a {@link ProfilerClient}.
-   *
-   * @param globals The global configuration.
-   * @return The {@link ProfilerClient}.
-   */
-  ProfilerClient create(Map<String, Object> globals);
-}
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 5a45571..a0e2bdf 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
@@ -20,12 +20,18 @@
 
 package org.apache.metron.profiler.client.stellar;
 
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.metron.hbase.HTableProvider;
+import org.apache.metron.hbase.TableProvider;
 import org.apache.metron.profiler.ProfileMeasurement;
 import org.apache.metron.profiler.ProfilePeriod;
-import org.apache.metron.profiler.client.HBaseProfilerClientFactory;
+import org.apache.metron.profiler.client.HBaseProfilerClient;
 import org.apache.metron.profiler.client.ProfilerClient;
-import org.apache.metron.profiler.client.ProfilerClientFactories;
-import org.apache.metron.profiler.client.ProfilerClientFactory;
+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.stellar.dsl.Context;
 import org.apache.metron.stellar.dsl.ParseException;
 import org.apache.metron.stellar.dsl.Stellar;
@@ -36,16 +42,21 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.concurrent.TimeUnit;
 
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_DEFAULT_VALUE;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_COLUMN_FAMILY;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE_PROVIDER;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_PERIOD;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_PERIOD_UNITS;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_SALT_DIVISOR;
 import static org.apache.metron.profiler.client.stellar.Util.getArg;
 import static org.apache.metron.profiler.client.stellar.Util.getEffectiveConfig;
-import static org.apache.metron.stellar.dsl.Context.Capabilities.GLOBAL_CONFIG;
+import static org.apache.metron.profiler.client.stellar.Util.getPeriodDurationInMillis;
 
 /**
  * A Stellar function that can retrieve data contained within a Profile.
@@ -80,135 +91,105 @@ import static org.apache.metron.stellar.dsl.Context.Capabilities.GLOBAL_CONFIG;
         name="GET",
         description="Retrieves a series of values from a stored profile.",
         params={
-                "profile - The name of the profile.",
-                "entity - The name of the entity.",
-                "periods - The list of profile periods to fetch. Use PROFILE_WINDOW or PROFILE_FIXED.",
-                "groups - Optional - The groups to retrieve. Must correspond to the 'groupBy' " +
-                        "list used during profile creation. Defaults to an empty list, meaning no groups.",
-                "config_overrides - Optional - Map (in curly braces) of name:value pairs, each overriding the global config parameter " +
-                        "of the same name. Default is the empty Map, meaning no overrides."
+          "profile - The name of the profile.",
+          "entity - The name of the entity.",
+          "periods - The list of profile periods to fetch. Use PROFILE_WINDOW or PROFILE_FIXED.",
+          "groups - Optional - The groups to retrieve. Must correspond to the 'groupBy' " +
+                    "list used during profile creation. Defaults to an empty list, meaning no groups.",
+          "config_overrides - Optional - Map (in curly braces) of name:value pairs, each overriding the global config parameter " +
+                  "of the same name. Default is the empty Map, meaning no overrides."
         },
         returns="The selected profile measurements."
 )
 public class GetProfile implements StellarFunction {
-  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private static final int PROFILE_ARG_INDEX = 0;
-  private static final int ENTITY_ARG_INDEX = 1;
-  private static final int PERIOD_ARG_INDEX = 2;
-  private static final int GROUPS_ARG_INDEX = 3;
-  private static final int CONFIG_OVERRIDES_ARG_INDEX = 4;
 
   /**
-   * Allows the function to retrieve persisted {@link ProfileMeasurement} values.
+   * Cached client that can retrieve profile values.
    */
-  private ProfilerClient profilerClient;
+  private ProfilerClient client;
 
   /**
-   * Creates the {@link ProfilerClient} used by this function.
+   * Cached value of config map actually used to construct the previously cached client.
    */
-  private ProfilerClientFactory profilerClientFactory;
+  private Map<String, Object> cachedConfigMap = new HashMap<String, Object>(6);
 
-  /**
-   * Last known global configuration used to create the {@link ProfilerClient}. If the
-   * global configuration changes, a new {@link ProfilerClient} needs to be constructed.
-   */
-  private Map<String, Object> lastKnownGlobals = new HashMap<>();
-
-  /**
-   * The default constructor used during Stellar function resolution.
-   */
-  public GetProfile() {
-    this(ProfilerClientFactories.DEFAULT);
-  }
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   /**
-   * The constructor used for testing.
-   * @param profilerClientFactory
+   * Initialization.  No longer need to do anything in initialization,
+   * as all setup is done lazily and cached.
    */
-  public GetProfile(ProfilerClientFactory profilerClientFactory) {
-    this.profilerClientFactory = profilerClientFactory;
-  }
-
   @Override
   public void initialize(Context context) {
-    Map<String, Object> globals = getGlobals(context);
-    profilerClient = profilerClientFactory.create(globals);
   }
 
+  /**
+   * Is the function initialized?
+   */
   @Override
   public boolean isInitialized() {
-    return profilerClient != null;
-  }
-
-  @Override
-  public void close() throws IOException {
-    if(profilerClient != null) {
-      profilerClient.close();
-    }
+    return true;
   }
 
+  /**
+   * Apply the function.
+   * @param args The function arguments.
+   * @param context
+   */
   @Override
   public Object apply(List<Object> args, Context context) throws ParseException {
-    // required arguments
-    String profile = getArg(PROFILE_ARG_INDEX, String.class, args);
-    String entity = getArg(ENTITY_ARG_INDEX, String.class, args);
-    List<ProfilePeriod> periods = getArg(PERIOD_ARG_INDEX, List.class, args);
-
-    // optional arguments
-    List<Object> groups = getGroups(args);
-    Map<String, Object> overrides = getOverrides(args);
-
-    // lazily create new profiler client if needed
-    Map<String, Object> effectiveConfig = getEffectiveConfig(context, overrides);
-    if (profilerClient == null || !lastKnownGlobals.equals(effectiveConfig)) {
-      profilerClient = profilerClientFactory.create(effectiveConfig);
-      lastKnownGlobals = effectiveConfig;
+
+    String profile = getArg(0, String.class, args);
+    String entity = getArg(1, String.class, args);
+    Optional<List<ProfilePeriod>> periods = Optional.ofNullable(getArg(2, List.class, args));
+    //Optional arguments
+    @SuppressWarnings("unchecked")
+    List<Object> groups = null;
+    Map configOverridesMap = null;
+    if (args.size() < 4) {
+      // no optional args, so default 'groups' and configOverridesMap remains null.
+      groups = new ArrayList<>(0);
+    }
+    else if (args.get(3) instanceof List) {
+      // correct extensible usage
+      groups = getArg(3, List.class, args);
+      if (args.size() >= 5) {
+        configOverridesMap = getArg(4, Map.class, args);
+        if (configOverridesMap.isEmpty()) configOverridesMap = null;
+      }
+    }
+    else {
+      // Deprecated "varargs" style usage for groups_list
+      // configOverridesMap cannot be specified so it remains null.
+      groups = getGroupsArg(3, args);
     }
 
-    // is there a default value?
-    Optional<Object> defaultValue = Optional.empty();
-    if(effectiveConfig != null) {
-      defaultValue = Optional.ofNullable(PROFILER_DEFAULT_VALUE.get(effectiveConfig));
+    Map<String, Object> effectiveConfig = getEffectiveConfig(context, configOverridesMap);
+    Object defaultValue = null;
+    //lazily create new profiler client if needed
+    if (client == null || !cachedConfigMap.equals(effectiveConfig)) {
+      RowKeyBuilder rowKeyBuilder = getRowKeyBuilder(effectiveConfig);
+      ColumnBuilder columnBuilder = getColumnBuilder(effectiveConfig);
+      HTableInterface table = getTable(effectiveConfig);
+      long periodDuration = getPeriodDurationInMillis(effectiveConfig);
+      client = new HBaseProfilerClient(table, rowKeyBuilder, columnBuilder, periodDuration);
+      cachedConfigMap = effectiveConfig;
+    }
+    if(cachedConfigMap != null) {
+      defaultValue = ProfilerClientConfig.PROFILER_DEFAULT_VALUE.get(cachedConfigMap);
     }
 
+    List<ProfileMeasurement> measurements = client.fetch(Object.class, profile, entity, groups,
+            periods.orElse(new ArrayList<>(0)), Optional.ofNullable(defaultValue));
+
     // return only the value of each profile measurement
-    List<ProfileMeasurement> measurements = profilerClient.fetch(Object.class, profile, entity, groups, periods, defaultValue);
     List<Object> values = new ArrayList<>();
     for(ProfileMeasurement m: measurements) {
       values.add(m.getProfileValue());
     }
-
     return values;
   }
 
-  private Map<String, Object> getOverrides(List<Object> args) {
-    Map<String, Object> configOverridesMap = null;
-    if(args.size() > CONFIG_OVERRIDES_ARG_INDEX && args.get(GROUPS_ARG_INDEX) instanceof List) {
-      configOverridesMap = getArg(CONFIG_OVERRIDES_ARG_INDEX, Map.class, args);
-      if (configOverridesMap.isEmpty()) {
-        configOverridesMap = null;
-      }
-    }
-    return configOverridesMap;
-  }
-
-  private List<Object> getGroups(List<Object> args) {
-    List<Object> groups;
-    if (args.size() < CONFIG_OVERRIDES_ARG_INDEX) {
-      // no optional args, so default 'groups' and configOverridesMap remains null.
-      groups = new ArrayList<>(0);
-
-    } else if (args.get(GROUPS_ARG_INDEX) instanceof List) {
-      // correct extensible usage
-      groups = getArg(GROUPS_ARG_INDEX, List.class, args);
-
-    } else {
-      // deprecated "varargs" style usage for groups_list
-      groups = getVarArgGroups(GROUPS_ARG_INDEX, args);
-    }
-    return groups;
-  }
-
   /**
    * Get the groups defined by the user.
    *
@@ -220,7 +201,7 @@ public class GetProfile implements StellarFunction {
    * @param args The function arguments.
    * @return The groups.
    */
-  private static List<Object> getVarArgGroups(int startIndex, List<Object> args) {
+  private List<Object> getGroupsArg(int startIndex, List<Object> args) {
     List<Object> groups = new ArrayList<>();
 
     for(int i=startIndex; i<args.size(); i++) {
@@ -231,8 +212,76 @@ public class GetProfile implements StellarFunction {
     return groups;
   }
 
-  private static Map<String, Object> getGlobals(Context context) {
-    return (Map<String, Object>) context.getCapability(GLOBAL_CONFIG)
-            .orElse(Collections.emptyMap());
+  /**
+   * Creates the ColumnBuilder to use in accessing the profile data.
+   * @param global The global configuration.
+   */
+  private ColumnBuilder getColumnBuilder(Map<String, Object> global) {
+    ColumnBuilder columnBuilder;
+
+    String columnFamily = PROFILER_COLUMN_FAMILY.get(global, String.class);
+    columnBuilder = new ValueOnlyColumnBuilder(columnFamily);
+
+    return columnBuilder;
+  }
+
+  /**
+   * Creates the ColumnBuilder to use in accessing the profile data.
+   * @param global The global configuration.
+   */
+  private RowKeyBuilder getRowKeyBuilder(Map<String, Object> global) {
+
+    // how long is the profile period?
+    long duration = PROFILER_PERIOD.get(global, Long.class);
+    LOG.debug("profiler client: {}={}", PROFILER_PERIOD, duration);
+
+    // which units are used to define the profile period?
+    String configuredUnits = PROFILER_PERIOD_UNITS.get(global, String.class);
+    TimeUnit units = TimeUnit.valueOf(configuredUnits);
+    LOG.debug("profiler client: {}={}", PROFILER_PERIOD_UNITS, units);
+
+    // what is the salt divisor?
+    Integer saltDivisor = PROFILER_SALT_DIVISOR.get(global, Integer.class);
+    LOG.debug("profiler client: {}={}", PROFILER_SALT_DIVISOR, saltDivisor);
+
+    return new SaltyRowKeyBuilder(saltDivisor, duration, units);
+  }
+
+  /**
+   * Create an HBase table used when accessing HBase.
+   * @param global The global configuration.
+   * @return
+   */
+  private HTableInterface getTable(Map<String, Object> global) {
+
+    String tableName = PROFILER_HBASE_TABLE.get(global, String.class);
+    TableProvider provider = getTableProvider(global);
+
+    try {
+      return provider.getTable(HBaseConfiguration.create(), tableName);
+
+    } catch (IOException e) {
+      throw new IllegalArgumentException(String.format("Unable to access table: %s", tableName), e);
+    }
+  }
+
+  /**
+   * Create the TableProvider to use when accessing HBase.
+   * @param global The global configuration.
+   */
+  private TableProvider getTableProvider(Map<String, Object> global) {
+    String clazzName = PROFILER_HBASE_TABLE_PROVIDER.get(global, String.class);
+
+    TableProvider provider;
+    try {
+      @SuppressWarnings("unchecked")
+      Class<? extends TableProvider> clazz = (Class<? extends TableProvider>) Class.forName(clazzName);
+      provider = clazz.getConstructor().newInstance();
+
+    } catch (Exception e) {
+      provider = new HTableProvider();
+    }
+
+    return provider;
   }
 }
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/ProfilerClientConfig.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/ProfilerClientConfig.java
index 08ee064..1715b23 100644
--- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/ProfilerClientConfig.java
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/ProfilerClientConfig.java
@@ -20,9 +20,8 @@
 
 package org.apache.metron.profiler.client.stellar;
 
-import org.apache.metron.hbase.HTableProvider;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
 import org.apache.metron.stellar.common.utils.ConversionUtils;
+import org.apache.metron.hbase.HTableProvider;
 
 import java.util.Map;
 
@@ -43,11 +42,6 @@ public enum ProfilerClientConfig {
   PROFILER_HBASE_TABLE_PROVIDER("hbase.provider.impl", HTableProvider.class.getName(), String.class),
 
   /**
-   * A global property that defines the name of the {@link HBaseConnectionFactory} implementation class.
-   */
-  PROFILER_HBASE_CONNECTION_FACTORY("hbase.connection.factory", HBaseConnectionFactory.class.getName(), String.class),
-
-  /**
    * A global property that defines the duration of each profile period.  This value
    * should be defined along with 'profiler.client.period.duration.units'.
    */
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/Util.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/Util.java
index 304cbe3..ea85c56 100644
--- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/Util.java
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/Util.java
@@ -75,7 +75,7 @@ public class Util {
    * @return effective config Map with overrides applied.
    * @throws ParseException - if any override values are of wrong type.
    */
-  public static Map<String, Object> getEffectiveConfig(Context context, Map configOverridesMap) throws ParseException {
+  public static Map<String, Object> getEffectiveConfig(Context context , Map configOverridesMap ) throws ParseException {
     // ensure the required capabilities are defined
     final Context.Capabilities[] required = { GLOBAL_CONFIG };
     validateCapabilities(context, required);
diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/VerboseProfile.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/VerboseProfile.java
index 31106da..9e857aa 100644
--- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/VerboseProfile.java
+++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/VerboseProfile.java
@@ -18,12 +18,18 @@
 
 package org.apache.metron.profiler.client.stellar;
 
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.metron.hbase.HTableProvider;
+import org.apache.metron.hbase.TableProvider;
 import org.apache.metron.profiler.ProfileMeasurement;
 import org.apache.metron.profiler.ProfilePeriod;
-import org.apache.metron.profiler.client.HBaseProfilerClientFactory;
+import org.apache.metron.profiler.client.HBaseProfilerClient;
 import org.apache.metron.profiler.client.ProfilerClient;
-import org.apache.metron.profiler.client.ProfilerClientFactories;
-import org.apache.metron.profiler.client.ProfilerClientFactory;
+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.stellar.dsl.Context;
 import org.apache.metron.stellar.dsl.ParseException;
 import org.apache.metron.stellar.dsl.Stellar;
@@ -39,9 +45,15 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_COLUMN_FAMILY;
 import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_DEFAULT_VALUE;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE_PROVIDER;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_SALT_DIVISOR;
 import static org.apache.metron.profiler.client.stellar.Util.getArg;
+import static org.apache.metron.profiler.client.stellar.Util.getPeriodDurationInMillis;
 import static org.apache.metron.stellar.dsl.Context.Capabilities.GLOBAL_CONFIG;
 
 /**
@@ -76,101 +88,69 @@ import static org.apache.metron.stellar.dsl.Context.Capabilities.GLOBAL_CONFIG;
         returns="A map for each profile measurement containing the profile name, entity, period, and value."
 )
 public class VerboseProfile implements StellarFunction {
-  protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  protected static final String PROFILE_KEY = "profile";
-  protected static final String ENTITY_KEY = "entity";
-  protected static final String PERIOD_KEY = "period";
-  protected static final String PERIOD_START_KEY = "period.start";
-  protected static final String PERIOD_END_KEY = "period.end";
-  protected static final String VALUE_KEY = "value";
-  protected static final String GROUPS_KEY = "groups";
-  private static final int PROFILE_ARG_INDEX = 0;
-  private static final int ENTITY_ARG_INDEX = 1;
-  private static final int PERIOD_ARG_INDEX = 2;
-  private static final int GROUPS_ARG_INDEX = 3;
-
-  /**
-   * The default constructor used during Stellar function resolution.
-   */
-  public VerboseProfile() {
-    this(ProfilerClientFactories.DEFAULT);
-  }
-
-  /**
-   * The constructor used for testing.
-   * @param profilerClientFactory
-   */
-  public VerboseProfile(ProfilerClientFactory profilerClientFactory) {
-    this.profilerClientFactory = profilerClientFactory;
-  }
-
-  /**
-   * Allows the function to retrieve persisted {@link ProfileMeasurement} values.
-   */
-  private ProfilerClient profilerClient;
 
-  /**
-   * Creates the {@link ProfilerClient} used by this function.
-   */
-  private ProfilerClientFactory profilerClientFactory;
+  protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  protected static String PROFILE_KEY = "profile";
+  protected static String ENTITY_KEY = "entity";
+  protected static String PERIOD_KEY = "period";
+  protected static String PERIOD_START_KEY = "period.start";
+  protected static String PERIOD_END_KEY = "period.end";
+  protected static String VALUE_KEY = "value";
+  protected static String GROUPS_KEY = "groups";
+  private ProfilerClient client;
 
   @Override
   public void initialize(Context context) {
-    // values stored in the global config that are used to initialize the ProfilerClient
-    // are read only once during initialization.  if those values change during a Stellar
-    // session, this function will not respond to them.  the Stellar session would need to be
-    // restarted for those changes to take effect.  this differs from the behavior of `PROFILE_GET`.
-    Map<String, Object> globals = getGlobals(context);
-    profilerClient = profilerClientFactory.create(globals);
+    // nothing to do
   }
 
   @Override
   public boolean isInitialized() {
-    return profilerClient != null;
-  }
-
-  @Override
-  public void close() throws IOException {
-    if(profilerClient != null) {
-      profilerClient.close();
-    }
+    return true;
   }
 
   @Override
   public Object apply(List<Object> args, Context context) throws ParseException {
     // required arguments
-    String profile = getArg(PROFILE_ARG_INDEX, String.class, args);
-    String entity = getArg(ENTITY_ARG_INDEX, String.class, args);
-    List<ProfilePeriod> periods = getArg(PERIOD_ARG_INDEX, List.class, args);
+    String profile = getArg(0, String.class, args);
+    String entity = getArg(1, String.class, args);
+    List<ProfilePeriod> periods = getArg(2, List.class, args);
 
     // optional 'groups' argument
     List<Object> groups = new ArrayList<>();
-    if(args.size() > GROUPS_ARG_INDEX) {
-      groups = getArg(GROUPS_ARG_INDEX, List.class, args);
+    if(args.size() >= 4) {
+      groups = getArg(3, List.class, args);
+    }
+
+    // get globals from the context
+    Map<String, Object> globals = (Map<String, Object>) context.getCapability(GLOBAL_CONFIG)
+            .orElse(Collections.emptyMap());
+
+    // lazily create the profiler client, if needed
+    if (client == null) {
+      RowKeyBuilder rowKeyBuilder = getRowKeyBuilder(globals);
+      ColumnBuilder columnBuilder = getColumnBuilder(globals);
+      HTableInterface table = getTable(globals);
+      long periodDuration = getPeriodDurationInMillis(globals);
+      client = new HBaseProfilerClient(table, rowKeyBuilder, columnBuilder, periodDuration);
     }
 
     // is there a default value?
     Optional<Object> defaultValue = Optional.empty();
-    Map<String, Object> globals = getGlobals(context);
     if(globals != null) {
       defaultValue = Optional.ofNullable(PROFILER_DEFAULT_VALUE.get(globals));
     }
 
+    List<ProfileMeasurement> measurements = client.fetch(Object.class, profile, entity, groups, periods, defaultValue);
+
     // render a view of each profile measurement
-    List<ProfileMeasurement> measurements = profilerClient.fetch(Object.class, profile, entity, groups, periods, defaultValue);
     List<Object> results = new ArrayList<>();
     for(ProfileMeasurement measurement: measurements) {
       results.add(render(measurement));
     }
-
     return results;
   }
 
-  private static Map<String, Object> getGlobals(Context context) {
-    return (Map<String, Object>) context.getCapability(GLOBAL_CONFIG)
-            .orElse(Collections.emptyMap());
-  }
-
   /**
    * Renders a view of the profile measurement.
    * @param measurement The profile measurement to render.
@@ -186,4 +166,57 @@ public class VerboseProfile implements StellarFunction {
     view.put(GROUPS_KEY, measurement.getGroups());
     return view;
   }
+
+  /**
+   * Creates the ColumnBuilder to use in accessing the profile data.
+   * @param global The global configuration.
+   */
+  private ColumnBuilder getColumnBuilder(Map<String, Object> global) {
+    String columnFamily = PROFILER_COLUMN_FAMILY.get(global, String.class);
+    return new ValueOnlyColumnBuilder(columnFamily);
+  }
+
+  /**
+   * Creates the ColumnBuilder to use in accessing the profile data.
+   * @param global The global configuration.
+   */
+  private RowKeyBuilder getRowKeyBuilder(Map<String, Object> global) {
+    Integer saltDivisor = PROFILER_SALT_DIVISOR.get(global, Integer.class);
+    return new SaltyRowKeyBuilder(saltDivisor, getPeriodDurationInMillis(global), TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Create an HBase table used when accessing HBase.
+   * @param global The global configuration.
+   * @return
+   */
+  private HTableInterface getTable(Map<String, Object> global) {
+    String tableName = PROFILER_HBASE_TABLE.get(global, String.class);
+    TableProvider provider = getTableProvider(global);
+    try {
+      return provider.getTable(HBaseConfiguration.create(), tableName);
+
+    } catch (IOException e) {
+      throw new IllegalArgumentException(String.format("Unable to access table: %s", tableName), e);
+    }
+  }
+
+  /**
+   * Create the TableProvider to use when accessing HBase.
+   * @param global The global configuration.
+   */
+  private TableProvider getTableProvider(Map<String, Object> global) {
+    String clazzName = PROFILER_HBASE_TABLE_PROVIDER.get(global, String.class);
+    TableProvider provider;
+    try {
+      @SuppressWarnings("unchecked")
+      Class<? extends TableProvider> clazz = (Class<? extends TableProvider>) Class.forName(clazzName);
+      provider = clazz.getConstructor().newInstance();
+
+    } catch (Exception e) {
+      provider = new HTableProvider();
+    }
+
+    return provider;
+  }
 }
diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientFactoryTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientFactoryTest.java
deleted file mode 100644
index 8465256..0000000
--- a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientFactoryTest.java
+++ /dev/null
@@ -1,108 +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.client;
-
-import org.apache.metron.hbase.client.FakeHBaseConnectionFactory;
-import org.apache.metron.profiler.hbase.SaltyRowKeyBuilder;
-import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_COLUMN_FAMILY;
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE;
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_CONNECTION_FACTORY;
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_PERIOD;
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_PERIOD_UNITS;
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_SALT_DIVISOR;
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests the {@link HBaseProfilerClientFactory}.
- */
-public class HBaseProfilerClientFactoryTest {
-
-  private static final String tableName = "table";
-  private static final String columnFamily = "columnFamily";
-  private static final Integer periodDuration = 23;
-  private static final TimeUnit periodDurationUnits = TimeUnit.MINUTES;
-  private static final Integer saltDivisor = 1000;
-  private static final long periodDurationMillis = periodDurationUnits.toMillis(23);
-  private HBaseProfilerClientFactory factory;
-
-  @Before
-  public void setup() {
-    factory = new HBaseProfilerClientFactory();
-  }
-
-  @Test
-  public void testCreate() {
-    Map<String, Object> globals = new HashMap<String, Object>() {{
-      put(PROFILER_HBASE_TABLE.getKey(), tableName);
-      put(PROFILER_COLUMN_FAMILY.getKey(), columnFamily);
-      put(PROFILER_SALT_DIVISOR.getKey(), saltDivisor.toString());
-      put(PROFILER_HBASE_CONNECTION_FACTORY.getKey(), FakeHBaseConnectionFactory.class.getName());
-      put(PROFILER_PERIOD.getKey(), periodDuration.toString());
-      put(PROFILER_PERIOD_UNITS.getKey(), periodDurationUnits.toString());
-    }};
-
-    HBaseProfilerClient client = factory.create(globals);
-    assertEquals(periodDurationMillis, client.getPeriodDurationMillis());
-
-    // validate the row key builder that is created
-    SaltyRowKeyBuilder rowKeyBuilder = (SaltyRowKeyBuilder) client.getRowKeyBuilder();
-    assertEquals(saltDivisor, (Integer) rowKeyBuilder.getSaltDivisor());
-    assertEquals(periodDurationMillis, rowKeyBuilder.getPeriodDurationMillis());
-
-    // validate the column builder that is created
-    ValueOnlyColumnBuilder columnBuilder = (ValueOnlyColumnBuilder) client.getColumnBuilder();
-    assertEquals(columnFamily, columnBuilder.getColumnFamily());
-  }
-
-  @Test
-  public void testCreateUsingDefaultValues() {
-    Map<String, Object> globals = new HashMap<String, Object>() {{
-      // without using a mock connection factory, the test will hang trying to connect to HBase
-      put(PROFILER_HBASE_CONNECTION_FACTORY.getKey(), FakeHBaseConnectionFactory.class.getName());
-    }};
-
-    // find what the default values should be
-    final long defaultPeriodDuration = (Long) PROFILER_PERIOD.getDefault();
-    final TimeUnit defaultPeriodDurationUnits = TimeUnit.valueOf((String) PROFILER_PERIOD_UNITS.getDefault());
-    final long defaultPeriodDurationMillis = defaultPeriodDurationUnits.toMillis(defaultPeriodDuration);
-    final long defaultSaltDivisor = (Long) PROFILER_SALT_DIVISOR.getDefault();
-    final String defaultColumnFamily = (String) PROFILER_COLUMN_FAMILY.getDefault();
-
-    HBaseProfilerClient client = factory.create(globals);
-    assertEquals(defaultPeriodDurationMillis, client.getPeriodDurationMillis());
-
-    // validate the row key builder that is created
-    SaltyRowKeyBuilder rowKeyBuilder = (SaltyRowKeyBuilder) client.getRowKeyBuilder();
-    assertEquals(defaultSaltDivisor, rowKeyBuilder.getSaltDivisor());
-    assertEquals(defaultPeriodDurationMillis, rowKeyBuilder.getPeriodDurationMillis());
-
-    // validate the column builder that is created
-    ValueOnlyColumnBuilder columnBuilder = (ValueOnlyColumnBuilder) client.getColumnBuilder();
-    assertEquals(defaultColumnFamily, columnBuilder.getColumnFamily());
-  }
-}
diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java
index f2dce99..cc3748e 100644
--- a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java
+++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java
@@ -20,187 +20,184 @@
 
 package org.apache.metron.profiler.client;
 
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.common.utils.SerDeUtils;
-import org.apache.metron.hbase.client.HBaseTableClient;
-import org.apache.metron.hbase.client.HBaseClient;
+import org.apache.metron.hbase.mock.MockHTable;
 import org.apache.metron.profiler.ProfileMeasurement;
-import org.apache.metron.profiler.ProfilePeriod;
 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.stellar.common.DefaultStellarStatefulExecutor;
+import org.apache.metron.stellar.common.StellarStatefulExecutor;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.ArgumentCaptor;
 
-import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.junit.Assert.assertTrue;
 
 /**
- * Tests the {@link HBaseProfilerClient}.
+ * Tests the HBaseProfilerClient.
+ *
+ * The naming used in this test attempts to be as similar to how the 'groupBy'
+ * functionality might be used 'in the wild'.  This test involves reading and
+ * writing two separate groups originating from the same Profile and Entity.
+ * There is a 'weekdays' group which contains all measurements taken on weekdays.
+ * There is also a 'weekend' group which contains all measurements taken on weekends.
  */
 public class HBaseProfilerClientTest {
+
   private static final String tableName = "profiler";
   private static final String columnFamily = "P";
-  private static final byte[] columnFamilyB = Bytes.toBytes(columnFamily);
-  private static final byte[] columnQualifier = Bytes.toBytes("column");
   private static final long periodDuration = 15;
   private static final TimeUnit periodUnits = TimeUnit.MINUTES;
   private static final int periodsPerHour = 4;
-  private static final byte[] expectedRowKey = Bytes.toBytes("some-row-key");
-  private static final String profileName = "profile1";
-  private static final String entityName = "entity1";
-  private static final int profileValue = 1231121;
-  private static final byte[] profileValueB = SerDeUtils.toBytes(profileValue);
-  private long periodDurationMillis = periodUnits.toMillis(periodDuration);
-
-  private HBaseClient hbaseClient;
-  private HBaseProfilerClient profilerClient;
-  private ProfileMeasurement expected;
-  private RowKeyBuilder rowKeyBuilder;
-  private ColumnBuilder columnBuilder;
-  private Result expectedResult;
-  private Result emptyResult;
+
+  private HBaseProfilerClient client;
+  private StellarStatefulExecutor executor;
+  private MockHTable table;
+  private ProfileWriter profileWriter;
 
   @Before
-  public void setup() {
-    // create a profile measurement used in the tests
-    expected = new ProfileMeasurement()
-            .withProfileName(profileName)
-            .withEntity(entityName)
-            .withPeriod(System.currentTimeMillis(), 5, TimeUnit.MINUTES)
-            .withProfileValue(profileValue);
-
-    // mock row key builder needs to return a row key for the profile measurement used in the tests
-    rowKeyBuilder = mock(RowKeyBuilder.class);
-    when(rowKeyBuilder.rowKey(any())).thenReturn(expectedRowKey);
-
-    // mock column builder - column family/qualifier comes from the column builder
-    columnBuilder = mock(ColumnBuilder.class);
-    when(columnBuilder.getColumnFamily()).thenReturn(columnFamily);
-    when(columnBuilder.getColumnQualifier(eq("value"))).thenReturn(columnQualifier);
-
-    // this mock is used to feed data to the profiler client while testing
-    hbaseClient = mock(HBaseTableClient.class);
-
-    // a result that matches the expected profile measurement that can be return by the mock hbase client
-    expectedResult = mock(Result.class);
-    when(expectedResult.containsColumn(eq(columnFamilyB), eq(columnQualifier))).thenReturn(true);
-    when(expectedResult.getValue(eq(columnFamilyB), eq(columnQualifier))).thenReturn(profileValueB);
-
-    // an empty result to use in the tests
-    emptyResult = mock(Result.class);
-    when(emptyResult.containsColumn(any(), any())).thenReturn(false);
-
-    // create the profiler client that will be tested
-    profilerClient = new HBaseProfilerClient(hbaseClient, rowKeyBuilder, columnBuilder, periodDurationMillis);
+  public void setup() throws Exception {
+    table = new MockHTable(tableName, columnFamily);
+    executor = new DefaultStellarStatefulExecutor();
+
+    // writes values to be read during testing
+    long periodDurationMillis = periodUnits.toMillis(periodDuration);
+    RowKeyBuilder rowKeyBuilder = new SaltyRowKeyBuilder();
+    ColumnBuilder columnBuilder = new ValueOnlyColumnBuilder(columnFamily);
+    profileWriter = new ProfileWriter(rowKeyBuilder, columnBuilder, table, periodDurationMillis);
+
+    client = new HBaseProfilerClient(table, rowKeyBuilder, columnBuilder, periodDurationMillis);
   }
 
-  @Test
-  public void shouldFetchProfileMeasurement() {
-    // need the hbase client to return a Result matching the expected profile measurement value
-    Result[] results = new Result[] { expectedResult };
-    when(hbaseClient.getAll()).thenReturn(results);
-
-    List<ProfileMeasurement> measurements = profilerClient.fetch(
-            Object.class,
-            expected.getProfileName(),
-            expected.getEntity(),
-            expected.getGroups(),
-            Arrays.asList(expected.getPeriod()),
-            Optional.empty());
-    assertEquals(1, measurements.size());
-    assertEquals(expected, measurements.get(0));
+  @After
+  public void tearDown() throws Exception {
+    table.clear();
   }
 
   @Test
-  public void shouldFetchNothingWhenNothingThere() {
-    // the hbase client will indicate their are no hits
-    Result[] results = new Result[] { emptyResult };
-    when(hbaseClient.getAll()).thenReturn(results);
-
-    List<ProfileMeasurement> measurements = profilerClient.fetch(
-            Object.class,
-            expected.getProfileName(),
-            expected.getEntity(),
-            expected.getGroups(),
-            Arrays.asList(expected.getPeriod()),
-            Optional.empty());
-    assertEquals(0, measurements.size());
+  public void Should_ReturnMeasurements_When_DataExistsForAGroup() throws Exception {
+    final String profile = "profile1";
+    final String entity = "entity1";
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final int count = hours * periodsPerHour + 1;
+    final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours);
+
+    // setup - write two groups of measurements - 'weekends' and 'weekdays'
+    ProfileMeasurement prototype = new ProfileMeasurement()
+            .withProfileName(profile)
+            .withEntity(entity)
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(prototype, count, Arrays.asList("weekdays"), val -> expectedValue);
+    profileWriter.write(prototype, count, Arrays.asList("weekends"), val -> 0);
+
+    long end = System.currentTimeMillis();
+    long start = end - TimeUnit.HOURS.toMillis(2);
+    {
+      //validate "weekday" results
+      List<Object> groups = Arrays.asList("weekdays");
+      List<ProfileMeasurement> results = client.fetch(Integer.class, profile, entity, groups, start, end, Optional.empty());
+      assertEquals(count, results.size());
+      results.forEach(actual -> {
+        assertEquals(profile, actual.getProfileName());
+        assertEquals(entity, actual.getEntity());
+        assertEquals(groups, actual.getGroups());
+        assertEquals(expectedValue, actual.getProfileValue());
+      });
+    }
+    {
+      //validate "weekend" results
+      List<Object> groups = Arrays.asList("weekends");
+      List<ProfileMeasurement> results = client.fetch(Integer.class, profile, entity, groups, start, end, Optional.empty());
+      assertEquals(count, results.size());
+      results.forEach(actual -> {
+        assertEquals(profile, actual.getProfileName());
+        assertEquals(entity, actual.getEntity());
+        assertEquals(groups, actual.getGroups());
+        assertEquals(0, actual.getProfileValue());
+      });
+    }
   }
 
   @Test
-  public void shouldFetchDefaultValueWhenNothingThere() {
-    // the hbase client will indicate their are no hits
-    Result[] results = new Result[] { emptyResult };
-    when(hbaseClient.getAll()).thenReturn(results);
-
-    List<ProfileMeasurement> measurements = profilerClient.fetch(
-            Object.class,
-            expected.getProfileName(),
-            expected.getEntity(),
-            expected.getGroups(),
-            Arrays.asList(expected.getPeriod()),
-            Optional.of(profileValue));
-
-    // expect the default value to be returned
-    assertEquals(1, measurements.size());
-    assertEquals(expected, measurements.get(0));
+  public void Should_ReturnResultFromGroup_When_MultipleGroupsExist() throws Exception {
+    final String profile = "profile1";
+    final String entity = "entity1";
+    final int periodsPerHour = 4;
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final int count = hours * periodsPerHour;
+    final long endTime = System.currentTimeMillis();
+    final long startTime = endTime - TimeUnit.HOURS.toMillis(hours);
+
+    // setup - write two groups of measurements - 'weekends' and 'weekdays'
+    ProfileMeasurement m = new ProfileMeasurement()
+            .withProfileName("profile1")
+            .withEntity("entity1")
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, count, Arrays.asList("weekdays"), val -> expectedValue);
+    profileWriter.write(m, count, Arrays.asList("weekends"), val -> 0);
+
+    List<Object> weekdays = Arrays.asList("weekdays");
+    List<ProfileMeasurement> results = client.fetch(Integer.class, profile, entity, weekdays, startTime, endTime, Optional.empty());
+
+    // should only return results from 'weekdays' group
+    assertEquals(count, results.size());
+    results.forEach(actual -> assertEquals(weekdays, actual.getGroups()));
   }
 
   @Test
-  public void shouldFetchMultipleProfilePeriods() {
-    // need the hbase client to return a Result matching the expected profile measurement value
-    Result[] results = new Result[] { expectedResult, expectedResult, expectedResult, expectedResult };
-    when(hbaseClient.getAll()).thenReturn(results);
-
-    // fetching across multiple periods
-    ProfilePeriod start = ProfilePeriod.fromPeriodId(1L, 15L, TimeUnit.MINUTES);
-    List<ProfilePeriod> periods = new ArrayList<ProfilePeriod>() {{
-      add(start);
-      add(start.next());
-      add(start.next());
-      add(start.next());
-    }};
-
-    List<ProfileMeasurement> measurements = profilerClient.fetch(
-            Object.class,
-            expected.getProfileName(),
-            expected.getEntity(),
-            expected.getGroups(),
-            periods,
-            Optional.empty());
-
-    // the row key builder should be called once for each profile period
-    ArgumentCaptor<ProfileMeasurement> captor = new ArgumentCaptor<>();
-    verify(rowKeyBuilder, times(4)).rowKey(captor.capture());
-
-    // the profile periods should match those originally submited
-    List<ProfileMeasurement> submitted = captor.getAllValues();
-    assertEquals(periods.get(0), submitted.get(0).getPeriod());
-    assertEquals(periods.get(1), submitted.get(1).getPeriod());
-    assertEquals(periods.get(2), submitted.get(2).getPeriod());
-    assertEquals(periods.get(3), submitted.get(3).getPeriod());
-
-    assertEquals(4, measurements.size());
+  public void Should_ReturnNoResults_When_GroupDoesNotExist() {
+    final String profile = "profile1";
+    final String entity = "entity1";
+    final int periodsPerHour = 4;
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final int count = hours * periodsPerHour;
+    final long endTime = System.currentTimeMillis();
+    final long startTime = endTime - TimeUnit.HOURS.toMillis(hours);
+
+    // create two groups of measurements - one on weekdays and one on weekends
+    ProfileMeasurement m = new ProfileMeasurement()
+            .withProfileName("profile1")
+            .withEntity("entity1")
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, count, Arrays.asList("weekdays"), val -> expectedValue);
+    profileWriter.write(m, count, Arrays.asList("weekends"), val -> 0);
+
+    // should return no results when the group does not exist
+    List<Object> groups = Arrays.asList("does-not-exist");
+    List<ProfileMeasurement> results = client.fetch(Integer.class, profile, entity, groups, startTime, endTime, Optional.empty());
+    assertEquals(0, results.size());
   }
 
   @Test
-  public void shouldCloseHBaseClient() throws IOException {
-    profilerClient.close();
-    verify(hbaseClient, times(1)).close();
+  public void Should_ReturnNoResults_When_NoDataInStartToEnd() throws Exception {
+    final String profile = "profile1";
+    final String entity = "entity1";
+    final int hours = 2;
+    int numberToWrite = hours * periodsPerHour;
+    final List<Object> group = Arrays.asList("weekends");
+    final long measurementTime = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(1);
+
+    // write some data with a timestamp of s1 day ago
+    ProfileMeasurement prototype = new ProfileMeasurement()
+            .withProfileName("profile1")
+            .withEntity("entity1")
+            .withPeriod(measurementTime, periodDuration, periodUnits);
+    profileWriter.write(prototype, numberToWrite, group, val -> 1000);
+
+    // should return no results when [start,end] is long after when test data was written
+    final long endFetchAt = System.currentTimeMillis();
+    final long startFetchAt = endFetchAt - TimeUnit.MILLISECONDS.toMillis(30);
+    List<ProfileMeasurement> results = client.fetch(Integer.class, profile, entity, group, startFetchAt, endFetchAt, Optional.empty());
+    assertEquals(0, results.size());
   }
 }
\ No newline at end of file
diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/ProfileWriter.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/ProfileWriter.java
new file mode 100644
index 0000000..4e00164
--- /dev/null
+++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/ProfileWriter.java
@@ -0,0 +1,125 @@
+/*
+ *
+ *  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.client;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.metron.hbase.HTableProvider;
+import org.apache.metron.hbase.ColumnList;
+import org.apache.metron.hbase.client.HBaseClient;
+import org.apache.metron.profiler.ProfileMeasurement;
+import org.apache.metron.profiler.ProfilePeriod;
+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 java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+/**
+ * Writes ProfileMeasurement values that can be read during automated testing.
+ */
+public class ProfileWriter {
+
+  private RowKeyBuilder rowKeyBuilder;
+  private ColumnBuilder columnBuilder;
+  private HBaseClient hbaseClient;
+  private HBaseProfilerClient client;
+
+  public ProfileWriter(RowKeyBuilder rowKeyBuilder, ColumnBuilder columnBuilder, HTableInterface table, long periodDurationMillis) {
+    this.rowKeyBuilder = rowKeyBuilder;
+    this.columnBuilder = columnBuilder;
+    this.hbaseClient = new HBaseClient((c, t) -> table, table.getConfiguration(), table.getName().getNameAsString());
+    this.client = new HBaseProfilerClient(table, rowKeyBuilder, columnBuilder, periodDurationMillis);
+  }
+
+  /**
+   * Writes profile measurements that can be used for testing.
+   *
+   * @param prototype      A prototype for the types of ProfileMeasurements that should be written.
+   * @param count          The number of profile measurements to write.
+   * @param group          The name of the group.
+   * @param valueGenerator A function that consumes the previous ProfileMeasurement value and produces the next.
+   */
+  public void write(ProfileMeasurement prototype, int count, List<Object> group, Function<Object, Object> valueGenerator) {
+
+    ProfileMeasurement m = prototype;
+    ProfilePeriod period = m.getPeriod();
+    for(int i=0; i<count; i++) {
+      // generate the next value that should be written
+      Object nextValue = valueGenerator.apply(m.getProfileValue());
+
+      // write the measurement
+      m = new ProfileMeasurement()
+              .withProfileName(prototype.getProfileName())
+              .withEntity(prototype.getEntity())
+              .withPeriod(period)
+              .withGroups(group)
+              .withProfileValue(nextValue);
+      write(m);
+
+      // advance to the next period
+      period = m.getPeriod().next();
+    }
+  }
+
+  /**
+   * Write a ProfileMeasurement.
+   * @param m The ProfileMeasurement to write.
+   */
+  private void write(ProfileMeasurement m) {
+
+    byte[] rowKey = rowKeyBuilder.rowKey(m);
+    ColumnList cols = columnBuilder.columns(m);
+
+    hbaseClient.addMutation(rowKey, cols, Durability.SKIP_WAL);
+    hbaseClient.mutate();
+  }
+
+  public static void main(String[] args) throws Exception {
+    RowKeyBuilder rowKeyBuilder = new SaltyRowKeyBuilder();
+    ColumnBuilder columnBuilder = new ValueOnlyColumnBuilder();
+
+    Configuration config = HBaseConfiguration.create();
+    config.set("hbase.master.hostname", "node1");
+    config.set("hbase.regionserver.hostname", "node1");
+    config.set("hbase.zookeeper.quorum", "node1");
+
+    HTableProvider provider = new HTableProvider();
+    HTableInterface table = provider.getTable(config, "profiler");
+
+    long periodDurationMillis = TimeUnit.MINUTES.toMillis(15);
+    long when = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(2);
+    ProfileMeasurement measure = new ProfileMeasurement()
+            .withProfileName("profile1")
+            .withEntity("192.168.66.121")
+            .withPeriod(when, periodDurationMillis, TimeUnit.MILLISECONDS);
+
+    ProfileWriter writer = new ProfileWriter(rowKeyBuilder, columnBuilder, table, periodDurationMillis);
+    writer.write(measure, 2 * 24 * 4, Collections.emptyList(), val -> new Random().nextInt(10));
+  }
+}
diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/GetProfileTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/GetProfileTest.java
index 41afa04..3fbed1c 100644
--- a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/GetProfileTest.java
+++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/GetProfileTest.java
@@ -20,191 +20,437 @@
 
 package org.apache.metron.profiler.client.stellar;
 
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.metron.hbase.mock.MockHBaseTableProvider;
 import org.apache.metron.profiler.ProfileMeasurement;
-import org.apache.metron.profiler.client.ProfilerClient;
+import org.apache.metron.profiler.client.ProfileWriter;
+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.stellar.common.DefaultStellarStatefulExecutor;
 import org.apache.metron.stellar.common.StellarStatefulExecutor;
 import org.apache.metron.stellar.dsl.Context;
-import org.apache.metron.stellar.dsl.functions.resolver.FunctionResolver;
+import org.apache.metron.stellar.dsl.ParseException;
 import org.apache.metron.stellar.dsl.functions.resolver.SimpleFunctionResolver;
+import org.apache.metron.stellar.dsl.functions.resolver.SingletonFunctionResolver;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_COLUMN_FAMILY;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE_PROVIDER;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_PERIOD;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_PERIOD_UNITS;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_SALT_DIVISOR;
 
 /**
- * Tests the 'PROFILE_GET' function in the {@link GetProfile} class.
+ * Tests the GetProfile class.
  */
 public class GetProfileTest {
 
+  private static final long periodDuration = 15;
+  private static final TimeUnit periodUnits = TimeUnit.MINUTES;
+  private static final int saltDivisor = 1000;
+  private static final String tableName = "profiler";
+  private static final String columnFamily = "P";
   private StellarStatefulExecutor executor;
-  private FunctionResolver functionResolver;
-  private Map<String, Object> globals;
-  private GetProfile function;
-  private ProfilerClient profilerClient;
-  private List<Integer> results;
-  private ProfileMeasurement expected;
-  private ProfileMeasurement defaultMeasurement;
-  private Object defaultValue;
-
-  private List run(String expression) {
-    return executor.execute(expression, new HashMap<>(), List.class);
+  private Map<String, Object> state;
+  private ProfileWriter profileWriter;
+  // different values of period and salt divisor, used to test config_overrides feature
+  private static final long periodDuration2 = 1;
+  private static final TimeUnit periodUnits2 = TimeUnit.HOURS;
+  private static final int saltDivisor2 = 2050;
+
+  private <T> T run(String expression, Class<T> clazz) {
+    return executor.execute(expression, state, clazz);
   }
 
+  /**
+   * This method sets up the configuration context for both writing profile data
+   * (using profileWriter to mock the complex process of what the Profiler topology
+   * actually does), and then reading that profile data (thereby testing the PROFILE_GET
+   * Stellar client implemented in GetProfile).
+   *
+   * It runs at @Before time, and sets testclass global variables used by the writers and readers.
+   * The various writers and readers are in each test case, not here.
+   *
+   * @return void
+   */
   @Before
   public void setup() {
-    // the mock profiler client used to feed profile measurement values to the function
-    profilerClient = mock(ProfilerClient.class);
+    state = new HashMap<>();
+    final HTableInterface table = MockHBaseTableProvider.addToCache(tableName, columnFamily);
+
+    // used to write values to be read during testing
+    long periodDurationMillis = TimeUnit.MINUTES.toMillis(15);
+    RowKeyBuilder rowKeyBuilder = new SaltyRowKeyBuilder();
+    ColumnBuilder columnBuilder = new ValueOnlyColumnBuilder(columnFamily);
+    profileWriter = new ProfileWriter(rowKeyBuilder, columnBuilder, table, periodDurationMillis);
 
     // global properties
-    globals = new HashMap<>();
-    Context context = new Context.Builder()
-            .with(Context.Capabilities.GLOBAL_CONFIG, () -> globals)
-            .build();
+    Map<String, Object> global = new HashMap<String, Object>() {{
+      put(PROFILER_HBASE_TABLE.getKey(), tableName);
+      put(PROFILER_COLUMN_FAMILY.getKey(), columnFamily);
+      put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockHBaseTableProvider.class.getName());
+      put(PROFILER_PERIOD.getKey(), Long.toString(periodDuration));
+      put(PROFILER_PERIOD_UNITS.getKey(), periodUnits.toString());
+      put(PROFILER_SALT_DIVISOR.getKey(), Integer.toString(saltDivisor));
+    }};
+
+    // create the stellar execution environment
+    executor = new DefaultStellarStatefulExecutor(
+            new SimpleFunctionResolver()
+                    .withClass(GetProfile.class)
+                    .withClass(FixedLookback.class),
+            new Context.Builder()
+                    .with(Context.Capabilities.GLOBAL_CONFIG, () -> global)
+                    .build());
+  }
+
+  /**
+   * This method is similar to setup(), in that it sets up profiler configuration context,
+   * but only for the client.  Additionally, it uses periodDuration2, periodUnits2
+   * and saltDivisor2, instead of periodDuration, periodUnits and saltDivisor respectively.
+   *
+   * This is used in the unit tests that test the config_overrides feature of PROFILE_GET.
+   * In these tests, the context from @Before setup() is used to write the data, then the global
+   * context is changed to context2 (from this method).  Each test validates that a default read
+   * using global context2 then gets no valid results (as expected), and that a read using
+   * original context values in the PROFILE_GET config_overrides argument gets all expected results.
+   *
+   * @return context2 - The profiler client configuration context created by this method.
+   *    The context2 values are also set in the configuration of the StellarStatefulExecutor
+   *    stored in the global variable 'executor'.  However, there is no API for querying the
+   *    context values from a StellarStatefulExecutor, so we output the context2 Context object itself,
+   *    for validation purposes (so that its values can be validated as being significantly
+   *    different from the setup() settings).
+   */
+  private Context setup2() {
+    state = new HashMap<>();
+
+    // global properties
+    Map<String, Object> global = new HashMap<String, Object>() {{
+      put(PROFILER_HBASE_TABLE.getKey(), tableName);
+      put(PROFILER_COLUMN_FAMILY.getKey(), columnFamily);
+      put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockHBaseTableProvider.class.getName());
+      put(PROFILER_PERIOD.getKey(), Long.toString(periodDuration2));
+      put(PROFILER_PERIOD_UNITS.getKey(), periodUnits2.toString());
+      put(PROFILER_SALT_DIVISOR.getKey(), Integer.toString(saltDivisor2));
+    }};
 
-    // the PROFILE_GET function that will be tested
-    function = new GetProfile(globals -> profilerClient);
-    function.initialize(context);
+    // create the modified context
+    Context context2 = new Context.Builder()
+            .with(Context.Capabilities.GLOBAL_CONFIG, () -> global)
+            .build();
 
     // create the stellar execution environment
-    functionResolver = new SimpleFunctionResolver()
-            .withClass(FixedLookback.class)
-            .withInstance(function);
-    executor = new DefaultStellarStatefulExecutor(functionResolver, context);
+    executor = new DefaultStellarStatefulExecutor(
+            new SimpleFunctionResolver()
+                    .withClass(GetProfile.class)
+                    .withClass(FixedLookback.class),
+            context2);
+
+    return context2; //because there is no executor.getContext() method
+  }
 
-    // create a profile measurement used in the tests
-    expected = new ProfileMeasurement()
+  /**
+   * Values should be retrievable that have NOT been stored within a group.
+   */
+  @Test
+  public void testWithNoGroups() {
+    final int periodsPerHour = 4;
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours);
+    final List<Object> group = Collections.emptyList();
+
+    // setup - write some measurements to be read later
+    final int count = hours * periodsPerHour;
+    ProfileMeasurement m = new ProfileMeasurement()
             .withProfileName("profile1")
             .withEntity("entity1")
-            .withPeriod(System.currentTimeMillis(), 5, TimeUnit.MINUTES)
-            .withProfileValue(1231121);
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, count, group, val -> expectedValue);
+
+    // execute - read the profile values - no groups
+    String expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))";
+    @SuppressWarnings("unchecked")
+    List<Integer> result = run(expr, List.class);
+
+    // validate - expect to read all values from the past 4 hours
+    Assert.assertEquals(count, result.size());
+    result.forEach(actual -> Assert.assertEquals(expectedValue, actual.intValue()));
+  }
+
+  /**
+   * Values should be retrievable that have been stored within a 'group'.
+   */
+  @Test
+  public void testWithOneGroup() {
+    final int periodsPerHour = 4;
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours);
+    final List<Object> group = Arrays.asList("weekends");
 
-    defaultValue = 7777777;
-    defaultMeasurement = new ProfileMeasurement()
+    // setup - write some measurements to be read later
+    final int count = hours * periodsPerHour;
+    ProfileMeasurement m = new ProfileMeasurement()
             .withProfileName("profile1")
             .withEntity("entity1")
-            .withPeriod(System.currentTimeMillis(), 5, TimeUnit.MINUTES)
-            .withProfileValue(defaultValue);
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, count, group, val -> expectedValue);
+
+    // create a variable that contains the groups to use
+    state.put("groups", group);
+
+    // execute - read the profile values
+    String expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), ['weekends'])";
+    @SuppressWarnings("unchecked")
+    List<Integer> result = run(expr, List.class);
+
+    // validate - expect to read all values from the past 4 hours
+    Assert.assertEquals(count, result.size());
+
+    // test the deprecated but allowed "varargs" form of groups specification
+    expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), 'weekends')";
+    result = run(expr, List.class);
+
+    // validate - expect to read all values from the past 4 hours
+    Assert.assertEquals(count, result.size());
+    result.forEach(actual -> Assert.assertEquals(expectedValue, actual.intValue()));
   }
 
+  /**
+   * Values should be retrievable that have been stored within a 'group'.
+   */
   @Test
-  public void testGetProfileValue() {
-    // only one profile measurement exists
-    when(profilerClient.fetch(
-            eq(Object.class),
-            eq(expected.getProfileName()),
-            eq(expected.getEntity()),
-            eq(expected.getGroups()),
-            any(),
-            any())).thenReturn(Arrays.asList(expected));
-
-    // expect the one measurement to be returned
-    results = run("PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))");
-    assertEquals(1, results.size());
-    assertEquals(expected.getProfileValue(), results.get(0));
+  public void testWithTwoGroups() {
+    final int periodsPerHour = 4;
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours);
+    final List<Object> group = Arrays.asList("weekdays", "tuesday");
+
+    // setup - write some measurements to be read later
+    final int count = hours * periodsPerHour;
+    ProfileMeasurement m = new ProfileMeasurement()
+            .withProfileName("profile1")
+            .withEntity("entity1")
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, count, group, val -> expectedValue);
+
+    // create a variable that contains the groups to use
+    state.put("groups", group);
+
+    // execute - read the profile values
+    String expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), ['weekdays', 'tuesday'])";
+    @SuppressWarnings("unchecked")
+    List<Integer> result = run(expr, List.class);
+
+    // validate - expect to read all values from the past 4 hours
+    Assert.assertEquals(count, result.size());
+
+    // test the deprecated but allowed "varargs" form of groups specification
+    expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), 'weekdays', 'tuesday')";
+    result = run(expr, List.class);
+
+    // validate - expect to read all values from the past 4 hours
+    Assert.assertEquals(count, result.size());
+    result.forEach(actual -> Assert.assertEquals(expectedValue, actual.intValue()));
   }
 
-  @Test
-  public void testGetProfileValueWithGroup() {
-    // the profile measurement is part of a group
-    expected.withGroups(Arrays.asList("group1"));
-
-    // only one profile measurement exists
-    when(profilerClient.fetch(
-            eq(Object.class),
-            eq(expected.getProfileName()),
-            eq(expected.getEntity()),
-            eq(expected.getGroups()),
-            any(),
-            any())).thenReturn(Arrays.asList(expected));
-
-    // expect the one measurement to be returned
-    results = run("PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), ['group1'])");
-    assertEquals(1, results.size());
-    assertEquals(expected.getProfileValue(), results.get(0));
+  /**
+   * Initialization should fail if the required context values are missing.
+   */
+  @Test(expected = ParseException.class)
+  public void testMissingContext() {
+    Context empty = Context.EMPTY_CONTEXT();
+
+    // 'unset' the context that was created during setup()
+    executor.setContext(empty);
+
+    // force re-initialization with no context
+    SingletonFunctionResolver.getInstance().initialize(empty);
+
+    // validate - function should be unable to initialize
+    String expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(1000, 'SECONDS'), groups)";
+    run(expr, List.class);
   }
 
+  /**
+   * If the time horizon specified does not include any profile measurements, then
+   * none should be returned.
+   */
   @Test
-  public void testGetProfileValueWithDifferentGroup() {
-    // the profile measurement is part of a group
-    expected.withGroups(Arrays.asList("group1"));
-
-    // only one profile measurement exists
-    when(profilerClient.fetch(
-            eq(Object.class),
-            eq(expected.getProfileName()),
-            eq(expected.getEntity()),
-            eq(expected.getGroups()),
-            any(),
-            any())).thenReturn(Arrays.asList(expected));
-
-    // expect the one measurement to be returned
-    results = run("PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), ['group999'])");
-    assertEquals(0, results.size());
+  public void testOutsideTimeHorizon() {
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours);
+    final List<Object> group = Collections.emptyList();
+
+    // setup - write a single value from 2 hours ago
+    ProfileMeasurement m = new ProfileMeasurement()
+            .withProfileName("profile1")
+            .withEntity("entity1")
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, 1, group, val -> expectedValue);
+
+    // create a variable that contains the groups to use
+    state.put("groups", group);
+
+    // execute - read the profile values
+    String expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'SECONDS'))";
+    @SuppressWarnings("unchecked")
+    List<Integer> result = run(expr, List.class);
+
+    // validate - there should be no values from only 4 seconds ago
+    Assert.assertEquals(0, result.size());
   }
 
+  /**
+   * Default value should be able to be specified
+   */
   @Test
-  public void shouldReturnNothingWhenNoMeasurementsExist() {
-    // no measurements exist
-    when(profilerClient.fetch(
-            eq(Object.class),
-            any(),
-            any(),
-            any(),
-            any(),
-            any())).thenReturn(Collections.emptyList());
-
-    // no measurements exist
-    results = run("PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))");
-    assertEquals(0, results.size());
+  public void testWithDefaultValue() {
+    String expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))";
+    @SuppressWarnings("unchecked")
+    List<Integer> result = run(expr, List.class);
+
+    // validate - expect to fail to read any values because we didn't write any.
+    Assert.assertEquals(0, result.size());
+
+    // execute - read the profile values - with config_override.
+    // first two override values are strings, third is deliberately a number.
+    testOverride("{'profiler.default.value' : 0}", 0);
+    testOverride("{'profiler.default.value' : 'metron'}", "metron");
+    testOverride("{'profiler.default.value' : []}", new ArrayList<>());
   }
 
+  private void testOverride(String overrides, Object defaultVal) {
+      String expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), [], " + overrides + ")";
+      List<Object> result = run(expr, List.class);
+
+      // validate - expect to read all values from the past 4 hours (16 or 17 values depending on start time)
+      // but they should all be the default value.
+      Assert.assertTrue(result.size() == 16 || result.size() == 17);
+      result.forEach(actual -> Assert.assertEquals(defaultVal, actual));
+  }
+
+  /**
+   * Values should be retrievable that were written with configuration different than current global config.
+   */
   @Test
-  public void shouldUseDefaultValueFromGlobals() {
-    // set a default value
-    globals.put("profiler.default.value", defaultValue);
-
-    // the underlying profile client is responsible for returning the default value, if no profiles are found
-    when(profilerClient.fetch(
-            eq(Object.class),
-            eq(expected.getProfileName()),
-            eq(expected.getEntity()),
-            eq(expected.getGroups()),
-            any(),
-            eq(Optional.of(defaultValue)))).thenReturn(Arrays.asList(defaultMeasurement));
-
-    results = run("PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))");
-    assertEquals(1, results.size());
-    assertEquals(defaultValue, results.get(0));
+  public void testWithConfigOverride() {
+    final int periodsPerHour = 4;
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours);
+    final List<Object> group = Collections.emptyList();
+
+    // setup - write some measurements to be read later
+    final int count = hours * periodsPerHour;
+    ProfileMeasurement m = new ProfileMeasurement()
+            .withProfileName("profile1")
+            .withEntity("entity1")
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, count, group, val -> expectedValue);
+
+    // now change the executor configuration
+    Context context2 = setup2();
+    // validate it is changed in significant way
+    @SuppressWarnings("unchecked")
+    Map<String, Object> global = (Map<String, Object>) context2.getCapability(Context.Capabilities.GLOBAL_CONFIG).get();
+    Assert.assertEquals(PROFILER_PERIOD.get(global), periodDuration2);
+    Assert.assertNotEquals(periodDuration, periodDuration2);
+
+    // execute - read the profile values - with (wrong) default global config values.
+    // No error message at this time, but returns empty results list, because
+    // row keys are not correctly calculated.
+    String expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))";
+    @SuppressWarnings("unchecked")
+    List<Integer> result = run(expr, List.class);
+
+    // validate - expect to fail to read any values
+    Assert.assertEquals(0, result.size());
+
+    // execute - read the profile values - with config_override.
+    // first two override values are strings, third is deliberately a number.
+    String overrides = "{'profiler.client.period.duration' : '" + periodDuration + "', "
+            + "'profiler.client.period.duration.units' : '" + periodUnits.toString() + "', "
+            + "'profiler.client.salt.divisor' : " + saltDivisor + " }";
+    expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS', " + overrides + "), [], " + overrides + ")"
+            ;
+    result = run(expr, List.class);
+
+    // validate - expect to read all values from the past 4 hours
+    Assert.assertEquals(count, result.size());
+    result.forEach(actual -> Assert.assertEquals(expectedValue, actual.intValue()));
   }
 
+  /**
+   * Values should be retrievable that have been stored within a 'group', with
+   * configuration different than current global config.
+   * This time put the config_override case before the non-override case.
+   */
   @Test
-  public void shouldUseDefaultValueFromOverrides() {
-    // the underlying profile client is responsible for returning the default value, if no profiles are found
-    when(profilerClient.fetch(
-            eq(Object.class),
-            eq(expected.getProfileName()),
-            eq(expected.getEntity()),
-            eq(expected.getGroups()),
-            any(),
-            eq(Optional.of(defaultValue)))).thenReturn(Arrays.asList(defaultMeasurement));
-
-    // set the default value in the overrides map
-    String overrides = String.format( "{ 'profiler.default.value': %s }", defaultValue);
-    results = run("PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), [], " + overrides + ")");
-    assertEquals(1, results.size());
-    assertEquals(defaultValue, results.get(0));
+  public void testWithConfigAndOneGroup() {
+    final int periodsPerHour = 4;
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours);
+    final List<Object> group = Arrays.asList("weekends");
+
+    // setup - write some measurements to be read later
+    final int count = hours * periodsPerHour;
+    ProfileMeasurement m = new ProfileMeasurement()
+            .withProfileName("profile1")
+            .withEntity("entity1")
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, count, group, val -> expectedValue);
+
+    // create a variable that contains the groups to use
+    state.put("groups", group);
+
+    // now change the executor configuration
+    Context context2 = setup2();
+    // validate it is changed in significant way
+    @SuppressWarnings("unchecked")
+    Map<String, Object> global = (Map<String, Object>) context2.getCapability(Context.Capabilities.GLOBAL_CONFIG).get();
+    Assert.assertEquals(global.get(PROFILER_PERIOD.getKey()), Long.toString(periodDuration2));
+    Assert.assertNotEquals(periodDuration, periodDuration2);
+
+    // execute - read the profile values - with config_override.
+    // first two override values are strings, third is deliberately a number.
+    String overrides = "{'profiler.client.period.duration' : '" + periodDuration + "', "
+            + "'profiler.client.period.duration.units' : '" + periodUnits.toString() + "', "
+            + "'profiler.client.salt.divisor' : " + saltDivisor + " }";
+    String expr = "PROFILE_GET('profile1', 'entity1'" +
+            ", PROFILE_FIXED(4, 'HOURS', " + overrides + "), ['weekends'], " +
+            overrides + ")";
+    @SuppressWarnings("unchecked")
+    List<Integer> result = run(expr, List.class);
+
+    // validate - expect to read all values from the past 4 hours
+    Assert.assertEquals(count, result.size());
+
+    // execute - read the profile values - with (wrong) default global config values.
+    // No error message at this time, but returns empty results list, because
+    // row keys are not correctly calculated.
+    expr = "PROFILE_GET('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), ['weekends'])";
+    result = run(expr, List.class);
+
+    // validate - expect to fail to read any values
+    Assert.assertEquals(0, result.size());
   }
 }
diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/VerboseProfileTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/VerboseProfileTest.java
index 6869046..bd39007 100644
--- a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/VerboseProfileTest.java
+++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/stellar/VerboseProfileTest.java
@@ -20,13 +20,19 @@
 
 package org.apache.metron.profiler.client.stellar;
 
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.metron.hbase.mock.MockHBaseTableProvider;
 import org.apache.metron.profiler.ProfileMeasurement;
-import org.apache.metron.profiler.client.ProfilerClient;
+import org.apache.metron.profiler.client.ProfileWriter;
+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.stellar.common.DefaultStellarStatefulExecutor;
 import org.apache.metron.stellar.common.StellarStatefulExecutor;
 import org.apache.metron.stellar.dsl.Context;
-import org.apache.metron.stellar.dsl.functions.resolver.FunctionResolver;
 import org.apache.metron.stellar.dsl.functions.resolver.SimpleFunctionResolver;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -35,176 +41,180 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_COLUMN_FAMILY;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE_PROVIDER;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_PERIOD;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_PERIOD_UNITS;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_SALT_DIVISOR;
+import static org.apache.metron.profiler.client.stellar.VerboseProfile.ENTITY_KEY;
+import static org.apache.metron.profiler.client.stellar.VerboseProfile.GROUPS_KEY;
+import static org.apache.metron.profiler.client.stellar.VerboseProfile.PERIOD_END_KEY;
+import static org.apache.metron.profiler.client.stellar.VerboseProfile.PERIOD_KEY;
+import static org.apache.metron.profiler.client.stellar.VerboseProfile.PERIOD_START_KEY;
+import static org.apache.metron.profiler.client.stellar.VerboseProfile.PROFILE_KEY;
+import static org.apache.metron.profiler.client.stellar.VerboseProfile.VALUE_KEY;
 
 /**
- * Tests the 'PROFILE_VERBOSE' function in the {@link VerboseProfile} class.
+ * Tests the VerboseProfile class.
  */
 public class VerboseProfileTest {
-
+  private static final long periodDuration = 15;
+  private static final TimeUnit periodUnits = TimeUnit.MINUTES;
+  private static final int saltDivisor = 1000;
+  private static final String tableName = "profiler";
+  private static final String columnFamily = "P";
   private StellarStatefulExecutor executor;
-  private FunctionResolver functionResolver;
-  private Map<String, Object> globals;
-  private VerboseProfile function;
-  private ProfilerClient profilerClient;
-  private List<Map<String, Object>> results;
-  private ProfileMeasurement expected;
+  private Map<String, Object> state;
+  private ProfileWriter profileWriter;
 
-  private List run(String expression) {
-    return executor.execute(expression, new HashMap<>(), List.class);
+  private <T> T run(String expression, Class<T> clazz) {
+    return executor.execute(expression, state, clazz);
   }
 
+  private Map<String, Object> globals;
+
   @Before
   public void setup() {
-    // the mock profiler client used to feed profile measurement values to the function
-    profilerClient = mock(ProfilerClient.class);
+    state = new HashMap<>();
+    final HTableInterface table = MockHBaseTableProvider.addToCache(tableName, columnFamily);
 
-    // global properties
-    globals = new HashMap<>();
-    Context context = new Context.Builder()
-            .with(Context.Capabilities.GLOBAL_CONFIG, () -> globals)
-            .build();
+    // used to write values to be read during testing
+    long periodDurationMillis = TimeUnit.MINUTES.toMillis(15);
+    RowKeyBuilder rowKeyBuilder = new SaltyRowKeyBuilder();
+    ColumnBuilder columnBuilder = new ValueOnlyColumnBuilder(columnFamily);
+    profileWriter = new ProfileWriter(rowKeyBuilder, columnBuilder, table, periodDurationMillis);
 
-    // the VERBOSE_PROFILE function that will be tested
-    function = new VerboseProfile(globals -> profilerClient);
-    function.initialize(context);
+    // global properties
+    globals = new HashMap<String, Object>() {{
+      put(PROFILER_HBASE_TABLE.getKey(), tableName);
+      put(PROFILER_COLUMN_FAMILY.getKey(), columnFamily);
+      put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockHBaseTableProvider.class.getName());
+      put(PROFILER_PERIOD.getKey(), Long.toString(periodDuration));
+      put(PROFILER_PERIOD_UNITS.getKey(), periodUnits.toString());
+      put(PROFILER_SALT_DIVISOR.getKey(), Integer.toString(saltDivisor));
+    }};
 
     // create the stellar execution environment
-    functionResolver = new SimpleFunctionResolver()
-            .withClass(FixedLookback.class)
-            .withInstance(function);
-    executor = new DefaultStellarStatefulExecutor(functionResolver, context);
+    executor = new DefaultStellarStatefulExecutor(
+            new SimpleFunctionResolver()
+                    .withClass(VerboseProfile.class)
+                    .withClass(FixedLookback.class),
+            new Context.Builder()
+                    .with(Context.Capabilities.GLOBAL_CONFIG, () -> globals)
+                    .build());
+  }
 
-    // create a profile measurement used in the tests
-    expected = new ProfileMeasurement()
+  @Test
+  public void shouldReturnMeasurementsWhenNotGrouped() {
+    final int periodsPerHour = 4;
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours);
+    final List<Object> group = Collections.emptyList();
+
+    // setup - write some measurements to be read later
+    final int count = hours * periodsPerHour;
+    ProfileMeasurement m = new ProfileMeasurement()
             .withProfileName("profile1")
             .withEntity("entity1")
-            .withPeriod(System.currentTimeMillis(), 5, TimeUnit.MINUTES)
-            .withProfileValue(1231121);
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, count, group, val -> expectedValue);
+
+    // expect to see all values over the past 4 hours
+    List<Map<String, Object>> results;
+    results = run("PROFILE_VERBOSE('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))", List.class);
+    Assert.assertEquals(count, results.size());
+    for(Map<String, Object> actual: results) {
+      Assert.assertEquals("profile1", actual.get(PROFILE_KEY));
+      Assert.assertEquals("entity1", actual.get(ENTITY_KEY));
+      Assert.assertNotNull(actual.get(PERIOD_KEY));
+      Assert.assertNotNull(actual.get(PERIOD_START_KEY));
+      Assert.assertNotNull(actual.get(PERIOD_END_KEY));
+      Assert.assertNotNull(actual.get(GROUPS_KEY));
+      Assert.assertEquals(expectedValue, actual.get(VALUE_KEY));
+    }
   }
 
   @Test
-  public void shouldRenderVerboseView() {
-    // only one profile measurement exists
-    when(profilerClient.fetch(
-            eq(Object.class),
-            eq(expected.getProfileName()),
-            eq(expected.getEntity()),
-            eq(expected.getGroups()),
-            any(),
-            any())).thenReturn(Arrays.asList(expected));
-
-    // expect the one measurement to be returned
-    results = run("PROFILE_VERBOSE('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))");
-    assertEquals(1, results.size());
-    Map<String, Object> actual = results.get(0);
-
-    // the measurement should be rendered as a map containing detailed information about the profile measurement
-    assertEquals(expected.getProfileName(),                 actual.get("profile"));
-    assertEquals(expected.getEntity(),                      actual.get("entity"));
-    assertEquals(expected.getPeriod().getPeriod(),          actual.get("period"));
-    assertEquals(expected.getPeriod().getStartTimeMillis(), actual.get("period.start"));
-    assertEquals(expected.getPeriod().getEndTimeMillis(),   actual.get("period.end"));
-    assertEquals(expected.getProfileValue(),                actual.get("value"));
-    assertEquals(expected.getGroups(),                      actual.get("groups"));
+  public void shouldReturnMeasurementsWhenGrouped() {
+    final int periodsPerHour = 4;
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours);
+    final List<Object> group = Arrays.asList("weekends");
+
+    // setup - write some measurements to be read later
+    final int count = hours * periodsPerHour;
+    ProfileMeasurement m = new ProfileMeasurement()
+            .withProfileName("profile1")
+            .withEntity("entity1")
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, count, group, val -> expectedValue);
+
+    // create a variable that contains the groups to use
+    state.put("groups", group);
+
+    // expect to see all values over the past 4 hours for the group
+    List<Map<String, Object>> results;
+    results = run("PROFILE_VERBOSE('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), groups)", List.class);
+    Assert.assertEquals(count, results.size());
+    for(Map<String, Object> actual: results) {
+      Assert.assertEquals("profile1", actual.get(PROFILE_KEY));
+      Assert.assertEquals("entity1", actual.get(ENTITY_KEY));
+      Assert.assertNotNull(actual.get(PERIOD_KEY));
+      Assert.assertNotNull(actual.get(PERIOD_START_KEY));
+      Assert.assertNotNull(actual.get(PERIOD_END_KEY));
+      Assert.assertNotNull(actual.get(GROUPS_KEY));
+      Assert.assertEquals(expectedValue, actual.get(VALUE_KEY));
+    }
   }
 
   @Test
-  public void shouldRenderVerboseViewWithGroup() {
-    // the profile measurement is part of a group
-    expected.withGroups(Arrays.asList("group1"));
-
-    // only one profile measurement exists
-    when(profilerClient.fetch(
-            eq(Object.class),
-            eq(expected.getProfileName()),
-            eq(expected.getEntity()),
-            eq(expected.getGroups()),
-            any(),
-            any())).thenReturn(Arrays.asList(expected));
-
-    // expect the one measurement to be returned
-    results = run("PROFILE_VERBOSE('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), ['group1'])");
-    assertEquals(1, results.size());
-    Map<String, Object> actual = results.get(0);
-
-    // the measurement should be rendered as a map containing detailed information about the profile measurement
-    assertEquals(expected.getProfileName(),                 actual.get("profile"));
-    assertEquals(expected.getEntity(),                      actual.get("entity"));
-    assertEquals(expected.getPeriod().getPeriod(),          actual.get("period"));
-    assertEquals(expected.getPeriod().getStartTimeMillis(), actual.get("period.start"));
-    assertEquals(expected.getPeriod().getEndTimeMillis(),   actual.get("period.end"));
-    assertEquals(expected.getProfileValue(),                actual.get("value"));
-    assertEquals(expected.getGroups(),                      actual.get("groups"));
-  }
+  public void shouldReturnNothingWhenNoMeasurementsExist() {
+    final int expectedValue = 2302;
+    final int hours = 2;
+    final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours);
+    final List<Object> group = Collections.emptyList();
 
-  @Test
-  public void shouldRenderVerboseViewWithDifferentGroup() {
-    // the profile measurement is part of a group
-    expected.withGroups(Arrays.asList("group1"));
-
-    // only one profile measurement exists
-    when(profilerClient.fetch(
-            eq(Object.class),
-            eq(expected.getProfileName()),
-            eq(expected.getEntity()),
-            eq(expected.getGroups()),
-            any(),
-            any())).thenReturn(Arrays.asList(expected));
-
-    // the profile measurement is not part of 'group999'
-    results = run("PROFILE_VERBOSE('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'), ['group999'])");
-    assertEquals(0, results.size());
-  }
+    // setup - write a single value from 2 hours ago
+    ProfileMeasurement m = new ProfileMeasurement()
+            .withProfileName("profile1")
+            .withEntity("entity1")
+            .withPeriod(startTime, periodDuration, periodUnits);
+    profileWriter.write(m, 1, group, val -> expectedValue);
 
-  @Test
-  public void shouldReturnNothingWhenNoMeasurementsExist() {
-    // no measurements exist
-    when(profilerClient.fetch(
-            eq(Object.class),
-            any(),
-            any(),
-            any(),
-            any(),
-            any())).thenReturn(Collections.emptyList());
-
-    // no measurements exist
-    results = run("PROFILE_VERBOSE('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))");
-    assertEquals(0, results.size());
+    // expect to get NO measurements over the past 4 seconds
+    List<Map<String, Object>> result;
+    result = run("PROFILE_VERBOSE('profile1', 'entity1', PROFILE_FIXED(4, 'SECONDS'))", List.class);
+    Assert.assertEquals(0, result.size());
   }
 
   @Test
-  public void shouldReturnDefaultValue() {
+  public void shouldReturnDefaultValueWhenNoMeasurementsExist() {
     // set a default value
-    globals.put("profiler.default.value", expected);
-
-    // the underlying profile client needs to be setup to return the default
-    when(profilerClient.fetch(
-            eq(Object.class),
-            eq(expected.getProfileName()),
-            eq(expected.getEntity()),
-            eq(expected.getGroups()),
-            any(),
-            eq(Optional.of(expected)))).thenReturn(Arrays.asList(expected));
-
-    results = run("PROFILE_VERBOSE('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))");
-    assertEquals(1, results.size());
-    Map<String, Object> actual = results.get(0);
-
-    // no measurements exist, but we expect the default value to be returned
-    assertEquals(expected.getProfileName(),                 actual.get("profile"));
-    assertEquals(expected.getEntity(),                      actual.get("entity"));
-    assertEquals(expected.getPeriod().getPeriod(),          actual.get("period"));
-    assertEquals(expected.getPeriod().getStartTimeMillis(), actual.get("period.start"));
-    assertEquals(expected.getPeriod().getEndTimeMillis(),   actual.get("period.end"));
-    assertEquals(expected.getProfileValue(),                actual.get("value"));
-    assertEquals(expected.getGroups(),                      actual.get("groups"));
+    String defaultVal = "this is the default value";
+    globals.put("profiler.default.value", defaultVal);
+
+    // no profiles exist
+    String expr = "PROFILE_VERBOSE('profile1', 'entity1', PROFILE_FIXED(4, 'HOURS'))";
+    List<Map<String, Object>> results = run(expr, List.class);
+
+    // expect to get the default value instead of no results
+    Assert.assertTrue(results.size() == 16 || results.size() == 17);
+    for(Map<String, Object> actual: results) {
+      Assert.assertEquals("profile1", actual.get(PROFILE_KEY));
+      Assert.assertEquals("entity1", actual.get(ENTITY_KEY));
+      Assert.assertNotNull(actual.get(PERIOD_KEY));
+      Assert.assertNotNull(actual.get(PERIOD_START_KEY));
+      Assert.assertNotNull(actual.get(PERIOD_END_KEY));
+      Assert.assertNotNull(actual.get(GROUPS_KEY));
+
+      // expect the default value
+      Assert.assertEquals(defaultVal, actual.get(VALUE_KEY));
+    }
+
   }
 }
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
index 5a9610e..3f889bc 100644
--- 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
@@ -241,18 +241,10 @@ public class SaltyRowKeyBuilder implements RowKeyBuilder {
     }
   }
 
-  public long getPeriodDurationMillis() {
-    return periodDurationMillis;
-  }
-
   public void withPeriodDuration(long duration, TimeUnit units) {
     periodDurationMillis = units.toMillis(duration);
   }
 
-  public int getSaltDivisor() {
-    return saltDivisor;
-  }
-
   public void setSaltDivisor(int saltDivisor) {
     this.saltDivisor = saltDivisor;
   }
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
index b580db8..0a4a99d 100644
--- 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
@@ -34,6 +34,7 @@ public class ValueOnlyColumnBuilder implements ColumnBuilder {
    * The column family storing the profile data.
    */
   private String columnFamily;
+
   private byte[] columnFamilyBytes;
 
   public ValueOnlyColumnBuilder() {
@@ -46,6 +47,7 @@ public class ValueOnlyColumnBuilder implements ColumnBuilder {
 
   @Override
   public ColumnList columns(ProfileMeasurement measurement) {
+
     ColumnList cols = new ColumnList();
     cols.addColumn(columnFamilyBytes, getColumnQualifier("value"), SerDeUtils.toBytes(measurement.getProfileValue()));
 
@@ -64,6 +66,7 @@ public class ValueOnlyColumnBuilder implements ColumnBuilder {
 
   @Override
   public byte[] getColumnQualifier(String fieldName) {
+
     if("value".equals(fieldName)) {
       return Bytes.toBytes("value");
     }
diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageDistributorTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageDistributorTest.java
index 31b41ff..e04404c 100644
--- a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageDistributorTest.java
+++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageDistributorTest.java
@@ -21,6 +21,7 @@
 package org.apache.metron.profiler;
 
 import com.github.benmanes.caffeine.cache.Ticker;
+import com.google.common.util.concurrent.MoreExecutors;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.common.configuration.profiler.ProfileConfig;
 import org.apache.metron.common.utils.JSONUtils;
@@ -32,6 +33,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
 import static java.util.concurrent.TimeUnit.HOURS;
diff --git a/metron-analytics/metron-profiler-spark/pom.xml b/metron-analytics/metron-profiler-spark/pom.xml
index caaed87..2d127e6 100644
--- a/metron-analytics/metron-profiler-spark/pom.xml
+++ b/metron-analytics/metron-profiler-spark/pom.xml
@@ -145,12 +145,6 @@
             <scope>test</scope>
         </dependency>
         <dependency>
-            <groupId>org.mockito</groupId>
-            <artifactId>mockito-core</artifactId>
-            <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>
diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfiler.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfiler.java
index d3cfdca..571545e 100644
--- a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfiler.java
+++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfiler.java
@@ -28,7 +28,6 @@ import org.apache.metron.profiler.spark.function.MessageRouterFunction;
 import org.apache.metron.profiler.spark.function.ProfileBuilderFunction;
 import org.apache.metron.profiler.spark.reader.TelemetryReader;
 import org.apache.metron.profiler.spark.reader.TelemetryReaders;
-import org.apache.spark.api.java.function.MapPartitionsFunction;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Encoders;
 import org.apache.spark.sql.SparkSession;
@@ -98,11 +97,8 @@ public class BatchProfiler implements Serializable {
     LOG.debug("Produced {} profile measurement(s)", measurements.cache().count());
 
     // write the profile measurements to HBase
-    MapPartitionsFunction<ProfileMeasurementAdapter, Integer> mapper = new HBaseWriterFunction.Builder()
-            .withProperties(profilerProps)
-            .build();
     long count = measurements
-            .mapPartitions(mapper, Encoders.INT())
+            .mapPartitions(new HBaseWriterFunction(profilerProps), Encoders.INT())
             .agg(sum("value"))
             .head()
             .getLong(0);
diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfilerConfig.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfilerConfig.java
index d120a29..148d970 100644
--- a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfilerConfig.java
+++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfilerConfig.java
@@ -20,14 +20,13 @@
 package org.apache.metron.profiler.spark;
 
 import org.apache.hadoop.hbase.client.Durability;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
-import org.apache.metron.hbase.client.HBaseTableClientFactory;
 import org.apache.metron.stellar.common.utils.ConversionUtils;
 
 import java.util.Map;
 import java.util.Properties;
 
 import static org.apache.metron.profiler.spark.reader.TelemetryReaders.JSON;
+import static org.apache.metron.profiler.spark.reader.TelemetryReaders.TEXT;
 
 /**
  * Defines the configuration values recognized by the Batch Profiler.
@@ -40,9 +39,7 @@ public enum BatchProfilerConfig {
 
   HBASE_SALT_DIVISOR("profiler.hbase.salt.divisor", 1000, Integer.class),
 
-  HBASE_CONNECTION_FACTORY("profiler.hbase.connection.provider", HBaseConnectionFactory.class.getName(), String.class),
-
-  HBASE_CLIENT_FACTORY("profiler.hbase.client.factory", HBaseTableClientFactory.class, String.class),
+  HBASE_TABLE_PROVIDER("profiler.hbase.table.provider", "org.apache.metron.hbase.HTableProvider", String.class),
 
   HBASE_TABLE_NAME("profiler.hbase.table", "profiler", String.class),
 
diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/HBaseWriterFunction.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/HBaseWriterFunction.java
index bb66af8..cfabd94 100644
--- a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/HBaseWriterFunction.java
+++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/HBaseWriterFunction.java
@@ -21,13 +21,12 @@ package org.apache.metron.profiler.spark.function;
 
 import org.apache.commons.collections4.IteratorUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Durability;
+import org.apache.metron.hbase.HTableProvider;
+import org.apache.metron.hbase.TableProvider;
 import org.apache.metron.hbase.client.HBaseClient;
-import org.apache.metron.hbase.client.HBaseClientFactory;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
-import org.apache.metron.hbase.client.HBaseTableClient;
-import org.apache.metron.hbase.client.HBaseTableClientFactory;
 import org.apache.metron.profiler.ProfileMeasurement;
 import org.apache.metron.profiler.hbase.ColumnBuilder;
 import org.apache.metron.profiler.hbase.RowKeyBuilder;
@@ -40,16 +39,16 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.lang.reflect.InvocationTargetException;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_CLIENT_FACTORY;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_COLUMN_FAMILY;
-import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_CONNECTION_FACTORY;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_SALT_DIVISOR;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_TABLE_NAME;
+import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_TABLE_PROVIDER;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_WRITE_DURABILITY;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.PERIOD_DURATION;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.PERIOD_DURATION_UNITS;
@@ -61,93 +60,7 @@ public class HBaseWriterFunction implements MapPartitionsFunction<ProfileMeasure
 
   protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static class Builder {
-    private HBaseConnectionFactory connectionFactory = new HBaseConnectionFactory();
-    private HBaseClientFactory hBaseClientFactory = new HBaseTableClientFactory();
-    private String tableName = String.class.cast(HBASE_TABLE_NAME.getDefault());
-    private Durability durability = Durability.class.cast(HBASE_WRITE_DURABILITY.getDefault());
-    private RowKeyBuilder rowKeyBuilder = new SaltyRowKeyBuilder();
-    private ColumnBuilder columnBuilder = new ValueOnlyColumnBuilder();
-
-    public Builder withConnectionFactory(HBaseConnectionFactory connectionFactory) {
-      this.connectionFactory = connectionFactory;
-      return this;
-    }
-
-    public Builder withClientFactory(HBaseClientFactory clientFactory) {
-      this.hBaseClientFactory = clientFactory;
-      return this;
-    }
-
-    public Builder withRowKeyBuilder(RowKeyBuilder rowKeyBuilder) {
-      this.rowKeyBuilder = rowKeyBuilder;
-      return this;
-    }
-
-    public Builder withColumnBuilder(ColumnBuilder columnBuilder) {
-      this.columnBuilder = columnBuilder;
-      return this;
-    }
-
-    public Builder withProperties(Properties properties) {
-      // row key builder
-      int saltDivisor = HBASE_SALT_DIVISOR.get(properties, Integer.class);
-      int periodDuration = PERIOD_DURATION.get(properties, Integer.class);
-      TimeUnit periodDurationUnits = TimeUnit.valueOf(PERIOD_DURATION_UNITS.get(properties, String.class));
-      rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodDurationUnits);
-
-      // column builder
-      String columnFamily = HBASE_COLUMN_FAMILY.get(properties, String.class);
-      columnBuilder = new ValueOnlyColumnBuilder(columnFamily);
-
-      // hbase
-      tableName = HBASE_TABLE_NAME.get(properties, String.class);
-      durability = HBASE_WRITE_DURABILITY.get(properties, Durability.class);
-
-      // connection factory
-      String factoryImpl = HBASE_CONNECTION_FACTORY.get(properties, String.class);
-      connectionFactory = createConnectionFactory(factoryImpl);
-
-      // client creator
-      String creatorImpl = HBASE_CLIENT_FACTORY.get(properties, String.class);
-      hBaseClientFactory = HBaseClientFactory.byName(creatorImpl, () -> new HBaseTableClientFactory());
-
-      return this;
-    }
-
-    private static HBaseConnectionFactory createConnectionFactory(String factoryImpl) {
-      LOG.trace("Creating table provider; className={}", factoryImpl);
-
-      // if class name not defined, use a reasonable default
-      if(StringUtils.isEmpty(factoryImpl) || factoryImpl.charAt(0) == '$') {
-        return new HBaseConnectionFactory();
-      }
-
-      // instantiate the table provider
-      return HBaseConnectionFactory.byName(factoryImpl);
-    }
-
-    public HBaseWriterFunction build() {
-      HBaseWriterFunction function = new HBaseWriterFunction();
-      function.connectionFactory = connectionFactory;
-      function.hBaseClientFactory = hBaseClientFactory;
-      function.tableName = tableName;
-      function.durability = durability;
-      function.rowKeyBuilder = rowKeyBuilder;
-      function.columnBuilder = columnBuilder;
-      return function;
-    }
-  }
-
-  /**
-   * Establishes connections to HBase.
-   */
-  private HBaseConnectionFactory connectionFactory;
-
-  /**
-   * Creates the {@link HBaseTableClient} when it is needed.
-   */
-  private HBaseClientFactory hBaseClientFactory;
+  private TableProvider tableProvider;
 
   /**
    * The name of the HBase table to write to.
@@ -169,11 +82,23 @@ public class HBaseWriterFunction implements MapPartitionsFunction<ProfileMeasure
    */
   private ColumnBuilder columnBuilder;
 
-  /**
-   * Use the {@link HBaseWriterFunction.Builder} instead.
-   */
-  private HBaseWriterFunction() {
-    // nothing to do
+  public HBaseWriterFunction(Properties properties) {
+    tableName = HBASE_TABLE_NAME.get(properties, String.class);
+    durability = HBASE_WRITE_DURABILITY.get(properties, Durability.class);
+
+    // row key builder
+    int saltDivisor = HBASE_SALT_DIVISOR.get(properties, Integer.class);
+    int periodDuration = PERIOD_DURATION.get(properties, Integer.class);
+    TimeUnit periodDurationUnits = TimeUnit.valueOf(PERIOD_DURATION_UNITS.get(properties, String.class));
+    rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodDurationUnits);
+
+    // column builder
+    String columnFamily = HBASE_COLUMN_FAMILY.get(properties, String.class);
+    columnBuilder = new ValueOnlyColumnBuilder(columnFamily);
+
+    // hbase table provider
+    String providerImpl = HBASE_TABLE_PROVIDER.get(properties, String.class);
+    tableProvider = createTableProvider(providerImpl);
   }
 
   /**
@@ -192,7 +117,8 @@ public class HBaseWriterFunction implements MapPartitionsFunction<ProfileMeasure
     if(measurements.size() > 0) {
 
       // open an HBase connection
-      try (HBaseClient client = hBaseClientFactory.create(connectionFactory, HBaseConfiguration.create(), tableName)) {
+      Configuration config = HBaseConfiguration.create();
+      try (HBaseClient client = new HBaseClient(tableProvider, config, tableName)) {
 
         for (ProfileMeasurementAdapter adapter : measurements) {
           ProfileMeasurement m = adapter.toProfileMeasurement();
@@ -209,4 +135,37 @@ public class HBaseWriterFunction implements MapPartitionsFunction<ProfileMeasure
     LOG.debug("{} profile measurement(s) written to HBase", count);
     return IteratorUtils.singletonIterator(count);
   }
+
+  /**
+   * Set the {@link TableProvider} using the class name of the provider.
+   * @param providerImpl The name of the class.
+   * @return
+   */
+  public HBaseWriterFunction withTableProviderImpl(String providerImpl) {
+    this.tableProvider = createTableProvider(providerImpl);
+    return this;
+  }
+
+  /**
+   * Creates a TableProvider based on a class name.
+   * @param providerImpl The class name of a TableProvider
+   */
+  private static TableProvider createTableProvider(String providerImpl) {
+    LOG.trace("Creating table provider; className={}", providerImpl);
+
+    // if class name not defined, use a reasonable default
+    if(StringUtils.isEmpty(providerImpl) || providerImpl.charAt(0) == '$') {
+      return new HTableProvider();
+    }
+
+    // instantiate the table provider
+    try {
+      Class<? extends TableProvider> clazz = (Class<? extends TableProvider>) Class.forName(providerImpl);
+      return clazz.getConstructor().newInstance();
+
+    } catch (InstantiationException | IllegalAccessException | IllegalStateException |
+            InvocationTargetException | NoSuchMethodException | ClassNotFoundException e) {
+      throw new IllegalStateException("Unable to instantiate connector", e);
+    }
+  }
 }
diff --git a/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/BatchProfilerIntegrationTest.java b/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/BatchProfilerIntegrationTest.java
index 673a0b6..b36cf8c 100644
--- a/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/BatchProfilerIntegrationTest.java
+++ b/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/BatchProfilerIntegrationTest.java
@@ -20,16 +20,10 @@
 package org.apache.metron.profiler.spark;
 
 import org.adrianwalker.multilinestring.Multiline;
-import org.apache.metron.hbase.client.FakeHBaseClient;
-import org.apache.metron.hbase.client.FakeHBaseClientFactory;
-import org.apache.metron.hbase.client.FakeHBaseConnectionFactory;
-import org.apache.metron.profiler.client.HBaseProfilerClient;
-import org.apache.metron.profiler.client.ProfilerClient;
+import org.apache.metron.hbase.mock.MockHBaseTableProvider;
 import org.apache.metron.profiler.client.stellar.FixedLookback;
 import org.apache.metron.profiler.client.stellar.GetProfile;
 import org.apache.metron.profiler.client.stellar.WindowLookback;
-import org.apache.metron.profiler.hbase.SaltyRowKeyBuilder;
-import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder;
 import org.apache.metron.stellar.common.DefaultStellarStatefulExecutor;
 import org.apache.metron.stellar.common.StellarStatefulExecutor;
 import org.apache.metron.stellar.dsl.Context;
@@ -52,16 +46,14 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.TimeUnit;
 
 import static org.apache.metron.common.configuration.profiler.ProfilerConfig.fromJSON;
 import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_COLUMN_FAMILY;
 import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE;
-import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_CONNECTION_FACTORY;
-import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_CLIENT_FACTORY;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE_PROVIDER;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_COLUMN_FAMILY;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_TABLE_NAME;
-import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_CONNECTION_FACTORY;
+import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_TABLE_PROVIDER;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.TELEMETRY_INPUT_BEGIN;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.TELEMETRY_INPUT_END;
 import static org.apache.metron.profiler.spark.BatchProfilerConfig.TELEMETRY_INPUT_FORMAT;
@@ -109,40 +101,28 @@ public class BatchProfilerIntegrationTest {
   public void setup() {
     readerProperties = new Properties();
     profilerProperties = new Properties();
+
+    // the output will be written to a mock HBase table
     String tableName = HBASE_TABLE_NAME.get(profilerProperties, String.class);
     String columnFamily = HBASE_COLUMN_FAMILY.get(profilerProperties, String.class);
-    profilerProperties.put(HBASE_CONNECTION_FACTORY.getKey(), FakeHBaseConnectionFactory.class.getName());
+    profilerProperties.put(HBASE_TABLE_PROVIDER.getKey(), MockHBaseTableProvider.class.getName());
+
+    // create the mock hbase table
+    MockHBaseTableProvider.addToCache(tableName, columnFamily);
 
     // define the globals required by `PROFILE_GET`
     Map<String, Object> global = new HashMap<String, Object>() {{
       put(PROFILER_HBASE_TABLE.getKey(), tableName);
       put(PROFILER_COLUMN_FAMILY.getKey(), columnFamily);
-      put(PROFILER_HBASE_CONNECTION_FACTORY.getKey(), FakeHBaseConnectionFactory.class.getName());
+      put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockHBaseTableProvider.class.getName());
     }};
 
-    // the batch profiler needs to use the `FakeHBaseClient` for these tests
-    profilerProperties.put(HBASE_CLIENT_FACTORY.getKey(), FakeHBaseClientFactory.class.getName());
-
-    // ensure that all of the static records are deleted before running the test
-    FakeHBaseClient hbaseClient = new FakeHBaseClient();
-    hbaseClient.deleteAll();
-
-    // create a `ProfilerClient` that uses the `FakeHBaseClient`
-    ProfilerClient profilerClient = new HBaseProfilerClient(
-            hbaseClient,
-            new SaltyRowKeyBuilder(),
-            new ValueOnlyColumnBuilder(),
-            TimeUnit.MINUTES.toMillis(15));
-
-    // create an instance of `PROFILE_GET` that indirectly uses the `FakeHBaseClient`
-    GetProfile profileGetFunction = new GetProfile(globals -> profilerClient);
-
     // create the stellar execution environment
     executor = new DefaultStellarStatefulExecutor(
             new SimpleFunctionResolver()
+                    .withClass(GetProfile.class)
                     .withClass(FixedLookback.class)
-                    .withClass(WindowLookback.class)
-                    .withInstance(profileGetFunction),
+                    .withClass(WindowLookback.class),
             new Context.Builder()
                     .with(Context.Capabilities.GLOBAL_CONFIG, () -> global)
                     .build());
diff --git a/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/HBaseWriterFunctionTest.java b/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/HBaseWriterFunctionTest.java
index c0f0370..55f3e21 100644
--- a/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/HBaseWriterFunctionTest.java
+++ b/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/HBaseWriterFunctionTest.java
@@ -21,14 +21,8 @@ package org.apache.metron.profiler.spark.function;
 
 import org.apache.commons.collections4.IteratorUtils;
 import org.apache.metron.common.configuration.profiler.ProfileConfig;
-import org.apache.metron.hbase.ColumnList;
-import org.apache.metron.hbase.client.FakeHBaseClient;
-import org.apache.metron.hbase.client.HBaseClientFactory;
+import org.apache.metron.hbase.mock.MockHBaseTableProvider;
 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.spark.ProfileMeasurementAdapter;
 import org.json.simple.JSONObject;
 import org.junit.Assert;
@@ -38,71 +32,64 @@ import org.junit.Test;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.metron.hbase.client.FakeHBaseClient.Mutation;
+import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_COLUMN_FAMILY;
+import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_TABLE_NAME;
 
 public class HBaseWriterFunctionTest {
 
-  private HBaseWriterFunction function;
-  private RowKeyBuilder rowKeyBuilder;
-  private ColumnBuilder columnBuilder;
-  private FakeHBaseClient hbaseClient;
-  private HBaseClientFactory hBaseClientFactory;
-
-  private static final JSONObject message = getMessage();
-  private static final String entity = (String) message.get("ip_src_addr");
-  private static final long timestamp = (Long) message.get("timestamp");
-  private static final ProfileConfig profile = getProfile();
+  Properties profilerProperties;
 
   @Before
   public void setup() {
-    hbaseClient = new FakeHBaseClient();
-    hbaseClient.deleteAll();
-    hBaseClientFactory = (x, y, z) -> hbaseClient;
-    rowKeyBuilder = new SaltyRowKeyBuilder();
-    columnBuilder = new ValueOnlyColumnBuilder();
-    function = new HBaseWriterFunction.Builder()
-            .withRowKeyBuilder(rowKeyBuilder)
-            .withColumnBuilder(columnBuilder)
-            .withClientFactory(hBaseClientFactory)
-            .build();
+    profilerProperties = getProfilerProperties();
+
+    // create a mock table for HBase
+    String tableName = HBASE_TABLE_NAME.get(profilerProperties, String.class);
+    String columnFamily = HBASE_COLUMN_FAMILY.get(profilerProperties, String.class);
+    MockHBaseTableProvider.addToCache(tableName, columnFamily);
   }
 
   @Test
   public void testWrite() throws Exception {
-    // write a profile measurement
+
+    JSONObject message = getMessage();
+    String entity = (String) message.get("ip_src_addr");
+    long timestamp = (Long) message.get("timestamp");
+    ProfileConfig profile = getProfile();
+
+    // setup the profile measurements that will be written
     List<ProfileMeasurementAdapter> measurements = createMeasurements(1, entity, timestamp, profile);
+
+    // setup the function to test
+    HBaseWriterFunction function = new HBaseWriterFunction(profilerProperties);
+    function.withTableProviderImpl(MockHBaseTableProvider.class.getName());
+
+    // write the measurements
     Iterator<Integer> results = function.call(measurements.iterator());
 
-    // validate the results
+    // validate the result
     List<Integer> counts = IteratorUtils.toList(results);
     Assert.assertEquals(1, counts.size());
     Assert.assertEquals(1, counts.get(0).intValue());
-
-    // 1 record should have been written to the hbase client
-    List<Mutation> written = hbaseClient.getAllPersisted();
-    Assert.assertEquals(1, written.size());
-
-    // validate the row key used to write to hbase
-    ProfileMeasurement m = measurements.get(0).toProfileMeasurement();
-    byte[] expectedRowKey = rowKeyBuilder.rowKey(m);
-    Assert.assertArrayEquals(expectedRowKey, written.get(0).rowKey);
-
-    // validate the columns used to write to hbase.
-    ColumnList expectedCols = columnBuilder.columns(m);
-    Assert.assertEquals(expectedCols, written.get(0).columnList);
   }
 
   @Test
   public void testWriteMany() throws Exception {
+
+    JSONObject message = getMessage();
+    String entity = (String) message.get("ip_src_addr");
+    long timestamp = (Long) message.get("timestamp");
+    ProfileConfig profile = getProfile();
+
     // setup the profile measurements that will be written
     List<ProfileMeasurementAdapter> measurements = createMeasurements(10, entity, timestamp, profile);
 
     // setup the function to test
-    HBaseWriterFunction function = new HBaseWriterFunction.Builder()
-            .withClientFactory(hBaseClientFactory)
-            .build();
+    HBaseWriterFunction function = new HBaseWriterFunction(profilerProperties);
+    function.withTableProviderImpl(MockHBaseTableProvider.class.getName());
 
     // write the measurements
     Iterator<Integer> results = function.call(measurements.iterator());
@@ -115,13 +102,13 @@ public class HBaseWriterFunctionTest {
 
   @Test
   public void testWriteNone() throws Exception {
+
     // there are no profile measurements to write
     List<ProfileMeasurementAdapter> measurements = new ArrayList<>();
 
     // setup the function to test
-    HBaseWriterFunction function = new HBaseWriterFunction.Builder()
-            .withClientFactory(hBaseClientFactory)
-            .build();
+    HBaseWriterFunction function = new HBaseWriterFunction(profilerProperties);
+    function.withTableProviderImpl(MockHBaseTableProvider.class.getName());
 
     // write the measurements
     Iterator<Integer> results = function.call(measurements.iterator());
@@ -160,7 +147,7 @@ public class HBaseWriterFunctionTest {
   /**
    * Returns a telemetry message to use for testing.
    */
-  private static JSONObject getMessage() {
+  private JSONObject getMessage() {
     JSONObject message = new JSONObject();
     message.put("ip_src_addr", "192.168.1.1");
     message.put("status", "red");
@@ -169,9 +156,16 @@ public class HBaseWriterFunctionTest {
   }
 
   /**
+   * Returns profiler properties to use for testing.
+   */
+  private Properties getProfilerProperties() {
+    return new Properties();
+  }
+
+  /**
    * Returns a profile definition to use for testing.
    */
-  private static ProfileConfig getProfile() {
+  private ProfileConfig getProfile() {
     return new ProfileConfig()
             .withProfile("profile1")
             .withForeach("ip_src_addr")
diff --git a/metron-analytics/metron-profiler-storm/pom.xml b/metron-analytics/metron-profiler-storm/pom.xml
index ea3e612..a51524f 100644
--- a/metron-analytics/metron-profiler-storm/pom.xml
+++ b/metron-analytics/metron-profiler-storm/pom.xml
@@ -29,6 +29,11 @@
     </properties>
     <dependencies>
         <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>3.2</version>
+        </dependency>
+        <dependency>
             <groupId>com.fasterxml.jackson.core</groupId>
             <artifactId>jackson-databind</artifactId>
             <version>${global_jackson_version}</version>
@@ -84,6 +89,10 @@
                     <groupId>asm</groupId>
                     <artifactId>asm</artifactId>
                 </exclusion>
+                <exclusion>
+                    <artifactId>commons-lang</artifactId>
+                    <groupId>commons-lang</groupId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -95,6 +104,10 @@
                     <artifactId>servlet-api</artifactId>
                     <groupId>javax.servlet</groupId>
                 </exclusion>
+                <exclusion>
+                    <artifactId>commons-lang</artifactId>
+                    <groupId>commons-lang</groupId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -106,6 +119,10 @@
                     <groupId>asm</groupId>
                     <artifactId>asm</artifactId>
                 </exclusion>
+                <exclusion>
+                    <artifactId>commons-lang</artifactId>
+                    <groupId>commons-lang</groupId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -188,6 +205,10 @@
                     <groupId>org.slf4j</groupId>
                     <artifactId>log4j-over-slf4j</artifactId>
                 </exclusion>
+                <exclusion>
+                    <artifactId>commons-lang</artifactId>
+                    <groupId>commons-lang</groupId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -212,6 +233,10 @@
                     <groupId>org.slf4j</groupId>
                     <artifactId>slf4j-log4j12</artifactId>
                 </exclusion>
+                <exclusion>
+                    <artifactId>commons-lang</artifactId>
+                    <groupId>commons-lang</groupId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -243,6 +268,10 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-mapreduce-client-core</artifactId>
                 </exclusion>
+                <exclusion>
+                    <artifactId>commons-lang</artifactId>
+                    <groupId>commons-lang</groupId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
diff --git a/metron-analytics/metron-profiler-storm/src/main/config/profiler.properties b/metron-analytics/metron-profiler-storm/src/main/config/profiler.properties
index 91a5d3b..dc30838 100644
--- a/metron-analytics/metron-profiler-storm/src/main/config/profiler.properties
+++ b/metron-analytics/metron-profiler-storm/src/main/config/profiler.properties
@@ -61,8 +61,6 @@ profiler.hbase.table=profiler
 profiler.hbase.column.family=P
 profiler.hbase.batch=10
 profiler.hbase.flush.interval.seconds=30
-profiler.hbase.client.factory=org.apache.metron.hbase.client.HBaseTableClientFactory
-profiler.hbase.connection.factory=org.apache.metron.hbase.client.HBaseConnectionFactory
 
 ##### Kafka #####
 
diff --git a/metron-analytics/metron-profiler-storm/src/main/flux/profiler/remote.yaml b/metron-analytics/metron-profiler-storm/src/main/flux/profiler/remote.yaml
index b3faf34..e16a782 100644
--- a/metron-analytics/metron-profiler-storm/src/main/flux/profiler/remote.yaml
+++ b/metron-analytics/metron-profiler-storm/src/main/flux/profiler/remote.yaml
@@ -137,9 +137,6 @@ components:
             - ${profiler.window.lag}
             - "${profiler.window.lag.units}"
 
-    -   id: "hbaseClientFactory"
-        className: "${profiler.hbase.client.factory}"
-
 spouts:
 
     -   id: "kafkaSpout"
@@ -182,10 +179,8 @@ bolts:
             - "${profiler.hbase.table}"
             - ref: "hbaseMapper"
         configMethods:
-            - name: "withClientFactory"
-              args: [ref: "hbaseClientFactory"]
-            - name: "withConnectionFactory"
-              args: ["${profiler.hbase.connection.factory}"]
+            - name: "withTableProvider"
+              args: ["${hbase.provider.impl}"]
             - name: "withBatchSize"
               args: [${profiler.hbase.batch}]
             - name: "withFlushIntervalSecs"
diff --git a/metron-analytics/metron-profiler-storm/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java b/metron-analytics/metron-profiler-storm/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java
index d5c2949..ec860a5 100644
--- a/metron-analytics/metron-profiler-storm/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java
+++ b/metron-analytics/metron-profiler-storm/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java
@@ -20,15 +20,18 @@
 
 package org.apache.metron.hbase.bolt;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Durability;
+import org.apache.metron.hbase.HTableProvider;
+import org.apache.metron.hbase.TableProvider;
 import org.apache.metron.hbase.ColumnList;
 import org.apache.metron.hbase.bolt.mapper.HBaseMapper;
 import org.apache.metron.hbase.client.HBaseClient;
-import org.apache.metron.hbase.client.HBaseClientFactory;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
-import org.apache.metron.hbase.client.HBaseTableClient;
-import org.apache.metron.hbase.client.HBaseTableClientFactory;
 import org.apache.storm.Config;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
@@ -38,10 +41,6 @@ import org.apache.storm.tuple.Tuple;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.lang.invoke.MethodHandles;
-import java.util.Map;
-import java.util.Optional;
-
 /**
  * A bolt that writes to HBase.
  *
@@ -77,32 +76,25 @@ public class HBaseBolt extends BaseRichBolt {
   protected HBaseMapper mapper;
 
   /**
-   * Defines when a batch needs flushed.
-   */
-  private BatchHelper batchHelper;
-
-  /**
-   * Establishes a connection to HBase.
+   * The name of the class that should be used as a table provider.
+   *
+   * <p>Defaults to 'org.apache.metron.hbase.HTableProvider'.
    */
-  private HBaseConnectionFactory connectionFactory;
+  protected String tableProviderClazzName = "org.apache.metron.hbase.HTableProvider";
 
   /**
-   * Creates the {@link HBaseTableClient} used by this bolt.
+   * The TableProvider
+   * May be loaded from tableProviderClazzName or provided
    */
-  private HBaseClientFactory hBaseClientFactory;
-
-  /**
-   * Used to write to HBase.
-   */
-  protected transient HBaseClient hbaseClient;
+  protected TableProvider tableProvider;
 
+  private BatchHelper batchHelper;
   protected OutputCollector collector;
+  protected transient HBaseClient hbaseClient;
 
   public HBaseBolt(String tableName, HBaseMapper mapper) {
     this.tableName = tableName;
     this.mapper = mapper;
-    this.connectionFactory = new HBaseConnectionFactory();
-    this.hBaseClientFactory = new HBaseTableClientFactory();
   }
 
   public HBaseBolt writeToWAL(boolean writeToWAL) {
@@ -110,34 +102,34 @@ public class HBaseBolt extends BaseRichBolt {
     return this;
   }
 
-  public HBaseBolt withBatchSize(int batchSize) {
-    this.batchSize = batchSize;
+  public HBaseBolt withTableProvider(String tableProvider) {
+    this.tableProviderClazzName = tableProvider;
     return this;
   }
 
-  public HBaseBolt withFlushIntervalSecs(int flushIntervalSecs) {
-    this.flushIntervalSecs = flushIntervalSecs;
+  public HBaseBolt withTableProviderInstance(TableProvider tableProvider){
+    this.tableProvider = tableProvider;
     return this;
   }
 
-  public HBaseBolt withConnectionFactory(HBaseConnectionFactory connectionFactory) {
-    this.connectionFactory = connectionFactory;
+  public HBaseBolt withBatchSize(int batchSize) {
+    this.batchSize = batchSize;
     return this;
   }
 
-  public HBaseBolt withConnectionFactory(String connectionFactoryImpl) {
-    this.connectionFactory = HBaseConnectionFactory.byName(connectionFactoryImpl);
+  public HBaseBolt withFlushIntervalSecs(int flushIntervalSecs) {
+    this.flushIntervalSecs = flushIntervalSecs;
     return this;
   }
 
-  public HBaseBolt withClientFactory(HBaseClientFactory clientFactory) {
-    this.hBaseClientFactory = clientFactory;
-    return this;
+  public void setClient(HBaseClient hbaseClient) {
+    this.hbaseClient = hbaseClient;
   }
 
   @Override
   public Map<String, Object> getComponentConfiguration() {
     LOG.debug("Tick tuples expected every {} second(s)", flushIntervalSecs);
+
     Config conf = new Config();
     conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, flushIntervalSecs);
     return conf;
@@ -147,7 +139,15 @@ public class HBaseBolt extends BaseRichBolt {
   public void prepare(Map map, TopologyContext topologyContext, OutputCollector collector) {
     this.collector = collector;
     this.batchHelper = new BatchHelper(batchSize, collector);
-    this.hbaseClient = hBaseClientFactory.create(connectionFactory, HBaseConfiguration.create(), tableName);
+
+    TableProvider provider;
+    if(this.tableProvider == null) {
+      provider = createTableProvider(tableProviderClazzName);
+    } else {
+      provider = this.tableProvider;
+    }
+
+    hbaseClient = new HBaseClient(provider, HBaseConfiguration.create(), tableName);
   }
 
   @Override
@@ -158,10 +158,12 @@ public class HBaseBolt extends BaseRichBolt {
   @Override
   public void execute(Tuple tuple) {
     LOG.trace("Received a tuple.");
+
     try {
       if (batchHelper.shouldHandle(tuple)) {
         save(tuple);
       }
+
       if (batchHelper.shouldFlush()) {
         flush();
       }
@@ -180,6 +182,7 @@ public class HBaseBolt extends BaseRichBolt {
     byte[] rowKey = mapper.rowKey(tuple);
     ColumnList cols = mapper.columns(tuple);
     Durability durability = writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL;
+
     Optional<Long> ttl = mapper.getTTL(tuple);
     if(ttl.isPresent()) {
       hbaseClient.addMutation(rowKey, cols, durability, ttl.get());
@@ -196,7 +199,31 @@ public class HBaseBolt extends BaseRichBolt {
    */
   private void flush() {
     LOG.debug("About to flush a batch of {} mutation(s)", batchHelper.getBatchSize());
+
     this.hbaseClient.mutate();
     batchHelper.ack();
   }
+
+  /**
+   * Creates a TableProvider based on a class name.
+   * @param connectorImpl The class name of a TableProvider
+   */
+  private static TableProvider createTableProvider(String connectorImpl) {
+    LOG.trace("Creating table provider; className={}", connectorImpl);
+
+    // if class name not defined, use a reasonable default
+    if(StringUtils.isEmpty(connectorImpl) || connectorImpl.charAt(0) == '$') {
+      return new HTableProvider();
+    }
+
+    // instantiate the table provider
+    try {
+      Class<? extends TableProvider> clazz = (Class<? extends TableProvider>) Class.forName(connectorImpl);
+      return clazz.getConstructor().newInstance();
+
+    } catch (InstantiationException | IllegalAccessException | IllegalStateException |
+              InvocationTargetException | NoSuchMethodException | ClassNotFoundException e) {
+      throw new IllegalStateException("Unable to instantiate connector", e);
+    }
+  }
 }
diff --git a/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/hbase/bolt/HBaseBoltTest.java b/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/hbase/bolt/HBaseBoltTest.java
index b5bf898..bae3728 100644
--- a/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/hbase/bolt/HBaseBoltTest.java
+++ b/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/hbase/bolt/HBaseBoltTest.java
@@ -20,10 +20,13 @@
 
 package org.apache.metron.hbase.bolt;
 
-import org.apache.metron.hbase.client.HBaseClient;
-import org.apache.metron.test.bolt.BaseBoltTest;
+import org.apache.metron.hbase.TableProvider;
 import org.apache.storm.Constants;
 import org.apache.storm.tuple.Tuple;
+import org.apache.metron.hbase.bolt.mapper.Widget;
+import org.apache.metron.hbase.bolt.mapper.WidgetMapper;
+import org.apache.metron.hbase.client.HBaseClient;
+import org.apache.metron.test.bolt.BaseBoltTest;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -50,6 +53,7 @@ public class HBaseBoltTest extends BaseBoltTest {
   private Tuple tuple2;
   private Widget widget1;
   private Widget widget2;
+  private TableProvider provider;
 
   @Before
   public void setupTuples() throws Exception {
@@ -64,10 +68,11 @@ public class HBaseBoltTest extends BaseBoltTest {
   }
 
   @Before
-  public void setup() {
+  public void setup() throws Exception {
     tuple1 = mock(Tuple.class);
     tuple2 = mock(Tuple.class);
     client = mock(HBaseClient.class);
+    provider = mock(TableProvider.class);
   }
 
   /**
@@ -75,9 +80,9 @@ public class HBaseBoltTest extends BaseBoltTest {
    */
   private HBaseBolt createBolt(int batchSize, WidgetMapper mapper) throws IOException {
     HBaseBolt bolt = new HBaseBolt(tableName, mapper)
-            .withBatchSize(batchSize)
-            .withClientFactory((f, c, t) -> client);
+            .withBatchSize(batchSize).withTableProviderInstance(provider);
     bolt.prepare(Collections.emptyMap(), topologyContext, outputCollector);
+    bolt.setClient(client);
     return bolt;
   }
 
diff --git a/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/hbase/bolt/Widget.java b/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/hbase/bolt/Widget.java
deleted file mode 100644
index e25d4d5..0000000
--- a/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/hbase/bolt/Widget.java
+++ /dev/null
@@ -1,84 +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.hbase.bolt;
-
-/**
- * A simple POJO used for testing.
- */
-public class Widget {
-
-  /**
-   * The name of the widget.
-   */
-  private String name;
-
-  /**
-   * The cost of the widget.
-   */
-  private int cost;
-
-  public Widget(String name, int cost) {
-    this.name = name;
-    this.cost = cost;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public int getCost() {
-    return cost;
-  }
-
-  public void setCost(int cost) {
-    this.cost = cost;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    Widget widget = (Widget) o;
-    if (cost != widget.cost) return false;
-    return name != null ? name.equals(widget.name) : widget.name == null;
-
-  }
-
-  @Override
-  public int hashCode() {
-    int result = name != null ? name.hashCode() : 0;
-    result = 31 * result + cost;
-    return result;
-  }
-
-  @Override
-  public String toString() {
-    return "Widget{" +
-            "name='" + name + '\'' +
-            ", cost=" + cost +
-            '}';
-  }
-}
diff --git a/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/hbase/bolt/WidgetMapper.java b/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/hbase/bolt/WidgetMapper.java
deleted file mode 100644
index ac6b561..0000000
--- a/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/hbase/bolt/WidgetMapper.java
+++ /dev/null
@@ -1,71 +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.hbase.bolt;
-
-import org.apache.metron.hbase.bolt.mapper.HBaseMapper;
-import org.apache.storm.tuple.Tuple;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.hbase.ColumnList;
-
-import java.util.Optional;
-
-
-/**
- * Maps a Widget to HBase.  Used only for testing.
- */
-public class WidgetMapper implements HBaseMapper {
-
-  private Optional<Long> ttl;
-
-  public WidgetMapper() {
-    this.ttl = Optional.empty();
-  }
-
-  public WidgetMapper(Long ttl) {
-    this.ttl = Optional.of(ttl);
-  }
-
-  @Override
-  public byte[] rowKey(Tuple tuple) {
-    Widget w = (Widget) tuple.getValueByField("widget");
-    return Bytes.toBytes(w.getName());
-  }
-
-  @Override
-  public ColumnList columns(Tuple tuple) {
-    Widget w = (Widget) tuple.getValueByField("widget");
-
-    ColumnList cols = new ColumnList();
-    cols.addColumn(CF, QNAME, Bytes.toBytes(w.getName()));
-    cols.addColumn(CF, QCOST, Bytes.toBytes(w.getCost()));
-    return cols;
-  }
-
-  @Override
-  public Optional<Long> getTTL(Tuple tuple) {
-    return ttl;
-  }
-
-  public static final String CF_STRING = "cfWidget";
-  public static final byte[] CF = Bytes.toBytes(CF_STRING);
-  public static final byte[] QNAME = Bytes.toBytes("qName");
-  public static final byte[] QCOST = Bytes.toBytes("qCost");
-}
diff --git a/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/profiler/storm/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/profiler/storm/integration/ProfilerIntegrationTest.java
index 4546a7c..9cf150a 100644
--- a/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/profiler/storm/integration/ProfilerIntegrationTest.java
+++ b/metron-analytics/metron-profiler-storm/src/test/java/org/apache/metron/profiler/storm/integration/ProfilerIntegrationTest.java
@@ -24,16 +24,14 @@ import org.adrianwalker.multilinestring.Multiline;
 import org.apache.commons.io.FileUtils;
 import org.apache.metron.common.Constants;
 import org.apache.metron.common.configuration.profiler.ProfilerConfig;
-import org.apache.metron.hbase.client.FakeHBaseClient;
-import org.apache.metron.hbase.client.FakeHBaseClientFactory;
-import org.apache.metron.hbase.client.FakeHBaseConnectionFactory;
+import org.apache.metron.hbase.mock.MockHBaseTableProvider;
+import org.apache.metron.hbase.mock.MockHTable;
 import org.apache.metron.integration.BaseIntegrationTest;
 import org.apache.metron.integration.ComponentRunner;
 import org.apache.metron.integration.UnableToStartException;
 import org.apache.metron.integration.components.FluxTopologyComponent;
 import org.apache.metron.integration.components.KafkaComponent;
 import org.apache.metron.integration.components.ZKServerComponent;
-import org.apache.metron.profiler.client.HBaseProfilerClientFactory;
 import org.apache.metron.profiler.client.stellar.FixedLookback;
 import org.apache.metron.profiler.client.stellar.GetProfile;
 import org.apache.metron.profiler.client.stellar.WindowLookback;
@@ -41,7 +39,6 @@ import org.apache.metron.statistics.OnlineStatisticsProvider;
 import org.apache.metron.stellar.common.DefaultStellarStatefulExecutor;
 import org.apache.metron.stellar.common.StellarStatefulExecutor;
 import org.apache.metron.stellar.dsl.Context;
-import org.apache.metron.stellar.dsl.functions.resolver.FunctionResolver;
 import org.apache.metron.stellar.dsl.functions.resolver.SimpleFunctionResolver;
 import org.apache.storm.Config;
 import org.json.simple.JSONObject;
@@ -67,6 +64,7 @@ import java.util.concurrent.TimeUnit;
 import static org.apache.metron.integration.utils.TestUtils.assertEventually;
 import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_COLUMN_FAMILY;
 import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE;
+import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_HBASE_TABLE_PROVIDER;
 import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_PERIOD;
 import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_PERIOD_UNITS;
 import static org.apache.metron.profiler.client.stellar.ProfilerClientConfig.PROFILER_SALT_DIVISOR;
@@ -111,6 +109,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
   private static KafkaComponent kafkaComponent;
   private static ConfigUploadComponent configUploadComponent;
   private static ComponentRunner runner;
+  private static MockHTable profilerTable;
   private static String message1;
   private static String message2;
   private static String message3;
@@ -399,6 +398,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
 
   @BeforeClass
   public static void setupBeforeClass() throws UnableToStartException {
+
     // create some messages that contain a timestamp - a really old timestamp; close to 1970
     message1 = getMessage(entity, startAt);
     message2 = getMessage(entity, startAt + 100);
@@ -433,8 +433,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
       setProperty("profiler.hbase.column.family", columnFamily);
       setProperty("profiler.hbase.batch", "10");
       setProperty("profiler.hbase.flush.interval.seconds", "1");
-      setProperty("profiler.hbase.connection.factory", FakeHBaseConnectionFactory.class.getName());
-      setProperty("profiler.hbase.client.factory", FakeHBaseClientFactory.class.getName());
+      setProperty("hbase.provider.impl", "" + MockHBaseTableProvider.class.getName());
 
       // profile settings
       setProperty("profiler.period.duration", Long.toString(periodDurationMillis));
@@ -448,6 +447,9 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
       setProperty("profiler.max.routes.per.bolt", Long.toString(maxRoutesPerBolt));
     }};
 
+    // create the mock table
+    profilerTable = (MockHTable) MockHBaseTableProvider.addToCache(tableName, columnFamily);
+
     zkComponent = getZKServerComponent(topologyProperties);
 
     // create the input and output topics
@@ -480,7 +482,8 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
   }
 
   @AfterClass
-  public static void tearDownAfterClass() {
+  public static void tearDownAfterClass() throws Exception {
+    MockHBaseTableProvider.clear();
     if (runner != null) {
       runner.stop();
     }
@@ -488,10 +491,14 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
 
   @Before
   public void setup() {
+    // create the mock table
+    profilerTable = (MockHTable) MockHBaseTableProvider.addToCache(tableName, columnFamily);
+
     // global properties
     Map<String, Object> global = new HashMap<String, Object>() {{
       put(PROFILER_HBASE_TABLE.getKey(), tableName);
       put(PROFILER_COLUMN_FAMILY.getKey(), columnFamily);
+      put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockHBaseTableProvider.class.getName());
 
       // client needs to use the same period duration
       put(PROFILER_PERIOD.getKey(), Long.toString(periodDurationMillis));
@@ -501,28 +508,21 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest {
       put(PROFILER_SALT_DIVISOR.getKey(), saltDivisor);
     }};
 
-    Context context = new Context.Builder()
-            .with(Context.Capabilities.GLOBAL_CONFIG, () -> global)
-            .build();
-
-    // create the GET_PROFILE function
-    GetProfile getProfileFunction = new GetProfile(new HBaseProfilerClientFactory(new FakeHBaseClientFactory()));
-
-    // ensure the functions that we need can be resolved
-    FunctionResolver functionResolver = new SimpleFunctionResolver()
-            .withClass(FixedLookback.class)
-            .withClass(WindowLookback.class)
-            .withInstance(getProfileFunction);
-
     // create the stellar execution environment
-    executor = new DefaultStellarStatefulExecutor(functionResolver, context);
-
-    // ensure that all HBase "records" are cleared before starting the test
-    new FakeHBaseClient().deleteAll();
+    executor = new DefaultStellarStatefulExecutor(
+            new SimpleFunctionResolver()
+                    .withClass(GetProfile.class)
+                    .withClass(FixedLookback.class)
+                    .withClass(WindowLookback.class),
+            new Context.Builder()
+                    .with(Context.Capabilities.GLOBAL_CONFIG, () -> global)
+                    .build());
   }
 
   @After
   public void tearDown() throws Exception {
+    MockHBaseTableProvider.clear();
+    profilerTable.clear();
     if (runner != null) {
       fluxComponent.stop();
       configUploadComponent.reset();
diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2 b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2
index d89dcab..d8bc13d 100644
--- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2
+++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2
@@ -61,8 +61,6 @@ profiler.hbase.table={{profiler_hbase_table}}
 profiler.hbase.column.family={{profiler_hbase_cf}}
 profiler.hbase.batch={{profiler_hbase_batch}}
 profiler.hbase.flush.interval.seconds={{profiler_hbase_flush_interval}}
-profiler.hbase.client.factory=org.apache.metron.hbase.client.HBaseTableClientFactory
-profiler.hbase.connection.factory=org.apache.metron.hbase.client.HBaseConnectionFactory
 
 ##### Kafka #####
 
diff --git a/metron-interface/metron-rest/pom.xml b/metron-interface/metron-rest/pom.xml
index 8718ed8..5207a99 100644
--- a/metron-interface/metron-rest/pom.xml
+++ b/metron-interface/metron-rest/pom.xml
@@ -1,15 +1,15 @@
 <?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. 
+<!--
+  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"
@@ -290,6 +290,10 @@
                     <groupId>com.google.guava</groupId>
                     <artifactId>guava</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>commons-httpclient</groupId>
+                    <artifactId>commons-httpclient</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
       <dependency>
@@ -417,6 +421,10 @@
                     <artifactId>javassist</artifactId>
                     <groupId>org.javassist</groupId>
                 </exclusion>
+                <exclusion>
+                    <groupId>commons-httpclient</groupId>
+                    <artifactId>commons-httpclient</artifactId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -497,6 +505,14 @@
                     <groupId>org.codehaus.groovy</groupId>
                     <artifactId>groovy-all</artifactId>
                 </exclusion>
+                <exclusion>
+                    <artifactId>httpcore</artifactId>
+                    <groupId>org.apache.httpcomponents</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>httpclient</artifactId>
+                    <groupId>org.apache.httpcomponents</groupId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java
index 0e79ee6..54f721c 100644
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java
+++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java
@@ -68,6 +68,7 @@ public class MetronRestConstants {
   public static final String SEARCH_MAX_GROUPS = "search.max.groups";
   public static final String SEARCH_FACET_FIELDS_SPRING_PROPERTY = "search.facet.fields";
   public static final String INDEX_DAO_IMPL = "index.dao.impl";
+  public static final String INDEX_HBASE_TABLE_PROVIDER_IMPL = "index.hbase.provider";
   public static final String INDEX_WRITER_NAME = "index.writer.name";
 
   public static final String META_DAO_IMPL = "meta.dao.impl";
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/HBaseConfig.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/HBaseConfig.java
index 44bcb12..7ce16f9 100644
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/HBaseConfig.java
+++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/HBaseConfig.java
@@ -17,97 +17,66 @@
  */
 package org.apache.metron.rest.config;
 
+import static org.apache.metron.rest.MetronRestConstants.TEST_PROFILE;
+
+import java.lang.reflect.InvocationTargetException;
+import java.util.Map;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.metron.common.configuration.EnrichmentConfigurations;
 import org.apache.metron.hbase.HTableProvider;
 import org.apache.metron.hbase.TableProvider;
 import org.apache.metron.hbase.client.HBaseClient;
-import org.apache.metron.hbase.client.HBaseClientFactory;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
-import org.apache.metron.hbase.client.HBaseTableClient;
-import org.apache.metron.hbase.client.HBaseTableClientFactory;
-import org.apache.metron.hbase.client.LegacyHBaseClient;
 import org.apache.metron.rest.RestException;
 import org.apache.metron.rest.service.GlobalConfigService;
-import org.apache.metron.rest.user.HBaseUserSettingsClient;
 import org.apache.metron.rest.user.UserSettingsClient;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Configuration;
 import org.springframework.context.annotation.Profile;
 
-import java.lang.reflect.InvocationTargetException;
-import java.util.Map;
-import java.util.function.Supplier;
-
-import static org.apache.metron.rest.MetronRestConstants.TEST_PROFILE;
-
 @Configuration
 @Profile("!" + TEST_PROFILE)
 public class HBaseConfig {
 
-  @Autowired
-  private GlobalConfigService globalConfigService;
+    @Autowired
+    private GlobalConfigService globalConfigService;
 
-  private Supplier<Map<String, Object>> globals = () -> {
-    try {
-      return globalConfigService.get();
-    } catch (RestException e) {
-      throw new IllegalStateException("Unable to retrieve the global config.", e);
+    @Autowired
+    public HBaseConfig(GlobalConfigService globalConfigService) {
+        this.globalConfigService = globalConfigService;
     }
-  };
-
-  @Autowired
-  public HBaseConfig(GlobalConfigService globalConfigService) {
-    this.globalConfigService = globalConfigService;
-  }
-
-  @Bean
-  public HBaseConnectionFactory hBaseConnectionFactory() {
-    return new HBaseConnectionFactory();
-  }
-
-  @Bean
-  org.apache.hadoop.conf.Configuration hBaseConfiguration() {
-    return HBaseConfiguration.create();
-  }
 
-  @Bean
-  HBaseClientFactory hBaseClientFactory() {
-    return new HBaseTableClientFactory();
-  }
-
-  @Bean(destroyMethod = "close")
-  public UserSettingsClient userSettingsClient(GlobalConfigService globalConfigService,
-                                               HBaseClientFactory hBaseClientFactory,
-                                               HBaseConnectionFactory hBaseConnectionFactory,
-                                               org.apache.hadoop.conf.Configuration hBaseConfiguration) {
-    UserSettingsClient userSettingsClient = new HBaseUserSettingsClient(
-            globals,
-            hBaseClientFactory,
-            hBaseConnectionFactory,
-            hBaseConfiguration);
-    userSettingsClient.init();
-    return userSettingsClient;
-  }
-
-  @Bean()
-  public LegacyHBaseClient legacyHBaseClient() {
-    Map<String, Object> restConfig = null;
-    try {
-      restConfig = globalConfigService.get();
-    } catch (RestException e) {
-      throw new IllegalStateException("Unable to retrieve the global config.", e);
+    @Bean()
+    public UserSettingsClient userSettingsClient() {
+      UserSettingsClient userSettingsClient = new UserSettingsClient();
+      userSettingsClient.init(() -> {
+        try {
+          return globalConfigService.get();
+        } catch (RestException e) {
+          throw new IllegalStateException("Unable to retrieve the global config.", e);
+        }
+      }, new HTableProvider());
+      return userSettingsClient;
     }
-    TableProvider provider = null;
-    try {
-      provider = TableProvider
-              .create((String) restConfig.get(EnrichmentConfigurations.TABLE_PROVIDER),
-                      HTableProvider::new);
-    } catch (ClassNotFoundException | InstantiationException | InvocationTargetException | IllegalAccessException | NoSuchMethodException e) {
-      throw new IllegalStateException("Unable to create table provider", e);
+
+    @Bean()
+    public HBaseClient hBaseClient() {
+      Map<String, Object> restConfig = null;
+      try {
+        restConfig = globalConfigService.get();
+      } catch (RestException e) {
+        throw new IllegalStateException("Unable to retrieve the global config.", e);
+      }
+      TableProvider provider = null;
+      try {
+        provider = TableProvider
+            .create((String) restConfig.get(EnrichmentConfigurations.TABLE_PROVIDER),
+                HTableProvider::new);
+      } catch (ClassNotFoundException | InstantiationException | InvocationTargetException | IllegalAccessException | NoSuchMethodException e) {
+        throw new IllegalStateException("Unable to create table provider", e);
+      }
+      return new HBaseClient(provider, HBaseConfiguration.create(),
+          (String) restConfig.get(EnrichmentConfigurations.TABLE_NAME));
     }
-    return new LegacyHBaseClient(provider, HBaseConfiguration.create(),
-            (String) restConfig.get(EnrichmentConfigurations.TABLE_NAME));
-  }
+
 }
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java
index 5ae8326..53b10f9 100644
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java
+++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java
@@ -17,8 +17,13 @@
  */
 package org.apache.metron.rest.config;
 
+import static org.apache.metron.rest.MetronRestConstants.INDEX_DAO_IMPL;
+import static org.apache.metron.rest.MetronRestConstants.INDEX_WRITER_NAME;
+
+import java.util.Optional;
 import org.apache.metron.common.zookeeper.ConfigurationsCache;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
+import org.apache.metron.hbase.HTableProvider;
+import org.apache.metron.hbase.TableProvider;
 import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.IndexDao;
 import org.apache.metron.indexing.dao.IndexDaoFactory;
@@ -32,11 +37,6 @@ import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Configuration;
 import org.springframework.core.env.Environment;
 
-import java.util.Optional;
-
-import static org.apache.metron.rest.MetronRestConstants.INDEX_DAO_IMPL;
-import static org.apache.metron.rest.MetronRestConstants.INDEX_WRITER_NAME;
-
 @Configuration
 public class IndexConfig {
 
@@ -55,55 +55,50 @@ public class IndexConfig {
   }
 
   @Bean
-  public AccessConfig accessConfig(HBaseConnectionFactory hBaseConnectionFactory,
-                                   org.apache.hadoop.conf.Configuration hBaseConfiguration) {
-    int searchMaxResults = environment.getProperty(MetronRestConstants.SEARCH_MAX_RESULTS, Integer.class, 1000);
-    int searchMaxGroups = environment.getProperty(MetronRestConstants.SEARCH_MAX_GROUPS, Integer.class, 1000);
-
-    AccessConfig config = new AccessConfig();
-    config.setHbaseConnectionFactory(hBaseConnectionFactory);
-    config.setHbaseConfiguration(hBaseConfiguration);
-    config.setMaxSearchResults(searchMaxResults);
-    config.setMaxSearchGroups(searchMaxGroups);
-    config.setGlobalConfigSupplier(() -> {
-      try {
-        return globalConfigService.get();
-      } catch (RestException e) {
-        throw new IllegalStateException("Unable to retrieve the global config.", e);
-      }
-    });
-    config.setIndexSupplier(IndexingCacheUtil.getIndexLookupFunction(cache, environment.getProperty(INDEX_WRITER_NAME)));
-    config.setKerberosEnabled(environment.getProperty(MetronRestConstants.KERBEROS_ENABLED_SPRING_PROPERTY, Boolean.class, false));
-    return config;
-  }
-
-  @Bean(destroyMethod = "close")
-  public IndexDao indexDao(AccessConfig accessConfig) {
+  public IndexDao indexDao() {
     try {
+      String hbaseProviderImpl = environment.getProperty(MetronRestConstants.INDEX_HBASE_TABLE_PROVIDER_IMPL, String.class, null);
       String indexDaoImpl = environment.getProperty(MetronRestConstants.INDEX_DAO_IMPL, String.class, null);
+      int searchMaxResults = environment.getProperty(MetronRestConstants.SEARCH_MAX_RESULTS, Integer.class, 1000);
+      int searchMaxGroups = environment.getProperty(MetronRestConstants.SEARCH_MAX_GROUPS, Integer.class, 1000);
+      String metaDaoImpl = environment.getProperty(MetronRestConstants.META_DAO_IMPL, String.class, null);
+      String metaDaoSort = environment.getProperty(MetronRestConstants.META_DAO_SORT, String.class, null);
+
+      AccessConfig config = new AccessConfig();
+      config.setMaxSearchResults(searchMaxResults);
+      config.setMaxSearchGroups(searchMaxGroups);
+      config.setGlobalConfigSupplier(() -> {
+        try {
+          return globalConfigService.get();
+        } catch (RestException e) {
+          throw new IllegalStateException("Unable to retrieve the global config.", e);
+        }
+      });
+      config.setIndexSupplier(IndexingCacheUtil.getIndexLookupFunction(cache, environment.getProperty(INDEX_WRITER_NAME)));
+      config.setTableProvider(TableProvider.create(hbaseProviderImpl, () -> new HTableProvider()));
+      config.setKerberosEnabled(environment.getProperty(MetronRestConstants.KERBEROS_ENABLED_SPRING_PROPERTY, Boolean.class, false));
       if (indexDaoImpl == null) {
         throw new IllegalStateException("You must provide an index DAO implementation via the " + INDEX_DAO_IMPL + " config");
       }
-
-      IndexDao indexDao = IndexDaoFactory.combine(IndexDaoFactory.create(indexDaoImpl, accessConfig));
+      IndexDao indexDao = IndexDaoFactory.combine(IndexDaoFactory.create(indexDaoImpl, config));
       if (indexDao == null) {
         throw new IllegalStateException("IndexDao is unable to be created.");
       }
-
-      String metaDaoImpl = environment.getProperty(MetronRestConstants.META_DAO_IMPL, String.class, null);
       if (metaDaoImpl == null) {
         // We're not using meta alerts.
         return indexDao;
-
-      } else {
-        // Create the meta alert dao and wrap it around the index dao.
-        String metaDaoSort = environment.getProperty(MetronRestConstants.META_DAO_SORT, String.class, null);
-        MetaAlertDao metaAlertDao = (MetaAlertDao) IndexDaoFactory.create(metaDaoImpl, accessConfig).get(0);
-        metaAlertDao.init(indexDao, Optional.ofNullable(metaDaoSort));
-        return metaAlertDao;
       }
 
-    } catch(Exception e) {
+      // Create the meta alert dao and wrap it around the index dao.
+      MetaAlertDao ret = (MetaAlertDao) IndexDaoFactory.create(metaDaoImpl, config).get(0);
+      ret.init(indexDao, Optional.ofNullable(metaDaoSort));
+      return ret;
+
+    }
+    catch(RuntimeException re) {
+      throw re;
+    }
+    catch(Exception e) {
       throw new IllegalStateException("Unable to create index DAO: " + e.getMessage(), e);
     }
   }
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/AlertsUIService.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/AlertsUIService.java
index 3b15879..cd9ff9d 100644
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/AlertsUIService.java
+++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/AlertsUIService.java
@@ -37,5 +37,5 @@ public interface AlertsUIService {
 
   void saveAlertsUIUserSettings(AlertsUIUserSettings userSettings) throws RestException;
 
-  boolean deleteAlertsUIUserSettings(String user) throws RestException;
+  boolean deleteAlertsUIUserSettings(String user);
 }
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/AlertsUIServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/AlertsUIServiceImpl.java
index 9854f83..6092ce2 100644
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/AlertsUIServiceImpl.java
+++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/AlertsUIServiceImpl.java
@@ -126,7 +126,7 @@ public class AlertsUIServiceImpl implements AlertsUIService {
   }
 
   @Override
-  public void saveAlertsUIUserSettings(AlertsUIUserSettings alertsUIUserSettings) throws RestException {
+  public void saveAlertsUIUserSettings(AlertsUIUserSettings alertsUIUserSettings) throws RestException{
     String user = SecurityUtils.getCurrentUser();
     try {
       userSettingsClient.save(user, ALERT_USER_SETTING_TYPE, _mapper.get().writeValueAsString(alertsUIUserSettings));
@@ -136,15 +136,12 @@ public class AlertsUIServiceImpl implements AlertsUIService {
   }
 
   @Override
-  public boolean deleteAlertsUIUserSettings(String user) throws RestException {
-    boolean success = false;
+  public boolean deleteAlertsUIUserSettings(String user) {
+    boolean success = true;
     try {
-      if(userSettingsClient.findOne(user, ALERT_USER_SETTING_TYPE).isPresent()) {
-        userSettingsClient.delete(user, ALERT_USER_SETTING_TYPE);
-        success = true;
-      }
+      userSettingsClient.delete(user, ALERT_USER_SETTING_TYPE);
     } catch (IOException e) {
-      throw new RestException(e);
+      success = false;
     }
     return success;
   }
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SensorEnrichmentConfigServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SensorEnrichmentConfigServiceImpl.java
index d7f7b7f..5c0f2e0 100644
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SensorEnrichmentConfigServiceImpl.java
+++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SensorEnrichmentConfigServiceImpl.java
@@ -32,7 +32,7 @@ import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.apache.metron.common.configuration.EnrichmentConfigurations;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.zookeeper.ConfigurationsCache;
-import org.apache.metron.hbase.client.LegacyHBaseClient;
+import org.apache.metron.hbase.client.HBaseClient;
 import org.apache.metron.rest.RestException;
 import org.apache.metron.rest.service.SensorEnrichmentConfigService;
 import org.apache.zookeeper.KeeperException;
@@ -48,12 +48,12 @@ public class SensorEnrichmentConfigServiceImpl implements SensorEnrichmentConfig
 
     private ConfigurationsCache cache;
 
-    private LegacyHBaseClient hBaseClient;
+    private HBaseClient hBaseClient;
 
     @Autowired
     public SensorEnrichmentConfigServiceImpl(final ObjectMapper objectMapper,
         final CuratorFramework client, final ConfigurationsCache cache,
-        final LegacyHBaseClient hBaseClient) {
+        final HBaseClient hBaseClient) {
       this.objectMapper = objectMapper;
       this.client = client;
       this.cache = cache;
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/user/HBaseUserSettingsClient.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/user/HBaseUserSettingsClient.java
deleted file mode 100644
index e5432c3..0000000
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/user/HBaseUserSettingsClient.java
+++ /dev/null
@@ -1,218 +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.rest.user;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.hbase.ColumnList;
-import org.apache.metron.hbase.HBaseProjectionCriteria;
-import org.apache.metron.hbase.client.HBaseClient;
-import org.apache.metron.hbase.client.HBaseClientFactory;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.nio.charset.StandardCharsets;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Optional;
-import java.util.function.Supplier;
-
-/**
- * A {@link UserSettingsClient} that interacts with user settings persisted in HBase.
- */
-public class HBaseUserSettingsClient implements UserSettingsClient {
-  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  public static final String USER_SETTINGS_HBASE_TABLE = "user.settings.hbase.table";
-  public static final String USER_SETTINGS_HBASE_CF = "user.settings.hbase.cf";
-  public static final String USER_SETTINGS_MAX_SCAN = "user.settings.max.scan";
-  private static final int DEFAULT_MAX_SCAN = 100_000;
-
-  private String columnFamily;
-  private Supplier<Map<String, Object>> globalConfigSupplier;
-  private HBaseConnectionFactory hBaseConnectionFactory;
-  private Configuration hBaseConfiguration;
-  private HBaseClientFactory hBaseClientFactory;
-  private HBaseClient hBaseClient;
-  private int maxScanCount;
-
-  public HBaseUserSettingsClient(Supplier<Map<String, Object>> globalConfigSupplier,
-                                 HBaseClientFactory hBaseClientFactory,
-                                 HBaseConnectionFactory hBaseConnectionFactory,
-                                 Configuration hBaseConfiguration) {
-    this.globalConfigSupplier = globalConfigSupplier;
-    this.hBaseClientFactory = hBaseClientFactory;
-    this.hBaseConnectionFactory = hBaseConnectionFactory;
-    this.hBaseConfiguration = hBaseConfiguration;
-  }
-
-  @Override
-  public synchronized void init() {
-    if (hBaseClient == null) {
-      Map<String, Object> globals = getGlobals();
-      columnFamily = getColumnFamily(globals);
-      maxScanCount = getMaxScanCount(globals);
-      hBaseClient = hBaseClientFactory.create(hBaseConnectionFactory, hBaseConfiguration,  getTableName(globals));
-    }
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-    if(hBaseClient != null) {
-      hBaseClient.close();
-    }
-  }
-
-  @Override
-  public Map<String, String> findOne(String user) throws IOException {
-    Result result = getResult(user);
-    return getAllUserSettings(result);
-  }
-
-  @Override
-  public Optional<String> findOne(String user, String type) throws IOException {
-    Result result = getResult(user);
-    return getUserSettings(result, type);
-  }
-
-  @Override
-  public Map<String, Map<String, String>> findAll() throws IOException {
-    Map<String, Map<String, String>> settings = new HashMap<>();
-    for (Result result : hBaseClient.scan(maxScanCount)) {
-      String user = new String(result.getRow());
-      settings.put(user, getAllUserSettings(result));
-    }
-    return settings;
-  }
-
-  @Override
-  public Map<String, Optional<String>> findAll(String type) throws IOException {
-    Map<String, Optional<String>> settings = new HashMap<>();
-    for (Result result : hBaseClient.scan(maxScanCount)) {
-      String user = new String(result.getRow());
-      settings.put(user, getUserSettings(result, type));
-    }
-    return settings;
-  }
-
-  @Override
-  public void save(String user, String type, String userSettings) throws IOException {
-    byte[] rowKey = userToRowKey(user);
-    ColumnList columns = new ColumnList().addColumn(columnFamily, type, userSettings);
-    hBaseClient.addMutation(rowKey, columns);
-    hBaseClient.mutate();
-  }
-
-  @Override
-  public void delete(String user) {
-    hBaseClient.delete(userToRowKey(user));
-  }
-
-  @Override
-  public void delete(String user, String type)  {
-    byte[] rowKey = userToRowKey(user);
-    ColumnList columns = new ColumnList().addColumn(columnFamily, type);
-    hBaseClient.delete(rowKey, columns);
-  }
-
-  private Result getResult(String user) {
-    // for the given user's row, retrieve the column family
-    byte[] rowKey = userToRowKey(user);
-    HBaseProjectionCriteria criteria = new HBaseProjectionCriteria().addColumnFamily(columnFamily);
-    hBaseClient.addGet(rowKey, criteria);
-    Result[] results = hBaseClient.getAll();
-
-    // expect 1 result, if the user settings exist
-    Result result = null;
-    if(results.length > 0) {
-      result = results[0];
-    } else {
-      LOG.debug("No result found; user={}, columnFamily={}", user, columnFamily);
-    }
-    return result;
-  }
-
-  private byte[] userToRowKey(String user) {
-    return Bytes.toBytes(user);
-  }
-
-  private Optional<String> getUserSettings(Result result, String type) throws IOException {
-    Optional<String> userSettings = Optional.empty();
-    if (result != null) {
-      byte[] value = result.getValue(Bytes.toBytes(columnFamily), Bytes.toBytes(type));
-      if (value != null) {
-        userSettings = Optional.of(new String(value, StandardCharsets.UTF_8));
-      }
-    }
-    return userSettings;
-  }
-
-  public Map<String, String> getAllUserSettings(Result result) {
-    if (result == null) {
-      return new HashMap<>();
-    }
-    NavigableMap<byte[], byte[]> columns = result.getFamilyMap(Bytes.toBytes(columnFamily));
-    if(columns == null || columns.size() == 0) {
-      return new HashMap<>();
-    }
-    Map<String, String> userSettingsMap = new HashMap<>();
-    for(Map.Entry<byte[], byte[]> column: columns.entrySet()) {
-      userSettingsMap.put(
-              new String(column.getKey(), StandardCharsets.UTF_8),
-              new String(column.getValue(), StandardCharsets.UTF_8));
-    }
-    return userSettingsMap;
-  }
-
-  private Map<String, Object> getGlobals() {
-    Map<String, Object> globalConfig = globalConfigSupplier.get();
-    if(globalConfig == null) {
-      throw new IllegalStateException("Cannot find the global config.");
-    }
-    return globalConfig;
-  }
-
-  private static String getTableName(Map<String, Object> globals) {
-    String table = (String) globals.get(USER_SETTINGS_HBASE_TABLE);
-    if(table == null) {
-      throw new IllegalStateException("You must configure " + USER_SETTINGS_HBASE_TABLE + "in the global config.");
-    }
-    return table;
-  }
-
-  private static String getColumnFamily(Map<String, Object> globals) {
-    String cf = (String) globals.get(USER_SETTINGS_HBASE_CF);
-    if(cf == null) {
-      throw new IllegalStateException("You must configure " + USER_SETTINGS_HBASE_CF + " in the global config.");
-    }
-    return cf;
-  }
-
-  private static int getMaxScanCount(Map<String, Object> globals) {
-    Integer maxScanCount = DEFAULT_MAX_SCAN;
-    if(globals.containsKey(USER_SETTINGS_MAX_SCAN)) {
-      String value = (String) globals.get(USER_SETTINGS_MAX_SCAN);
-      maxScanCount = Integer.valueOf(value);
-    }
-    return maxScanCount;
-  }
-}
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/user/UserSettingsClient.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/user/UserSettingsClient.java
index 8efd4dd..a08b775 100644
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/user/UserSettingsClient.java
+++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/user/UserSettingsClient.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -17,85 +17,158 @@
  */
 package org.apache.metron.rest.user;
 
-import java.io.Closeable;
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
 import java.util.Map;
+import java.util.NavigableMap;
 import java.util.Optional;
+import java.util.function.Supplier;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.hbase.TableProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-/**
- * A client that interacts with persisted user settings data.  Enables create, read, and
- * delete operations on user settings.
- */
-public interface UserSettingsClient extends Closeable {
-
-  /**
-   * Initialize the client.
-   */
-  void init();
-
-  /**
-   * Finds all settings for a particular user.
-   *
-   * <p>There may be more than one type of setting associated with each user. This returns all
-   * setting types for a particular user.
-   *
-   * @param user The user.
-   * @return All of the user's settings.
-   * @throws IOException
-   */
-  Map<String, String> findOne(String user) throws IOException;
-
-  /**
-   * Finds one setting for a particular user.
-   *
-   * @param user The user.
-   * @param type The setting type.
-   * @return The value of the setting.
-   * @throws IOException
-   */
-  Optional<String> findOne(String user, String type) throws IOException;
-
-  /**
-   * Finds all settings for all users.
-   *
-   * @return All settings value for all users.
-   * @throws IOException
-   */
-  Map<String, Map<String, String>> findAll() throws IOException;
-
-  /**
-   * Finds the value of a particular setting type for all users.
-   *
-   * @param type The setting type.
-   * @return The value of the setting type for all users.
-   * @throws IOException
-   */
-  Map<String, Optional<String>> findAll(String type) throws IOException;
-
-  /**
-   * Save a user setting.
-   *
-   * @param user The user.
-   * @param type The setting type.
-   * @param value The value of the user setting.
-   * @throws IOException
-   */
-  void save(String user, String type, String value) throws IOException;
-
-  /**
-   * Delete all user settings for a particular user.
-   *
-   * @param user The user.
-   * @throws IOException
-   */
-  void delete(String user) throws IOException;
-
-  /**
-   * Delete one user setting value.
-   *
-   * @param user The user.
-   * @param type The setting type.
-   * @throws IOException
-   */
-  void delete(String user, String type) throws IOException;
+public class UserSettingsClient {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static String USER_SETTINGS_HBASE_TABLE = "user.settings.hbase.table";
+  public static String USER_SETTINGS_HBASE_CF = "user.settings.hbase.cf";
+
+  private HTableInterface userSettingsTable;
+  private byte[] cf;
+  private Supplier<Map<String, Object>> globalConfigSupplier;
+  private TableProvider tableProvider;
+
+  public UserSettingsClient() {
+  }
+
+  public UserSettingsClient(Supplier<Map<String, Object>> globalConfigSupplier, TableProvider tableProvider) {
+    this.globalConfigSupplier = globalConfigSupplier;
+    this.tableProvider = tableProvider;
+  }
+
+  public UserSettingsClient(HTableInterface userSettingsTable, byte[] cf) {
+    this.userSettingsTable = userSettingsTable;
+    this.cf = cf;
+  }
+
+  public synchronized void init(Supplier<Map<String, Object>> globalConfigSupplier, TableProvider tableProvider) {
+    if (this.userSettingsTable == null) {
+      Map<String, Object> globalConfig = globalConfigSupplier.get();
+      if(globalConfig == null) {
+        throw new IllegalStateException("Cannot find the global config.");
+      }
+      String table = (String)globalConfig.get(USER_SETTINGS_HBASE_TABLE);
+      String cf = (String) globalConfigSupplier.get().get(USER_SETTINGS_HBASE_CF);
+      if(table == null || cf == null) {
+        throw new IllegalStateException("You must configure " + USER_SETTINGS_HBASE_TABLE + " and " + USER_SETTINGS_HBASE_CF + " in the global config.");
+      }
+      try {
+        userSettingsTable = tableProvider.getTable(HBaseConfiguration.create(), table);
+        this.cf = cf.getBytes();
+      } catch (IOException e) {
+        throw new IllegalStateException("Unable to initialize HBaseDao: " + e.getMessage(), e);
+      }
+
+    }
+  }
+
+  public HTableInterface getTableInterface() {
+    if(userSettingsTable == null) {
+      init(globalConfigSupplier, tableProvider);
+    }
+    return userSettingsTable;
+  }
+
+  public Map<String, String> findOne(String user) throws IOException {
+    Result result = getResult(user);
+    return getAllUserSettings(result);
+  }
+
+  public Optional<String> findOne(String user, String type) throws IOException {
+    Result result = getResult(user);
+    return getUserSettings(result, type);
+  }
+
+  public Map<String, Map<String, String>> findAll() throws IOException {
+    Scan scan = new Scan();
+    ResultScanner results = getTableInterface().getScanner(scan);
+    Map<String, Map<String, String>> allUserSettings = new HashMap<>();
+    for (Result result : results) {
+      allUserSettings.put(new String(result.getRow()), getAllUserSettings(result));
+    }
+    return allUserSettings;
+  }
+
+  public Map<String, Optional<String>> findAll(String type) throws IOException {
+    Scan scan = new Scan();
+    ResultScanner results = getTableInterface().getScanner(scan);
+    Map<String, Optional<String>> allUserSettings = new HashMap<>();
+    for (Result result : results) {
+      allUserSettings.put(new String(result.getRow()), getUserSettings(result, type));
+    }
+    return allUserSettings;
+  }
+
+  public void save(String user, String type, String userSettings) throws IOException {
+    byte[] rowKey = Bytes.toBytes(user);
+    Put put = new Put(rowKey);
+    put.addColumn(cf, Bytes.toBytes(type), Bytes.toBytes(userSettings));
+    getTableInterface().put(put);
+  }
+
+  public void delete(String user) throws IOException {
+    Delete delete = new Delete(Bytes.toBytes(user));
+    getTableInterface().delete(delete);
+  }
+
+  public void delete(String user, String type) throws IOException {
+    Delete delete = new Delete(Bytes.toBytes(user));
+    delete.addColumn(cf, Bytes.toBytes(type));
+    getTableInterface().delete(delete);
+  }
+
+  private Result getResult(String user) throws IOException {
+    byte[] rowKey = Bytes.toBytes(user);
+    Get get = new Get(rowKey);
+    get.addFamily(cf);
+    return getTableInterface().get(get);
+  }
+
+  private Optional<String> getUserSettings(Result result, String type) throws IOException {
+    Optional<String> userSettings = Optional.empty();
+    if (result != null) {
+      byte[] value = result.getValue(cf, Bytes.toBytes(type));
+      if (value != null) {
+        userSettings = Optional.of(new String(value, StandardCharsets.UTF_8));
+      }
+    }
+    return userSettings;
+  }
+
+  public Map<String, String> getAllUserSettings(Result result) {
+    if (result == null) {
+      return new HashMap<>();
+    }
+    NavigableMap<byte[], byte[]> columns = result.getFamilyMap(cf);
+    if(columns == null || columns.size() == 0) {
+      return new HashMap<>();
+    }
+    Map<String, String> userSettingsMap = new HashMap<>();
+    for(Map.Entry<byte[], byte[]> column: columns.entrySet()) {
+      userSettingsMap.put(new String(column.getKey(), StandardCharsets.UTF_8), new String(column.getValue(), StandardCharsets.UTF_8));
+    }
+    return userSettingsMap;
+  }
 }
diff --git a/metron-interface/metron-rest/src/main/resources/application-test.yml b/metron-interface/metron-rest/src/main/resources/application-test.yml
index 3f99311..0e794cb 100644
--- a/metron-interface/metron-rest/src/main/resources/application-test.yml
+++ b/metron-interface/metron-rest/src/main/resources/application-test.yml
@@ -49,8 +49,8 @@ search:
 
 index:
   dao:
-  # By default, we use the InMemoryDao for our tests
-     impl: org.apache.metron.indexing.dao.InMemoryDao
+  # By default, we use the InMemoryDao for our tests and HBaseDao for backing updates.
+     impl: org.apache.metron.indexing.dao.InMemoryDao,org.apache.metron.indexing.dao.HBaseDao
   hbase:
   # HBase is provided via a mock provider, so no actual HBase infrastructure is started.
      provider: org.apache.metron.hbase.mock.MockHBaseTableProvider
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/HBaseConfigTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/HBaseConfigTest.java
index 80cc34f..0d45f18 100644
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/HBaseConfigTest.java
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/HBaseConfigTest.java
@@ -17,19 +17,22 @@
  */
 package org.apache.metron.rest.config;
 
+import static org.apache.metron.rest.user.UserSettingsClient.USER_SETTINGS_HBASE_CF;
+import static org.apache.metron.rest.user.UserSettingsClient.USER_SETTINGS_HBASE_TABLE;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.mockStatic;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+import java.util.HashMap;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.metron.common.configuration.EnrichmentConfigurations;
 import org.apache.metron.hbase.HTableProvider;
-import org.apache.metron.hbase.client.FakeHBaseClientFactory;
-import org.apache.metron.hbase.client.HBaseClientFactory;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
 import org.apache.metron.hbase.mock.MockHBaseTableProvider;
 import org.apache.metron.rest.service.GlobalConfigService;
-import org.apache.metron.rest.user.UserSettingsClient;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -37,70 +40,42 @@ import org.junit.runner.RunWith;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
-import java.io.IOException;
-import java.util.HashMap;
-
-import static org.apache.metron.rest.user.HBaseUserSettingsClient.USER_SETTINGS_HBASE_CF;
-import static org.apache.metron.rest.user.HBaseUserSettingsClient.USER_SETTINGS_HBASE_TABLE;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyZeroInteractions;
-import static org.mockito.Mockito.when;
-import static org.powermock.api.mockito.PowerMockito.mockStatic;
-import static org.powermock.api.mockito.PowerMockito.whenNew;
-
 @RunWith(PowerMockRunner.class)
-@PrepareForTest({Table.class, HBaseConfiguration.class, HBaseConfig.class})
+@PrepareForTest({HTableProvider.class, HBaseConfiguration.class, HBaseConfig.class})
 public class HBaseConfigTest {
 
-  private HBaseConnectionFactory hBaseConnectionFactory;
-  private HBaseClientFactory hBaseClientCreator;
-  private HBaseConfiguration hBaseConfiguration;
-  private Configuration configuration;
   private GlobalConfigService globalConfigService;
   private HBaseConfig hBaseConfig;
-  private Connection connection;
-  private Table table;
 
   @Before
-  public void setUp() throws IOException {
-    connection = mock(Connection.class);
-    table = mock(Table.class);
-    hBaseConnectionFactory = mock(HBaseConnectionFactory.class);
-    configuration = mock(Configuration.class);
-    hBaseConfiguration = mock(HBaseConfiguration.class);
-    hBaseClientCreator = mock(FakeHBaseClientFactory.class);
+  public void setUp() throws Exception {
     globalConfigService = mock(GlobalConfigService.class);
     hBaseConfig = new HBaseConfig(globalConfigService);
     mockStatic(HBaseConfiguration.class);
-    when(HBaseConfiguration.create()).thenReturn(configuration);
   }
 
   @Test
-  public void userSettingsShouldBeCreated() throws Exception {
-    final String expectedTable = "hbase-table-name";
-    final String expectedColumnFamily = "hbase-column-family";
+  public void userSettingsTableShouldBeReturnedFromGlobalConfigByDefault() throws Exception {
     when(globalConfigService.get()).thenReturn(new HashMap<String, Object>() {{
-      put(USER_SETTINGS_HBASE_TABLE, expectedTable);
-      put(USER_SETTINGS_HBASE_CF, expectedColumnFamily);
+      put(USER_SETTINGS_HBASE_TABLE, "global_config_user_settings_table");
+      put(USER_SETTINGS_HBASE_CF, "global_config_user_settings_cf");
     }});
+    HTableProvider htableProvider = mock(HTableProvider.class);
+    whenNew(HTableProvider.class).withNoArguments().thenReturn(htableProvider);
+    Configuration configuration = mock(Configuration.class);
+    when(HBaseConfiguration.create()).thenReturn(configuration);
 
-    // connection factory needs to return the mock connection
-    when(hBaseConnectionFactory.createConnection(any()))
-            .thenReturn(connection);
-
-    // connection should return the table, if the expected table name is used
-    when(connection.getTable(eq(TableName.valueOf(expectedTable))))
-            .thenReturn(table);
+    hBaseConfig.userSettingsClient();
+    verify(htableProvider).getTable(configuration, "global_config_user_settings_table");
+    verifyZeroInteractions(htableProvider);
+  }
 
-    UserSettingsClient client = hBaseConfig.userSettingsClient(
-            globalConfigService,
-            hBaseClientCreator,
-            hBaseConnectionFactory,
-            hBaseConfiguration);
-    Assert.assertNotNull(client);
+  @Test
+  public void hBaseClientShouldBeCreatedWithDefaultProvider() throws Exception {
+    when(globalConfigService.get()).thenReturn(new HashMap<String, Object>() {{
+      put(EnrichmentConfigurations.TABLE_NAME, "enrichment_list_hbase_table_name");
+    }});
+    Assert.assertNotNull(hBaseConfig.hBaseClient());
   }
 
   @Test
@@ -109,6 +84,7 @@ public class HBaseConfigTest {
       put(EnrichmentConfigurations.TABLE_PROVIDER, MockHBaseTableProvider.class.getName());
       put(EnrichmentConfigurations.TABLE_NAME, "enrichment_list_hbase_table_name");
     }});
-    Assert.assertNotNull(hBaseConfig.legacyHBaseClient());
+    Assert.assertNotNull(hBaseConfig.hBaseClient());
   }
+
 }
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java
index 3d13341..1509ae2 100644
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java
@@ -17,6 +17,19 @@
  */
 package org.apache.metron.rest.config;
 
+import static org.apache.metron.rest.MetronRestConstants.TEST_PROFILE;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import kafka.admin.AdminUtils$;
 import kafka.utils.ZKStringSerializer$;
 import kafka.utils.ZkUtils;
 import org.I0Itec.zkclient.ZkClient;
@@ -37,13 +50,7 @@ import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.apache.metron.common.zookeeper.ConfigurationsCache;
 import org.apache.metron.common.zookeeper.ZKConfigurationsCache;
-import org.apache.metron.hbase.client.FakeHBaseClient;
-import org.apache.metron.hbase.client.FakeHBaseClientFactory;
-import org.apache.metron.hbase.client.FakeHBaseConnectionFactory;
 import org.apache.metron.hbase.client.HBaseClient;
-import org.apache.metron.hbase.client.HBaseClientFactory;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
-import org.apache.metron.hbase.client.LegacyHBaseClient;
 import org.apache.metron.hbase.mock.MockHBaseTableProvider;
 import org.apache.metron.integration.ComponentRunner;
 import org.apache.metron.integration.UnableToStartException;
@@ -61,7 +68,6 @@ import org.apache.metron.rest.service.StormStatusService;
 import org.apache.metron.rest.service.impl.CachedStormStatusServiceImpl;
 import org.apache.metron.rest.service.impl.PcapToPdmlScriptWrapper;
 import org.apache.metron.rest.service.impl.StormCLIWrapper;
-import org.apache.metron.rest.user.HBaseUserSettingsClient;
 import org.apache.metron.rest.user.UserSettingsClient;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.beans.factory.annotation.Qualifier;
@@ -72,31 +78,13 @@ import org.springframework.kafka.core.ConsumerFactory;
 import org.springframework.kafka.core.DefaultKafkaConsumerFactory;
 import org.springframework.web.client.RestTemplate;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import static org.apache.metron.rest.MetronRestConstants.TEST_PROFILE;
-
 @Configuration
 @Profile(TEST_PROFILE)
 public class TestConfig {
 
-  @Autowired
-  private HBaseConnectionFactory hBaseConnectionFactory;
-
-  @Autowired
-  private org.apache.hadoop.conf.Configuration hBaseConfiguration;
-
-  @Autowired
-  private HBaseClientFactory hBaseClientFactory;
+  static {
+    MockHBaseTableProvider.addToCache("updates", "t");
+  }
 
   @Bean
   public Properties zkProperties() {
@@ -214,44 +202,13 @@ public class TestConfig {
     return AdminClient.create(adminConfig);
   }
 
-  @Bean(destroyMethod = "close")
-  public UserSettingsClient userSettingsClient() {
-    Map<String, Object> globals = new HashMap<String, Object>() {{
-      put(HBaseUserSettingsClient.USER_SETTINGS_HBASE_TABLE, "user_settings");
-      put(HBaseUserSettingsClient.USER_SETTINGS_HBASE_CF, "cf");
-    }};
-
-    UserSettingsClient userSettingsClient = new HBaseUserSettingsClient(
-            () -> globals,
-            hBaseClientFactory,
-            hBaseConnectionFactory,
-            hBaseConfiguration);
-    userSettingsClient.init();
-    return userSettingsClient;
-  }
-
-  @Bean
-  public HBaseConnectionFactory hBaseConnectionFactory() {
-    return new FakeHBaseConnectionFactory();
-  }
-
-  @Bean
-  org.apache.hadoop.conf.Configuration hBaseConfiguration() {
-    return HBaseConfiguration.create();
-  }
-
-  @Bean
-  HBaseClientFactory hBaseClientFactory() {
-    return new FakeHBaseClientFactory();
-  }
-
-  @Bean(destroyMethod = "close")
-  public HBaseClient hBaseClient() {
-    return new FakeHBaseClient();
+  @Bean()
+  public UserSettingsClient userSettingsClient() throws RestException, IOException {
+    return new UserSettingsClient(new MockHBaseTableProvider().addToCache("user_settings", "cf"), Bytes.toBytes("cf"));
   }
 
   @Bean()
-  public LegacyHBaseClient legacyHBaseClient() throws RestException, IOException {
+  public HBaseClient hBaseClient() throws RestException, IOException {
     final String cf = "t";
     final String cq = "v";
     HTableInterface table = MockHBaseTableProvider.addToCache("enrichment_list", cf);
@@ -265,7 +222,8 @@ public class TestConfig {
       put.addColumn(Bytes.toBytes(cf), Bytes.toBytes(cq), "{}".getBytes(StandardCharsets.UTF_8));
       table.put(put);
     }
-    return new LegacyHBaseClient(new MockHBaseTableProvider(), HBaseConfiguration.create(), "enrichment_list");
+    return new HBaseClient(new MockHBaseTableProvider(), HBaseConfiguration.create(),
+        "enrichment_list");
   }
 
   @Bean
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/AlertsUIControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/AlertsUIControllerIntegrationTest.java
index 40363f9..49863d6 100644
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/AlertsUIControllerIntegrationTest.java
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/AlertsUIControllerIntegrationTest.java
@@ -30,7 +30,6 @@ import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.
 import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status;
 
 import org.adrianwalker.multilinestring.Multiline;
-import org.apache.metron.hbase.client.FakeHBaseClient;
 import org.apache.metron.integration.ComponentRunner;
 import org.apache.metron.integration.UnableToStartException;
 import org.apache.metron.integration.components.KafkaComponent;
@@ -142,9 +141,6 @@ public class AlertsUIControllerIntegrationTest {
       alertsUIService.deleteAlertsUIUserSettings(user);
     }
     this.mockMvc = MockMvcBuilders.webAppContextSetup(this.wac).apply(springSecurity()).build();
-
-    // ensure the fake hbase client is reset before each test
-    new FakeHBaseClient().deleteAll();
   }
 
   @Test
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java
index dcc3a0d..bd3f5bd 100644
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java
@@ -30,7 +30,8 @@ import java.util.List;
 import java.util.Map;
 
 public class DaoControllerTest {
-
+  public static final String TABLE = "updates";
+  public static final String CF = "t";
   public void loadTestData(Map<String, String> indicesToDataMap) throws ParseException {
     Map<String, List<String>> backingStore = new HashMap<>();
     for(Map.Entry<String, String> indices : indicesToDataMap.entrySet())
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SensorEnrichmentConfigControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SensorEnrichmentConfigControllerIntegrationTest.java
index 690ed0d..d77e3ae 100644
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SensorEnrichmentConfigControllerIntegrationTest.java
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SensorEnrichmentConfigControllerIntegrationTest.java
@@ -17,10 +17,19 @@
  */
 package org.apache.metron.rest.controller;
 
+import static org.apache.metron.integration.utils.TestUtils.assertEventually;
+import static org.apache.metron.rest.MetronRestConstants.TEST_PROFILE;
+import static org.springframework.security.test.web.servlet.request.SecurityMockMvcRequestPostProcessors.csrf;
+import static org.springframework.security.test.web.servlet.request.SecurityMockMvcRequestPostProcessors.httpBasic;
+import static org.springframework.security.test.web.servlet.setup.SecurityMockMvcConfigurers.springSecurity;
+import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.delete;
+import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get;
+import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post;
+import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content;
+import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.jsonPath;
+import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status;
+
 import org.adrianwalker.multilinestring.Multiline;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.hbase.ColumnList;
-import org.apache.metron.hbase.client.FakeHBaseClient;
 import org.apache.metron.rest.service.SensorEnrichmentConfigService;
 import org.hamcrest.core.IsCollectionContaining;
 import org.junit.Before;
@@ -32,86 +41,71 @@ import org.springframework.http.MediaType;
 import org.springframework.test.context.ActiveProfiles;
 import org.springframework.test.context.junit4.SpringRunner;
 import org.springframework.test.web.servlet.MockMvc;
-import org.springframework.test.web.servlet.MvcResult;
-import org.springframework.test.web.servlet.request.MockHttpServletRequestBuilder;
 import org.springframework.test.web.servlet.setup.MockMvcBuilders;
 import org.springframework.web.context.WebApplicationContext;
 
-import static org.apache.metron.integration.utils.TestUtils.assertEventually;
-import static org.apache.metron.rest.MetronRestConstants.TEST_PROFILE;
-import static org.springframework.security.test.web.servlet.request.SecurityMockMvcRequestPostProcessors.csrf;
-import static org.springframework.security.test.web.servlet.request.SecurityMockMvcRequestPostProcessors.httpBasic;
-import static org.springframework.security.test.web.servlet.setup.SecurityMockMvcConfigurers.springSecurity;
-import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.delete;
-import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get;
-import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post;
-import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content;
-import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.jsonPath;
-import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status;
-
 @RunWith(SpringRunner.class)
 @SpringBootTest(webEnvironment= SpringBootTest.WebEnvironment.RANDOM_PORT)
 @ActiveProfiles(TEST_PROFILE)
 public class SensorEnrichmentConfigControllerIntegrationTest {
 
   /**
-   * {
-   *    "enrichment":{
-   *       "fieldMap":{
-   *          "geo":[
-   *             "ip_dst_addr"
-   *          ],
-   *          "host":[
-   *             "ip_dst_addr"
-   *          ],
-   *          "hbaseEnrichment":[
-   *             "ip_src_addr"
-   *          ],
-   *          "stellar":{
-   *             "config":{
-   *                "group1":{
-   *                   "foo":"1 + 1",
-   *                   "bar":"foo"
-   *                },
-   *                "group2":{
-   *                   "ALL_CAPS":"TO_UPPER(source.type)"
-   *                }
-   *             }
-   *          }
-   *       },
-   *       "fieldToTypeMap":{
-   *          "ip_src_addr":[
-   *             "sample"
-   *          ]
-   *       }
-   *    },
-   *    "threatIntel":{
-   *       "fieldMap":{
-   *          "hbaseThreatIntel":[
-   *             "ip_src_addr",
-   *             "ip_dst_addr"
-   *          ]
-   *       },
-   *       "fieldToTypeMap":{
-   *          "ip_src_addr":[
-   *             "malicious_ip"
-   *          ],
-   *          "ip_dst_addr":[
-   *             "malicious_ip"
-   *          ]
-   *       },
-   *       "triageConfig":{
-   *          "riskLevelRules":[
-   *             {
-   *                "rule":"ip_src_addr == '10.122.196.204' or ip_dst_addr == '10.122.196.204'",
-   *                "score":10
-   *             }
-   *          ],
-   *          "aggregator":"MAX"
-   *       }
-   *    }
-   * }
-   *
+   {
+   "enrichment": {
+   "fieldMap": {
+   "geo": [
+   "ip_dst_addr"
+   ],
+   "host": [
+   "ip_dst_addr"
+   ],
+   "hbaseEnrichment": [
+   "ip_src_addr"
+   ],
+   "stellar": {
+   "config": {
+   "group1": {
+   "foo": "1 + 1",
+   "bar": "foo"
+   },
+   "group2": {
+   "ALL_CAPS": "TO_UPPER(source.type)"
+   }
+   }
+   }
+   },
+   "fieldToTypeMap": {
+   "ip_src_addr": [
+   "sample"
+   ]
+   }
+   },
+   "threatIntel": {
+   "fieldMap": {
+   "hbaseThreatIntel": [
+   "ip_src_addr",
+   "ip_dst_addr"
+   ]
+   },
+   "fieldToTypeMap": {
+   "ip_src_addr": [
+   "malicious_ip"
+   ],
+   "ip_dst_addr": [
+   "malicious_ip"
+   ]
+   },
+   "triageConfig": {
+   "riskLevelRules": [
+   {
+   "rule": "ip_src_addr == '10.122.196.204' or ip_dst_addr == '10.122.196.204'",
+   "score": 10
+   }
+   ],
+   "aggregator": "MAX"
+   }
+   }
+   }
    */
   @Multiline
   public static String broJson;
@@ -123,6 +117,7 @@ public class SensorEnrichmentConfigControllerIntegrationTest {
   private WebApplicationContext wac;
 
   private MockMvc mockMvc;
+
   private String sensorEnrichmentConfigUrl = "/api/v1/sensor/enrichment/config";
   private String user = "user";
   private String password = "password";
@@ -130,9 +125,6 @@ public class SensorEnrichmentConfigControllerIntegrationTest {
   @Before
   public void setup() throws Exception {
     this.mockMvc = MockMvcBuilders.webAppContextSetup(this.wac).apply(springSecurity()).build();
-
-    // clear any fake hbase data between tests
-    new FakeHBaseClient().deleteAll();
   }
 
   @Test
@@ -250,6 +242,12 @@ public class SensorEnrichmentConfigControllerIntegrationTest {
             .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8")))
             .andExpect(jsonPath("$[?(@.sensorTopic == 'broTest')]").doesNotExist());
 
+    this.mockMvc.perform(get(sensorEnrichmentConfigUrl + "/list/available/enrichments").with(httpBasic(user,password)))
+            .andExpect(status().isOk())
+            .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8")))
+            .andExpect(jsonPath("$.length()").value("3"))
+            .andExpect(jsonPath("$.*").value(IsCollectionContaining.hasItems("foo", "bar", "baz")));
+
     this.mockMvc.perform(get(sensorEnrichmentConfigUrl + "/list/available/threat/triage/aggregators").with(httpBasic(user,password)))
             .andExpect(status().isOk())
             .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8")))
@@ -257,7 +255,8 @@ public class SensorEnrichmentConfigControllerIntegrationTest {
             .andExpect(jsonPath("$[1]").value("MIN"))
             .andExpect(jsonPath("$[2]").value("SUM"))
             .andExpect(jsonPath("$[3]").value("MEAN"))
-            .andExpect(jsonPath("$[4]").value("POSITIVE_MEAN"));
+            .andExpect(jsonPath("$[4]").value("POSITIVE_MEAN"))
+    ;
 
     sensorEnrichmentConfigService.delete("broTest");
   }
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java
index 47fd9d9..7995557 100644
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java
@@ -22,6 +22,8 @@ import org.adrianwalker.multilinestring.Multiline;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.metron.hbase.mock.MockHBaseTableProvider;
+import org.apache.metron.hbase.mock.MockHTable;
 import org.apache.metron.indexing.dao.HBaseDao;
 import org.apache.metron.indexing.dao.SearchIntegrationTest;
 import org.apache.metron.indexing.dao.update.CommentAddRemoveRequest;
@@ -160,6 +162,10 @@ public class UpdateControllerIntegrationTest extends DaoControllerTest {
             .andExpect(jsonPath("$.project").doesNotExist())
             .andExpect(jsonPath("$.timestamp").value(2));
 
+    // nothing is recorded in HBase
+    MockHTable table = (MockHTable) MockHBaseTableProvider.getFromCache(TABLE);
+    Assert.assertEquals(0,table.size());
+
     // patch the document
     this.mockMvc.perform(patchRequest)
             .andExpect(status().isOk());
@@ -172,6 +178,16 @@ public class UpdateControllerIntegrationTest extends DaoControllerTest {
             .andExpect(jsonPath("$.guid").value(guid))
             .andExpect(jsonPath("$.project").value("metron"))
             .andExpect(jsonPath("$.timestamp").value(2));
+
+    // the change should be recorded in HBase
+    Assert.assertEquals(1,table.size());
+    {
+        //ensure hbase is up to date
+        Get g = new Get(new HBaseDao.Key(guid,"bro").toBytes());
+        Result r = table.get(g);
+        NavigableMap<byte[], byte[]> columns = r.getFamilyMap(CF.getBytes());
+        Assert.assertEquals(1, columns.size());
+    }
   }
 
   @Test
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/AlertsUIServiceImplTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/AlertsUIServiceImplTest.java
index ffaae1e..016a08b 100644
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/AlertsUIServiceImplTest.java
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/AlertsUIServiceImplTest.java
@@ -17,17 +17,30 @@
  */
 package org.apache.metron.rest.service.impl;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.mock;
+
 import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.hadoop.hbase.HBaseConfiguration;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.common.system.FakeClock;
-import org.apache.metron.hbase.client.FakeHBaseClientFactory;
-import org.apache.metron.hbase.client.FakeHBaseConnectionFactory;
 import org.apache.metron.rest.MetronRestConstants;
 import org.apache.metron.rest.model.AlertsUIUserSettings;
 import org.apache.metron.rest.service.KafkaService;
-import org.apache.metron.rest.user.HBaseUserSettingsClient;
 import org.apache.metron.rest.user.UserSettingsClient;
 import org.junit.Before;
 import org.junit.Test;
@@ -37,34 +50,27 @@ import org.springframework.security.core.Authentication;
 import org.springframework.security.core.context.SecurityContextHolder;
 import org.springframework.security.core.userdetails.UserDetails;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-
-import static org.apache.metron.rest.service.impl.AlertsUIServiceImpl.ALERT_USER_SETTING_TYPE;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyZeroInteractions;
-import static org.mockito.Mockito.when;
-import static org.powermock.api.mockito.PowerMockito.mock;
-
-/**
- * Tests the {@link AlertsUIServiceImpl} class.
- */
 @SuppressWarnings("unchecked")
 public class AlertsUIServiceImplTest {
 
   public static ThreadLocal<ObjectMapper> _mapper = ThreadLocal.withInitial(() ->
           new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL));
 
-  private static final String field = "field";
-  private static final String value1 = "value1";
-  private static final String value2 = "value2";
-  private static final String escalationTopic = "escalation";
+  /**
+   * {
+   *   "tableColumns": ["user1_field"]
+   * }
+   */
+  @Multiline
+  public static String user1AlertUserSettings;
+
+  /**
+   * {
+   *   "tableColumns": ["user2_field"]
+   * }
+   */
+  @Multiline
+  public static String user2AlertUserSettings;
 
   private KafkaService kafkaService;
   private Environment environment;
@@ -79,19 +85,7 @@ public class AlertsUIServiceImplTest {
   public void setUp() throws Exception {
     kafkaService = mock(KafkaService.class);
     environment = mock(Environment.class);
-
-    Map<String, Object> globals = new HashMap<String, Object>() {{
-      put(HBaseUserSettingsClient.USER_SETTINGS_HBASE_TABLE, "some_table");
-      put(HBaseUserSettingsClient.USER_SETTINGS_HBASE_CF, "column_family");
-      put(HBaseUserSettingsClient.USER_SETTINGS_MAX_SCAN, "100000");
-    }};
-
-    userSettingsClient = new HBaseUserSettingsClient(
-            () -> globals,
-            new FakeHBaseClientFactory(),
-            new FakeHBaseConnectionFactory(),
-            HBaseConfiguration.create());
-    userSettingsClient.init();
+    userSettingsClient = mock(UserSettingsClient.class);
     alertsUIService = new AlertsUIServiceImpl(kafkaService, environment, userSettingsClient);
 
     // use a fake clock for testing
@@ -109,7 +103,12 @@ public class AlertsUIServiceImplTest {
 
   @Test
   public void escalateAlertShouldSendMessageToKafka() throws Exception {
+    final String field = "field";
+    final String value1 = "value1";
+    final String value2 = "value2";
+
     // define the escalation topic
+    final String escalationTopic = "escalation";
     when(environment.getProperty(MetronRestConstants.KAFKA_TOPICS_ESCALATION_PROPERTY)).thenReturn(escalationTopic);
 
     // create an alert along with the expected escalation message that is sent to kafka
@@ -127,63 +126,63 @@ public class AlertsUIServiceImplTest {
   }
 
   @Test
-  public void shouldGetActiveProfile() throws Exception {
-    AlertsUIUserSettings expected = new AlertsUIUserSettings();
-    expected.setTableColumns(Collections.singletonList("user1_field"));
-
-    // save a profile for current user
-    userSettingsClient.save(user1, ALERT_USER_SETTING_TYPE, toJSON(expected));
-
-    // retrieve the active profile
-    assertEquals(expected, alertsUIService.getAlertsUIUserSettings().get());
+  public void getShouldProperlyReturnActiveProfile() throws Exception {
+    when(userSettingsClient.findOne(user1, AlertsUIServiceImpl.ALERT_USER_SETTING_TYPE)).thenReturn(Optional.of(user1AlertUserSettings));
+
+    AlertsUIUserSettings expectedAlertsUIUserSettings = new AlertsUIUserSettings();
+    expectedAlertsUIUserSettings.setTableColumns(Collections.singletonList("user1_field"));
+    assertEquals(expectedAlertsUIUserSettings, alertsUIService.getAlertsUIUserSettings().get());
+    verify(userSettingsClient, times(1)).findOne(user1, AlertsUIServiceImpl.ALERT_USER_SETTING_TYPE);
+    verifyNoMoreInteractions(userSettingsClient);
   }
 
   @Test
-  public void shouldFindAllActiveProfiles() throws Exception {
-    AlertsUIUserSettings settings1 = new AlertsUIUserSettings();
-    settings1.setTableColumns(Collections.singletonList("user1_field"));
-
-    AlertsUIUserSettings settings2 = new AlertsUIUserSettings();
-    settings2.setTableColumns(Collections.singletonList("user2_field"));
-
-    // save some profiles
-    userSettingsClient.save(user1, ALERT_USER_SETTING_TYPE, toJSON(settings1));
-    userSettingsClient.save(user2, ALERT_USER_SETTING_TYPE, toJSON(settings2));
-
-    // retrieve all active profiles
+  public void findAllShouldProperlyReturnActiveProfiles() throws Exception {
+    AlertsUIUserSettings alertsProfile1 = new AlertsUIUserSettings();
+    alertsProfile1.setUser(user1);
+    AlertsUIUserSettings alertsProfile2 = new AlertsUIUserSettings();
+    alertsProfile2.setUser(user1);
+    when(userSettingsClient.findAll(AlertsUIServiceImpl.ALERT_USER_SETTING_TYPE))
+            .thenReturn(new HashMap<String, Optional<String>>() {{
+              put(user1,  Optional.of(user1AlertUserSettings));
+              put(user2, Optional.of(user2AlertUserSettings));
+              }});
+
+    AlertsUIUserSettings expectedAlertsUIUserSettings1 = new AlertsUIUserSettings();
+    expectedAlertsUIUserSettings1.setTableColumns(Collections.singletonList("user1_field"));
+    AlertsUIUserSettings expectedAlertsUIUserSettings2 = new AlertsUIUserSettings();
+    expectedAlertsUIUserSettings2.setTableColumns(Collections.singletonList("user2_field"));
     Map<String, AlertsUIUserSettings> actualAlertsProfiles = alertsUIService.findAllAlertsUIUserSettings();
     assertEquals(2, actualAlertsProfiles.size());
-    assertEquals(settings1, actualAlertsProfiles.get(user1));
-    assertEquals(settings2, actualAlertsProfiles.get(user2));
+    assertEquals(expectedAlertsUIUserSettings1, actualAlertsProfiles.get(user1));
+    assertEquals(expectedAlertsUIUserSettings2, actualAlertsProfiles.get(user2));
+
+    verify(userSettingsClient, times(1)).findAll(AlertsUIServiceImpl.ALERT_USER_SETTING_TYPE);
+    verifyNoMoreInteractions(userSettingsClient);
   }
 
   @Test
-  public void shouldSaveActiveProfile() throws Exception {
-    AlertsUIUserSettings expected = new AlertsUIUserSettings();
-    expected.setTableColumns(Collections.singletonList("user1_field"));
+  public void saveShouldProperlySaveActiveProfile() throws Exception {
+    AlertsUIUserSettings alertsUIUserSettings = new AlertsUIUserSettings();
+    alertsUIUserSettings.setTableColumns(Collections.singletonList("user1_field"));
 
-    // save an active profile
-    alertsUIService.saveAlertsUIUserSettings(expected);
+    alertsUIService.saveAlertsUIUserSettings(alertsUIUserSettings);
 
-    // get the active profile
-    Optional<AlertsUIUserSettings> actual = alertsUIService.getAlertsUIUserSettings();
-    assertEquals(expected, actual.get());
+    String expectedAlertUserSettings = _mapper.get().writeValueAsString(alertsUIUserSettings);
+    verify(userSettingsClient, times(1))
+            .save(user1, AlertsUIServiceImpl.ALERT_USER_SETTING_TYPE, expectedAlertUserSettings);
+    verifyNoMoreInteractions(userSettingsClient);
   }
 
-
   @Test
-  public void shouldDeleteActiveProfile() throws Exception {
-    AlertsUIUserSettings expected = new AlertsUIUserSettings();
-    expected.setTableColumns(Collections.singletonList("user1_field"));
-
-    userSettingsClient.save(user1, ALERT_USER_SETTING_TYPE, toJSON(expected));
+  public void deleteShouldProperlyDeleteActiveProfile() throws Exception {
     assertTrue(alertsUIService.deleteAlertsUIUserSettings(user1));
-  }
 
-  @Test
-  public void shouldNotDeleteMissingProfile() throws Exception {
-    // no profile saved for 'user999'
-    assertFalse(alertsUIService.deleteAlertsUIUserSettings("user999"));
+    doThrow(new IOException()).when(userSettingsClient).delete(user1, AlertsUIServiceImpl.ALERT_USER_SETTING_TYPE);
+    assertFalse(alertsUIService.deleteAlertsUIUserSettings(user1));
+
+    verify(userSettingsClient, times(2)).delete(user1, AlertsUIServiceImpl.ALERT_USER_SETTING_TYPE);
+    verifyNoMoreInteractions(userSettingsClient);
   }
 
   /**
@@ -210,8 +209,4 @@ public class AlertsUIServiceImplTest {
     map.put(key, value);
     return map;
   }
-
-  private String toJSON(AlertsUIUserSettings settings) throws JsonProcessingException {
-    return _mapper.get().writeValueAsString(settings);
-  }
 }
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/SensorEnrichmentConfigServiceImplTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/SensorEnrichmentConfigServiceImplTest.java
index 4176121..3d07da5 100644
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/SensorEnrichmentConfigServiceImplTest.java
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/service/impl/SensorEnrichmentConfigServiceImplTest.java
@@ -42,7 +42,7 @@ import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
 import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
 import org.apache.metron.common.configuration.enrichment.threatintel.ThreatIntelConfig;
 import org.apache.metron.common.zookeeper.ConfigurationsCache;
-import org.apache.metron.hbase.client.LegacyHBaseClient;
+import org.apache.metron.hbase.client.HBaseClient;
 import org.apache.metron.rest.RestException;
 import org.apache.metron.rest.service.SensorEnrichmentConfigService;
 import org.apache.zookeeper.KeeperException;
@@ -82,14 +82,14 @@ public class SensorEnrichmentConfigServiceImplTest {
   public static String broJson;
 
   ConfigurationsCache cache;
-  private LegacyHBaseClient hBaseClient;
+  private HBaseClient hBaseClient;
 
   @Before
   public void setUp() throws Exception {
     objectMapper = mock(ObjectMapper.class);
     curatorFramework = mock(CuratorFramework.class);
     cache = mock(ConfigurationsCache.class);
-    hBaseClient = mock(LegacyHBaseClient.class);
+    hBaseClient = mock(HBaseClient.class);
     sensorEnrichmentConfigService = new SensorEnrichmentConfigServiceImpl(objectMapper, curatorFramework, cache, hBaseClient);
   }
 
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/user/HBaseUserSettingsClientTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/user/HBaseUserSettingsClientTest.java
deleted file mode 100644
index c4b3ada..0000000
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/user/HBaseUserSettingsClientTest.java
+++ /dev/null
@@ -1,194 +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.rest.user;
-
-
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.hbase.client.FakeHBaseClient;
-import org.apache.metron.hbase.client.FakeHBaseClientFactory;
-import org.apache.metron.hbase.client.FakeHBaseConnectionFactory;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-
-import static org.apache.metron.rest.user.HBaseUserSettingsClient.USER_SETTINGS_HBASE_CF;
-import static org.apache.metron.rest.user.HBaseUserSettingsClient.USER_SETTINGS_HBASE_TABLE;
-import static org.apache.metron.rest.user.HBaseUserSettingsClient.USER_SETTINGS_MAX_SCAN;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-public class HBaseUserSettingsClientTest {
-
-  @Rule
-  public final ExpectedException exception = ExpectedException.none();
-  private static final String tableName = "some_table";
-  private static final String columnFamily = "cf";
-  private static byte[] cf = Bytes.toBytes(columnFamily);
-
-  private Map<String, Object> globals;
-  private HBaseUserSettingsClient userSettingsClient;
-  private FakeHBaseClient hBaseClient;
-
-  @Before
-  public void setUp() throws Exception {
-    globals = new HashMap<String, Object>() {{
-      put(USER_SETTINGS_HBASE_TABLE, tableName);
-      put(USER_SETTINGS_HBASE_CF, columnFamily);
-      put(USER_SETTINGS_MAX_SCAN, "100000");
-    }};
-
-    hBaseClient = new FakeHBaseClient();
-    hBaseClient.deleteAll();
-
-    userSettingsClient = new HBaseUserSettingsClient(
-            () -> globals,
-            new FakeHBaseClientFactory(hBaseClient),
-            new FakeHBaseConnectionFactory(),
-            HBaseConfiguration.create());
-    userSettingsClient.init();
-
-    // timezone settings
-    userSettingsClient.save("user1", "timezone", "EST");
-    userSettingsClient.save("user2", "timezone", "MST");
-    userSettingsClient.save("user3", "timezone", "PST");
-
-    // language settings
-    userSettingsClient.save("user1", "language", "EN");
-    userSettingsClient.save("user2", "language", "ES");
-    userSettingsClient.save("user3", "language", "FR");
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    userSettingsClient.close();
-  }
-
-  @Test
-  public void shouldFindSetting() throws Exception {
-    assertEquals("EST", userSettingsClient.findOne("user1", "timezone").get());
-    assertEquals("MST", userSettingsClient.findOne("user2", "timezone").get());
-    assertEquals("PST", userSettingsClient.findOne("user3", "timezone").get());
-
-    assertEquals("EN", userSettingsClient.findOne("user1", "language").get());
-    assertEquals("ES", userSettingsClient.findOne("user2", "language").get());
-    assertEquals("FR", userSettingsClient.findOne("user3", "language").get());
-  }
-
-  @Test
-  public void shouldNotFindSetting() throws Exception {
-    assertFalse(userSettingsClient.findOne("user999", "timezone").isPresent());
-    assertFalse(userSettingsClient.findOne("user999", "language").isPresent());
-  }
-
-  @Test
-  public void shouldFindSettingsForUser() throws Exception {
-    Map<String, String> settings = userSettingsClient.findOne("user1");
-    assertEquals(2, settings.size());
-    assertEquals("EN", settings.get("language"));
-    assertEquals("EST", settings.get("timezone"));
-  }
-
-  @Test
-  public void shouldNotFindSettingsForUser() throws Exception {
-    Map<String, String> settings = userSettingsClient.findOne("user999");
-    assertEquals(0, settings.size());
-  }
-
-  @Test
-  public void shouldFindAllSettingsByType() throws Exception {
-    Map<String, Optional<String>> settings = userSettingsClient.findAll("timezone");
-
-    // there should be a 'timezone' setting defined for each user
-    int numberOfUsers = 3;
-    assertEquals(numberOfUsers, settings.size());
-    assertEquals("EST", settings.get("user1").get());
-    assertEquals("MST", settings.get("user2").get());
-    assertEquals("PST", settings.get("user3").get());
-  }
-
-  @Test
-  public void shouldFindAllSettings() throws Exception {
-    Map<String, Map<String, String>> settings = userSettingsClient.findAll();
-
-    // there should be a set of settings defined for each user
-    int numberOfUsers = 3;
-    assertEquals(numberOfUsers, settings.size());
-
-    assertEquals("EST", settings.get("user1").get("timezone"));
-    assertEquals("MST", settings.get("user2").get("timezone"));
-    assertEquals("PST", settings.get("user3").get("timezone"));
-    assertEquals("EN", settings.get("user1").get("language"));
-    assertEquals("ES", settings.get("user2").get("language"));
-    assertEquals("FR", settings.get("user3").get("language"));
-  }
-
-  @Test
-  public void shouldDeleteSetting() throws Exception {
-    // before deleting the "timezone" setting, they should all exist
-    assertEquals("EST", userSettingsClient.findOne("user1", "timezone").get());
-    assertEquals("EN", userSettingsClient.findOne("user1", "language").get());
-
-    // delete the "timezone" setting for "user1"
-    userSettingsClient.delete("user1", "timezone");
-
-    assertFalse(userSettingsClient.findOne("user1", "timezone").isPresent());
-    assertEquals("EN", userSettingsClient.findOne("user1", "language").get());
-  }
-
-  @Test
-  public void shouldNotDeleteSetting() throws Exception {
-    // delete the "timezone" setting for a user that does not exist
-    userSettingsClient.delete("user999", "timezone");
-
-    // the settings should still not exist and nothing should blow-up
-    assertFalse(userSettingsClient.findOne("user999", "timezone").isPresent());
-    assertFalse(userSettingsClient.findOne("user999", "language").isPresent());
-  }
-
-  @Test
-  public void shouldDeleteSettingsByUser() throws Exception {
-    // before deleting user1, all the settings should exist
-    assertEquals("EST", userSettingsClient.findOne("user1", "timezone").get());
-    assertEquals("EN", userSettingsClient.findOne("user1", "language").get());
-
-    // delete the user
-    userSettingsClient.delete("user1");
-
-    // none of user1's settings should exist any longer
-    assertFalse(userSettingsClient.findOne("user1", "timezone").isPresent());
-    assertFalse(userSettingsClient.findOne("user1", "language").isPresent());
-  }
-
-  @Test
-  public void shouldNotDeleteSettingsByUser() throws Exception {
-    // delete the settings for a user that does not exist
-    userSettingsClient.delete("user999");
-
-    // the settings should still not exist and nothing should blow-up
-    Map<String, String> settings = userSettingsClient.findOne("user999");
-    assertEquals(0, settings.size());
-  }
-}
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/user/UserSettingsClientTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/user/UserSettingsClientTest.java
new file mode 100644
index 0000000..5b4f786
--- /dev/null
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/user/UserSettingsClientTest.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.rest.user;
+
+
+import static org.apache.metron.rest.user.UserSettingsClient.USER_SETTINGS_HBASE_CF;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.Supplier;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class UserSettingsClientTest {
+
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  private static ThreadLocal<ObjectMapper> _mapper = ThreadLocal.withInitial(() ->
+          new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL));
+
+  private HTableInterface userSettingsTable;
+  private Supplier<Map<String, Object>> globalConfigSupplier;
+  private UserSettingsClient userSettingsClient;
+  private static byte[] cf = Bytes.toBytes("cf");
+
+  @Before
+  public void setUp() throws Exception {
+    userSettingsTable = mock(HTableInterface.class);
+    globalConfigSupplier = () -> new HashMap<String, Object>() {{
+      put(USER_SETTINGS_HBASE_CF, "cf");
+    }};
+  }
+
+  @Test
+  public void shouldFindOne() throws Exception {
+    Result result = mock(Result.class);
+    when(result.getValue(cf, Bytes.toBytes("type"))).thenReturn("userSettings1String".getBytes());
+    Get get = new Get("user1".getBytes());
+    get.addFamily(cf);
+    when(userSettingsTable.get(get)).thenReturn(result);
+
+    UserSettingsClient userSettingsClient = new UserSettingsClient(userSettingsTable, cf);
+    assertEquals("userSettings1String", userSettingsClient.findOne("user1", "type").get());
+    assertFalse(userSettingsClient.findOne("missingUser", "type").isPresent());
+  }
+
+  @Test
+  public void shouldFindAll() throws Exception {
+    ResultScanner resultScanner = mock(ResultScanner.class);
+    Result result1 = mock(Result.class);
+    Result result2 = mock(Result.class);
+    when(result1.getRow()).thenReturn("user1".getBytes());
+    when(result2.getRow()).thenReturn("user2".getBytes());
+    when(result1.getValue(cf, Bytes.toBytes("type"))).thenReturn("userSettings1String".getBytes());
+    when(result2.getValue(cf, Bytes.toBytes("type"))).thenReturn("userSettings2String".getBytes());
+    when(resultScanner.iterator()).thenReturn(Arrays.asList(result1, result2).iterator());
+    when(userSettingsTable.getScanner(any(Scan.class))).thenReturn(resultScanner);
+
+    UserSettingsClient userSettingsClient = new UserSettingsClient(userSettingsTable, cf);
+    assertEquals(new HashMap<String, Optional<String>>() {{
+      put("user1", Optional.of("userSettings1String"));
+      put("user2", Optional.of("userSettings2String"));
+    }}, userSettingsClient.findAll("type"));
+  }
+
+}
diff --git a/metron-platform/elasticsearch-shaded/pom.xml b/metron-platform/elasticsearch-shaded/pom.xml
index a1ebe50..811c699 100644
--- a/metron-platform/elasticsearch-shaded/pom.xml
+++ b/metron-platform/elasticsearch-shaded/pom.xml
@@ -28,6 +28,16 @@
     </properties>
     <dependencies>
         <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpcore</artifactId>
+            <version>4.4.5</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+            <version>4.5.2</version>
+        </dependency>
+        <dependency>
             <groupId>com.google.guava</groupId>
             <artifactId>guava</artifactId>
             <version>${guava_version}</version>
diff --git a/metron-platform/metron-elasticsearch/metron-elasticsearch-common/pom.xml b/metron-platform/metron-elasticsearch/metron-elasticsearch-common/pom.xml
index ac4e0a2..e915d99 100644
--- a/metron-platform/metron-elasticsearch/metron-elasticsearch-common/pom.xml
+++ b/metron-platform/metron-elasticsearch/metron-elasticsearch-common/pom.xml
@@ -59,23 +59,9 @@
                     <groupId>log4j</groupId>
                     <artifactId>log4j</artifactId>
                 </exclusion>
-                <exclusion>
-                    <artifactId>commons-httpclient</artifactId>
-                    <groupId>commons-httpclient</groupId>
-                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
-            <groupId>org.apache.httpcomponents</groupId>
-            <artifactId>httpclient</artifactId>
-            <version>4.4.1</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.httpcomponents</groupId>
-            <artifactId>httpcore</artifactId>
-            <version>4.4.9</version>
-        </dependency>
-        <dependency>
             <groupId>com.google.guava</groupId>
             <artifactId>guava</artifactId>
             <version>${global_guava_version}</version>
diff --git a/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java b/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java
index 9cd3867..7226c30 100644
--- a/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java
+++ b/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java
@@ -119,11 +119,6 @@ public class ElasticsearchDao implements IndexDao {
   }
 
   @Override
-  public void close() throws IOException {
-    // nothing to do
-  }
-
-  @Override
   public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException {
     return this.searchDao.search(searchRequest);
   }
diff --git a/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java b/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java
index aa064d3..ac5417e 100644
--- a/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java
+++ b/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java
@@ -161,11 +161,6 @@ public class ElasticsearchMetaAlertDao implements MetaAlertDao {
   }
 
   @Override
-  public void close() throws IOException {
-    // nothing to do
-  }
-
-  @Override
   public Map<String, FieldType> getColumnMetadata(List<String> indices) throws IOException {
     return indexDao.getColumnMetadata(indices);
   }
diff --git a/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDaoTest.java b/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDaoTest.java
index f1cd265..cabb992 100644
--- a/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDaoTest.java
+++ b/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDaoTest.java
@@ -68,10 +68,6 @@ public class ElasticsearchMetaAlertDaoTest {
       }
 
       @Override
-      public void close() throws IOException {
-      }
-
-      @Override
       public Document getLatest(String guid, String sensorType) {
         return null;
       }
diff --git a/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java b/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java
index 97afd41..16760d2 100644
--- a/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java
+++ b/metron-platform/metron-elasticsearch/metron-elasticsearch-common/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java
@@ -160,6 +160,11 @@ public class ElasticsearchUpdateIntegrationTest extends UpdateIntegrationTest {
     return es.getAllIndexedDocs(index, SENSOR_NAME + "_doc");
   }
 
+  @Override
+  protected MockHTable getMockHTable() {
+    return table;
+  }
+
   /**
    * Install the index template to ensure that "guid" is of type "keyword". The
    * {@link org.apache.metron.elasticsearch.dao.ElasticsearchRetrieveLatestDao} cannot find
diff --git a/metron-platform/metron-hbase-server/src/main/java/org/apache/metron/hbase/coprocessor/HBaseCacheWriter.java b/metron-platform/metron-hbase-server/src/main/java/org/apache/metron/hbase/coprocessor/HBaseCacheWriter.java
index 5d19960..b1bbdde 100644
--- a/metron-platform/metron-hbase-server/src/main/java/org/apache/metron/hbase/coprocessor/HBaseCacheWriter.java
+++ b/metron-platform/metron-hbase-server/src/main/java/org/apache/metron/hbase/coprocessor/HBaseCacheWriter.java
@@ -26,7 +26,7 @@ import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.metron.hbase.TableProvider;
-import org.apache.metron.hbase.client.LegacyHBaseClient;
+import org.apache.metron.hbase.client.HBaseClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -59,7 +59,7 @@ public class HBaseCacheWriter implements CacheWriter<String, String> {
   @Override
   public void write(@Nonnull String key, @Nonnull String value) {
     LOG.debug("Calling hbase cache writer with key='{}', value='{}'", key, value);
-    try (LegacyHBaseClient hbClient = new LegacyHBaseClient(this.tableProvider, this.config, this.tableName)) {
+    try (HBaseClient hbClient = new HBaseClient(this.tableProvider, this.config, this.tableName)) {
       LOG.debug("rowKey={}, columnFamily={}, columnQualifier={}, value={}", key, columnFamily,
           columnQualifier, value);
       hbClient.put(key, columnFamily, columnQualifier, value);
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/ColumnList.java b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/ColumnList.java
index 33af434..01a5cc6 100644
--- a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/ColumnList.java
+++ b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/ColumnList.java
@@ -20,12 +20,8 @@
 
 package org.apache.metron.hbase;
 
-import org.apache.hadoop.hbase.util.Bytes;
-
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
-import java.util.Objects;
 
 /**
  * Represents a list of HBase columns.
@@ -80,30 +76,6 @@ public class ColumnList {
     public long getTs() {
       return ts;
     }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) return true;
-      if (!(o instanceof Column)) return false;
-      Column column = (Column) o;
-      return ts == column.ts &&
-              Arrays.equals(value, column.value);
-    }
-
-    @Override
-    public int hashCode() {
-      int result = Objects.hash(ts);
-      result = 31 * result + Arrays.hashCode(value);
-      return result;
-    }
-
-    @Override
-    public String toString() {
-      return "Column{" +
-              "value=" + Arrays.toString(value) +
-              ", ts=" + ts +
-              '}';
-    }
   }
 
   public static class Counter extends AbstractColumn {
@@ -116,26 +88,6 @@ public class ColumnList {
     public long getIncrement() {
       return incr;
     }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) return true;
-      if (!(o instanceof Counter)) return false;
-      Counter counter = (Counter) o;
-      return incr == counter.incr;
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(incr);
-    }
-
-    @Override
-    public String toString() {
-      return "Counter{" +
-              "incr=" + incr +
-              '}';
-    }
   }
 
   private ArrayList<ColumnList.Column> columns;
@@ -171,24 +123,6 @@ public class ColumnList {
     return this;
   }
 
-  public ColumnList addColumn(byte[] family, byte[] qualifier){
-    addColumn(new Column(family, qualifier, -1, null));
-    return this;
-  }
-
-  public ColumnList addColumn(String family, String qualifier){
-    addColumn(Bytes.toBytes(family), Bytes.toBytes(qualifier));
-    return this;
-  }
-
-  public ColumnList addColumn(String family, String qualifier, byte[] value){
-    return addColumn(Bytes.toBytes(family), Bytes.toBytes(qualifier), value);
-  }
-
-  public ColumnList addColumn(String family, String qualifier, String value){
-    return addColumn(Bytes.toBytes(family), Bytes.toBytes(qualifier), Bytes.toBytes(value));
-  }
-
   /**
    * Add a standard HBase column given an instance of a class that implements
    * the <code>IColumn</code> interface.
@@ -197,11 +131,6 @@ public class ColumnList {
     return this.addColumn(column.family(), column.qualifier(), column.timestamp(), column.value());
   }
 
-  public ColumnList addColumn(Column column){
-    columns().add(column);
-    return this;
-  }
-
   /**
    * Add an HBase counter column.
    */
@@ -223,14 +152,14 @@ public class ColumnList {
    * Query to determine if we have column definitions.
    */
   public boolean hasColumns(){
-    return columns != null && columns.size() > 0;
+    return this.columns != null;
   }
 
   /**
    * Query to determine if we have counter definitions.
    */
   public boolean hasCounters(){
-    return this.counters != null && counters.size() > 0;
+    return this.counters != null;
   }
 
   /**
@@ -246,26 +175,4 @@ public class ColumnList {
   public List<Counter> getCounters(){
     return this.counters;
   }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (!(o instanceof ColumnList)) return false;
-    ColumnList that = (ColumnList) o;
-    return Objects.equals(columns, that.columns) &&
-            Objects.equals(counters, that.counters);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(columns, counters);
-  }
-
-  @Override
-  public String toString() {
-    return "ColumnList{" +
-            "columns=" + columns +
-            ", counters=" + counters +
-            '}';
-  }
 }
\ No newline at end of file
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/HBaseProjectionCriteria.java b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/HBaseProjectionCriteria.java
index 6bacfb2..7432b02 100644
--- a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/HBaseProjectionCriteria.java
+++ b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/HBaseProjectionCriteria.java
@@ -40,16 +40,12 @@ public class HBaseProjectionCriteria implements Serializable {
 
   public static class ColumnMetaData implements Serializable {
 
-    private byte[] columnFamily;
+    private byte[]  columnFamily;
     private byte[] qualifier;
 
     public ColumnMetaData(String columnFamily, String qualifier) {
-      this(columnFamily.getBytes(), qualifier.getBytes());
-    }
-
-    public ColumnMetaData(byte[] columnFamily, byte[] qualifier) {
-      this.columnFamily = columnFamily;
-      this.qualifier = qualifier;
+      this.columnFamily = columnFamily.getBytes();
+      this.qualifier = qualifier.getBytes();
     }
 
     public byte[] getColumnFamily() {
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/TableConfig.java b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/TableConfig.java
index 0033cb7..de2e929 100644
--- a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/TableConfig.java
+++ b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/TableConfig.java
@@ -32,6 +32,7 @@ public class TableConfig implements Serializable {
     private String connectorImpl;
 
     public TableConfig() {
+
     }
 
     public TableConfig(String tableName) {
@@ -84,7 +85,6 @@ public class TableConfig implements Serializable {
     public void setBatch(boolean batch) {
         this.batch = batch;
     }
-
     /**
      * @param writeBufferSize
      *          Overrides the client-side write buffer size.
@@ -107,11 +107,12 @@ public class TableConfig implements Serializable {
     public long getWriteBufferSize() {
         return writeBufferSize;
     }
-
     /**
      * @return A Set of configured column families
      */
     public Set<String> getColumnFamilies() {
         return this.columnFamilies.keySet();
     }
+
+
 }
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseClient.java b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseClient.java
index f51927a..d0d934e 100644
--- a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseClient.java
+++ b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseClient.java
@@ -17,114 +17,321 @@
  *  limitations under the License.
  *
  */
+
 package org.apache.metron.hbase.client;
 
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.metron.hbase.ColumnList;
-import org.apache.metron.hbase.HBaseProjectionCriteria;
+import static org.apache.commons.collections4.CollectionUtils.size;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
 import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.hbase.TableProvider;
+import org.apache.metron.hbase.ColumnList;
+import org.apache.metron.hbase.HBaseProjectionCriteria;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A client that interacts with HBase.
  */
-public interface HBaseClient extends Closeable {
+public class HBaseClient implements Closeable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   /**
-   * Enqueues a 'get' request that will be submitted when {@link #getAll()} is called.
-   * @param rowKey The row key to be retrieved.
+   * The batch of queued Mutations.
    */
-  void addGet(byte[] rowKey, HBaseProjectionCriteria criteria);
+  List<Mutation> mutations;
 
   /**
-   * Submits all pending get operations and returns the result of each.
-   * @return The result of each pending get request.
+   * The batch of queued Gets.
    */
-  Result[] getAll();
+  List<Get> gets;
 
   /**
-   * Clears all pending get operations.
+   * The HBase table this client interacts with.
    */
-  void clearGets();
+  private HTableInterface table;
+
+  public HBaseClient(TableProvider provider, final Configuration configuration, final String tableName) {
+    this.mutations = new ArrayList<>();
+    this.gets = new ArrayList<>();
+    try {
+      this.table = provider.getTable(configuration, tableName);
+    } catch (Exception e) {
+      String msg = String.format("Unable to open connection to HBase for table '%s'", tableName);
+      LOG.error(msg, e);
+      throw new RuntimeException(msg, e);
+    }
+  }
 
   /**
-   * Scans an entire table returning all row keys as a List of Strings.
-   *
-   * <p><b>**WARNING**:</b> Do not use this method unless you're absolutely crystal clear about the performance
-   * impact. Doing full table scans in HBase can adversely impact performance.
+   * Add a Mutation such as a Put or Increment to the batch.  The Mutation is only queued for
+   * later execution.
    *
-   * @return List of all row keys as Strings for this table.
+   * @param rowKey     The row key of the Mutation.
+   * @param cols       The columns affected by the Mutation.
+   * @param durability The durability of the mutation.
    */
-  List<String> scanRowKeys() throws IOException;
+  public void addMutation(byte[] rowKey, ColumnList cols, Durability durability) {
+
+    if (cols.hasColumns()) {
+      Put put = createPut(rowKey, cols, durability);
+      mutations.add(put);
+    }
+
+    if (cols.hasCounters()) {
+      Increment inc = createIncrement(rowKey, cols, durability);
+      mutations.add(inc);
+    }
+
+    if (mutations.isEmpty()) {
+      mutations.add(new Put(rowKey));
+    }
+  }
 
   /**
-   * Scans the table and returns each result.
+   * Adds a Mutation such as a Put or Increment with a time to live.  The Mutation is only queued
+   * for later execution.
    *
-   * <p><b>**WARNING**:</b> Do not use this method unless you're absolutely crystal clear about the performance
-   * impact. Doing full table scans in HBase can adversely impact performance.
+   * @param rowKey           The row key of the Mutation.
+   * @param cols             The columns affected by the Mutation.
+   * @param durability       The durability of the mutation.
+   * @param timeToLiveMillis The time to live in milliseconds.
+   */
+  public void addMutation(byte[] rowKey, ColumnList cols, Durability durability, Long timeToLiveMillis) {
+
+    if (cols.hasColumns()) {
+      Put put = createPut(rowKey, cols, durability, timeToLiveMillis);
+      mutations.add(put);
+    }
+
+    if (cols.hasCounters()) {
+      Increment inc = createIncrement(rowKey, cols, durability, timeToLiveMillis);
+      mutations.add(inc);
+    }
+
+    if (mutations.isEmpty()) {
+      Put put = new Put(rowKey);
+      put.setTTL(timeToLiveMillis);
+      mutations.add(put);
+    }
+  }
+
+  /**
+   * Remove all queued Mutations from the batch.
+   */
+  public void clearMutations() {
+    mutations.clear();
+  }
+
+  /**
+   * Submits all queued Mutations.
+   * @return The number of mutation submitted.
+   */
+  public int mutate() {
+    int mutationCount = mutations.size();
+    Object[] result = new Object[mutationCount];
+    try {
+      table.batch(mutations, result);
+      mutations.clear();
+
+    } catch (Exception e) {
+      String msg = String.format("'%d' HBase write(s) failed on table '%s'", size(mutations), tableName(table));
+      LOG.error(msg, e);
+      throw new RuntimeException(msg, e);
+    }
+
+    return mutationCount;
+  }
+
+  /**
+   * Adds a Get to the batch.
    *
-   * @return The results from the scan.
-   * @throws IOException
+   * @param rowKey   The row key of the Get
+   * @param criteria Defines the columns/families that will be retrieved.
+   */
+  public void addGet(byte[] rowKey, HBaseProjectionCriteria criteria) {
+    Get get = new Get(rowKey);
+
+    if (criteria != null) {
+      criteria.getColumnFamilies().forEach(cf -> get.addFamily(cf));
+      criteria.getColumns().forEach(col -> get.addColumn(col.getColumnFamily(), col.getQualifier()));
+    }
+
+    // queue the get
+    this.gets.add(get);
+  }
+
+  /**
+   * Clears all queued Gets from the batch.
    */
-  Result[] scan(int numRows) throws IOException;
+  public void clearGets() {
+    gets.clear();
+  }
 
   /**
-   * Enqueues a {@link org.apache.hadoop.hbase.client.Mutation} such as a put or
-   * increment.  The operation is enqueued for later execution.
+   * Submit all queued Gets.
    *
-   * @param rowKey     The row key of the Mutation.
-   * @param cols       The columns affected by the Mutation.
+   * @return The Result of each queued Get.
    */
-  void addMutation(byte[] rowKey, ColumnList cols);
+  public Result[] getAll() {
+    try {
+      Result[] results = table.get(gets);
+      gets.clear();
+      return results;
+
+    } catch (Exception e) {
+      String msg = String.format("'%d' HBase read(s) failed on table '%s'", size(gets), tableName(table));
+      LOG.error(msg, e);
+      throw new RuntimeException(msg, e);
+    }
+  }
 
   /**
-   * Enqueues a {@link org.apache.hadoop.hbase.client.Mutation} such as a put or
-   * increment.  The operation is enqueued for later execution.
+   * Close the table.
+   */
+  @Override
+  public void close() throws IOException {
+    if(table != null) {
+      table.close();
+    }
+  }
+
+  /**
+   * Creates an HBase Put.
    *
-   * @param rowKey     The row key of the Mutation.
-   * @param cols       The columns affected by the Mutation.
-   * @param durability The durability of the mutation.
+   * @param rowKey     The row key.
+   * @param cols       The columns to put.
+   * @param durability The durability of the put.
    */
-  void addMutation(byte[] rowKey, ColumnList cols, Durability durability);
+  private Put createPut(byte[] rowKey, ColumnList cols, Durability durability) {
+    Put put = new Put(rowKey);
+    put.setDurability(durability);
+    addColumns(cols, put);
+    return put;
+  }
 
   /**
-   * Enqueues a {@link org.apache.hadoop.hbase.client.Mutation} such as a put or
-   * increment.  The operation is enqueued for later execution.
+   * Creates an HBase Put.
    *
-   * @param rowKey           The row key of the Mutation.
-   * @param cols             The columns affected by the Mutation.
-   * @param durability       The durability of the mutation.
-   * @param timeToLiveMillis The time to live in milliseconds.
+   * @param rowKey           The row key.
+   * @param cols             The columns to put.
+   * @param durability       The durability of the put.
+   * @param timeToLiveMillis The TTL in milliseconds.
    */
-  void addMutation(byte[] rowKey, ColumnList cols, Durability durability, Long timeToLiveMillis);
+  private Put createPut(byte[] rowKey, ColumnList cols, Durability durability, long timeToLiveMillis) {
+    Put put = new Put(rowKey);
+    put.setDurability(durability);
+    put.setTTL(timeToLiveMillis);
+    addColumns(cols, put);
+    return put;
+  }
 
   /**
-   * Ensures that all pending mutations have completed.
+   * Adds the columns to the Put
    *
-   * @return The number of operations completed.
+   * @param cols The columns to add.
+   * @param put  The Put.
    */
-  int mutate();
+  private void addColumns(ColumnList cols, Put put) {
+    for (ColumnList.Column col : cols.getColumns()) {
+
+      if (col.getTs() > 0) {
+        put.add(col.getFamily(), col.getQualifier(), col.getTs(), col.getValue());
+
+      } else {
+        put.add(col.getFamily(), col.getQualifier(), col.getValue());
+      }
+    }
+  }
 
   /**
-   * Clears all pending mutations.
+   * Creates an HBase Increment for a counter.
+   *
+   * @param rowKey     The row key.
+   * @param cols       The columns to include.
+   * @param durability The durability of the increment.
    */
-  void clearMutations();
+  private Increment createIncrement(byte[] rowKey, ColumnList cols, Durability durability) {
+    Increment inc = new Increment(rowKey);
+    inc.setDurability(durability);
+    cols.getCounters().forEach(cnt -> inc.addColumn(cnt.getFamily(), cnt.getQualifier(), cnt.getIncrement()));
+    return inc;
+  }
 
   /**
-   * Delete a record by row key.
+   * Creates an HBase Increment for a counter.
    *
-   * @param rowKey The row key to delete.
+   * @param rowKey     The row key.
+   * @param cols       The columns to include.
+   * @param durability The durability of the increment.
+   */
+  private Increment createIncrement(byte[] rowKey, ColumnList cols, Durability durability, long timeToLiveMillis) {
+    Increment inc = new Increment(rowKey);
+    inc.setDurability(durability);
+    inc.setTTL(timeToLiveMillis);
+    cols.getCounters().forEach(cnt -> inc.addColumn(cnt.getFamily(), cnt.getQualifier(), cnt.getIncrement()));
+    return inc;
+  }
+
+  /**
+   * Returns the name of the HBase table.
+   * <p>Attempts to avoid any null pointers that might be encountered along the way.
+   * @param table The table to retrieve the name of.
+   * @return The name of the table
    */
-  void delete(byte[] rowKey);
+  private static String tableName(HTableInterface table) {
+    String tableName = "null";
+    if(table != null) {
+      if(table.getName() != null) {
+        tableName = table.getName().getNameAsString();
+      }
+    }
+    return tableName;
+  }
 
   /**
-   * Delete a column or set of columns by row key.
+   * Puts a record into the configured HBase table synchronously (not batched).
+   */
+  public void put(String rowKey, String columnFamily, String columnQualifier, String value)
+      throws IOException {
+    Put put = new Put(Bytes.toBytes(rowKey));
+    put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes(columnQualifier),
+        Bytes.toBytes(value));
+    table.put(put);
+  }
+
+  /**
+   * Scans an entire table returning all row keys as a List of Strings.
    *
-   * @param rowKey The row key to delete.
-   * @param columnList The set of columns to delete.
+   * <p>
+   * <b>**WARNING**:</b> Do not use this method unless you're absolutely crystal clear about the performance
+   * impact. Doing full table scans in HBase can adversely impact performance.
+   *
+   * @return List of all row keys as Strings for this table.
    */
-  void delete(byte[] rowKey, ColumnList columnList);
+  public List<String> readRecords() throws IOException {
+    Scan scan = new Scan();
+    ResultScanner scanner = table.getScanner(scan);
+    List<String> rows = new ArrayList<>();
+    for (Result r = scanner.next(); r != null; r = scanner.next()) {
+      rows.add(Bytes.toString(r.getRow()));
+    }
+    return rows;
+  }
+
 }
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseClientFactory.java b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseClientFactory.java
deleted file mode 100644
index b38b07b..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseClientFactory.java
+++ /dev/null
@@ -1,70 +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.hbase.client;
-
-import org.apache.hadoop.conf.Configuration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-import java.lang.invoke.MethodHandles;
-import java.lang.reflect.InvocationTargetException;
-import java.util.function.Supplier;
-
-/**
- * Responsible for creating an {@link HBaseTableClient}.
- */
-public interface HBaseClientFactory extends Serializable {
-  Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  /**
-   * @param factory The connection factory for creating connections to HBase.
-   * @param configuration The HBase configuration.
-   * @param tableName The name of the HBase table.
-   * @return An {@link HBaseTableClient}.
-   */
-  HBaseClient create(HBaseConnectionFactory factory, Configuration configuration, String tableName);
-
-  /**
-   * Instantiates a new {@link HBaseClientFactory} by class name.
-   *
-   * @param className The class name of the {@link HBaseClientFactory} to instantiate.
-   * @param defaultImpl The default instance to instantiate if the className is invalid.
-   * @return A new {@link HBaseClientFactory}.
-   */
-  static HBaseClientFactory byName(String className, Supplier<HBaseClientFactory> defaultImpl) {
-    LOG.debug("Creating HBase client creator; className={}", className);
-
-    if(className == null || className.length() == 0 || className.charAt(0) == '$') {
-      LOG.debug("Using default hbase client creator");
-      return defaultImpl.get();
-
-    } else {
-      try {
-        Class<? extends HBaseClientFactory> clazz = (Class<? extends HBaseClientFactory>) Class.forName(className);
-        return clazz.getConstructor().newInstance();
-
-      } catch(InstantiationException | IllegalAccessException | InvocationTargetException |
-              NoSuchMethodException | ClassNotFoundException e) {
-        throw new IllegalStateException("Unable to instantiate connector.", e);
-      }
-    }
-  }
-}
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseConnectionFactory.java b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseConnectionFactory.java
deleted file mode 100644
index bda26c5..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseConnectionFactory.java
+++ /dev/null
@@ -1,59 +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.hbase.client;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.lang.invoke.MethodHandles;
-import java.lang.reflect.InvocationTargetException;
-
-/**
- * Establishes a {@link Connection} to HBase.
- */
-public class HBaseConnectionFactory implements Serializable {
-  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  public Connection createConnection(Configuration configuration) throws IOException {
-    return ConnectionFactory.createConnection(configuration);
-  }
-
-  /**
-   * Creates an {@link HBaseConnectionFactory} based on a fully-qualified class name.
-   *
-   * @param className The fully-qualified class name to instantiate.
-   * @return A {@link HBaseConnectionFactory}.
-   */
-  public static HBaseConnectionFactory byName(String className) {
-    LOG.debug("Creating HBase connection factory; className={}", className);
-    try {
-      Class<? extends HBaseConnectionFactory> clazz = (Class<? extends HBaseConnectionFactory>) Class.forName(className);
-      return clazz.getConstructor().newInstance();
-
-    } catch (InstantiationException | NoSuchMethodException | IllegalAccessException | ClassNotFoundException | InvocationTargetException e) {
-      throw new IllegalStateException("Unable to instantiate HBaseConnectionFactory.", e);
-    }
-  }
-}
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseTableClient.java b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseTableClient.java
deleted file mode 100644
index 4591f9e..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseTableClient.java
+++ /dev/null
@@ -1,285 +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.hbase.client;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.hbase.ColumnList;
-import org.apache.metron.hbase.HBaseProjectionCriteria;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.commons.collections4.CollectionUtils.size;
-
-/**
- * An {@link HBaseClient} that uses the {@link Table} API to interact with HBase.
- */
-public class HBaseTableClient implements HBaseClient {
-  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private List<Mutation> mutations;
-  private List<Get> gets;
-  private Connection connection;
-  private Table table;
-
-  /**
-   * @param connectionFactory Creates connections to HBase.
-   * @param configuration The HBase configuration.
-   * @param tableName The name of the HBase table.
-   */
-  public HBaseTableClient(HBaseConnectionFactory connectionFactory, Configuration configuration, String tableName) throws IOException {
-    gets = new ArrayList<>();
-    mutations = new ArrayList<>();
-    connection = connectionFactory.createConnection(configuration);
-    table = connection.getTable(TableName.valueOf(tableName));
-  }
-
-  @Override
-  public void close() {
-    try {
-      if(table != null) {
-        table.close();
-      }
-    } catch(IOException e) {
-      LOG.error("Error while closing HBase table", e);
-    }
-
-    try {
-      if(connection != null) {
-        connection.close();
-      }
-    } catch(IOException e) {
-      LOG.error("Error while closing HBase connection", e);
-    }
-  }
-
-  @Override
-  public void addGet(byte[] rowKey, HBaseProjectionCriteria criteria) {
-    Get get = new Get(rowKey);
-
-    // define which column families and columns are needed
-    if (criteria != null) {
-      criteria.getColumnFamilies().forEach(cf -> get.addFamily(cf));
-      criteria.getColumns().forEach(col -> get.addColumn(col.getColumnFamily(), col.getQualifier()));
-    }
-
-    // queue the get
-    this.gets.add(get);
-  }
-
-  @Override
-  public Result[] getAll() {
-    try {
-      return table.get(gets);
-
-    } catch (Exception e) {
-      String msg = String.format("'%d' HBase read(s) failed on table '%s'", size(gets), tableName(table));
-      LOG.error(msg, e);
-      throw new RuntimeException(msg, e);
-
-    } finally {
-      gets.clear();
-    }
-  }
-
-  @Override
-  public void clearGets() {
-    gets.clear();
-  }
-
-  @Override
-  public List<String> scanRowKeys() throws IOException {
-    List<String> rowKeys = new ArrayList<>();
-    ResultScanner scanner = getScanner();
-    for (Result r = scanner.next(); r != null; r = scanner.next()) {
-      String rowKeyAsString = Bytes.toString(r.getRow());
-      rowKeys.add(rowKeyAsString);
-    }
-    return rowKeys;
-  }
-
-  @Override
-  public Result[] scan(int numRows) throws IOException {
-    return getScanner().next(numRows);
-  }
-
-  private ResultScanner getScanner() throws IOException {
-    Scan scan = new Scan();
-    return table.getScanner(scan);
-  }
-
-  /**
-   * Returns the name of the HBase table.
-   * <p>Attempts to avoid any null pointers that might be encountered along the way.
-   * @param table The table to retrieve the name of.
-   * @return The name of the table
-   */
-  private static String tableName(Table table) {
-    String tableName = "null";
-    if(table != null) {
-      if(table.getName() != null) {
-        tableName = table.getName().getNameAsString();
-      }
-    }
-    return tableName;
-  }
-
-  @Override
-  public void addMutation(byte[] rowKey, ColumnList cols) {
-    HBaseWriterParams params = new HBaseWriterParams();
-    addMutation(rowKey, cols, params);
-  }
-
-  @Override
-  public void addMutation(byte[] rowKey, ColumnList cols, Durability durability) {
-    HBaseWriterParams params = new HBaseWriterParams()
-            .withDurability(durability);
-    addMutation(rowKey, cols, params);
-  }
-
-  @Override
-  public void addMutation(byte[] rowKey, ColumnList cols, Durability durability, Long timeToLiveMillis) {
-    HBaseWriterParams params = new HBaseWriterParams()
-            .withDurability(durability)
-            .withTimeToLive(timeToLiveMillis);
-    addMutation(rowKey, cols, params);
-  }
-
-  private void addMutation(byte[] rowKey, ColumnList cols, HBaseWriterParams params) {
-    if (cols.hasColumns()) {
-      Put put = createPut(rowKey, params);
-      addColumns(cols, put);
-      mutations.add(put);
-    }
-    if (cols.hasCounters()) {
-      Increment inc = createIncrement(rowKey, params);
-      addColumns(cols, inc);
-      mutations.add(inc);
-    }
-  }
-
-  @Override
-  public void clearMutations() {
-    mutations.clear();
-  }
-
-  @Override
-  public int mutate() {
-    int mutationCount = mutations.size();
-    if(mutationCount > 0) {
-      doMutate();
-    }
-
-    return mutationCount;
-  }
-
-  @Override
-  public void delete(byte[] rowKey) {
-    try {
-      Delete delete = new Delete(rowKey);
-      table.delete(delete);
-
-    } catch (Exception e) {
-      String msg = String.format("Unable to delete; table=%s", tableName(table));
-      LOG.error(msg, e);
-      throw new RuntimeException(msg, e);
-    }
-  }
-
-  @Override
-  public void delete(byte[] rowKey, ColumnList columnList) {
-    try {
-      Delete delete = new Delete(rowKey);
-      for(ColumnList.Column column: columnList.getColumns()) {
-        delete.addColumn(column.getFamily(), column.getQualifier());
-      }
-      table.delete(delete);
-
-    } catch (Exception e) {
-      String msg = String.format("Unable to delete; table=%s", tableName(table));
-      LOG.error(msg, e);
-      throw new RuntimeException(msg, e);
-    }
-  }
-
-  private void doMutate() {
-    Object[] result = new Object[mutations.size()];
-    try {
-      table.batch(mutations, result);
-
-    } catch (Exception e) {
-      String msg = String.format("'%d' HBase write(s) failed on table '%s'", size(mutations), tableName(table));
-      LOG.error(msg, e);
-      throw new RuntimeException(msg, e);
-
-    } finally {
-      mutations.clear();
-    }
-  }
-
-  private Put createPut(byte[] rowKey, HBaseWriterParams params) {
-    Put put = new Put(rowKey);
-    if(params.getTimeToLiveMillis() > 0) {
-      put.setTTL(params.getTimeToLiveMillis());
-    }
-    put.setDurability(params.getDurability());
-    return put;
-  }
-
-  private void addColumns(ColumnList cols, Put put) {
-    for (ColumnList.Column col: cols.getColumns()) {
-      if (col.getTs() > 0) {
-        put.addColumn(col.getFamily(), col.getQualifier(), col.getTs(), col.getValue());
-      } else {
-        put.addColumn(col.getFamily(), col.getQualifier(), col.getValue());
-      }
-    }
-  }
-
-  private void addColumns(ColumnList cols, Increment inc) {
-    cols.getCounters().forEach(cnt ->
-            inc.addColumn(cnt.getFamily(), cnt.getQualifier(), cnt.getIncrement()));
-  }
-
-  private Increment createIncrement(byte[] rowKey, HBaseWriterParams params) {
-    Increment inc = new Increment(rowKey);
-    if(params.getTimeToLiveMillis() > 0) {
-      inc.setTTL(params.getTimeToLiveMillis());
-    }
-    inc.setDurability(params.getDurability());
-    return inc;
-  }
-}
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseTableClientFactory.java b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseTableClientFactory.java
deleted file mode 100644
index 8697e72..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseTableClientFactory.java
+++ /dev/null
@@ -1,54 +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.hbase.client;
-
-import org.apache.hadoop.conf.Configuration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.invoke.MethodHandles;
-
-/**
- * Creates an {@link HBaseTableClient}.
- */
-public class HBaseTableClientFactory implements HBaseClientFactory {
-  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  /**
-   * @param factory The factory that creates connections to HBase.
-   * @param configuration The HBase configuration.
-   * @param tableName The name of the HBase table.
-   * @return An {@link HBaseTableClient} that behaves synchronously.
-   */
-  @Override
-  public HBaseClient create(HBaseConnectionFactory factory,
-                            Configuration configuration,
-                            String tableName) {
-    try {
-      LOG.debug("Creating HBase client; table={}", tableName);
-      return new HBaseTableClient(factory, configuration, tableName);
-
-    } catch (Exception e) {
-      String msg = String.format("Unable to open connection to HBase for table '%s'", tableName);
-      LOG.error(msg, e);
-      throw new RuntimeException(msg, e);
-    }
-  }
-}
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseWriterParams.java b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseWriterParams.java
deleted file mode 100644
index ec93177..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/HBaseWriterParams.java
+++ /dev/null
@@ -1,51 +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.hbase.client;
-
-import org.apache.hadoop.hbase.client.Durability;
-
-/**
- * Parameters that define how the {@link HBaseWriter} writes to HBase.
- */
-public class HBaseWriterParams {
-  private Durability durability;
-  private Long timeToLiveMillis;
-
-  public HBaseWriterParams() {
-    durability = Durability.USE_DEFAULT;
-    timeToLiveMillis = 0L;
-  }
-
-  public HBaseWriterParams withDurability(Durability durability) {
-    this.durability = durability;
-    return this;
-  }
-
-  public HBaseWriterParams withTimeToLive(Long timeToLiveMillis) {
-    this.timeToLiveMillis = timeToLiveMillis;
-    return this;
-  }
-
-  public Durability getDurability() {
-    return durability;
-  }
-
-  public Long getTimeToLiveMillis() {
-    return timeToLiveMillis;
-  }
-}
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/LegacyHBaseClient.java b/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/LegacyHBaseClient.java
deleted file mode 100644
index d3f22c0..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/main/java/org/apache/metron/hbase/client/LegacyHBaseClient.java
+++ /dev/null
@@ -1,337 +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.hbase.client;
-
-import static org.apache.commons.collections4.CollectionUtils.size;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.hbase.TableProvider;
-import org.apache.metron.hbase.ColumnList;
-import org.apache.metron.hbase.HBaseProjectionCriteria;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A client that interacts with HBase.
- */
-public class LegacyHBaseClient implements Closeable {
-
-  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  /**
-   * The batch of queued Mutations.
-   */
-  List<Mutation> mutations;
-
-  /**
-   * The batch of queued Gets.
-   */
-  List<Get> gets;
-
-  /**
-   * The HBase table this client interacts with.
-   */
-  private HTableInterface table;
-
-  public LegacyHBaseClient(TableProvider provider, final Configuration configuration, final String tableName) {
-    this.mutations = new ArrayList<>();
-    this.gets = new ArrayList<>();
-    try {
-      this.table = provider.getTable(configuration, tableName);
-    } catch (Exception e) {
-      String msg = String.format("Unable to open connection to HBase for table '%s'", tableName);
-      LOG.error(msg, e);
-      throw new RuntimeException(msg, e);
-    }
-  }
-
-  /**
-   * Add a Mutation such as a Put or Increment to the batch.  The Mutation is only queued for
-   * later execution.
-   *
-   * @param rowKey     The row key of the Mutation.
-   * @param cols       The columns affected by the Mutation.
-   * @param durability The durability of the mutation.
-   */
-  public void addMutation(byte[] rowKey, ColumnList cols, Durability durability) {
-
-    if (cols.hasColumns()) {
-      Put put = createPut(rowKey, cols, durability);
-      mutations.add(put);
-    }
-
-    if (cols.hasCounters()) {
-      Increment inc = createIncrement(rowKey, cols, durability);
-      mutations.add(inc);
-    }
-
-    if (mutations.isEmpty()) {
-      mutations.add(new Put(rowKey));
-    }
-  }
-
-  /**
-   * Adds a Mutation such as a Put or Increment with a time to live.  The Mutation is only queued
-   * for later execution.
-   *
-   * @param rowKey           The row key of the Mutation.
-   * @param cols             The columns affected by the Mutation.
-   * @param durability       The durability of the mutation.
-   * @param timeToLiveMillis The time to live in milliseconds.
-   */
-  public void addMutation(byte[] rowKey, ColumnList cols, Durability durability, Long timeToLiveMillis) {
-
-    if (cols.hasColumns()) {
-      Put put = createPut(rowKey, cols, durability, timeToLiveMillis);
-      mutations.add(put);
-    }
-
-    if (cols.hasCounters()) {
-      Increment inc = createIncrement(rowKey, cols, durability, timeToLiveMillis);
-      mutations.add(inc);
-    }
-
-    if (mutations.isEmpty()) {
-      Put put = new Put(rowKey);
-      put.setTTL(timeToLiveMillis);
-      mutations.add(put);
-    }
-  }
-
-  /**
-   * Remove all queued Mutations from the batch.
-   */
-  public void clearMutations() {
-    mutations.clear();
-  }
-
-  /**
-   * Submits all queued Mutations.
-   * @return The number of mutation submitted.
-   */
-  public int mutate() {
-    int mutationCount = mutations.size();
-    Object[] result = new Object[mutationCount];
-    try {
-      table.batch(mutations, result);
-      mutations.clear();
-
-    } catch (Exception e) {
-      String msg = String.format("'%d' HBase write(s) failed on table '%s'", size(mutations), tableName(table));
-      LOG.error(msg, e);
-      throw new RuntimeException(msg, e);
-    }
-
-    return mutationCount;
-  }
-
-  /**
-   * Adds a Get to the batch.
-   *
-   * @param rowKey   The row key of the Get
-   * @param criteria Defines the columns/families that will be retrieved.
-   */
-  public void addGet(byte[] rowKey, HBaseProjectionCriteria criteria) {
-    Get get = new Get(rowKey);
-
-    if (criteria != null) {
-      criteria.getColumnFamilies().forEach(cf -> get.addFamily(cf));
-      criteria.getColumns().forEach(col -> get.addColumn(col.getColumnFamily(), col.getQualifier()));
-    }
-
-    // queue the get
-    this.gets.add(get);
-  }
-
-  /**
-   * Clears all queued Gets from the batch.
-   */
-  public void clearGets() {
-    gets.clear();
-  }
-
-  /**
-   * Submit all queued Gets.
-   *
-   * @return The Result of each queued Get.
-   */
-  public Result[] getAll() {
-    try {
-      Result[] results = table.get(gets);
-      gets.clear();
-      return results;
-
-    } catch (Exception e) {
-      String msg = String.format("'%d' HBase read(s) failed on table '%s'", size(gets), tableName(table));
-      LOG.error(msg, e);
-      throw new RuntimeException(msg, e);
-    }
-  }
-
-  /**
-   * Close the table.
-   */
-  @Override
-  public void close() throws IOException {
-    if(table != null) {
-      table.close();
-    }
-  }
-
-  /**
-   * Creates an HBase Put.
-   *
-   * @param rowKey     The row key.
-   * @param cols       The columns to put.
-   * @param durability The durability of the put.
-   */
-  private Put createPut(byte[] rowKey, ColumnList cols, Durability durability) {
-    Put put = new Put(rowKey);
-    put.setDurability(durability);
-    addColumns(cols, put);
-    return put;
-  }
-
-  /**
-   * Creates an HBase Put.
-   *
-   * @param rowKey           The row key.
-   * @param cols             The columns to put.
-   * @param durability       The durability of the put.
-   * @param timeToLiveMillis The TTL in milliseconds.
-   */
-  private Put createPut(byte[] rowKey, ColumnList cols, Durability durability, long timeToLiveMillis) {
-    Put put = new Put(rowKey);
-    put.setDurability(durability);
-    put.setTTL(timeToLiveMillis);
-    addColumns(cols, put);
-    return put;
-  }
-
-  /**
-   * Adds the columns to the Put
-   *
-   * @param cols The columns to add.
-   * @param put  The Put.
-   */
-  private void addColumns(ColumnList cols, Put put) {
-    for (ColumnList.Column col : cols.getColumns()) {
-
-      if (col.getTs() > 0) {
-        put.add(col.getFamily(), col.getQualifier(), col.getTs(), col.getValue());
-
-      } else {
-        put.add(col.getFamily(), col.getQualifier(), col.getValue());
-      }
-    }
-  }
-
-  /**
-   * Creates an HBase Increment for a counter.
-   *
-   * @param rowKey     The row key.
-   * @param cols       The columns to include.
-   * @param durability The durability of the increment.
-   */
-  private Increment createIncrement(byte[] rowKey, ColumnList cols, Durability durability) {
-    Increment inc = new Increment(rowKey);
-    inc.setDurability(durability);
-    cols.getCounters().forEach(cnt -> inc.addColumn(cnt.getFamily(), cnt.getQualifier(), cnt.getIncrement()));
-    return inc;
-  }
-
-  /**
-   * Creates an HBase Increment for a counter.
-   *
-   * @param rowKey     The row key.
-   * @param cols       The columns to include.
-   * @param durability The durability of the increment.
-   */
-  private Increment createIncrement(byte[] rowKey, ColumnList cols, Durability durability, long timeToLiveMillis) {
-    Increment inc = new Increment(rowKey);
-    inc.setDurability(durability);
-    inc.setTTL(timeToLiveMillis);
-    cols.getCounters().forEach(cnt -> inc.addColumn(cnt.getFamily(), cnt.getQualifier(), cnt.getIncrement()));
-    return inc;
-  }
-
-  /**
-   * Returns the name of the HBase table.
-   * <p>Attempts to avoid any null pointers that might be encountered along the way.
-   * @param table The table to retrieve the name of.
-   * @return The name of the table
-   */
-  private static String tableName(HTableInterface table) {
-    String tableName = "null";
-    if(table != null) {
-      if(table.getName() != null) {
-        tableName = table.getName().getNameAsString();
-      }
-    }
-    return tableName;
-  }
-
-  /**
-   * Puts a record into the configured HBase table synchronously (not batched).
-   */
-  public void put(String rowKey, String columnFamily, String columnQualifier, String value)
-      throws IOException {
-    Put put = new Put(Bytes.toBytes(rowKey));
-    put.addColumn(Bytes.toBytes(columnFamily), Bytes.toBytes(columnQualifier),
-        Bytes.toBytes(value));
-    table.put(put);
-  }
-
-  /**
-   * Scans an entire table returning all row keys as a List of Strings.
-   *
-   * <p>
-   * <b>**WARNING**:</b> Do not use this method unless you're absolutely crystal clear about the performance
-   * impact. Doing full table scans in HBase can adversely impact performance.
-   *
-   * @return List of all row keys as Strings for this table.
-   */
-  public List<String> readRecords() throws IOException {
-    Scan scan = new Scan();
-    ResultScanner scanner = table.getScanner(scan);
-    List<String> rows = new ArrayList<>();
-    for (Result r = scanner.next(); r != null; r = scanner.next()) {
-      rows.add(Bytes.toString(r.getRow()));
-    }
-    return rows;
-  }
-
-}
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseClient.java b/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseClient.java
deleted file mode 100644
index a2fce1f..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseClient.java
+++ /dev/null
@@ -1,335 +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.hbase.client;
-
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.hbase.ColumnList;
-import org.apache.metron.hbase.HBaseProjectionCriteria;
-import org.mockito.AdditionalMatchers;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.lang.invoke.MethodHandles;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.stream.Collectors;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * An {@link HBaseClient} useful for testing.
- *
- * <p>Maintains a static, in-memory set of records to mimic the behavior of
- * an {@link HBaseClient} that interacts with HBase.
- */
-public class FakeHBaseClient implements HBaseClient, Serializable {
-  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  /**
-   * The mutations that have been persisted.  Represents mutations that would
-   * have been written to HBase.
-   *
-   * <p>This is static so that all instantiated clients 'see' the same set of records.
-   */
-  private static Map<ByteBuffer, ColumnList> persisted = Collections.synchronizedMap(new HashMap<>());
-
-  /**
-   * Represents a mutation that was submitted to the {@link FakeHBaseClient}.
-   */
-  public static class Mutation {
-    public byte[] rowKey;
-    public ColumnList columnList;
-
-    public Mutation(byte[] rowKey, ColumnList columnList) {
-      this.rowKey = rowKey;
-      this.columnList = columnList;
-    }
-  }
-
-  /**
-   * The set of queued or pending mutations.
-   */
-  private List<Mutation> queuedMutations;
-
-  /**
-   * The set of queued or pending gets.
-   */
-  private List<ByteBuffer> queuedGets;
-
-  public FakeHBaseClient() {
-    queuedGets = new ArrayList<>();
-    queuedMutations = new ArrayList<>();
-  }
-
-  /**
-   * Deletes all records persisted in the static, in-memory collection.
-   */
-  public void deleteAll() {
-    persisted.clear();
-  }
-
-  /**
-   * Returns all mutations that have been persisted in the static, in-memory collection.
-   */
-  public List<Mutation> getAllPersisted() {
-    return persisted.entrySet()
-            .stream()
-            .map(entry -> new Mutation(entry.getKey().array(), entry.getValue()))
-            .collect(Collectors.toList());
-  }
-
-  @Override
-  public void addGet(byte[] rowKey, HBaseProjectionCriteria criteria) {
-    queuedGets.add(ByteBuffer.wrap(rowKey));
-  }
-
-  @Override
-  public Result[] getAll() {
-    LOG.debug("Looking for {} get(s) amongst {} persisted record(s)", queuedGets.size(), persisted.size());
-    List<Result> results = new ArrayList<>();
-    for (int i = 0; i < queuedGets.size(); i++) {
-      ByteBuffer rowKey = queuedGets.get(i);
-      Result result;
-      if (persisted.containsKey(rowKey)) {
-        ColumnList cols = persisted.get(rowKey);
-        result = matchingResult(rowKey.array(), cols);
-
-      } else {
-        result = emptyResult();
-      }
-      results.add(result);
-    }
-
-    clearGets();
-    return results.stream().toArray(Result[]::new);
-  }
-
-  /**
-   * Builds a mock {@link Result} that will respond correctly to the following methods calls.
-   *
-   * Result.containsColumn(family, qualifier)
-   * Result.getValue(family, qualifier)
-   * Result.getFamilyMap(family)
-   *
-   * @param columns The columns.
-   * @return A {@link Result}.
-   */
-  private static Result matchingResult(byte[] rowKey, ColumnList columns) {
-    // Result.getRow() should return the row key
-    Result result = mock(Result.class);
-    when(result.getRow())
-            .thenReturn(rowKey);
-
-    // find all column families
-    Set<ByteBuffer> families = new HashSet<>();
-    for (ColumnList.Column column : columns.getColumns()) {
-      families.add(ByteBuffer.wrap(column.getFamily()));
-    }
-
-    // for each column family
-    for (ByteBuffer family: families) {
-
-      // build the family map
-      NavigableMap<byte[], byte[]> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-      for (ColumnList.Column column : columns.getColumns()) {
-
-        // is this column in the current family?
-        if (family.equals(ByteBuffer.wrap(column.getFamily()))) {
-          LOG.debug("Found column in family; {}:{}", Bytes.toString(column.getFamily()),
-                  Bytes.toString(column.getQualifier()));
-
-          // Result.containsColumn(family, qualifier) should return true
-          when(result.containsColumn(eq(column.getFamily()), eq(column.getQualifier())))
-                  .thenReturn(true);
-
-          // Result.getValue(family, qualifier) should return the value
-          when(result.getValue(eq(column.getFamily()), eq(column.getQualifier())))
-                  .thenReturn(column.getValue());
-
-          familyMap.put(column.getQualifier(), column.getValue());
-        }
-      }
-      LOG.debug("Built family map; family={}, size={}", Bytes.toString(family.array()), familyMap.size());
-
-      // Result.getFamilyMap(family) should return all values in that family
-      when(result.getFamilyMap(AdditionalMatchers.aryEq(family.array())))
-              .thenReturn(familyMap);
-    }
-
-    return result;
-  }
-
-  private static Result emptyResult() {
-    Result result = mock(Result.class);
-    when(result.containsColumn(any(), any()))
-            .thenReturn(false);
-    when(result.isEmpty())
-            .thenReturn(true);
-    return result;
-  }
-
-  @Override
-  public void clearGets() {
-    queuedGets.clear();
-  }
-
-  @Override
-  public List<String> scanRowKeys() {
-    return persisted
-            .keySet()
-            .stream()
-            .map(buffer -> Bytes.toString(buffer.array()))
-            .collect(Collectors.toList());
-  }
-
-  @Override
-  public Result[] scan(int numRows) throws IOException {
-    return persisted
-            .entrySet()
-            .stream()
-            .limit(numRows)
-            .map((entry) -> matchingResult(entry.getKey().array(), entry.getValue()))
-            .toArray(Result[]::new);
-  }
-
-  @Override
-  public void addMutation(byte[] rowKey, ColumnList cols) {
-    for (ColumnList.Column column : cols.getColumns()) {
-      String family = Bytes.toString(column.getFamily());
-      String qualifier = Bytes.toString(column.getQualifier());
-      LOG.debug("Queuing mutation column; {}:{}", family, qualifier);
-    }
-    queuedMutations.add(new Mutation(rowKey, cols));
-  }
-
-  @Override
-  public void addMutation(byte[] rowKey, ColumnList cols, Durability durability) {
-    // ignore durability
-    addMutation(rowKey, cols);
-  }
-
-  @Override
-  public void addMutation(byte[] rowKey, ColumnList cols, Durability durability, Long timeToLiveMillis) {
-    // ignore durability and time-to-live
-    addMutation(rowKey, cols);
-  }
-
-  @Override
-  public int mutate() {
-    int numberOfMutations = queuedMutations.size();
-
-    // persist each queued mutation
-    for(Mutation mutation: queuedMutations) {
-      ByteBuffer key = ByteBuffer.wrap(mutation.rowKey);
-      ColumnList columns = mutation.columnList;
-
-      // if there are existing columns, need to merge the columns
-      ColumnList existing = persisted.get(key);
-      if(existing != null) {
-        for(ColumnList.Column col: existing.getColumns()) {
-          columns.addColumn(col);
-        }
-      }
-
-      persisted.put(key, columns);
-    }
-    clearMutations();
-
-    LOG.debug("Wrote {} record(s); now have {} record(s) in all", numberOfMutations, persisted.size());
-    return numberOfMutations;
-  }
-
-  @Override
-  public void clearMutations() {
-    queuedMutations.clear();
-  }
-
-  @Override
-  public void delete(byte[] rowKey) {
-    persisted.remove(ByteBuffer.wrap(rowKey));
-  }
-
-  @Override
-  public void delete(byte[] rowKey, ColumnList toDelete) {
-    ColumnList existingColumns = persisted.get(ByteBuffer.wrap(rowKey));
-    if(existingColumns != null) {
-      // the names of columns that need deleted
-      List<String> columnsToDelete = nameOf(toDelete);
-      LOG.debug("About to delete columns; existing={}, toDelete={}", nameOf(existingColumns), columnsToDelete);
-
-      // build a new set of columns that removes any columns that need deleted
-      ColumnList newColumns = new ColumnList();
-      for(ColumnList.Column existing: existingColumns.getColumns()) {
-
-        // only keep the columns that do not need to be deleted
-        boolean keepColumn = !columnsToDelete.contains(nameOf(existing));
-        if(keepColumn) {
-          newColumns.addColumn(existing);
-        } else {
-          LOG.debug("Column was deleted; column={}", existing);
-        }
-      }
-
-      if(newColumns.hasColumns()) {
-        // persist the new columns
-        persisted.put(ByteBuffer.wrap(rowKey), newColumns);
-
-      } else {
-        // there are no columns left, so remove the row
-        persisted.remove(ByteBuffer.wrap(rowKey));
-      }
-
-    } else {
-      LOG.debug("Nothing to delete");
-    }
-  }
-
-  private List<String> nameOf(ColumnList columnList) {
-    return columnList
-            .getColumns()
-            .stream()
-            .map(col -> nameOf(col))
-            .collect(Collectors.toList());
-  }
-
-  private String nameOf(ColumnList.Column column) {
-    return Bytes.toString(column.getFamily()) + ":" + Bytes.toString(column.getQualifier());
-  }
-
-  @Override
-  public void close() {
-    // nothing to do
-  }
-}
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseClientFactory.java b/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseClientFactory.java
deleted file mode 100644
index 135e693..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseClientFactory.java
+++ /dev/null
@@ -1,43 +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.hbase.client;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Creates a {@link FakeHBaseClient}.
- */
-public class FakeHBaseClientFactory implements HBaseClientFactory {
-
-  private FakeHBaseClient hBaseClient;
-
-  public FakeHBaseClientFactory() {
-    this(new FakeHBaseClient());
-  }
-
-  public FakeHBaseClientFactory(FakeHBaseClient hBaseClient) {
-    this.hBaseClient = hBaseClient;
-  }
-
-  @Override
-  public HBaseClient create(HBaseConnectionFactory factory, Configuration configuration, String tableName) {
-    return hBaseClient;
-  }
-}
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseClientTest.java b/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseClientTest.java
deleted file mode 100644
index 9dca390..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseClientTest.java
+++ /dev/null
@@ -1,268 +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.hbase.client;
-
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.hbase.ColumnList;
-import org.apache.metron.hbase.HBaseProjectionCriteria;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-/**
- * Tests the {@link FakeHBaseClient} class.
- */
-public class FakeHBaseClientTest {
-  private static final String columnFamily = "W";
-  private static final byte[] columnFamilyB = Bytes.toBytes(columnFamily);
-  private static final String columnQualifier = "column";
-  private static final byte[] columnQualifierB = Bytes.toBytes(columnQualifier);
-  private static final String rowKey1String = "row-key-1";
-  private static final byte[] rowKey1 = Bytes.toBytes(rowKey1String);
-  private static final String rowKey2String = "row-key-2";
-  private static final byte[] rowKey2 = Bytes.toBytes(rowKey2String);
-
-  private FakeHBaseClient client;
-
-  @Before
-  public void setup() {
-    client = new FakeHBaseClient();
-    client.deleteAll();
-  }
-
-  @Test
-  public void testMutate() throws Exception {
-    // write some values
-    ColumnList columns = new ColumnList()
-            .addColumn(columnFamily, columnQualifier, "value1");
-    client.addMutation(rowKey1, columns, Durability.SKIP_WAL);
-    client.mutate();
-
-    // read back the value
-    client.addGet(rowKey1, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Result[] results = client.getAll();
-
-    // validate
-    assertEquals(1, results.length);
-    assertEquals("value1", getValue(results[0], columnFamily, columnQualifier));
-  }
-
-  @Test
-  public void testMutateMultipleColumns() throws Exception {
-    // write some values
-    ColumnList columns = new ColumnList()
-            .addColumn(columnFamily, "col1", "value1")
-            .addColumn(columnFamily, "col2", "value2");
-    client.addMutation(rowKey1, columns, Durability.SKIP_WAL);
-    client.mutate();
-
-    // read back the value
-    client.addGet(rowKey1, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Result[] results = client.getAll();
-
-    // validate
-    assertEquals(1, results.length);
-    assertEquals("value1", getValue(results[0], columnFamily, "col1"));
-    assertEquals("value2", getValue(results[0], columnFamily, "col2"));
-  }
-
-  @Test
-  public void testNoMutations() throws Exception {
-    // do not add any mutations before attempting to write
-    int count = client.mutate();
-    Assert.assertEquals(0, count);
-
-    // attempt to read
-    HBaseProjectionCriteria criteria = new HBaseProjectionCriteria().addColumnFamily(columnFamily);
-    client.addGet(rowKey1, criteria);
-    client.addGet(rowKey2, criteria);
-    Result[] results = client.getAll();
-
-    // nothing should have been read
-    assertEquals(2, results.length);
-    for(Result result : results) {
-      Assert.assertTrue(result.isEmpty());
-    }
-  }
-
-  /**
-   * Unfortunately, the {@link Result} returned by the {@link FakeHBaseClient} is a mock and needs
-   * to respond like an actual {@link Result}.  This test ensures that {@link Result#getFamilyMap(byte[])}
-   * works correctly.
-   */
-  @Test
-  public void testResultFamilyMap() {
-    // write some values
-    ColumnList columns = new ColumnList()
-            .addColumn(columnFamily, "col1", "value1")
-            .addColumn(columnFamily, "col2", "value2");
-    client.addMutation(rowKey1, columns, Durability.SKIP_WAL);
-    client.mutate();
-
-    // read back the value
-    HBaseProjectionCriteria criteria = new HBaseProjectionCriteria()
-            .addColumnFamily(columnFamily);
-    client.addGet(rowKey1, criteria);
-    Result[] results = client.getAll();
-
-    // validate
-    assertEquals(1, results.length);
-    Result result = results[0];
-
-    // the first column
-    String value1 = Bytes.toString(result.getFamilyMap(columnFamilyB).get(Bytes.toBytes("col1")));
-    assertEquals("value1", value1);
-
-    // the second column
-    String value2 = Bytes.toString(result.getFamilyMap(columnFamilyB).get(Bytes.toBytes("col2")));
-    assertEquals("value2", value2);
-  }
-
-  @Test
-  public void testScan() throws Exception {
-    // write some values
-    client.addMutation(rowKey1, new ColumnList().addColumn(columnFamily, columnQualifier, "value1"), Durability.SKIP_WAL);
-    client.mutate();
-
-    // scan the table
-    Result[] results = client.scan(10);
-    assertEquals(1, results.length);
-
-    assertArrayEquals(rowKey1, results[0].getRow());
-    String actual1 = Bytes.toString(results[0].getValue(columnFamilyB, columnQualifierB));
-    assertEquals("value1", actual1);
-  }
-
-  @Test
-  public void testScanLimit() throws Exception {
-    // write some values
-    client.addMutation(rowKey1, new ColumnList().addColumn(columnFamily, columnQualifier, "value1"), Durability.SKIP_WAL);
-    client.addMutation(rowKey2, new ColumnList().addColumn(columnFamily, columnQualifier, "value2"), Durability.SKIP_WAL);
-    client.mutate();
-
-    // scan the table, but limit to 1 result
-    Result[] results = client.scan(1);
-    assertEquals(1, results.length);
-  }
-
-  @Test
-  public void testScanNothing() throws Exception {
-    // scan the table, but there is nothing there
-    Result[] results = client.scan(1);
-    assertEquals(0, results.length);
-  }
-
-  @Test
-  public void testScanRowKeys() throws Exception {
-    // write some values
-    client.addMutation(rowKey1, new ColumnList().addColumn(columnFamily, columnQualifier, "value1"), Durability.SKIP_WAL);
-    client.addMutation(rowKey2, new ColumnList().addColumn(columnFamily, columnQualifier, "value2"), Durability.SKIP_WAL);
-    client.mutate();
-
-    // scan the table
-    List<String> rowKeys = client.scanRowKeys();
-    List<String> expected = Arrays.asList(rowKey1String, rowKey2String);
-    assertEquals(new HashSet<>(expected), new HashSet<>(rowKeys));
-  }
-
-  @Test
-  public void testDelete() {
-    // write some values
-    client.addMutation(rowKey1, new ColumnList().addColumn(columnFamily, columnQualifier, "value1"), Durability.SKIP_WAL);
-    client.addMutation(rowKey2, new ColumnList().addColumn(columnFamily, columnQualifier, "value2"), Durability.SKIP_WAL);
-    client.mutate();
-
-    client.delete(rowKey1);
-
-    // the deleted row key should no longer exist
-    client.addGet(rowKey1, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Assert.assertTrue(client.getAll()[0].isEmpty());
-
-    // the other row key should remain
-    client.addGet(rowKey2, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Assert.assertFalse(client.getAll()[0].isEmpty());
-  }
-
-  @Test
-  public void testDeleteNothing() {
-    // nothing should blow-up if we attempt to delete something that does not exist
-    client.delete(rowKey1);
-  }
-
-  @Test
-  public void testDeleteColumn() {
-    // write some values
-    ColumnList columns = new ColumnList()
-            .addColumn(columnFamily, "col1", "value1")
-            .addColumn(columnFamily, "col2", "value2");
-    client.addMutation(rowKey1, columns, Durability.SKIP_WAL);
-    client.mutate();
-
-    // delete a column
-    client.delete(rowKey1, new ColumnList().addColumn(columnFamily, "col1"));
-
-    // read back the value
-    client.addGet(rowKey1, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Result[] results = client.getAll();
-
-    // validate
-    assertEquals(1, results.length);
-    assertNull(getValue(results[0], columnFamily, "col1"));
-    assertEquals("value2", getValue(results[0], columnFamily, "col2"));
-  }
-
-  @Test
-  public void testDeleteAllColumns() {
-    // write some values
-    ColumnList columns = new ColumnList()
-            .addColumn(columnFamily, "col1", "value1")
-            .addColumn(columnFamily, "col2", "value2");
-    client.addMutation(rowKey1, columns, Durability.SKIP_WAL);
-    client.mutate();
-
-    // delete both columns individually
-    client.delete(rowKey1, new ColumnList().addColumn(columnFamily, "col1"));
-    client.delete(rowKey1, new ColumnList().addColumn(columnFamily, "col2"));
-
-    // read back the value
-    client.addGet(rowKey1, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Result[] results = client.getAll();
-
-    // validate
-    assertEquals(1, results.length);
-    assertNull(getValue(results[0], columnFamily, "col1"));
-    assertNull(getValue(results[0], columnFamily, "col2"));
-  }
-
-  private String getValue(Result result, String columnFamily, String columnQualifier) {
-    byte[] value = result.getValue(Bytes.toBytes(columnFamily), Bytes.toBytes(columnQualifier));
-    return Bytes.toString(value);
-  }
-}
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseConnectionFactory.java b/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseConnectionFactory.java
deleted file mode 100644
index d44e110..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/FakeHBaseConnectionFactory.java
+++ /dev/null
@@ -1,89 +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.hbase.client;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Table;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * A mock {@link HBaseConnectionFactory} useful for testing.
- */
-public class FakeHBaseConnectionFactory extends HBaseConnectionFactory {
-
-  /**
-   * A set of {@link Table}s that will be returned by all {@link Connection}
-   * objects created by this factory.
-   */
-  private Map<TableName, Table> tables;
-
-  public FakeHBaseConnectionFactory() {
-    this.tables = new HashMap<>();
-  }
-
-  /**
-   * The {@link Connection} returned by this factory will return the given table by
-   * name when {@link Connection#getTable(TableName)} is called.
-   *
-   * @param tableName The name of the table.
-   * @param table The table.
-   * @return
-   */
-  public FakeHBaseConnectionFactory withTable(String tableName, Table table) {
-    this.tables.put(TableName.valueOf(tableName), table);
-    return this;
-  }
-
-  /**
-   * The {@link Connection} returned by this factory will return a table by
-   * name when {@link Connection#getTable(TableName)} is called.
-   *
-   * @param tableName The name of the table.
-   * @return
-   */
-  public FakeHBaseConnectionFactory withTable(String tableName) {
-    return withTable(tableName, mock(Table.class));
-  }
-
-  public Table getTable(String tableName) {
-    return tables.get(TableName.valueOf(tableName));
-  }
-
-  @Override
-  public Connection createConnection(Configuration configuration) throws IOException {
-    Connection connection = mock(Connection.class);
-
-    // the connection must return each of the given tables by name
-    for(Map.Entry<TableName, Table> entry: tables.entrySet()) {
-      TableName tableName = entry.getKey();
-      Table table = entry.getValue();
-      when(connection.getTable(eq(tableName))).thenReturn(table);
-    }
-
-    return connection;
-  }
-}
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java b/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java
index f9a9f1f..1983fc7 100644
--- a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java
+++ b/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java
@@ -63,7 +63,7 @@ public class HBaseClientTest {
   private static final String tableName = "table";
 
   private static HBaseTestingUtility util;
-  private static LegacyHBaseClient client;
+  private static HBaseClient client;
   private static HTableInterface table;
   private static Admin admin;
   private static byte[] cf = Bytes.toBytes("cf");
@@ -87,7 +87,7 @@ public class HBaseClientTest {
     table = util.createTable(Bytes.toBytes(tableName), cf);
     util.waitTableEnabled(table.getName());
     // setup the client
-    client = new LegacyHBaseClient((c, t) -> table, table.getConfiguration(), tableName);
+    client = new HBaseClient((c,t) -> table, table.getConfiguration(), tableName);
   }
 
   @AfterClass
@@ -259,7 +259,7 @@ public class HBaseClientTest {
     TableProvider tableProvider = mock(TableProvider.class);
     when(tableProvider.getTable(any(), any())).thenThrow(new IllegalArgumentException("test exception"));
 
-    client = new LegacyHBaseClient(tableProvider, HBaseConfiguration.create(), tableName);
+    client = new HBaseClient(tableProvider, HBaseConfiguration.create(), tableName);
   }
 
   @Test(expected = RuntimeException.class)
@@ -271,7 +271,7 @@ public class HBaseClientTest {
     TableProvider tableProvider = mock(TableProvider.class);
     when(tableProvider.getTable(any(), any())).thenReturn(table);
 
-    client = new LegacyHBaseClient(tableProvider, HBaseConfiguration.create(), tableName);
+    client = new HBaseClient(tableProvider, HBaseConfiguration.create(), tableName);
     client.addMutation(rowKey1, cols1, Durability.SYNC_WAL);
     client.mutate();
   }
@@ -288,7 +288,7 @@ public class HBaseClientTest {
     HBaseProjectionCriteria criteria = new HBaseProjectionCriteria();
     criteria.addColumnFamily(Bytes.toString(cf));
 
-    client = new LegacyHBaseClient(tableProvider, HBaseConfiguration.create(), tableName);
+    client = new HBaseClient(tableProvider, HBaseConfiguration.create(), tableName);
     client.addGet(rowKey1, criteria);
     client.addGet(rowKey2, criteria);
     client.getAll();
diff --git a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/integration/HBaseTableClientIntegrationTest.java b/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/integration/HBaseTableClientIntegrationTest.java
deleted file mode 100644
index d9db3cf..0000000
--- a/metron-platform/metron-hbase/metron-hbase-common/src/test/java/org/apache/metron/hbase/client/integration/HBaseTableClientIntegrationTest.java
+++ /dev/null
@@ -1,286 +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.hbase.client.integration;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.metron.hbase.ColumnList;
-import org.apache.metron.hbase.HBaseProjectionCriteria;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
-import org.apache.metron.hbase.client.HBaseTableClient;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-/**
- * An integration test for the {@link HBaseTableClient}.
- */
-public class HBaseTableClientIntegrationTest {
-  private static final String tableName = "widgets";
-  private static final String columnFamily = "W";
-  private static final byte[] columnFamilyB = Bytes.toBytes(columnFamily);
-  private static final String columnQualifier = "column";
-  private static final byte[] columnQualifierB = Bytes.toBytes(columnQualifier);
-  private static final String rowKey1String = "row-key-1";
-  private static final byte[] rowKey1 = Bytes.toBytes(rowKey1String);
-  private static final String rowKey2String = "row-key-2";
-  private static final byte[] rowKey2 = Bytes.toBytes(rowKey2String);
-  private static HBaseTestingUtility util;
-  private static Table table;
-  private HBaseTableClient client;
-
-  @BeforeClass
-  public static void startHBase() throws Exception {
-    Configuration config = HBaseConfiguration.create();
-    config.set("hbase.master.hostname", "localhost");
-    config.set("hbase.regionserver.hostname", "localhost");
-
-    util = new HBaseTestingUtility(config);
-    util.startMiniCluster();
-
-    // create the table
-    table = util.createTable(TableName.valueOf(tableName), columnFamily);
-    util.waitTableEnabled(table.getName());
-  }
-
-  @AfterClass
-  public static void stopHBase() throws Exception {
-    util.deleteTable(table.getName());
-    util.shutdownMiniCluster();
-    util.cleanupTestDir();
-  }
-
-  @Before
-  public void setup() throws IOException {
-    client = new HBaseTableClient(new HBaseConnectionFactory(), util.getConfiguration(), tableName);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    // delete all records in the table
-    List<Delete> deletions = new ArrayList<>();
-    for(Result r : table.getScanner(new Scan())) {
-      deletions.add(new Delete(r.getRow()));
-    }
-    table.delete(deletions);
-
-    if(client != null) {
-      client.close();
-    }
-  }
-
-  @Test
-  public void testMutate() throws Exception {
-    // write some values
-    ColumnList columns = new ColumnList()
-            .addColumn(columnFamily, columnQualifier, "value1");
-    client.addMutation(rowKey1, columns, Durability.SKIP_WAL);
-    client.mutate();
-
-    // read back the value
-    client.addGet(rowKey1, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Result[] results = client.getAll();
-
-    // validate
-    assertEquals(1, results.length);
-    assertEquals("value1", getValue(results[0], columnFamily, columnQualifier));
-  }
-
-  @Test
-  public void testMutateMultipleColumns() throws Exception {
-    // write some values
-    ColumnList columns = new ColumnList()
-            .addColumn(columnFamily, "col1", "value1")
-            .addColumn(columnFamily, "col2", "value2");
-    client.addMutation(rowKey1, columns, Durability.SKIP_WAL);
-    client.mutate();
-
-    // read back the value
-    client.addGet(rowKey1, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Result[] results = client.getAll();
-
-    // validate
-    assertEquals(1, results.length);
-    assertEquals("value1", getValue(results[0], columnFamily, "col1"));
-    assertEquals("value2", getValue(results[0], columnFamily, "col2"));
-  }
-
-  @Test
-  public void testNoMutations() throws Exception {
-    // do not add any mutations before attempting to write
-    int count = client.mutate();
-    Assert.assertEquals(0, count);
-
-    // attempt to read
-    HBaseProjectionCriteria criteria = new HBaseProjectionCriteria().addColumnFamily(columnFamily);
-    client.addGet(rowKey1, criteria);
-    client.addGet(rowKey2, criteria);
-    Result[] results = client.getAll();
-
-    // nothing should have been read
-    assertEquals(2, results.length);
-    for(Result result : results) {
-      Assert.assertTrue(result.isEmpty());
-    }
-  }
-
-  @Test
-  public void testScan() throws Exception {
-    // write some values
-    client.addMutation(rowKey1, new ColumnList().addColumn(columnFamily, columnQualifier, "value1"), Durability.SKIP_WAL);
-    client.mutate();
-
-    // scan the table
-    Result[] results = client.scan(10);
-    assertEquals(1, results.length);
-
-    assertArrayEquals(rowKey1, results[0].getRow());
-    String actual1 = Bytes.toString(results[0].getValue(columnFamilyB, columnQualifierB));
-    assertEquals("value1", actual1);
-  }
-
-  @Test
-  public void testScanLimit() throws Exception {
-    // write some values
-    client.addMutation(rowKey1, new ColumnList().addColumn(columnFamily, columnQualifier, "value1"), Durability.SKIP_WAL);
-    client.addMutation(rowKey2, new ColumnList().addColumn(columnFamily, columnQualifier, "value2"), Durability.SKIP_WAL);
-    client.mutate();
-
-    // scan the table, but limit to 1 result
-    Result[] results = client.scan(1);
-    assertEquals(1, results.length);
-  }
-
-  @Test
-  public void testScanNothing() throws Exception {
-    // scan the table, but there is nothing there
-    Result[] results = client.scan(1);
-    assertEquals(0, results.length);
-  }
-
-  @Test
-  public void testScanRowKeys() throws Exception {
-    // write some values
-    client.addMutation(rowKey1, new ColumnList().addColumn(columnFamily, columnQualifier, "value1"), Durability.SKIP_WAL);
-    client.addMutation(rowKey2, new ColumnList().addColumn(columnFamily, columnQualifier, "value2"), Durability.SKIP_WAL);
-    client.mutate();
-
-    // scan the table
-    List<String> rowKeys = client.scanRowKeys();
-    List<String> expected = Arrays.asList(rowKey1String, rowKey2String);
-    assertEquals(new HashSet<>(expected), new HashSet<>(rowKeys));
-  }
-
-  @Test
-  public void testDelete() {
-    // write some values
-    client.addMutation(rowKey1, new ColumnList().addColumn(columnFamily, columnQualifier, "value1"), Durability.SKIP_WAL);
-    client.addMutation(rowKey2, new ColumnList().addColumn(columnFamily, columnQualifier, "value2"), Durability.SKIP_WAL);
-    client.mutate();
-
-    client.delete(rowKey1);
-
-    // the deleted row key should no longer exist
-    client.addGet(rowKey1, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Assert.assertTrue(client.getAll()[0].isEmpty());
-
-    // the other row key should remain
-    client.addGet(rowKey2, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Assert.assertFalse(client.getAll()[0].isEmpty());
-  }
-
-  @Test
-  public void testDeleteNothing() {
-    // nothing should blow-up if we attempt to delete something that does not exist
-    client.delete(rowKey1);
-  }
-
-  @Test
-  public void testDeleteColumn() {
-    // write some values
-    ColumnList columns = new ColumnList()
-            .addColumn(columnFamily, "col1", "value1")
-            .addColumn(columnFamily, "col2", "value2");
-    client.addMutation(rowKey1, columns, Durability.SKIP_WAL);
-    client.mutate();
-
-    // delete a column
-    client.delete(rowKey1, new ColumnList().addColumn(columnFamily, "col1"));
-
-    // read back the value
-    client.addGet(rowKey1, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Result[] results = client.getAll();
-
-    // validate
-    assertEquals(1, results.length);
-    assertNull(getValue(results[0], columnFamily, "col1"));
-    assertEquals("value2", getValue(results[0], columnFamily, "col2"));
-  }
-
-  @Test
-  public void testDeleteAllColumns() {
-    // write some values
-    ColumnList columns = new ColumnList()
-            .addColumn(columnFamily, "col1", "value1")
-            .addColumn(columnFamily, "col2", "value2");
-    client.addMutation(rowKey1, columns, Durability.SKIP_WAL);
-    client.mutate();
-
-    // delete both columns individually
-    client.delete(rowKey1, new ColumnList().addColumn(columnFamily, "col1"));
-    client.delete(rowKey1, new ColumnList().addColumn(columnFamily, "col2"));
-
-    // read back the value
-    client.addGet(rowKey1, new HBaseProjectionCriteria().addColumnFamily(columnFamily));
-    Result[] results = client.getAll();
-
-    // validate
-    assertEquals(1, results.length);
-    assertNull(getValue(results[0], columnFamily, "col1"));
-    assertNull(getValue(results[0], columnFamily, "col2"));
-  }
-
-  private String getValue(Result result, String columnFamily, String columnQualifier) {
-    byte[] value = result.getValue(Bytes.toBytes(columnFamily), Bytes.toBytes(columnQualifier));
-    return Bytes.toString(value);
-  }
-}
diff --git a/metron-platform/metron-indexing/metron-indexing-common/pom.xml b/metron-platform/metron-indexing/metron-indexing-common/pom.xml
index e79758f..d409531 100644
--- a/metron-platform/metron-indexing/metron-indexing-common/pom.xml
+++ b/metron-platform/metron-indexing/metron-indexing-common/pom.xml
@@ -181,34 +181,6 @@
             <scope>test</scope>
             <type>test-jar</type>
         </dependency>
-        <dependency>
-            <groupId>org.apache.hbase</groupId>
-            <artifactId>hbase-testing-util</artifactId>
-            <version>${global_hbase_version}</version>
-            <scope>test</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>
-                <exclusion>
-                    <groupId>org.apache.hadoop</groupId>
-                    <artifactId>hadoop-hdfs</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hadoop</groupId>
-                    <artifactId>hadoop-minicluster</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
     </dependencies>
     <build>
         <plugins>
diff --git a/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java b/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java
index 9ebecb6..b1df46a 100644
--- a/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java
+++ b/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java
@@ -17,11 +17,8 @@
  */
 package org.apache.metron.indexing.dao;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.metron.hbase.TableProvider;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
-
 import java.util.function.Function;
+import org.apache.metron.hbase.TableProvider;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -33,12 +30,9 @@ public class AccessConfig {
   private Supplier<Map<String, Object>> globalConfigSupplier;
   private Function<String, String> indexSupplier;
   private Map<String, String> optionalSettings = new HashMap<>();
-  private HBaseConnectionFactory hbaseConnectionFactory;
-  private Configuration hbaseConfiguration;
   private TableProvider tableProvider = null;
   private Boolean isKerberosEnabled = false;
 
-
   /**
    * @return A supplier which will return the current global config.
    */
@@ -92,30 +86,6 @@ public class AccessConfig {
   }
 
   /**
-   * @return The {@link HBaseConnectionFactory} that establishes connections to HBase.
-   */
-  public HBaseConnectionFactory getHbaseConnectionFactory() {
-    return hbaseConnectionFactory;
-  }
-
-  public AccessConfig setHbaseConnectionFactory(HBaseConnectionFactory hbaseConnectionFactory) {
-    this.hbaseConnectionFactory = hbaseConnectionFactory;
-    return this;
-  }
-
-  /**
-   * @return The configuration used to connect to HBase.
-   */
-  public Configuration getHbaseConfiguration() {
-    return hbaseConfiguration;
-  }
-
-  public AccessConfig setHbaseConfiguration(Configuration hbaseConfiguration) {
-    this.hbaseConfiguration = hbaseConfiguration;
-    return this;
-  }
-
-  /**
    * @return The table provider to use for NoSql DAOs
    */
   public TableProvider getTableProvider() {
diff --git a/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java
index 0256307..71d0544 100644
--- a/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java
+++ b/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java
@@ -18,18 +18,25 @@
 
 package org.apache.metron.indexing.dao;
 
-import org.apache.hadoop.conf.Configuration;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.metron.common.utils.JSONUtils;
 import org.apache.metron.common.utils.KeyUtil;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
 import org.apache.metron.indexing.dao.search.AlertComment;
 import org.apache.metron.indexing.dao.search.FieldType;
 import org.apache.metron.indexing.dao.search.GetRequest;
@@ -41,18 +48,6 @@ import org.apache.metron.indexing.dao.search.SearchResponse;
 import org.apache.metron.indexing.dao.update.CommentAddRemoveRequest;
 import org.apache.metron.indexing.dao.update.Document;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Optional;
-import java.util.stream.Collectors;
-
 /**
  * The HBaseDao is an index dao which only supports the following actions:
  * * Update
@@ -65,12 +60,11 @@ import java.util.stream.Collectors;
  *
  */
 public class HBaseDao implements IndexDao {
-  public static final String HBASE_TABLE = "update.hbase.table";
-  public static final String HBASE_CF = "update.hbase.cf";
-  private Table table;
-  private byte[] columnFamily;
-  private AccessConfig accessConfig;
-  private Connection connection;
+  public static String HBASE_TABLE = "update.hbase.table";
+  public static String HBASE_CF = "update.hbase.cf";
+  private HTableInterface tableInterface;
+  private byte[] cf;
+  private AccessConfig config;
 
   /**
    * Implements the HBaseDao row key and exposes convenience methods for serializing/deserializing the row key.
@@ -79,7 +73,6 @@ public class HBaseDao implements IndexDao {
   public static class Key {
     private String guid;
     private String sensorType;
-
     public Key(String guid, String sensorType) {
       this.guid = guid;
       this.sensorType = sensorType;
@@ -139,6 +132,7 @@ public class HBaseDao implements IndexDao {
   }
 
   public HBaseDao() {
+
   }
 
   @Override
@@ -152,72 +146,40 @@ public class HBaseDao implements IndexDao {
   }
 
   @Override
-  public synchronized void init(AccessConfig accessConfig) {
-    if(table == null) {
-      this.accessConfig = accessConfig;
-      Map<String, Object> globals = getGlobals(accessConfig);
-      String tableName = getTableName(globals);
-      columnFamily = Bytes.toBytes(getColumnFamily(globals));
-
+  public synchronized void init(AccessConfig config) {
+    if(this.tableInterface == null) {
+      this.config = config;
+      Map<String, Object> globalConfig = config.getGlobalConfigSupplier().get();
+      if(globalConfig == null) {
+        throw new IllegalStateException("Cannot find the global config.");
+      }
+      String table = (String)globalConfig.get(HBASE_TABLE);
+      String cf = (String) config.getGlobalConfigSupplier().get().get(HBASE_CF);
+      if(table == null || cf == null) {
+        throw new IllegalStateException("You must configure " + HBASE_TABLE + " and " + HBASE_CF + " in the global config.");
+      }
       try {
-        HBaseConnectionFactory connectionFactory = accessConfig.getHbaseConnectionFactory();
-        Configuration conf = accessConfig.getHbaseConfiguration();
-        connection = connectionFactory.createConnection(conf);
-        table = connection.getTable(TableName.valueOf(tableName));
-
+        tableInterface = config.getTableProvider().getTable(HBaseConfiguration.create(), table);
+        this.cf = cf.getBytes();
       } catch (IOException e) {
         throw new IllegalStateException("Unable to initialize HBaseDao: " + e.getMessage(), e);
       }
     }
   }
 
-  @Override
-  public void close() throws IOException {
-    if(table != null) {
-      table.close();
-    }
-    if(connection != null) {
-      connection.close();
-    }
-  }
-
-  public Table getTable() {
-    if(table == null) {
-      init(accessConfig);
-    }
-    return table;
-  }
-
-  private Map<String, Object> getGlobals(AccessConfig accessConfig) {
-    Map<String, Object> globalConfig = accessConfig.getGlobalConfigSupplier().get();
-    if(globalConfig == null) {
-      throw new IllegalStateException("Cannot find the global config.");
-    }
-    return globalConfig;
-  }
-
-  private static String getTableName(Map<String, Object> globalConfig) {
-    String table = (String) globalConfig.get(HBASE_TABLE);
-    if(table == null) {
-      throw new IllegalStateException("You must configure " + HBASE_TABLE + "in the global config.");
-    }
-    return table;
-  }
-
-  private static String getColumnFamily(Map<String, Object> globalConfig) {
-    String cf = (String) globalConfig.get(HBASE_CF);
-    if(cf == null) {
-      throw new IllegalStateException("You must configure " + HBASE_CF + " in the global config.");
+  public HTableInterface getTableInterface() {
+    if(tableInterface == null) {
+      init(config);
     }
-    return cf;
+    return tableInterface;
   }
 
   @Override
   public synchronized Document getLatest(String guid, String sensorType) throws IOException {
     Key k = new Key(guid, sensorType);
     Get get = new Get(Key.toBytes(k));
-    get.addFamily(columnFamily);
-    Result result = getTable().get(get);
+    get.addFamily(cf);
+    Result result = getTableInterface().get(get);
     return getDocumentFromResult(result);
   }
 
@@ -228,7 +190,7 @@ public class HBaseDao implements IndexDao {
     for (GetRequest getRequest: getRequests) {
       gets.add(buildGet(getRequest));
     }
-    Result[] results = getTable().get(gets);
+    Result[] results = getTableInterface().get(gets);
     List<Document> allLatest = new ArrayList<>();
     for (Result result: results) {
       Document d = getDocumentFromResult(result);
@@ -240,7 +202,7 @@ public class HBaseDao implements IndexDao {
   }
 
   private Document getDocumentFromResult(Result result) throws IOException {
-    NavigableMap<byte[], byte[]> columns = result.getFamilyMap(columnFamily);
+    NavigableMap<byte[], byte[]> columns = result.getFamilyMap( cf);
     if(columns == null || columns.size() == 0) {
       return null;
     }
@@ -278,7 +240,7 @@ public class HBaseDao implements IndexDao {
   @Override
   public synchronized Document update(Document update, Optional<String> index) throws IOException {
     Put put = buildPut(update);
-    getTable().put(put);
+    getTableInterface().put(put);
     return update;
   }
 
@@ -291,14 +253,14 @@ public class HBaseDao implements IndexDao {
       Put put = buildPut(update);
       puts.add(put);
     }
-    getTable().put(puts);
+    getTableInterface().put(puts);
     return updates;
   }
 
   protected Get buildGet(GetRequest getRequest) throws IOException {
     Key k = new Key(getRequest.getGuid(), getRequest.getSensorType());
     Get get = new Get(Key.toBytes(k));
-    get.addFamily(columnFamily);
+    get.addFamily(cf);
     return get;
   }
 
@@ -308,7 +270,7 @@ public class HBaseDao implements IndexDao {
     long ts = update.getTimestamp() == null || update.getTimestamp() == 0 ? System.currentTimeMillis() : update.getTimestamp();
     byte[] columnQualifier = Bytes.toBytes(ts);
     byte[] doc = JSONUtils.INSTANCE.toJSONPretty(update.getDocument());
-    put.addColumn(columnFamily, columnQualifier, doc);
+    put.addColumn(cf, columnQualifier, doc);
     return put;
   }
 
diff --git a/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/IndexDao.java b/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/IndexDao.java
index 79b9101..11b2ff0 100644
--- a/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/IndexDao.java
+++ b/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/IndexDao.java
@@ -20,13 +20,11 @@ package org.apache.metron.indexing.dao;
 import org.apache.metron.indexing.dao.search.SearchDao;
 import org.apache.metron.indexing.dao.update.UpdateDao;
 
-import java.io.Closeable;
-
 /**
  * The IndexDao provides a common interface for retrieving and storing data in a variety of persistent stores.
  * Document reads and writes require a GUID and sensor type with an index being optional.
  */
-public interface IndexDao extends UpdateDao, SearchDao, RetrieveLatestDao, ColumnMetadataDao, Closeable {
+public interface IndexDao extends UpdateDao, SearchDao, RetrieveLatestDao, ColumnMetadataDao {
 
   String COMMENTS_FIELD = "comments";
 
diff --git a/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java b/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java
index d762605..a240c56 100644
--- a/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java
+++ b/metron-platform/metron-indexing/metron-indexing-common/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java
@@ -256,13 +256,6 @@ public class MultiIndexDao implements IndexDao {
   }
 
   @Override
-  public void close() throws IOException {
-    for(IndexDao dao : indices) {
-      dao.close();
-    }
-  }
-
-  @Override
   public Document getLatest(final String guid, String sensorType) throws IOException {
     List<DocumentContainer> output = indices
             .parallelStream()
diff --git a/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java b/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java
index 8425798..f49a6ad 100644
--- a/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java
+++ b/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java
@@ -215,11 +215,6 @@ public class InMemoryDao implements IndexDao {
   }
 
   @Override
-  public void close() {
-    // nothing to do
-  }
-
-  @Override
   public Document getLatest(String guid, String sensorType) throws IOException {
     for(Map.Entry<String, List<String>> kv: BACKING_STORE.entrySet()) {
       if(kv.getKey().startsWith(sensorType)) {
diff --git a/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/InMemoryMetaAlertDao.java b/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/InMemoryMetaAlertDao.java
index 6b683da..5dbefdd 100644
--- a/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/InMemoryMetaAlertDao.java
+++ b/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/InMemoryMetaAlertDao.java
@@ -115,12 +115,6 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
     // Ignore threatSort for test.
   }
 
-  @Override
-  public void close() throws IOException {
-    if(indexDao != null) {
-      indexDao.close();
-    }
-  }
 
   @Override
   public Document getLatest(String guid, String sensorType) throws IOException {
diff --git a/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java b/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java
index f9e6be6..6333d32 100644
--- a/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java
+++ b/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java
@@ -17,12 +17,12 @@ package org.apache.metron.indexing.dao;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.commons.collections.MapUtils;
 import org.apache.metron.common.Constants;
+import org.apache.metron.hbase.mock.MockHTable;
 import org.apache.metron.indexing.dao.search.AlertComment;
 import org.apache.metron.indexing.dao.update.CommentAddRemoveRequest;
 import org.apache.metron.indexing.dao.update.Document;
 import org.apache.metron.indexing.dao.update.OriginalNotFoundException;
 import org.apache.metron.indexing.dao.update.PatchRequest;
-import org.json.simple.parser.ParseException;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -35,9 +35,8 @@ import java.util.Optional;
 import java.util.UUID;
 import java.util.stream.Collectors;
 
-import static org.hamcrest.CoreMatchers.hasItem;
-
 import static org.apache.metron.indexing.dao.IndexDao.COMMENTS_FIELD;
+import static org.hamcrest.CoreMatchers.hasItem;
 
 public abstract class UpdateIntegrationTest {
 
@@ -273,81 +272,29 @@ public abstract class UpdateIntegrationTest {
     return new Document(message1, guid, SENSOR_NAME, timestamp);
   }
 
-  private static List<AlertComment> getComments(Document withComment) throws ParseException {
-    return getComments(withComment.getDocument());
-  }
-
-  private static List<AlertComment> getComments(Map<String, Object> fields) throws ParseException {
+  private List<AlertComment> getComments(Document withComment) {
+    List<Map<String, Object>> commentsField = List.class.cast(withComment.getDocument().get(COMMENTS_FIELD));
     List<AlertComment> comments = new ArrayList<>();
-    boolean hasComments = fields.containsKey(COMMENTS_FIELD);
-    if(hasComments) {
-      List<Object> commentsField = List.class.cast(fields.get(COMMENTS_FIELD));
-      for (Object commentObject: commentsField) {
-        if (commentObject instanceof Map) {
-          // comments are stored as maps in Elasticsearch
-          Map<String, Object> commentAsMap = (Map<String, Object>) commentObject;
-          comments.add(new AlertComment(commentAsMap));
-
-        } else if (commentObject instanceof String) {
-          // comments are stored as json strings in Solr
-          String commentAsString = (String) commentObject;
-          comments.add(new AlertComment(commentAsString));
-
-        } else {
-          throw new IllegalArgumentException(String.format("Unexpected comment value; %s", commentObject));
-        }
-      }
+    if(commentsField != null) {
+      comments = commentsField
+              .stream()
+              .map(map -> new AlertComment(map))
+              .collect(Collectors.toList());
     }
 
     return comments;
   }
 
-  /**
-   * Reformats the format of stored comments.
-   *
-   * <p>Comments are serialized differently when stored in Elasticsearch and Solr.  Comments
-   * are stored as maps in Elasticsearch and JSON strings in Solr.  This reformats all comments
-   * as maps, so they look the same when validation occurs in the integration tests.
-   * @param fields
-   */
-  protected static void reformatComments(Map<String, Object> fields) {
-    @SuppressWarnings("unchecked")
-    List<Object> commentValues = (List<Object>) fields.get(COMMENTS_FIELD);
-    if (commentValues != null) {
-      try {
-        List<AlertComment> comments = getComments(fields);
-        if(comments.size() > 0) {
-          // overwrite the comments field
-          List<Map<String, Object>> serializedComments = comments
-                  .stream()
-                  .map(AlertComment::asMap)
-                  .collect(Collectors.toList());
-          fields.put(COMMENTS_FIELD, serializedComments);
-
-        } else {
-          // there are no longer any comments
-          fields.remove(COMMENTS_FIELD);
-        }
-
-      } catch (ParseException e) {
-        throw new IllegalStateException("Unable to parse comment", e);
-      }
-    }
-  }
-
-  protected Document findUpdatedDoc(Map<String, Object> expected, String guid, String sensorType)
+  protected Document findUpdatedDoc(Map<String, Object> message0, String guid, String sensorType)
       throws InterruptedException, IOException, OriginalNotFoundException {
-    // comments are stored differently in Solr and Elasticsearch
-    reformatComments(expected);
-
     for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) {
-      Document found = getDao().getLatest(guid, sensorType);
-      if (found != null && expected.equals(found.getDocument())) {
-        return found;
+      Document doc = getDao().getLatest(guid, sensorType);
+      if (doc != null && message0.equals(doc.getDocument())) {
+        return doc;
       }
       if (t == MAX_RETRIES -1) {
-        MapUtils.debugPrint(System.out, "Expected", expected);
-        MapUtils.debugPrint(System.out, "Actual", found.getDocument());
+        MapUtils.debugPrint(System.out, "Expected", message0);
+        MapUtils.debugPrint(System.out, "actual", doc.getDocument());
       }
     }
     throw new OriginalNotFoundException("Count not find " + guid + " after " + MAX_RETRIES + " tries");
@@ -362,6 +309,7 @@ public abstract class UpdateIntegrationTest {
   }
 
   protected abstract String getIndexName();
+  protected abstract MockHTable getMockHTable();
   protected abstract void addTestData(String indexName, String sensorType, List<Map<String,Object>> docs) throws Exception;
   protected abstract List<Map<String,Object>> getIndexedTestData(String indexName, String sensorType) throws Exception;
 }
diff --git a/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/integration/HBaseDaoIntegrationTest.java b/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/integration/HBaseDaoIntegrationTest.java
index d1d0707..8148b69 100644
--- a/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/integration/HBaseDaoIntegrationTest.java
+++ b/metron-platform/metron-indexing/metron-indexing-common/src/test/java/org/apache/metron/indexing/integration/HBaseDaoIntegrationTest.java
@@ -18,15 +18,20 @@
 
 package org.apache.metron.indexing.integration;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.metron.hbase.client.HBaseConnectionFactory;
+import static org.apache.metron.indexing.dao.HBaseDao.HBASE_CF;
+import static org.apache.metron.indexing.dao.HBaseDao.HBASE_TABLE;
+import static org.apache.metron.indexing.dao.IndexDao.COMMENTS_FIELD;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.metron.hbase.mock.MockHBaseTableProvider;
+import org.apache.metron.hbase.mock.MockHTable;
 import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.HBaseDao;
 import org.apache.metron.indexing.dao.IndexDao;
@@ -35,34 +40,18 @@ import org.apache.metron.indexing.dao.search.AlertComment;
 import org.apache.metron.indexing.dao.search.GetRequest;
 import org.apache.metron.indexing.dao.update.Document;
 import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.stream.Collectors;
-
-import static org.apache.metron.indexing.dao.HBaseDao.HBASE_CF;
-import static org.apache.metron.indexing.dao.HBaseDao.HBASE_TABLE;
-import static org.apache.metron.indexing.dao.IndexDao.COMMENTS_FIELD;
-
-/**
- * An integration test for the {@link HBaseDao}.
- */
 public class HBaseDaoIntegrationTest extends UpdateIntegrationTest  {
 
   private static final String TABLE_NAME = "metron_update";
   private static final String COLUMN_FAMILY = "cf";
   private static final String SENSOR_TYPE = "test";
-  private static final byte[] expectedKeySerialization = new byte[] {
+
+  private static IndexDao hbaseDao;
+  private static byte[] expectedKeySerialization = new byte[] {
           (byte)0xf5,0x53,0x76,(byte)0x96,0x67,0x3a,
           (byte)0xc1,(byte)0xaf,(byte)0xff,0x41,0x33,(byte)0x9d,
           (byte)0xac,(byte)0xb9,0x1a,(byte)0xb0,0x00,0x04,
@@ -71,43 +60,17 @@ public class HBaseDaoIntegrationTest extends UpdateIntegrationTest  {
           0x54,0x79,0x70,0x65
   };
 
-  private static HBaseTestingUtility testUtil;
-  private static Configuration conf;
-  private static Table table;
-  private IndexDao hbaseDao;
-
-  @BeforeClass
-  public static void startHBase() throws Exception {
-    Configuration config = HBaseConfiguration.create();
-    config.set("hbase.master.hostname", "localhost");
-    config.set("hbase.regionserver.hostname", "localhost");
-
-    testUtil = new HBaseTestingUtility(config);
-    testUtil.startMiniCluster();
-
-    // create the table
-    table = testUtil.createTable(TableName.valueOf(TABLE_NAME), COLUMN_FAMILY);
-    testUtil.waitTableEnabled(table.getName());
-  }
-
-  @AfterClass
-  public static void stopHBase() throws Exception {
-    testUtil.deleteTable(table.getName());
-    testUtil.shutdownMiniCluster();
-    testUtil.cleanupTestDir();
-  }
-
   @Before
-  public void setup() throws Exception {
+  public void startHBase() throws Exception {
     AccessConfig accessConfig = new AccessConfig();
-    accessConfig.setHbaseConnectionFactory(new HBaseConnectionFactory());
-    accessConfig.setHbaseConfiguration(testUtil.getConfiguration());
     accessConfig.setMaxSearchResults(1000);
     accessConfig.setMaxSearchGroups(1000);
     accessConfig.setGlobalConfigSupplier(() -> new HashMap<String, Object>() {{
       put(HBASE_TABLE, TABLE_NAME);
       put(HBASE_CF, COLUMN_FAMILY);
     }});
+    MockHBaseTableProvider.addToCache(TABLE_NAME, COLUMN_FAMILY);
+    accessConfig.setTableProvider(new MockHBaseTableProvider());
 
     hbaseDao = new HBaseDao();
     hbaseDao.init(accessConfig);
@@ -115,11 +78,7 @@ public class HBaseDaoIntegrationTest extends UpdateIntegrationTest  {
 
   @After
   public void clearTable() throws Exception {
-    // clear all records from the the table
-    List<Delete> deletions = new ArrayList<>();
-    for(Result r : table.getScanner(new Scan())) {
-      deletions.add(new Delete(r.getRow()));
-    }
+    MockHBaseTableProvider.clear();
   }
 
   /**
@@ -136,6 +95,7 @@ public class HBaseDaoIntegrationTest extends UpdateIntegrationTest  {
     Assert.assertArrayEquals(raw, expectedKeySerialization);
   }
 
+
   @Test
   public void testKeySerialization() throws Exception {
     HBaseDao.Key k = new HBaseDao.Key("guid", "sensorType");
@@ -271,6 +231,11 @@ public class HBaseDaoIntegrationTest extends UpdateIntegrationTest  {
   }
 
   @Override
+  protected MockHTable getMockHTable() {
+    return null;
+  }
+
+  @Override
   protected void addTestData(String indexName, String sensorType, List<Map<String, Object>> docs) {
   }
 
diff --git a/metron-platform/metron-pcap/pom.xml b/metron-platform/metron-pcap/pom.xml
index 02180fb..eb891b1 100644
--- a/metron-platform/metron-pcap/pom.xml
+++ b/metron-platform/metron-pcap/pom.xml
@@ -55,6 +55,16 @@
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpcore</artifactId>
+            <version>${global_httpclient_version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.httpcomponents</groupId>
+            <artifactId>httpclient</artifactId>
+            <version>${global_httpclient_version}</version>
+        </dependency>
+        <dependency>
             <groupId>com.fasterxml.jackson.core</groupId>
             <artifactId>jackson-core</artifactId>
             <version>${global_jackson_version}</version>
@@ -97,6 +107,14 @@
                     <artifactId>commons-beanutils</artifactId>
                     <groupId>commons-beanutils</groupId>
                 </exclusion>
+                <exclusion>
+                    <artifactId>httpcore</artifactId>
+                    <groupId>org.apache.httpcomponents</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>httpclient</artifactId>
+                    <groupId>org.apache.httpcomponents</groupId>
+                </exclusion>
             </exclusions>
         </dependency>
         <dependency>
@@ -131,6 +149,16 @@
             <groupId>org.apache.metron</groupId>
             <artifactId>metron-storm-kafka</artifactId>
             <version>${project.parent.version}</version>
+            <exclusions>
+                <exclusion>
+                    <artifactId>httpcore</artifactId>
+                    <groupId>org.apache.httpcomponents</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>httpclient</artifactId>
+                    <groupId>org.apache.httpcomponents</groupId>
+                </exclusion>
+            </exclusions>
         </dependency>
         <dependency>
             <groupId>org.apache.metron</groupId>
@@ -175,10 +203,5 @@
                 </exclusion>
             </exclusions>
         </dependency>
-        <dependency>
-            <groupId>org.apache.httpcomponents</groupId>
-            <artifactId>httpclient</artifactId>
-            <version>${global_httpclient_version}</version>
-        </dependency>
     </dependencies>
 </project>
diff --git a/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrDao.java b/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrDao.java
index 0ef67a0..c37b93e 100644
--- a/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrDao.java
+++ b/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrDao.java
@@ -91,11 +91,6 @@ public class SolrDao implements IndexDao {
     }
   }
 
-  @Override
-  public void close() {
-    // nothing to do
-  }
-
   public Optional<String> getIndex(String sensorName, Optional<String> index) {
     if (index.isPresent()) {
       return index;
diff --git a/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertDao.java b/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertDao.java
index 1e19648..26dc25d 100644
--- a/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertDao.java
+++ b/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertDao.java
@@ -161,16 +161,6 @@ public class SolrMetaAlertDao implements MetaAlertDao {
   }
 
   @Override
-  public void close() throws IOException {
-    if(indexDao != null) {
-      indexDao.close();
-    }
-    if(solrDao != null) {
-      solrDao.close();
-    }
-  }
-
-  @Override
   public void init(AccessConfig config) {
     // Do nothing. We're just wrapping a child dao
   }
diff --git a/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java b/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java
index eb2ff06..d41b7e4 100644
--- a/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java
+++ b/metron-platform/metron-solr/metron-solr-common/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java
@@ -64,9 +64,8 @@ public class SolrUtilities {
     insertChildAlerts(solrDocument, document);
 
     return new Document(document,
-            (String) solrDocument.getFieldValue(Constants.GUID),
-            (String) solrDocument.getFieldValue(Constants.SENSOR_TYPE),
-            (Long) solrDocument.getFieldValue(Constants.Fields.TIMESTAMP.getName()));
+        (String) solrDocument.getFieldValue(Constants.GUID),
+        (String) solrDocument.getFieldValue(Constants.SENSOR_TYPE), 0L);
   }
 
   protected static void reformatComments(SolrDocument solrDocument, Map<String, Object> document) {
diff --git a/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java b/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java
index 0adcf49..6817299 100644
--- a/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java
+++ b/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java
@@ -95,10 +95,6 @@ public class SolrMetaAlertDaoTest {
       }
 
       @Override
-      public void close() throws IOException {
-      }
-
-      @Override
       public Document getLatest(String guid, String sensorType) {
         return null;
       }
diff --git a/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrSearchDaoTest.java b/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrSearchDaoTest.java
index f4ef16b..bf9458f 100644
--- a/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrSearchDaoTest.java
+++ b/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrSearchDaoTest.java
@@ -38,8 +38,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.UUID;
-
 import org.apache.metron.common.Constants;
 import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.search.GetRequest;
@@ -221,7 +219,7 @@ public class SolrSearchDaoTest {
 
   @Test
   public void getLatestShouldProperlyReturnDocument() throws Exception {
-    SolrDocument solrDocument = createSolrDocument("bro", 123456789L);
+    SolrDocument solrDocument = mock(SolrDocument.class);
 
     solrSearchDao = spy(new SolrSearchDao(client, accessConfig));
     when(client.getById("collection", "guid")).thenReturn(solrDocument);
@@ -239,10 +237,10 @@ public class SolrSearchDaoTest {
     GetRequest broRequest2 = new GetRequest("bro-2", "bro");
     GetRequest snortRequest1 = new GetRequest("snort-1", "snort");
     GetRequest snortRequest2 = new GetRequest("snort-2", "snort");
-    SolrDocument broSolrDoc1 = createSolrDocument("bro", 12345L);
-    SolrDocument broSolrDoc2 = createSolrDocument("bro", 34567L);
-    SolrDocument snortSolrDoc1 = createSolrDocument("snort", 12345L);
-    SolrDocument snortSolrDoc2 = createSolrDocument("snort", 67890L);
+    SolrDocument broSolrDoc1 = mock(SolrDocument.class);
+    SolrDocument broSolrDoc2 = mock(SolrDocument.class);
+    SolrDocument snortSolrDoc1 = mock(SolrDocument.class);
+    SolrDocument snortSolrDoc2 = mock(SolrDocument.class);
     Document broDoc1 = SolrUtilities.toDocument(broSolrDoc1);
     Document broDoc2 = SolrUtilities.toDocument(broSolrDoc2);
     Document snortDoc1 = SolrUtilities.toDocument(snortSolrDoc1);
@@ -512,13 +510,5 @@ public class SolrSearchDaoTest {
     assertNull(level2GroupResults.get(1).getGroupResults());
   }
 
-  private SolrDocument createSolrDocument(String sensorType, Long timestamp) {
-    SolrDocument solrDocument = new SolrDocument();
-    solrDocument.addField(SolrDao.VERSION_FIELD, 1.0);
-    solrDocument.addField(Constants.GUID, UUID.randomUUID().toString());
-    solrDocument.addField(Constants.SENSOR_TYPE, sensorType);
-    solrDocument.addField(Constants.Fields.TIMESTAMP.getName(), timestamp);
-    solrDocument.addField("ip_src_addr", "192.168.1.1");
-    return solrDocument;
-  }
+
 }
diff --git a/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrUtilitiesTest.java b/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrUtilitiesTest.java
index 89441c0..f284f25 100644
--- a/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrUtilitiesTest.java
+++ b/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/dao/SolrUtilitiesTest.java
@@ -30,20 +30,17 @@ public class SolrUtilitiesTest {
 
   @Test
   public void toDocumentShouldProperlyReturnDocument() throws Exception {
-    long expectedTimestamp = System.currentTimeMillis();
     SolrDocument solrDocument = new SolrDocument();
     solrDocument.addField(SolrDao.VERSION_FIELD, 1.0);
     solrDocument.addField(Constants.GUID, "guid");
     solrDocument.addField(Constants.SENSOR_TYPE, "bro");
-    solrDocument.addField(Constants.Fields.TIMESTAMP.getName(), expectedTimestamp);
     solrDocument.addField("field", "value");
 
     Document expectedDocument = new Document(new HashMap<String, Object>() {{
       put("field", "value");
       put(Constants.GUID, "guid");
       put(Constants.SENSOR_TYPE, "bro");
-      put(Constants.Fields.TIMESTAMP.getName(), expectedTimestamp);
-    }}, "guid", "bro", expectedTimestamp);
+    }}, "guid", "bro", 0L);
 
     Document actualDocument = SolrUtilities.toDocument(solrDocument);
     assertEquals(expectedDocument, actualDocument);
diff --git a/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java b/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java
index d147202..8c18981 100644
--- a/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java
+++ b/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java
@@ -51,7 +51,7 @@ public class SolrRetrieveLatestIntegrationTest {
   protected static final String TEST_COLLECTION = "test";
   protected static final String TEST_SENSOR = "test_sensor";
   protected static final String BRO_SENSOR = "bro";
-  protected final long expectedTimestamp = 123456789L;
+
   private static IndexDao dao;
 
   @BeforeClass
@@ -62,7 +62,8 @@ public class SolrRetrieveLatestIntegrationTest {
 
   @Before
   public void setup() throws Exception {
-    solrComponent.addCollection(TEST_COLLECTION, "./src/test/resources/config/test/conf");
+    solrComponent
+        .addCollection(TEST_COLLECTION, "./src/test/resources/config/test/conf");
     solrComponent.addCollection(BRO_SENSOR, "./src/main/config/schema/bro");
 
     AccessConfig accessConfig = new AccessConfig();
@@ -74,8 +75,8 @@ public class SolrRetrieveLatestIntegrationTest {
 
     dao = new SolrDao();
     dao.init(accessConfig);
-    addData(BRO_SENSOR, BRO_SENSOR, expectedTimestamp);
-    addData(TEST_COLLECTION, TEST_SENSOR, expectedTimestamp);
+    addData(BRO_SENSOR, BRO_SENSOR);
+    addData(TEST_COLLECTION, TEST_SENSOR);
   }
 
   @After
@@ -130,11 +131,8 @@ public class SolrRetrieveLatestIntegrationTest {
     requests.add(buildGetRequest(BRO_SENSOR, 2));
 
     Iterable<Document> actual = dao.getAllLatest(requests);
-    Document expected1 = buildExpectedDocument(BRO_SENSOR, 1);
-    assertTrue(Iterables.contains(actual, expected1));
-
-    Document expected2 = buildExpectedDocument(BRO_SENSOR, 2);
-    assertTrue(Iterables.contains(actual, expected2));
+    assertTrue(Iterables.contains(actual, buildExpectedDocument(BRO_SENSOR, 1)));
+    assertTrue(Iterables.contains(actual, buildExpectedDocument(BRO_SENSOR, 2)));
     assertEquals(2, Iterables.size(actual));
   }
 
@@ -181,9 +179,8 @@ public class SolrRetrieveLatestIntegrationTest {
   protected Document buildExpectedDocument(String sensor, int i) {
     Map<String, Object> expectedMapOne = new HashMap<>();
     expectedMapOne.put("source.type", sensor);
-    expectedMapOne.put(Constants.Fields.TIMESTAMP.getName(), expectedTimestamp);
     expectedMapOne.put(Constants.GUID, buildGuid(sensor, i));
-    return new Document(expectedMapOne, buildGuid(sensor, i), sensor, expectedTimestamp);
+    return new Document(expectedMapOne, buildGuid(sensor, i), sensor, 0L);
   }
 
   protected GetRequest buildGetRequest(String sensor, int i) {
@@ -193,7 +190,7 @@ public class SolrRetrieveLatestIntegrationTest {
     return requestOne;
   }
 
-  protected static void addData(String collection, String sensorName, Long timestamp)
+  protected static void addData(String collection, String sensorName)
       throws IOException, SolrServerException {
     List<Map<String, Object>> inputData = new ArrayList<>();
     for (int i = 0; i < 3; ++i) {
@@ -201,7 +198,6 @@ public class SolrRetrieveLatestIntegrationTest {
       HashMap<String, Object> inputMap = new HashMap<>();
       inputMap.put("source.type", sensorName);
       inputMap.put(Constants.GUID, name);
-      inputMap.put(Constants.Fields.TIMESTAMP.getName(), timestamp);
       inputData.add(inputMap);
     }
     solrComponent.addDocs(collection, inputData);
diff --git a/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java b/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java
index 852ca10..d49f4b4 100644
--- a/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java
+++ b/metron-platform/metron-solr/metron-solr-common/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java
@@ -17,19 +17,32 @@
  */
 package org.apache.metron.solr.integration;
 
+import static org.apache.metron.solr.SolrConstants.SOLR_ZOOKEEPER;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.metron.common.configuration.ConfigurationsUtils;
 import org.apache.metron.common.zookeeper.ZKConfigurationsCache;
+import org.apache.metron.hbase.mock.MockHBaseTableProvider;
+import org.apache.metron.hbase.mock.MockHTable;
 import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.HBaseDao;
+import org.apache.metron.indexing.dao.IndexDao;
+import org.apache.metron.indexing.dao.MultiIndexDao;
 import org.apache.metron.indexing.dao.UpdateIntegrationTest;
 import org.apache.metron.indexing.dao.update.Document;
 import org.apache.metron.indexing.util.IndexingCacheUtil;
 import org.apache.metron.solr.client.SolrClientFactory;
 import org.apache.metron.solr.dao.SolrDao;
 import org.apache.metron.solr.integration.components.SolrComponent;
-import org.apache.solr.common.SolrException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -38,15 +51,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import static org.apache.metron.solr.SolrConstants.SOLR_ZOOKEEPER;
-import static org.junit.Assert.assertEquals;
-
 public class SolrUpdateIntegrationTest extends UpdateIntegrationTest {
   @Rule
   public final ExpectedException exception = ExpectedException.none();
@@ -55,6 +59,8 @@ public class SolrUpdateIntegrationTest extends UpdateIntegrationTest {
 
   private static final String TABLE_NAME = "modifications";
   private static final String CF = "p";
+  private static MockHTable table;
+  private static IndexDao hbaseDao;
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -67,21 +73,28 @@ public class SolrUpdateIntegrationTest extends UpdateIntegrationTest {
     solrComponent.addCollection(SENSOR_NAME, "./src/test/resources/config/test/conf");
     solrComponent.addCollection("error", "./src/main/config/schema/error");
 
+    Configuration config = HBaseConfiguration.create();
+    MockHBaseTableProvider tableProvider = new MockHBaseTableProvider();
+    MockHBaseTableProvider.addToCache(TABLE_NAME, CF);
+    table = (MockHTable) tableProvider.getTable(config, TABLE_NAME);
+
+    hbaseDao = new HBaseDao();
+    AccessConfig accessConfig = new AccessConfig();
+    accessConfig.setTableProvider(tableProvider);
     Map<String, Object> globalConfig = createGlobalConfig();
     globalConfig.put(HBaseDao.HBASE_TABLE, TABLE_NAME);
     globalConfig.put(HBaseDao.HBASE_CF, CF);
+    accessConfig.setGlobalConfigSupplier(() -> globalConfig);
+    accessConfig.setIndexSupplier(s -> s);
 
-    CuratorFramework client = ConfigurationsUtils.getClient(solrComponent.getZookeeperUrl());
+    CuratorFramework client = ConfigurationsUtils
+        .getClient(solrComponent.getZookeeperUrl());
     client.start();
     ZKConfigurationsCache cache = new ZKConfigurationsCache(client);
     cache.start();
-
-    AccessConfig accessConfig = new AccessConfig();
-    accessConfig.setGlobalConfigSupplier(() -> globalConfig);
-    accessConfig.setIndexSupplier(s -> s);
     accessConfig.setIndexSupplier(IndexingCacheUtil.getIndexLookupFunction(cache, "solr"));
 
-    SolrDao dao = new SolrDao();
+    MultiIndexDao dao = new MultiIndexDao(hbaseDao, new SolrDao());
     dao.init(accessConfig);
     setDao(dao);
   }
@@ -89,6 +102,7 @@ public class SolrUpdateIntegrationTest extends UpdateIntegrationTest {
   @After
   public void reset() {
     solrComponent.reset();
+    table.clear();
   }
 
   @AfterClass
@@ -102,6 +116,11 @@ public class SolrUpdateIntegrationTest extends UpdateIntegrationTest {
     return SENSOR_NAME;
   }
 
+  @Override
+  protected MockHTable getMockHTable() {
+    return table;
+  }
+
   private static Map<String, Object> createGlobalConfig() {
     return new HashMap<String, Object>() {{
       put(SOLR_ZOOKEEPER, solrComponent.getZookeeperUrl());
@@ -165,9 +184,8 @@ public class SolrUpdateIntegrationTest extends UpdateIntegrationTest {
     documentMap.put("error_hash", hugeString);
     errorDoc = new Document(documentMap, "error", "error", 0L);
 
-    exception.expect(SolrException.class);
+    exception.expect(IOException.class);
     exception.expectMessage("Document contains at least one immense term in field=\"error_hash\"");
-
     getDao().update(errorDoc, Optional.of("error"));
   }
 }
diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/StellarFunctionInfo.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/StellarFunctionInfo.java
index 042706d..8606723 100644
--- a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/StellarFunctionInfo.java
+++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/StellarFunctionInfo.java
@@ -77,11 +77,6 @@ public class StellarFunctionInfo {
     return function;
   }
 
-  public StellarFunctionInfo setFunction(StellarFunction function) {
-    this.function = function;
-    return this;
-  }
-
   @Override
   public boolean equals(Object o) {
     if (this == o) return true;
@@ -95,6 +90,7 @@ public class StellarFunctionInfo {
     // Probably incorrect - comparing Object[] arrays with Arrays.equals
     if (!Arrays.equals(params, that.params)) return false;
     return function != null ? function.equals(that.function) : that.function == null;
+
   }
 
   @Override
diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/BaseFunctionResolver.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/BaseFunctionResolver.java
index 6fd669b..38a32d1 100644
--- a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/BaseFunctionResolver.java
+++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/BaseFunctionResolver.java
@@ -52,7 +52,7 @@ public abstract class BaseFunctionResolver implements FunctionResolver, Serializ
   /**
    * Maps a function name to the metadata necessary to execute the Stellar function.
    */
-  protected Supplier<Map<String, StellarFunctionInfo>> functions;
+  private Supplier<Map<String, StellarFunctionInfo>> functions;
 
   /**
    * The Stellar execution context that can be used to inform the function resolution process.
@@ -145,7 +145,6 @@ public abstract class BaseFunctionResolver implements FunctionResolver, Serializ
    */
   @Override
   public StellarFunction apply(String functionName) {
-    LOG.debug("Resolving function; functionName={}", functionName);
     StellarFunctionInfo info = functions.get().get(functionName);
     if(info == null) {
       throw new IllegalStateException(format("Unknown function: `%s`", functionName));
@@ -157,6 +156,7 @@ public abstract class BaseFunctionResolver implements FunctionResolver, Serializ
    * Performs the core process of function resolution.
    */
   protected Map<String, StellarFunctionInfo> resolveFunctions() {
+
     // maps a function name to its definition
     Map<String, StellarFunctionInfo> functions = new HashMap<>();
 
@@ -242,18 +242,4 @@ public abstract class BaseFunctionResolver implements FunctionResolver, Serializ
       return null;
     }
   }
-
-  @Override
-  public BaseFunctionResolver withInstance(StellarFunction function) {
-    // perform function resolution on the instance that was passed in
-    StellarFunctionInfo functionInfo = resolveFunction(function.getClass());
-    functionInfo.setFunction(function);
-
-    // add the function to the set of resolvable functions
-    Map<String, StellarFunctionInfo> currentFunctions = this.functions.get();
-    currentFunctions.put(functionInfo.getName(), functionInfo);
-
-    this.functions = Suppliers.memoize(() -> currentFunctions);
-    return this;
-  }
 }
diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/FunctionResolver.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/FunctionResolver.java
index 38eb067..4047586 100644
--- a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/FunctionResolver.java
+++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/FunctionResolver.java
@@ -17,13 +17,12 @@
  */
 package org.apache.metron.stellar.dsl.functions.resolver;
 
-import org.apache.metron.stellar.dsl.Context;
-import org.apache.metron.stellar.dsl.StellarFunction;
-import org.apache.metron.stellar.dsl.StellarFunctionInfo;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.function.Function;
+import org.apache.metron.stellar.dsl.Context;
+import org.apache.metron.stellar.dsl.StellarFunction;
+import org.apache.metron.stellar.dsl.StellarFunctionInfo;
 
 /**
  * Responsible for function resolution in Stellar.
@@ -47,16 +46,6 @@ public interface FunctionResolver extends Function<String, StellarFunction>, Clo
   void initialize(Context context);
 
   /**
-   * Attempts to resolve a function defined within the provided {@link StellarFunction}
-   * instance.
-   *
-   * <p>This can be useful for instrumenting a Stellar function before it is tested.
-   *
-   * @param function The Stellar function to resolve.
-   */
-  FunctionResolver withInstance(StellarFunction function);
-
-    /**
    * Perform any cleanup necessary for the loaded Stellar functions.
    */
   @Override
diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/SimpleFunctionResolver.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/SimpleFunctionResolver.java
index a5a5b0c..d2d0e62 100644
--- a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/SimpleFunctionResolver.java
+++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/dsl/functions/resolver/SimpleFunctionResolver.java
@@ -18,16 +18,12 @@
 
 package org.apache.metron.stellar.dsl.functions.resolver;
 
-import com.google.common.base.Suppliers;
-import org.apache.metron.stellar.dsl.StellarFunction;
-import org.apache.metron.stellar.dsl.StellarFunctionInfo;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.lang.invoke.MethodHandles;
 import java.util.HashSet;
-import java.util.Map;
 import java.util.Set;
+import org.apache.metron.stellar.dsl.StellarFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A simple Stellar function resolver that resolves functions from specific
@@ -53,8 +49,7 @@ public class SimpleFunctionResolver extends BaseFunctionResolver {
   }
 
   /**
-   * Attempts to resolve any functions defined within a specific class.
-   *
+   * Will attempt to resolve any Stellar functions defined within the specified class.
    * @param clazz The class which may contain a Stellar function.
    */
   public SimpleFunctionResolver withClass(Class<? extends StellarFunction> clazz) {