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) {