From e2bd987f1df94174ec60f33c30efa09a0a3c4ace Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 23 Aug 2016 13:03:15 -0400 Subject: [PATCH 01/10] METRON-389 Create Java API to Read Profile Data During Model Scoring --- .../metron-profiler-client/pom.xml | 291 +++++++++++++++ .../src/main/assembly/assembly.xml | 37 ++ .../metron/profiler/HBaseProfilerClient.java | 118 ++++++ .../metron/profiler/ProfilerClient.java | 45 +++ .../profiler/HBaseProfilerClientTest.java | 215 +++++++++++ .../src/test/resources/log4j.properties | 28 ++ metron-analytics/metron-profiler/README.md | 108 +++++- metron-analytics/metron-profiler/pom.xml | 123 +++---- .../src/main/config/profiler.properties | 4 +- .../src/main/flux/profiler/remote.yaml | 25 +- .../metron/profiler/ProfileMeasurement.java | 73 ++-- .../apache/metron/profiler/ProfilePeriod.java | 197 ++++++++++ .../profiler/bolt/ProfileBuilderBolt.java | 67 ++-- .../profiler/bolt/ProfileHBaseMapper.java | 154 ++++---- .../profiler/bolt/ProfileSplitterBolt.java | 18 +- .../metron/profiler/hbase/ColumnBuilder.java | 45 +++ .../metron/profiler/hbase/RowKeyBuilder.java | 60 +++ .../profiler/hbase/SaltyRowKeyBuilder.java | 211 +++++++++++ .../metron/profiler/hbase/Serializer.java | 92 +++++ .../hbase/ValueOnlyColumnBuilder.java | 60 +++ .../stellar/DefaultStellarExecutor.java | 34 +- .../profiler/stellar/StellarExecutor.java | 15 +- .../metron/profiler/ProfilePeriodTest.java | 341 ++++++++++++++++++ .../profiler/bolt/ProfileBuilderBoltTest.java | 1 + .../profiler/bolt/ProfileHBaseMapperTest.java | 114 ++++++ .../hbase/SaltyRowKeyBuilderTest.java | 275 ++++++++++++++ .../metron/profiler/hbase/SerializerTest.java | 71 ++++ .../integration/ProfilerIntegrationTest.java | 19 +- .../DefaultStellarExecutorTest.java | 30 +- metron-analytics/pom.xml | 1 + .../configuration/profiler/ProfileConfig.java | 44 ++- .../metron/common/dsl/StellarFunctions.java | 8 + .../common/dsl/functions/DateFunctions.java | 219 +++++++++-- .../functions/StellarStatisticsFunctions.java | 105 ++---- .../common/stellar/DateFunctionsTest.java | 104 ++++++ .../StellarStatisticsFunctionsTest.java | 76 ++-- .../metron/hbase/client/HBaseClientTest.java | 26 +- .../apache/metron/test/mock/MockHTable.java | 3 +- 38 files changed, 2976 insertions(+), 481 deletions(-) create mode 100644 metron-analytics/metron-profiler-client/pom.xml create mode 100644 metron-analytics/metron-profiler-client/src/main/assembly/assembly.xml create mode 100644 metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/HBaseProfilerClient.java create mode 100644 metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/ProfilerClient.java create mode 100644 metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/HBaseProfilerClientTest.java create mode 100644 metron-analytics/metron-profiler-client/src/test/resources/log4j.properties create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/Serializer.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/hbase/SerializerTest.java rename metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/{util => stellar}/DefaultStellarExecutorTest.java (80%) create mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/DateFunctionsTest.java diff --git a/metron-analytics/metron-profiler-client/pom.xml b/metron-analytics/metron-profiler-client/pom.xml new file mode 100644 index 0000000000..f27abba815 --- /dev/null +++ b/metron-analytics/metron-profiler-client/pom.xml @@ -0,0 +1,291 @@ + + + + + 4.0.0 + + org.apache.metron + metron-analytics + 0.2.0BETA + + metron-profiler-client + + UTF-8 + UTF-8 + + + + org.apache.metron + metron-profiler + ${project.parent.version} + + + com.google.guava + guava + + + + + org.apache.metron + metron-hbase + ${project.parent.version} + + + org.apache.metron + metron-test-utilities + ${project.parent.version} + test + + + org.apache.hadoop + hadoop-auth + ${global_hadoop_version} + + + org.apache.hadoop + hadoop-client + ${global_hadoop_version} + + + org.apache.hadoop + hadoop-common + tests + ${global_hadoop_version} + + + org.apache.hadoop + hadoop-hdfs + tests + ${global_hadoop_version} + + + org.apache.hbase + hbase-client + ${global_hbase_version} + provided + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + com.google.guava + guava + + + + + org.apache.storm + storm-core + ${global_storm_version} + provided + + + servlet-api + javax.servlet + + + log4j-over-slf4j + org.slf4j + + + disruptor + com.googlecode.disruptor + + + + + org.apache.storm + storm-hbase + ${global_storm_version} + provided + + + servlet-api + javax.servlet + + + log4j-over-slf4j + org.slf4j + + + + + org.apache.hbase + hbase-server + ${global_hbase_version} + test + + + org.apache.hbase + hbase-testing-util + ${global_hbase_version} + test + + + org.mockito + mockito-all + ${global_mockito_version} + test + + + + + + + org.apache.maven.plugins + maven-project-info-reports-plugin + 2.7 + + + false + + + + + org.codehaus.mojo + emma-maven-plugin + 1.0-alpha-3 + true + + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.12.4 + + + -Xmx2048m + true + + false + + + + unit-tests + test + + test + + + + false + + + **/*Test.java + + + + **/*IntegrationTest.java + + + + + + integration-tests + integration-test + + test + + + + false + + + **/*IntegrationTest.java + + + + + + + org.apache.maven.plugins + maven-shade-plugin + ${global_shade_version} + + true + + + + package + + shade + + + + + com.lmax + org.apache.metron.lmax.metron-profiler + + + com.google.common + org.apache.metron.guava.metron-profiler + + + com.fasterxml.jackson.core + com.fasterxml.jackson.core.metron.elasticsearch + + + + + storm:storm-core:* + storm:storm-lib:* + org.slf4j.impl* + org.slf4j:slf4j-log4j* + + + + + .yaml + + + + + + + + + + + + maven-assembly-plugin + + src/main/assembly/assembly.xml + + + + make-assembly + package + + single + + + + + + + \ No newline at end of file diff --git a/metron-analytics/metron-profiler-client/src/main/assembly/assembly.xml b/metron-analytics/metron-profiler-client/src/main/assembly/assembly.xml new file mode 100644 index 0000000000..b34bd980f3 --- /dev/null +++ b/metron-analytics/metron-profiler-client/src/main/assembly/assembly.xml @@ -0,0 +1,37 @@ + + + + archive + + tar.gz + + false + + + ${project.basedir}/target + + ${project.artifactId}-${project.version}.jar + + /lib + true + + + diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/HBaseProfilerClient.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/HBaseProfilerClient.java new file mode 100644 index 0000000000..7140d3f9fb --- /dev/null +++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/HBaseProfilerClient.java @@ -0,0 +1,118 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler; + +import org.apache.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.profiler.hbase.ColumnBuilder; +import org.apache.metron.profiler.hbase.RowKeyBuilder; +import org.apache.metron.profiler.hbase.Serializer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * The default implementation of a ProfilerClient that fetches profile data persisted in HBase. + */ +public class HBaseProfilerClient implements ProfilerClient { + + /** + * Used to access the profile data stored in HBase. + */ + private HTableInterface table; + + /** + * Generates the row keys necessary to scan HBase. + */ + private RowKeyBuilder rowKeyBuilder; + + /** + * The column family used to store the profile data. + */ + private String columnFamily; + + private byte[] columnFamilyBytes; + + public HBaseProfilerClient(HTableInterface table, RowKeyBuilder rowKeyBuilder, String columnFamily) { + setTable(table); + setRowKeyBuilder(rowKeyBuilder); + setColumnFamily(columnFamily); + } + + /** + * Fetches all of the data values associated with a Profile. + * + * @param profile The name of the profile. + * @param entity The name of the entity. + * @param durationAgo How far in the past to fetch values from. + * @param unit The time unit of 'durationAgo'. + * @param groups The groups + * @param The type of values stored by the Profile. + * @return A list of profile values. + */ + @Override + public List fetch(String profile, String entity, long durationAgo, TimeUnit unit, Class clazz, List groups) { + + // find all the row keys that satisfy this fetch + List keysToFetch = rowKeyBuilder.rowKeys(profile, entity, groups, durationAgo, unit); + + // create a Get for each of the row keys + List gets = keysToFetch + .stream() + .map(k -> new Get(k).addColumn(Bytes.toBytes(columnFamily), ColumnBuilder.QVALUE)) + .collect(Collectors.toList()); + + // submit the gets to HBase + try { + List values = new ArrayList<>(); + + Result[] results = table.get(gets); + Arrays.stream(results) + .filter(r -> r.containsColumn(columnFamilyBytes, ColumnBuilder.QVALUE)) + .map(r -> r.getValue(columnFamilyBytes, ColumnBuilder.QVALUE)) + .forEach(val -> values.add(Serializer.fromBytes(val, clazz))); + + return values; + + } catch(IOException e) { + throw new RuntimeException(e); + } + } + + public void setTable(HTableInterface table) { + this.table = table; + } + + public void setRowKeyBuilder(RowKeyBuilder rowKeyBuilder) { + this.rowKeyBuilder = rowKeyBuilder; + } + + public void setColumnFamily(String columnFamily) { + this.columnFamily = columnFamily; + this.columnFamilyBytes = Bytes.toBytes(columnFamily); + } +} diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/ProfilerClient.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/ProfilerClient.java new file mode 100644 index 0000000000..72b591db7c --- /dev/null +++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/ProfilerClient.java @@ -0,0 +1,45 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * An interface for a client capable of retrieving the profile data that has been persisted by the Profiler. + */ +public interface ProfilerClient { + + /** + * Fetch the measurement values associated with a profile. + * + * @param profile The name of the profile. + * @param entity The name of the entity. + * @param durationAgo How far in the past to fetch values from. + * @param unit The time unit of 'durationAgo'. + * @param clazz The type of values stored by the profile. + * @param groups The groups used to sort the profile data. + * @param The type of values stored by the Profile. + * @return An array of profile values. + */ + List fetch(String profile, String entity, long durationAgo, TimeUnit unit, Class clazz, List groups); + +} diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/HBaseProfilerClientTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/HBaseProfilerClientTest.java new file mode 100644 index 0000000000..95273d9c11 --- /dev/null +++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/HBaseProfilerClientTest.java @@ -0,0 +1,215 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.metron.hbase.client.HBaseClient; +import org.apache.metron.profiler.hbase.ColumnBuilder; +import org.apache.metron.profiler.hbase.SaltyRowKeyBuilder; +import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder; +import org.apache.metron.profiler.hbase.RowKeyBuilder; +import org.apache.metron.profiler.stellar.DefaultStellarExecutor; +import org.apache.metron.profiler.stellar.StellarExecutor; +import org.apache.storm.hbase.common.ColumnList; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; + +/** + * 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 HBaseProfilerClient client; + private HBaseClient hbaseClient; + private RowKeyBuilder rowKeyBuilder; + private ColumnBuilder columnBuilder; + private HTableInterface table; + private StellarExecutor executor; + private static HBaseTestingUtility util; + + @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(); + } + + @AfterClass + public static void stopHBase() throws Exception { + util.shutdownMiniCluster(); + util.cleanupTestDir(); + } + + @Before + public void setup() throws Exception { + + // setup all of the necessary dependencies + table = util.createTable(Bytes.toBytes(tableName), Bytes.toBytes(columnFamily)); + hbaseClient = new HBaseClient((c,t) -> table, table.getConfiguration(), tableName); + executor = new DefaultStellarExecutor(); + rowKeyBuilder = new SaltyRowKeyBuilder(); + columnBuilder = new ValueOnlyColumnBuilder(); + + // what we're actually testing + client = new HBaseProfilerClient(table, rowKeyBuilder, columnFamily); + } + + /** + * Writes profile measurements that can be used for testing. + * @param count The number of profile measurements to write. + * @param profileName Name of the profile. + * @param entityName Name of the entity. + * @param value The measurement value. + * @param periodsPerHour Number of profile periods per hour. + * @param startTime When measurements should start to be written. + * @param group The name of the group. + */ + private void writeMeasurements(int count, String profileName, String entityName, int value, int periodsPerHour, long startTime, List group) { + + // create the first measurement + ProfileMeasurement m = new ProfileMeasurement(profileName, entityName, startTime, periodsPerHour); + m.setValue(value); + + for(int i=0; i groups) { + + byte[] rowKey = rowKeyBuilder.rowKey(m, groups); + ColumnList cols = columnBuilder.columns(m); + + List mutations = hbaseClient.constructMutationReq(rowKey, cols, Durability.SKIP_WAL); + hbaseClient.batchMutate(mutations); + } + + /** + * The client should be able to distinguish between groups and only fetch those in the correct group. + */ + @Test + public void testFetchOneGroup() throws Exception { + + final int periodsPerHour = 4; + final int expectedValue = 2302; + final int hours = 2; + final int count = hours * periodsPerHour; + final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours); + + // setup - write two groups of measurements - 'weekends' and 'weekdays' + writeMeasurements(count, "profile1", "entity1", expectedValue, periodsPerHour, startTime, Arrays.asList("weekdays")); + writeMeasurements(count, "profile1", "entity1", 0, periodsPerHour, startTime, Arrays.asList("weekends")); + + // execute + List results = client.fetch("profile1", "entity1", hours, TimeUnit.HOURS, Integer.class, Arrays.asList("weekdays")); + + // validate + assertEquals(count, results.size()); + results.forEach(actual -> assertEquals(expectedValue, (int) actual)); + } + + /** + * Attempt to fetch a group that does not exist. + */ + @Test + public void testFetchNoGroup() { + + // setup + final int periodsPerHour = 4; + final int expectedValue = 2302; + final int hours = 2; + final int count = hours * periodsPerHour; + final long startTime = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hours); + + // create two groups of measurements - one on weekdays and one on weekends + writeMeasurements(count, "profile1", "entity1", expectedValue, periodsPerHour, startTime, Arrays.asList("weekdays")); + writeMeasurements(count, "profile1", "entity1", 0, periodsPerHour, startTime, Arrays.asList("weekends")); + + // execute + List results = client.fetch("profile1", "entity1", hours, TimeUnit.HOURS, Integer.class, Arrays.asList("does-not-exist")); + + // validate + assertEquals(0, results.size()); + } + + /** + * Profile data only within 'milliseconds ago' should be fetched. Data outside of that time horizon should + * not be fetched. + */ + @Test + public void testFetchOutsideTimeWindow() throws Exception { + + // setup - create some measurement values from a day ago + final int periodsPerHour = 4; + final int hours = 2; + final List group = Arrays.asList("weekends"); + final long startTime = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(1); + writeMeasurements(hours * periodsPerHour, "profile1", "entity1", 1000, 4, startTime, group); + + // execute + List results = client.fetch("profile1", "entity1", 2, TimeUnit.MILLISECONDS, Integer.class, group); + + // validate - there should NOT be any results from just 2 milliseconds ago + assertEquals(0, results.size()); + } +} diff --git a/metron-analytics/metron-profiler-client/src/test/resources/log4j.properties b/metron-analytics/metron-profiler-client/src/test/resources/log4j.properties new file mode 100644 index 0000000000..70be8ae64f --- /dev/null +++ b/metron-analytics/metron-profiler-client/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# + +# Root logger option +log4j.rootLogger=ERROR, stdout + +# Direct log messages to stdout +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md index f1ba0457cf..ad1fc8cca1 100644 --- a/metron-analytics/metron-profiler/README.md +++ b/metron-analytics/metron-profiler/README.md @@ -12,14 +12,79 @@ Any field contained within a message can be used to generate a profile. A profi The Profiler configuration requires a JSON-formatted set of elements, many of which can contain Stellar code. The configuration contains the following elements. -| Name | | Description | -|--- |--- |--- | -| profile | Required | A unique name identifying the profile. The field is treated as a string. | -| foreach | Required | A separate profile is maintained *for each* of these. This is effectively the entity that the profile is describing. The field is expected to contain a Stellar expression whose result is the entity name. For example, if `ip_src_addr` then a separate profile would be maintained for each unique IP source address in the data; 10.0.0.1, 10.0.0.2, etc. | -| onlyif | Optional | An expression that determines if a message should be applied to the profile. A Stellar expression is expected that when executed returns a boolean. A message is only applied to a profile if this condition is true. This allows a profile to filter the messages that it receives. | -| init | Optional | A set of expressions that is executed at the start of a window period. A map is expected where the key is the variable name and the value is a Stellar expression. The map can contain 0 or more variables/expressions. At the start of each window period the expression is executed once and stored in a variable with the given name. | -| update | Required | A set of expressions that is executed when a message is applied to the profile. A map is expected where the key is the variable name and the value is a Stellar expression. The map can include 0 or more variables/expressions. | -| result | Required | A Stellar expression that is executed when the window period expires. The expression is expected to in some way summarize the messages that were applied to the profile over the window period. The expression must result in a numeric value such as a Double, Long, Float, Short, or Integer. | +| Name | | Description +|--- |--- |--- +| [profile](#profile) | Required | Unique name identifying the profile. +| [foreach](#foreach) | Required | A separate profile is maintained "for each" of these. +| [onlyif](#onlyif) | Optional | Boolean expression that determines if a message should be applied to the profile. +| [groupBy](#groupby) | Optional | One or more Stellar expressions used to group the profile measurements when persisted. +| [init](#init) | Optional | One or more expressions executed at the start of a window period. +| [update](#update) | Required | One or more expressions executed when a message is applied to the profile. +| [result](#result) | Required | A Stellar expression that is executed when the window period expires. + +#### `profile` + +*Required* + +A unique name identifying the profile. The field is treated as a string. + +#### `foreach` + +*Required* + +A separate profile is maintained 'for each' of these. This is effectively the entity that the profile is describing. The field is expected to contain a Stellar expression whose result is the entity name. + +For example, if `ip_src_addr` then a separate profile would be maintained for each unique IP source address in the data; 10.0.0.1, 10.0.0.2, etc. + +#### `onlyif` + +*Optional* + +An expression that determines if a message should be applied to the profile. A Stellar expression that returns a Boolean is expected. A message is only applied to a profile if this expression is true. This allows a profile to filter the messages that get applied to it. + +#### `groupBy` + +*Optional* + +One or more Stellar expressions used to group the profile measurements when persisted. This is intended to sort the Profile data to allow for a contiguous scan when accessing subsets of the data. + +The 'groupBy' expressions can refer to any field within a `org.apache.metron.profiler.ProfileMeasurement`. A common use case would be grouping by day of week. This allows a contiguous scan to access all profile data for Mondays only. Using the following definition would achieve this. + +``` +"groupBy": [ "DAY_OF_WEEK()" ] +``` + +#### `init` + +*Optional* + +One or more expressions executed at the start of a window period. A map is expected where the key is the variable name and the value is a Stellar expression. The map can contain 0 or more variables/expressions. At the start of each window period the expression is executed once and stored in a variable with the given name. + +``` +"init": { + "var1": "0", + "var2": "1" +} +``` + +#### `update` + +*Required* + +One or more expressions executed when a message is applied to the profile. A map is expected where the key is the variable name and the value is a Stellar expression. The map can include 0 or more variables/expressions. When each message is applied to the profile, the expression is executed and stored in a variable with the given name. + +``` +"update": { + "var1": "var1 + 1", + "var2": "var2 + 1" +} +``` + +#### `result` + +*Required* + +A Stellar expression that is executed when the window period expires. The expression is expected to summarize the messages that were applied to the profile over the window period. The expression must result in a numeric value such as a Double, Long, Float, Short, or Integer. ### Examples @@ -220,23 +285,28 @@ This section will describe the steps required to get your first profile running. ## Implementation -## Topology +## Key Classes -The Profiler is implemented as a Storm topology using the following bolts and spouts. +* `ProfileMeasurement` - Represents a single data point within a Profile. A Profile is effectively a time series. To this end a Profile is composed of many ProfileMeasurement values which in aggregate form a time series. -### KafkaSpout +* `ProfilePeriod` - The Profiler captures one `ProfileMeasurement` each `ProfilePeriod`. A `ProfilePeriod` will occur at fixed, deterministic points in time. This allows for efficient retrieval of profile data. -A spout that consumes messages from a single Kafka topic. In most cases, the Profiler topology will consume messages from the `indexing` topic. This topic contains fully enriched messages that are ready to be indexed. This ensures that profiles can take advantage of all the available data elements. +* `RowKeyBuilder` - Builds row keys that can be used to read or write profile data to HBase. -### ProfileSplitterBolt - -The bolt responsible for filtering incoming messages and directing each to the one or more downstream bolts that are responsible for building a Profile. Each message may be needed by 0, 1 or even many Profiles. Each emitted tuple contains the 'resolved' entity name, the profile definition, and the input message. +* `ColumnBuilder` - Defines the columns of data stored with a profile measurement. + +* `ProfileHBaseMapper` - Defines for the `HBaseBolt` how profile measurements are stored in HBase. This class leverages a `RowKeyBuilder` and `ColumnBuilder`. + +## Storm Topology + +The Profiler is implemented as a Storm topology using the following bolts and spouts. + +* `KafkaSpout` - A spout that consumes messages from a single Kafka topic. In most cases, the Profiler topology will consume messages from the `indexing` topic. This topic contains fully enriched messages that are ready to be indexed. This ensures that profiles can take advantage of all the available data elements. -### ProfileBuilderBolt +* `ProfileSplitterBolt` - The bolt responsible for filtering incoming messages and directing each to the one or more downstream bolts that are responsible for building a profile. Each message may be needed by 0, 1 or even many profiles. Each emitted tuple contains the 'resolved' entity name, the profile definition, and the input message. -This bolt maintains all of the state required to build a Profile. When the window period expires, the data is summarized as a ProfileMeasurement, all state is flushed, and the ProfileMeasurement is emitted. Each instance of this bolt is responsible for maintaining the state for a single Profile-Entity pair. +* `ProfileBuilderBolt` - This bolt maintains all of the state required to build a profile. When the window period expires, the data is summarized as a `ProfileMeasurement`, all state is flushed, and the `ProfileMeasurement` is emitted. Each instance of this bolt is responsible for maintaining the state for a single Profile-Entity pair. -### HBaseBolt +* `HBaseBolt` - A bolt that is responsible for writing to HBase. Most profiles will be flushed every 15 minutes or so. If each `ProfileBuilderBolt` were responsible for writing to HBase itself, there would be little to no opportunity to optimize these writes. By aggregating the writes from multiple Profile-Entity pairs these writes can be batched, for example. -A bolt that is responsible for writing to HBase. Most profiles will be flushed every 15 minutes or so. If each ProfileBuilderBolt were responsible for writing to HBase itself, there would be little to no opportunity to optimize these writes. By aggregating the writes from multiple Profile-Entity pairs these writes can be batched, for example. diff --git a/metron-analytics/metron-profiler/pom.xml b/metron-analytics/metron-profiler/pom.xml index b31df67981..ebb51a2f9e 100644 --- a/metron-analytics/metron-profiler/pom.xml +++ b/metron-analytics/metron-profiler/pom.xml @@ -26,6 +26,17 @@ UTF-8 + + org.apache.metron + metron-common + ${project.parent.version} + + + com.google.guava + guava + + + org.apache.metron metron-hbase @@ -42,20 +53,6 @@ metron-test-utilities ${project.parent.version} test - - - org.slf4j - slf4j-log4j12 - - - log4j - log4j - - - com.google.guava - guava - - org.apache.hadoop @@ -80,14 +77,22 @@ ${global_hadoop_version} - org.adrianwalker - multiline-string - 0.1.2 - test + org.apache.hbase + hbase-client + ${global_hbase_version} + provided - tools - sun.jdk + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + com.google.guava + guava @@ -95,6 +100,7 @@ org.apache.storm storm-core ${global_storm_version} + provided servlet-api @@ -108,23 +114,35 @@ org.apache.storm - storm-hdfs + storm-hbase ${global_storm_version} + provided - org.apache.storm - storm-core + servlet-api + javax.servlet - org.apache.hadoop - hadoop-client + log4j-over-slf4j + org.slf4j org.apache.storm - storm-hbase + storm-hdfs ${global_storm_version} + provided + + + + + + + + + + org.apache.kafka @@ -137,51 +155,6 @@ - - org.apache.hbase - hbase-client - ${global_hbase_version} - - - org.slf4j - slf4j-log4j12 - - - log4j - log4j - - - com.google.guava - guava - - - - - org.apache.hbase - hbase-server - ${global_hbase_version} - test - - - org.apache.hbase - hbase-testing-util - ${global_hbase_version} - test - - - org.slf4j - slf4j-log4j12 - - - log4j - log4j - - - com.google.guava - guava - - - org.mockito mockito-all @@ -202,12 +175,10 @@ org.apache.maven.plugins maven-project-info-reports-plugin 2.7 - false - org.codehaus.mojo emma-maven-plugin @@ -284,6 +255,10 @@ + + com.lmax + org.apache.metron.lmax.metron-profiler + com.google.common org.apache.metron.guava.metron-profiler @@ -334,4 +309,4 @@ - + \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/main/config/profiler.properties b/metron-analytics/metron-profiler/src/main/config/profiler.properties index 80fd2da4fe..4a9cdb4f69 100644 --- a/metron-analytics/metron-profiler/src/main/config/profiler.properties +++ b/metron-analytics/metron-profiler/src/main/config/profiler.properties @@ -23,12 +23,14 @@ profiler.workers=1 profiler.executors=0 profiler.input.topic=indexing -profiler.flush.interval.seconds=900 +profiler.periods.per.hour=4 profiler.hbase.salt.divisor=1000 profiler.hbase.table=profiler +profiler.hbase.column.family=P profiler.hbase.batch=10 profiler.hbase.flush.interval.seconds=30 + ##### Kafka ##### kafka.zk=node1:2181 diff --git a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml index d99510a036..835c609815 100644 --- a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml +++ b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml @@ -25,11 +25,28 @@ components: - id: "defaultExecutor" className: "org.apache.metron.profiler.stellar.DefaultStellarExecutor" - - id: "hbaseMapper" - className: "org.apache.metron.profiler.bolt.ProfileHBaseMapper" + - id: "rowKeyBuilder" + className: "org.apache.metron.profiler.hbase.SaltyRowKeyBuilder" properties: - name: "saltDivisor" value: ${profiler.hbase.salt.divisor} + - name: "periodsPerHour" + value: ${profiler.periods.per.hour} + + - id: "columnBuilder" + className: "org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder" + constructorArgs: + - "${profiler.hbase.column.family}" + + - id: "hbaseMapper" + className: "org.apache.metron.profiler.bolt.ProfileHBaseMapper" + properties: + - name: "rowKeyBuilder" + ref: "rowKeyBuilder" + - name: "columnBuilder" + ref: "columnBuilder" + - name: "executor" + ref: "defaultExecutor" - id: "zkHosts" className: "storm.kafka.ZkHosts" @@ -76,8 +93,8 @@ bolts: properties: - name: "executor" ref: "defaultExecutor" - - name: "flushFrequency" - value: ${profiler.flush.interval.seconds} + - name: "periodsPerHour" + value: ${profiler.periods.per.hour} - id: "hbaseBolt" className: "org.apache.metron.hbase.bolt.HBaseBolt" diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java index 6db76feb6e..2688740172 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java @@ -20,6 +20,8 @@ package org.apache.metron.profiler; +import java.util.List; + /** * Represents a single data point within a Profile. * @@ -39,58 +41,58 @@ public class ProfileMeasurement { private String entity; /** - * When the measurement window was started in milliseconds since the epoch. + * The actual measurement itself. */ - private long start; + private Object value; /** - * When the measurement window closed in milliseconds since the epoch. + * A set of expressions used to group the profile measurements when persisted. */ - private long end; + private List groupBy; /** - * The actual measurement itself. + * The period in which the ProfileMeasurement was taken. */ - private Object value; + private ProfilePeriod period; - public String getProfileName() { - return profileName; + /** + * @param profileName The name of the profile. + * @param entity The name of the entity. + * @param epochMillis The timestamp when the measurement period has been started in milliseconds since the epoch. + * @param periodsPerHour The number of profile periods per hour. + */ + public ProfileMeasurement(String profileName, String entity, long epochMillis, int periodsPerHour) { + this.profileName = profileName; + this.entity = entity; + this.period = new ProfilePeriod(epochMillis, periodsPerHour); } - public void setProfileName(String profileName) { - this.profileName = profileName; + public String getProfileName() { + return profileName; } public String getEntity() { return entity; } - public void setEntity(String entity) { - this.entity = entity; - } - - public long getStart() { - return start; - } - - public void setStart(long start) { - this.start = start; + public Object getValue() { + return value; } - public long getEnd() { - return end; + public void setValue(Object value) { + this.value = value; } - public void setEnd(long end) { - this.end = end; + public List getGroupBy() { + return groupBy; } - public Object getValue() { - return value; + public void setGroupBy(List groupBy) { + this.groupBy = groupBy; } - public void setValue(Object value) { - this.value = value; + public ProfilePeriod getPeriod() { + return period; } @Override @@ -100,21 +102,20 @@ public boolean equals(Object o) { ProfileMeasurement that = (ProfileMeasurement) o; - if (start != that.start) return false; - if (end != that.end) return false; if (profileName != null ? !profileName.equals(that.profileName) : that.profileName != null) return false; if (entity != null ? !entity.equals(that.entity) : that.entity != null) return false; - return value != null ? value.equals(that.value) : that.value == null; - + if (value != null ? !value.equals(that.value) : that.value != null) return false; + if (groupBy != null ? !groupBy.equals(that.groupBy) : that.groupBy != null) return false; + return period != null ? period.equals(that.period) : that.period == null; } @Override public int hashCode() { int result = profileName != null ? profileName.hashCode() : 0; result = 31 * result + (entity != null ? entity.hashCode() : 0); - result = 31 * result + (int) (start ^ (start >>> 32)); - result = 31 * result + (int) (end ^ (end >>> 32)); result = 31 * result + (value != null ? value.hashCode() : 0); + result = 31 * result + (groupBy != null ? groupBy.hashCode() : 0); + result = 31 * result + (period != null ? period.hashCode() : 0); return result; } @@ -123,9 +124,9 @@ public String toString() { return "ProfileMeasurement{" + "profileName='" + profileName + '\'' + ", entity='" + entity + '\'' + - ", start=" + start + - ", end=" + end + ", value=" + value + + ", groupBy=" + groupBy + + ", period=" + period + '}'; } } diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java new file mode 100644 index 0000000000..91b8dd1567 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java @@ -0,0 +1,197 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler; + +import java.util.Calendar; +import java.util.TimeZone; +import java.util.concurrent.TimeUnit; + +import static java.lang.String.format; + +/** + * The Profiler captures a ProfileMeasurement once every ProfilePeriod. There can be + * multiple ProfilePeriods every hour. + */ +public class ProfilePeriod { + + /** + * The year. + */ + private int year; + + /** + * Day of the year; [1, 366] + */ + private int dayOfYear; + + /** + * Hour of the day; [0, 23] + */ + private int hour; + + /** + * The period within the hour; [0, periodsPerHour) + */ + private int period; + + /** + * The number of periods per hour. This value must be a divisor of 60; 1, 2, 3, 4, 6, 10, etc. + */ + private int periodsPerHour; + + /** + * The actual time used to initialize the ProfilePeriod. This value should not be + * used for anything other than troubleshooting. + */ + private long epochMillis; + + /** + * @param epochMillis A timestamp contained somewhere within the profile period. + * @param periodsPerHour The number of periods per hour. Must be an divisor or multiple of 60; 1, 2, 3, 4, 6, 240, etc. + */ + public ProfilePeriod(long epochMillis, int periodsPerHour) { + + // periods per hour must be a divisor or multiple of 60 + if(60 % periodsPerHour != 0 && periodsPerHour % 60 != 0) { + throw new RuntimeException(format("invalid periodsPerHour: expected=divisor/multiple of 60, actual=%d", periodsPerHour)); + } + + Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC")); + cal.setTimeInMillis(epochMillis); + + this.periodsPerHour = periodsPerHour; + this.period = findPeriod(cal.get(Calendar.MINUTE), cal.get(Calendar.SECOND), periodsPerHour); + this.hour = cal.get(Calendar.HOUR_OF_DAY); + this.dayOfYear = cal.get(Calendar.DAY_OF_YEAR); + this.year = cal.get(Calendar.YEAR); + this.epochMillis = epochMillis; + } + + /** + * Returns the next ProfilePeriod in time. + */ + public ProfilePeriod next() { + long nextMillis = this.getTimeInMillis() + millisPerPeriod(periodsPerHour); + return new ProfilePeriod(nextMillis, periodsPerHour); + } + + /** + * @return The time in milliseconds since the epoch. + */ + public long getTimeInMillis() { + + int millisPastHour = (int) millisPerPeriod(periodsPerHour) * period; + + Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC")); + cal.set(Calendar.YEAR, year); + cal.set(Calendar.DAY_OF_YEAR, dayOfYear); + cal.set(Calendar.HOUR_OF_DAY, hour); + cal.set(Calendar.MINUTE, (millisPastHour / 1000) / 60); + cal.set(Calendar.SECOND, (millisPastHour / 1000) % 60); + cal.set(Calendar.MILLISECOND, 0); + + return cal.getTimeInMillis(); + } + + public int getYear() { + return year; + } + + public int getDayOfYear() { + return dayOfYear; + } + + public int getHour() { + return hour; + } + + public int getPeriod() { + return period; + } + + public int getPeriodsPerHour() { + return periodsPerHour; + } + + /** + * Determines the period within the hour based on the minutes/seconds on the clock. + * + * @param minutes The minute within the hour; 0-59. + * @param seconds The second within the minute; 0-59. + * @return The period within the hour. + */ + private static int findPeriod(int minutes, int seconds, int periodsPerHour) { + final int secondsInHour = minutes * 60 + seconds; + return (int) (secondsInHour / secondsPerPeriod(periodsPerHour)); + } + + /** + * The number of seconds in each period. + * @param periodsPerHour The number of periods per hour. + */ + private static double secondsPerPeriod(int periodsPerHour) { + return millisPerPeriod(periodsPerHour) / 1000L; + } + + /** + * The number of milliseconds in each period. + * @param periodsPerHour The number of periods per hour. + */ + private static long millisPerPeriod(int periodsPerHour) { + final long millisPerHour = 60L * 60L * 1000L; + return millisPerHour / periodsPerHour; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ProfilePeriod that = (ProfilePeriod) o; + + if (year != that.year) return false; + if (dayOfYear != that.dayOfYear) return false; + if (hour != that.hour) return false; + if (period != that.period) return false; + return periodsPerHour == that.periodsPerHour; + } + + @Override + public int hashCode() { + int result = year; + result = 31 * result + dayOfYear; + result = 31 * result + hour; + result = 31 * result + period; + result = 31 * result + periodsPerHour; + return result; + } + + @Override + public String toString() { + return "ProfilePeriod{" + + "year=" + year + + ", dayOfYear=" + dayOfYear + + ", hour=" + hour + + ", period=" + period + + ", periodsPerHour=" + periodsPerHour + + '}'; + } +} diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java index 6cf505398f..017892be36 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java @@ -32,7 +32,6 @@ import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.common.dsl.Context; import org.apache.metron.common.dsl.ParseException; -import org.apache.metron.common.dsl.StellarFunctions; import org.apache.metron.profiler.ProfileMeasurement; import org.apache.metron.profiler.stellar.StellarExecutor; import org.json.simple.JSONObject; @@ -40,7 +39,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.Map; import static java.lang.String.format; @@ -65,9 +63,10 @@ public class ProfileBuilderBolt extends ConfiguredProfilerBolt { private StellarExecutor executor; /** - * The number of seconds between when the Profile is flushed. + * The number of times per hour that a profile is flushed and a measurement + * is written. This should be a divisor or multiple of 60; 1, 2, 3, 4, 6, 240, etc. */ - private int flushFrequency; + private int periodsPerHour; /** * A ProfileMeasurement is created and emitted each window period. A Profile @@ -85,11 +84,6 @@ public class ProfileBuilderBolt extends ConfiguredProfilerBolt { */ private transient JSONParser parser; - /** - * Stellar context - */ - private Context stellarContext; - private OutputCollector collector; /** @@ -106,14 +100,19 @@ public ProfileBuilderBolt(String zookeeperUrl) { @Override public Map getComponentConfiguration() { Config conf = new Config(); - conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, getFlushFrequency()); + + // how frequently should the bold receive tick tuples? + long freqInSeconds = ((60 * 60) / periodsPerHour); + conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, freqInSeconds); + return conf; } protected void initializeStellar() { - stellarContext = new Context.Builder() - .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client) - .build(); + Context context = new Context.Builder() + .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client) + .build(); + executor.setContext(context); } @Override @@ -157,18 +156,12 @@ public void execute(Tuple input) { private void doExecute(Tuple input) { if(!isTickTuple(input)) { - - // if this is the first tuple in a window period, initialization is needed if (!isInitialized()) { init(input); } - - // update the profile with data from a new message update(input); } else { - - // flush the profile - can only flush if it has been initialized if(isInitialized()) { flush(input); } @@ -186,20 +179,21 @@ private void init(Tuple input) { profileConfig = (ProfileConfig) input.getValueByField("profile"); // create the measurement which will be saved at the end of the window period - measurement = new ProfileMeasurement(); - measurement.setStart(getTimestamp()); - measurement.setEntity(input.getStringByField("entity")); - measurement.setProfileName(profileConfig.getProfile()); + measurement = new ProfileMeasurement( + profileConfig.getProfile(), + input.getStringByField("entity"), + getTimestamp(), + periodsPerHour); // execute the 'init' expression try { JSONObject message = (JSONObject) input.getValueByField("message"); Map expressions = profileConfig.getInit(); - expressions.forEach((var, expr) -> executor.assign(var, expr, message, stellarContext)); + expressions.forEach((var, expr) -> executor.assign(var, expr, message)); } catch(ParseException e) { - String msg = format("Bad 'init' expression: %s, profile=%s, entity=%s, start=%d", - e.getMessage(), measurement.getProfileName(), measurement.getEntity(), measurement.getStart()); + String msg = format("Bad 'init' expression: %s, profile=%s, entity=%s", + e.getMessage(), measurement.getProfileName(), measurement.getEntity()); throw new ParseException(msg, e); } } @@ -214,11 +208,11 @@ private void update(Tuple input) { // execute each of the 'update' expressions try { Map expressions = profileConfig.getUpdate(); - expressions.forEach((var, expr) -> executor.assign(var, expr, message, stellarContext)); + expressions.forEach((var, expr) -> executor.assign(var, expr, message)); } catch(ParseException e) { - String msg = format("Bad 'update' expression: %s, profile=%s, entity=%s, start=%d", - e.getMessage(), measurement.getProfileName(), measurement.getEntity(), measurement.getStart()); + String msg = format("Bad 'update' expression: %s, profile=%s, entity=%s", + e.getMessage(), measurement.getProfileName(), measurement.getEntity()); throw new ParseException(msg, e); } } @@ -231,21 +225,20 @@ private void update(Tuple input) { * the next window period. */ private void flush(Tuple tickTuple) { - LOG.info(String.format("Flushing profile: profile=%s, entity=%s, start=%d", - measurement.getProfileName(), measurement.getEntity(), measurement.getStart())); + LOG.info(String.format("Flushing profile: profile=%s, entity=%s", + measurement.getProfileName(), measurement.getEntity())); // execute the 'result' expression Object result; try { String resultExpr = profileConfig.getResult(); - result = executor.execute(resultExpr, new JSONObject(), Object.class, stellarContext); + result = executor.execute(resultExpr, new JSONObject(), Object.class); } catch(ParseException e) { throw new ParseException("Bad 'result' expression", e); } // emit the completed profile measurement - measurement.setEnd(getTimestamp()); measurement.setValue(result); emit(measurement, tickTuple); @@ -298,11 +291,9 @@ public void setExecutor(StellarExecutor executor) { this.executor = executor; } - public int getFlushFrequency() { - return flushFrequency; + public void setPeriodsPerHour(int periodsPerHour) { + this.periodsPerHour = periodsPerHour; } - public void setFlushFrequency(int flushFrequency) { - this.flushFrequency = flushFrequency; - } + } diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java index 492437eabf..96758ac27b 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java @@ -21,14 +21,22 @@ package org.apache.metron.profiler.bolt; import backtype.storm.tuple.Tuple; -import org.apache.hadoop.hbase.util.Bytes; +import org.apache.commons.beanutils.BeanMap; +import org.apache.metron.common.dsl.ParseException; import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.metron.profiler.hbase.ColumnBuilder; +import org.apache.metron.profiler.hbase.SaltyRowKeyBuilder; +import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder; +import org.apache.metron.profiler.hbase.RowKeyBuilder; +import org.apache.metron.profiler.stellar.StellarExecutor; import org.apache.storm.hbase.bolt.mapper.HBaseMapper; import org.apache.storm.hbase.common.ColumnList; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; -import java.util.Calendar; +import java.util.ArrayList; +import java.util.List; + +import static java.lang.String.format; +import static org.apache.commons.collections.CollectionUtils.isEmpty; /** * An HbaseMapper that defines how a ProfileMeasurement is persisted within an HBase table. @@ -36,20 +44,28 @@ public class ProfileHBaseMapper implements HBaseMapper { /** - * A salt is prepended to the row key to help prevent hotspotting. This constant is used - * to generate the salt. Ideally, this constant should be roughly equal to the number of - * nodes in the Hbase cluster. + * Executes Stellar code and maintains state across multiple invocations. */ - private int saltDivisor; + private StellarExecutor executor; /** - * The name of the column family. + * Generates the row keys necessary to store profile data in HBase. */ - private String columnFamily; + private RowKeyBuilder rowKeyBuilder; + + /** + * Generates the ColumnList necesary to store profile data in HBase. + */ + private ColumnBuilder columnBuilder; public ProfileHBaseMapper() { - setColumnFamily("P"); - setSaltDivisor(1000); + setRowKeyBuilder(new SaltyRowKeyBuilder()); + setColumnBuilder(new ValueOnlyColumnBuilder()); + } + + public ProfileHBaseMapper(RowKeyBuilder rowKeyBuilder, ColumnBuilder columnBuilder) { + setRowKeyBuilder(rowKeyBuilder); + setColumnBuilder(columnBuilder); } /** @@ -60,19 +76,8 @@ public ProfileHBaseMapper() { @Override public byte[] rowKey(Tuple tuple) { ProfileMeasurement m = (ProfileMeasurement) tuple.getValueByField("measurement"); - - // create a calendar to determine day-of-week, etc - Calendar calendar = Calendar.getInstance(); - calendar.setTimeInMillis(m.getStart()); - - return Bytes.toBytes(getSalt(m.getStart()) + - m.getProfileName() + - calendar.get(Calendar.DAY_OF_WEEK) + - calendar.get(Calendar.WEEK_OF_MONTH) + - calendar.get(Calendar.MONTH) + - calendar.get(Calendar.YEAR) + - m.getEntity() + - m.getStart()); + List groups = executeGroupBy(m); + return rowKeyBuilder.rowKey(m, groups); } /** @@ -82,85 +87,48 @@ public byte[] rowKey(Tuple tuple) { @Override public ColumnList columns(Tuple tuple) { ProfileMeasurement measurement = (ProfileMeasurement) tuple.getValueByField("measurement"); - - byte[] cfBytes = Bytes.toBytes(columnFamily); - ColumnList cols = new ColumnList(); - cols.addColumn(cfBytes, QPROFILE, Bytes.toBytes(measurement.getProfileName())); - cols.addColumn(cfBytes, QENTITY, Bytes.toBytes(measurement.getEntity())); - cols.addColumn(cfBytes, QSTART, Bytes.toBytes(measurement.getStart())); - cols.addColumn(cfBytes, QEND, Bytes.toBytes(measurement.getEnd())); - cols.addColumn(cfBytes, QVALUE, toBytes(measurement.getValue())); - - return cols; + return columnBuilder.columns(measurement); } /** - * Serialize a profile measurement's value. - * - * The value produced by a Profile definition can be any numeric data type. The data - * type depends on how the profile is defined by the user. The user should be able to - * choose the data type that is most suitable for their use case. - * - * @param value The value to serialize. + * Executes each of the 'groupBy' expressions. The result of each + * expression are the groups used to sort the data as part of the + * row key. + * @param m The profile measurement. + * @return The result of executing the 'groupBy' expressions. */ - private byte[] toBytes(Object value) { - byte[] result; - - if(value instanceof Integer) { - result = Bytes.toBytes((Integer) value); - } else if(value instanceof Double) { - result = Bytes.toBytes((Double) value); - } else if(value instanceof Short) { - result = Bytes.toBytes((Short) value); - } else if(value instanceof Long) { - result = Bytes.toBytes((Long) value); - } else if(value instanceof Float) { - result = Bytes.toBytes((Float) value); - } else { - throw new RuntimeException("Expected 'Number': actual=" + value); + private List executeGroupBy(ProfileMeasurement m) { + List groups = new ArrayList<>(); + + if(!isEmpty(m.getGroupBy())) { + try { + // allows each 'groupBy' expression to refer to the fields of the ProfileMeasurement + BeanMap measureAsMap = new BeanMap(m); + + for (String expr : m.getGroupBy()) { + Object result = executor.execute(expr, measureAsMap, Object.class); + groups.add(result); + } + + } catch(Throwable e) { + String msg = format("Bad 'groupBy' expression: %s, profile=%s, entity=%s", + e.getMessage(), m.getProfileName(), m.getEntity()); + throw new ParseException(msg, e); + } } - return result; + return groups; } - /** - * Calculates a salt value that is used as part of the row key. - * - * The salt is calculated as 'md5(timestamp) % N' where N is a configurable value that ideally - * is close to the number of nodes in the Hbase cluster. - * - * @param epoch The timestamp in epoch millis to use in generating the salt. - */ - private int getSalt(long epoch) { - try { - MessageDigest digest = MessageDigest.getInstance("MD5"); - byte[] hash = digest.digest(Bytes.toBytes(epoch)); - return Bytes.toInt(hash) % saltDivisor; - - } catch(NoSuchAlgorithmException e) { - throw new RuntimeException(e); - } - } - - public String getColumnFamily() { - return columnFamily; - } - - public void setColumnFamily(String columnFamily) { - this.columnFamily = columnFamily; + public void setExecutor(StellarExecutor executor) { + this.executor = executor; } - public int getSaltDivisor() { - return saltDivisor; + public void setRowKeyBuilder(RowKeyBuilder rowKeyBuilder) { + this.rowKeyBuilder = rowKeyBuilder; } - public void setSaltDivisor(int saltDivisor) { - this.saltDivisor = saltDivisor; + public void setColumnBuilder(ColumnBuilder columnBuilder) { + this.columnBuilder = columnBuilder; } - - public static final byte[] QPROFILE = Bytes.toBytes("profile"); - public static final byte[] QENTITY = Bytes.toBytes("entity"); - public static final byte[] QSTART = Bytes.toBytes("start"); - public static final byte[] QEND = Bytes.toBytes("end"); - public static final byte[] QVALUE = Bytes.toBytes("value"); } diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java index c9af519705..cbd7423a5d 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java @@ -31,7 +31,6 @@ import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.common.configuration.profiler.ProfilerConfig; import org.apache.metron.common.dsl.Context; -import org.apache.metron.common.dsl.StellarFunctions; import org.apache.metron.profiler.stellar.StellarExecutor; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; @@ -65,12 +64,6 @@ public class ProfileSplitterBolt extends ConfiguredProfilerBolt { */ private StellarExecutor executor; - /** - * Stellar context - */ - private Context stellarContext; - - /** * @param zookeeperUrl The Zookeeper URL that contains the configuration for this bolt. */ @@ -87,9 +80,14 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll } protected void initializeStellar() { - stellarContext = new Context.Builder() + /* + * the context provides global configuration values that Stellar may need + * during initialization to connect to external resources like Zookeeper or HBase. + */ + Context context = new Context.Builder() .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client) .build(); + executor.setContext(context); } @Override @@ -134,10 +132,10 @@ private void applyProfile(ProfileConfig profile, Tuple input, JSONObject message // is this message needed by this profile? String onlyIf = profile.getOnlyif(); - if (StringUtils.isBlank(onlyIf) || executor.execute(onlyIf, message, Boolean.class, stellarContext)) { + if (StringUtils.isBlank(onlyIf) || executor.execute(onlyIf, message, Boolean.class)) { // what is the name of the entity in this message? - String entity = executor.execute(profile.getForeach(), message, String.class, stellarContext); + String entity = executor.execute(profile.getForeach(), message, String.class); // emit a message for the bolt responsible for building this profile collector.emit(input, new Values(entity, profile, message)); diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java new file mode 100644 index 0000000000..20b71ffb3c --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java @@ -0,0 +1,45 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler.hbase; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.storm.hbase.common.ColumnList; + +import java.io.Serializable; + +/** + * Defines how fields in a ProfileMeasurement will be mapped to columns in HBase. + */ +public interface ColumnBuilder extends Serializable { + + /** + * Generate the columns used to store a ProfileMeasurement. + * @param measurement The profile measurement. + */ + ColumnList columns(ProfileMeasurement measurement); + + byte[] QPROFILE = Bytes.toBytes("profile"); + byte[] QENTITY = Bytes.toBytes("entity"); + byte[] QSTART = Bytes.toBytes("start"); + byte[] QEND = Bytes.toBytes("end"); + byte[] QVALUE = Bytes.toBytes("value"); +} diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java new file mode 100644 index 0000000000..bb8cc80606 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/RowKeyBuilder.java @@ -0,0 +1,60 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler.hbase; + +import org.apache.metron.profiler.ProfileMeasurement; + +import java.io.Serializable; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * Builds a row key that can be used to read or write ProfileMeasurement data + * to HBase. + */ +public interface RowKeyBuilder extends Serializable { + + /** + * Build a row key for a given ProfileMeasurement. + * + * This method is useful when writing ProfileMeasurements to HBase. + * + * @param measurement The profile measurement. + * @param groups The groups used to sort the profile data. + * @return The HBase row key. + */ + byte[] rowKey(ProfileMeasurement measurement, List groups); + + /** + * Builds a list of row keys necessary to retrieve a profile's measurements over + * a time horizon. + * + * This method is useful when attempting to read ProfileMeasurements stored in HBase. + * + * @param profile The name of the profile. + * @param entity The name of the entity. + * @param groups The group(s) used to sort the profile data. + * @param durationAgo How long ago? + * @param unit The time units of how long ago. + * @return All of the row keys necessary to retrieve the profile measurements. + */ + List rowKeys(String profile, String entity, List groups, long durationAgo, TimeUnit unit); +} diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java new file mode 100644 index 0000000000..e78a8ea80a --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilder.java @@ -0,0 +1,211 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler.hbase; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.metron.profiler.ProfilePeriod; + +import java.nio.ByteBuffer; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * A RowKeyBuilder that uses a salt to prevent hot-spotting. + * + * Responsible for building the row keys used to store profile data in HBase. The row key is composed of the following + * fields in the given order. + *
    + *
  • salt - A salt that helps prevent hot-spotting. + *
  • profile - The name of the profile. + *
  • entity - The name of the entity being profiled. + *
  • group(s) - The group(s) used to sort the data in HBase. For example, a group may distinguish between weekends and weekdays. + *
  • year - The year based on UTC. + *
  • day of year - The current day within the year based on UTC; [1, 366] + *
  • hour - The hour within the day based on UTC; [0, 23] + *
period - The period within the hour. The number of periods per hour can be defined by the user; defaults to 4. + */ +public class SaltyRowKeyBuilder implements RowKeyBuilder { + + /** + * A salt can be prepended to the row key to help prevent hot-spotting. The salt + * divisor is used to generate the salt. The salt divisor should be roughly equal + * to the number of nodes in the Hbase cluster. + */ + private int saltDivisor; + + /** + * An hour is divided into multiple periods. This defines how many periods + * will exist within each given hour. + */ + private int periodsPerHour; + + public SaltyRowKeyBuilder() { + this.saltDivisor = 1000; + this.periodsPerHour = 4; + } + + public SaltyRowKeyBuilder(int saltDivisor, int periodsPerHour) { + this.saltDivisor = saltDivisor; + this.periodsPerHour = periodsPerHour; + } + + /** + * Builds a list of row keys necessary to retrieve profile measurements over + * a time horizon. + * + * @param profile The name of the profile. + * @param entity The name of the entity. + * @param groups The group(s) used to sort the profile data. + * @param durationAgo How long ago? + * @param unit The time units of how long ago. + * @return All of the row keys necessary to retrieve the profile measurements. + */ + @Override + public List rowKeys(String profile, String entity, List groups, long durationAgo, TimeUnit unit) { + List rowKeys = new ArrayList<>(); + + // find the time horizon + long endTime = System.currentTimeMillis(); + long startTime = endTime - unit.toMillis(durationAgo); + + // find the starting period and advance until the end time is reached + ProfilePeriod period = new ProfilePeriod(startTime, periodsPerHour); + while(period.getTimeInMillis() <= endTime) { + + byte[] k = rowKey(profile, entity, period, groups); + rowKeys.add(k); + + // advance to the next period + period = period.next(); + } + + return rowKeys; + } + + /** + * Builds the row key for a given profile measurement. + * @param m The profile measurement. + * @param groups The groups used to sort the profile data. + * @return The HBase row key. + */ + @Override + public byte[] rowKey(ProfileMeasurement m, List groups) { + return rowKey(m.getProfileName(), m.getEntity(), m.getPeriod(), groups); + } + + /** + * Build the row key. + * @param profile The name of the profile. + * @param entity The name of the entity. + * @param period The period in which the measurement was taken. + * @param groups The groups. + * @return The HBase row key. + */ + public byte[] rowKey(String profile, String entity, ProfilePeriod period, List groups) { + + // row key = salt + prefix + group(s) + time + byte[] salt = getSalt(period, saltDivisor); + byte[] prefixKey = prefixKey(profile, entity); + byte[] groupKey = groupKey(groups); + byte[] timeKey = timeKey(period); + + int capacity = salt.length + prefixKey.length + groupKey.length + timeKey.length; + return ByteBuffer + .allocate(capacity) + .put(salt) + .put(prefixKey) + .put(groupKey) + .put(timeKey) + .array(); + } + + /** + * Builds the 'prefix' component of the row key. + * @param profile The name of the profile. + * @param entity The name of the entity. + */ + private static byte[] prefixKey(String profile, String entity) { + return ByteBuffer + .allocate(profile.length() + entity.length()) + .put(profile.getBytes()) + .put(entity.getBytes()) + .array(); + } + + /** + * Builds the 'group' component of the row key. + * @param groups The groups to include in the row key. + */ + private static byte[] groupKey(List groups) { + + StringBuilder builder = new StringBuilder(); + groups.forEach(g -> builder.append(g)); + String groupStr = builder.toString(); + + return ByteBuffer + .allocate(groupStr.length()) + .put(groupStr.getBytes()) + .array(); + } + + /** + * Builds the 'time' portion of the row key + * @param period The ProfilePeriod in which the ProfileMeasurement was taken. + */ + private static byte[] timeKey(ProfilePeriod period) { + return ByteBuffer + .allocate(4 * Integer.BYTES) + .putInt(period.getYear()) + .putInt(period.getDayOfYear()) + .putInt(period.getHour()) + .putInt(period.getPeriod()) + .array(); + } + + /** + * Calculates a salt value that is used as part of the row key. + * + * The salt is calculated as 'md5(timestamp) % N' where N is a configurable value that ideally + * is close to the number of nodes in the Hbase cluster. + * + * @param period The period in which a profile measurement is taken. + */ + public static byte[] getSalt(ProfilePeriod period, int saltDivisor) { + try { + MessageDigest digest = MessageDigest.getInstance("MD5"); + byte[] hash = digest.digest(timeKey(period)); + int salt = Bytes.toInt(hash) % saltDivisor; + return ByteBuffer + .allocate(Integer.BYTES) + .putInt(salt) + .array(); + + } catch(NoSuchAlgorithmException e) { + throw new RuntimeException(e); + } + } + + private static final double MINS_PER_HOUR = 60.0; +} diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/Serializer.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/Serializer.java new file mode 100644 index 0000000000..c0fe16f580 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/Serializer.java @@ -0,0 +1,92 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler.hbase; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Provides basic functionality to serialize and deserialize the allowed + * value types for a ProfileMeasurement. + */ +public class Serializer { + + private Serializer() { + // do not instantiate + } + + /** + * Serialize a profile measurement's value. + * + * The value produced by a Profile definition can be any numeric data type. The data + * type depends on how the profile is defined by the user. The user should be able to + * choose the data type that is most suitable for their use case. + * + * @param value The value to serialize. + */ + public static byte[] toBytes(Object value) { + byte[] result; + + if(value instanceof Integer) { + result = Bytes.toBytes((Integer) value); + } else if(value instanceof Double) { + result = Bytes.toBytes((Double) value); + } else if(value instanceof Short) { + result = Bytes.toBytes((Short) value); + } else if(value instanceof Long) { + result = Bytes.toBytes((Long) value); + } else if(value instanceof Float) { + result = Bytes.toBytes((Float) value); + } else { + throw new RuntimeException("Expected 'Number': actual=" + value); + } + + return result; + } + + /** + * Deserialize a profile measurement's value. + * + * The value produced by a Profile definition can be any numeric data type. The data + * type depends on how the profile is defined by the user. The user should be able to + * choose the data type that is most suitable for their use case. + * + * @param value The value to deserialize. + */ + public static T fromBytes(byte[] value, Class clazz) { + T result; + + if(clazz == Integer.class) { + result = clazz.cast(Bytes.toInt(value)); + } else if(clazz == Double.class) { + result = clazz.cast(Bytes.toDouble(value)); + } else if(clazz == Short.class) { + result = clazz.cast(Bytes.toShort(value)); + } else if(clazz == Long.class) { + result = clazz.cast(Bytes.toLong(value)); + } else if(clazz == Float.class) { + result = clazz.cast(Bytes.toFloat(value)); + } else { + throw new RuntimeException("Expected 'Number': actual=" + clazz); + } + + return result; + } +} diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java new file mode 100644 index 0000000000..a75e382e0f --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java @@ -0,0 +1,60 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler.hbase; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.storm.hbase.common.ColumnList; + +/** + * A ColumnBuilder that writes only the value of a ProfileMeasurement. + */ +public class ValueOnlyColumnBuilder implements ColumnBuilder { + + /** + * The column family storing the profile data. + */ + private String columnFamily; + + private byte[] columnFamilyBytes; + + public ValueOnlyColumnBuilder() { + setColumnFamily("P"); + } + + public ValueOnlyColumnBuilder(String columnFamily) { + setColumnFamily(columnFamily); + } + + @Override + public ColumnList columns(ProfileMeasurement measurement) { + + ColumnList cols = new ColumnList(); + cols.addColumn(columnFamilyBytes, QVALUE, Serializer.toBytes(measurement.getValue())); + + return cols; + } + + public void setColumnFamily(String columnFamily) { + this.columnFamily = columnFamily; + this.columnFamilyBytes = Bytes.toBytes(columnFamily); + } +} diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java index 7eaf38adb8..ae786f2b57 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java @@ -27,7 +27,6 @@ import org.apache.metron.common.dsl.VariableResolver; import org.apache.metron.common.stellar.StellarProcessor; import org.apache.metron.common.utils.ConversionUtils; -import org.json.simple.JSONObject; import java.io.Serializable; import java.util.HashMap; @@ -43,14 +42,22 @@ public class DefaultStellarExecutor implements StellarExecutor, Serializable { */ private Map state; + /** + * Provides additional context for initializing certain Stellar functions. For + * example, references to find Zookeeper or HBase. + */ + private Context context; + public DefaultStellarExecutor() { clearState(); + context = Context.EMPTY_CONTEXT(); } /** * @param initialState Initial state loaded into the execution environment. */ public DefaultStellarExecutor(Map initialState) { + this(); this.state = new HashMap<>(initialState); } @@ -65,11 +72,10 @@ public Map getState() { * @param variable The variable name to assign to. * @param expression The expression to execute. * @param message The message that provides additional context for the expression. - * @param stellarContext The context which holds global state for Stellar functions */ @Override - public void assign(String variable, String expression, JSONObject message, Context stellarContext) { - Object result = execute(expression, message, stellarContext); + public void assign(String variable, String expression, Map message) { + Object result = execute(expression, message); state.put(variable, result); } @@ -80,11 +86,10 @@ public void assign(String variable, String expression, JSONObject message, Conte * @param message The message that is accessible when Stellar is executed. * @param clazz The expected class of the expression's result. * @param The expected class of the expression's result. - * @param stellarContext The context which holds global state for Stellar functions */ @Override - public T execute(String expr, JSONObject message, Class clazz, Context stellarContext) { - Object resultObject = execute(expr, message, stellarContext); + public T execute(String expr, Map message, Class clazz) { + Object resultObject = execute(expr, message); // perform type conversion, if necessary T result = ConversionUtils.convert(resultObject, clazz); @@ -101,18 +106,27 @@ public void clearState() { this.state = new HashMap<>(); } + /** + * Sets the Context for the Stellar execution environment. This provides global data used + * to initialize Stellar functions. + * @param context The Stellar context. + */ + @Override + public void setContext(Context context) { + this.context = context; + } + /** * Execute a Stellar expression. * * @param expr The expression to execute. * @param msg The message that is accessible when Stellar is executed. - * @param stellarContext The context which holds global state for Stellar functions */ - private Object execute(String expr, JSONObject msg, Context stellarContext) { + private Object execute(String expr, Map msg) { try { VariableResolver resolver = new MapVariableResolver(state, msg); StellarProcessor processor = new StellarProcessor(); - return processor.parse(expr, resolver, StellarFunctions.FUNCTION_RESOLVER(), stellarContext); + return processor.parse(expr, resolver, StellarFunctions.FUNCTION_RESOLVER(), context); } catch (ParseException e) { throw new ParseException(String.format("Bad expression: expr=%s, msg=%s, state=%s", expr, msg, state)); diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java index 8d6660c0bf..66e3ad15d3 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java @@ -40,20 +40,18 @@ public interface StellarExecutor { * @param variable The name of the variable to assign to. * @param expression The expression to execute. * @param message The message that provides additional context for the expression. - * @param stellarContext The context which holds global state for Stellar functions */ - void assign(String variable, String expression, JSONObject message, Context stellarContext); + void assign(String variable, String expression, Map message); /** * Execute a Stellar expression and return the result. * * @param expression The expression to execute. - * @param message The message that is accessible when Stellar is executed. + * @param message A map of values, most often the JSON message itself, that is accessible within Stellar. * @param clazz The expected class of the expression's result. * @param The expected class of the expression's result. - * @param stellarContext The context which holds global state for Stellar functions */ - T execute(String expression, JSONObject message, Class clazz, Context stellarContext); + T execute(String expression, Map message, Class clazz); /** * The current state of the Stellar execution environment. @@ -64,4 +62,11 @@ public interface StellarExecutor { * Removes all state from the execution environment. */ void clearState(); + + /** + * Sets the Context for the Stellar execution environment. This provides global data used + * to initialize Stellar functions. + * @param context The Stellar context. + */ + void setContext(Context context); } diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java new file mode 100644 index 0000000000..ada7365d9d --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java @@ -0,0 +1,341 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Tests the ProfilePeriod class. + */ +public class ProfilePeriodTest { + + /** + * Thu, Aug 25 2016 09:27:10 EST + * Thu, Aug 25 2016 13:27:10 GMT + * + * 238th day of the year + */ + private long AUG2016 = 1472131630748L; + + /** + * The number of periods per hour must always ensure that the first period falls on the start of each hour. This + * means that the number of periods must be a divisor or multiple of 60. + */ + @Test(expected = RuntimeException.class) + public void testInvalidPeriodsPerHour() { + new ProfilePeriod(AUG2016, 241); + } + + @Test + public void test1PeriodPerHour() { + ProfilePeriod period = new ProfilePeriod(AUG2016, 1); + + assertEquals(2016, period.getYear()); + assertEquals(238, period.getDayOfYear()); + assertEquals(13, period.getHour()); + assertEquals(0, period.getPeriod()); + } + + @Test + public void test2PeriodsPerHour() { + ProfilePeriod period = new ProfilePeriod(AUG2016, 2); + + assertEquals(2016, period.getYear()); + assertEquals(238, period.getDayOfYear()); + assertEquals(13, period.getHour()); + assertEquals(0, period.getPeriod()); + } + + @Test + public void test3PeriodsPerHour() { + ProfilePeriod period = new ProfilePeriod(AUG2016, 3); + + assertEquals(2016, period.getYear()); + assertEquals(238, period.getDayOfYear()); + assertEquals(13, period.getHour()); + assertEquals(1, period.getPeriod()); + } + + @Test + public void test4PeriodsPerHour() { + ProfilePeriod period = new ProfilePeriod(AUG2016, 4); + + assertEquals(2016, period.getYear()); + assertEquals(238, period.getDayOfYear()); + assertEquals(13, period.getHour()); + assertEquals(1, period.getPeriod()); + } + + @Test + public void test60PeriodsPerHour() { + ProfilePeriod period = new ProfilePeriod(AUG2016, 60); + + assertEquals(2016, period.getYear()); + assertEquals(238, period.getDayOfYear()); + assertEquals(13, period.getHour()); + assertEquals(27, period.getPeriod()); + } + + @Test + public void test240PeriodsPerHour() { + ProfilePeriod period = new ProfilePeriod(AUG2016, 240); + + assertEquals(2016, period.getYear()); + assertEquals(238, period.getDayOfYear()); + assertEquals(13, period.getHour()); + assertEquals(108, period.getPeriod()); + } + + + @Test + public void testNextWith2PeriodsPerHour() { + int periodsPerHour = 2; + + ProfilePeriod first = new ProfilePeriod(AUG2016, periodsPerHour); + assertEquals(2016, first.getYear()); + assertEquals(238, first.getDayOfYear()); + assertEquals(13, first.getHour()); + assertEquals(0, first.getPeriod()); + + // find the next period + ProfilePeriod second = first.next(); + assertEquals(2016, second.getYear()); + assertEquals(238, second.getDayOfYear()); + assertEquals(13, second.getHour()); + assertEquals(1, second.getPeriod()); + assertEquals(periodsPerHour, second.getPeriodsPerHour()); + + // find the next period + ProfilePeriod third = second.next(); + assertEquals(2016, third.getYear()); + assertEquals(238, third.getDayOfYear()); + assertEquals(14, third.getHour()); + assertEquals(0, third.getPeriod()); + assertEquals(periodsPerHour, third.getPeriodsPerHour()); + + // find the next period + ProfilePeriod fourth = third.next(); + assertEquals(2016, fourth.getYear()); + assertEquals(238, fourth.getDayOfYear()); + assertEquals(14, fourth.getHour()); + assertEquals(1, fourth.getPeriod()); + assertEquals(periodsPerHour, fourth.getPeriodsPerHour()); + + // find the next period + ProfilePeriod fifth = fourth.next(); + assertEquals(2016, fifth.getYear()); + assertEquals(238, fifth.getDayOfYear()); + assertEquals(15, fifth.getHour()); + assertEquals(0, fifth.getPeriod()); + assertEquals(periodsPerHour, fifth.getPeriodsPerHour()); + + // find the next period + ProfilePeriod sixth = fifth.next(); + assertEquals(2016, sixth.getYear()); + assertEquals(238, sixth.getDayOfYear()); + assertEquals(15, sixth.getHour()); + assertEquals(1, sixth.getPeriod()); + assertEquals(periodsPerHour, sixth.getPeriodsPerHour()); + + // find the next period + ProfilePeriod seventh = sixth.next(); + assertEquals(2016, seventh.getYear()); + assertEquals(238, seventh.getDayOfYear()); + assertEquals(16, seventh.getHour()); + assertEquals(0, seventh.getPeriod()); + assertEquals(periodsPerHour, seventh.getPeriodsPerHour()); + } + + @Test + public void testNextWith4PeriodsPerHour() { + int periodsPerHour = 4; + + ProfilePeriod first = new ProfilePeriod(AUG2016, periodsPerHour); + assertEquals(2016, first.getYear()); + assertEquals(238, first.getDayOfYear()); + assertEquals(13, first.getHour()); + assertEquals(1, first.getPeriod()); + + // find the next period + ProfilePeriod second = first.next(); + assertEquals(2016, second.getYear()); + assertEquals(238, second.getDayOfYear()); + assertEquals(13, second.getHour()); + assertEquals(2, second.getPeriod()); + assertEquals(periodsPerHour, second.getPeriodsPerHour()); + + // find the next period + ProfilePeriod third = second.next(); + assertEquals(2016, third.getYear()); + assertEquals(238, third.getDayOfYear()); + assertEquals(13, third.getHour()); + assertEquals(3, third.getPeriod()); + assertEquals(periodsPerHour, third.getPeriodsPerHour()); + + // find the next period + ProfilePeriod fourth = third.next(); + assertEquals(2016, fourth.getYear()); + assertEquals(238, fourth.getDayOfYear()); + assertEquals(14, fourth.getHour()); + assertEquals(0, fourth.getPeriod()); + assertEquals(periodsPerHour, fourth.getPeriodsPerHour()); + } + + @Test + public void testNextWith10PeriodsPerHour() { + int periodsPerHour = 10; + + ProfilePeriod first = new ProfilePeriod(AUG2016, periodsPerHour); + assertEquals(2016, first.getYear()); + assertEquals(238, first.getDayOfYear()); + assertEquals(13, first.getHour()); + assertEquals(4, first.getPeriod()); + + // find the next period + ProfilePeriod second = first.next(); + assertEquals(2016, second.getYear()); + assertEquals(238, second.getDayOfYear()); + assertEquals(13, second.getHour()); + assertEquals(5, second.getPeriod()); + assertEquals(periodsPerHour, second.getPeriodsPerHour()); + + // find the next period + ProfilePeriod third = second.next(); + assertEquals(2016, third.getYear()); + assertEquals(238, third.getDayOfYear()); + assertEquals(13, third.getHour()); + assertEquals(6, third.getPeriod()); + assertEquals(periodsPerHour, third.getPeriodsPerHour()); + + // find the next period + ProfilePeriod fourth = third.next(); + assertEquals(2016, fourth.getYear()); + assertEquals(238, fourth.getDayOfYear()); + assertEquals(13, fourth.getHour()); + assertEquals(7, fourth.getPeriod()); + assertEquals(periodsPerHour, fourth.getPeriodsPerHour()); + + // find the next period + ProfilePeriod fifth = fourth.next(); + assertEquals(2016, fifth.getYear()); + assertEquals(238, fifth.getDayOfYear()); + assertEquals(13, fifth.getHour()); + assertEquals(8, fifth.getPeriod()); + assertEquals(periodsPerHour, fifth.getPeriodsPerHour()); + + // find the next period + ProfilePeriod sixth = fifth.next(); + assertEquals(2016, sixth.getYear()); + assertEquals(238, sixth.getDayOfYear()); + assertEquals(13, sixth.getHour()); + assertEquals(9, sixth.getPeriod()); + assertEquals(periodsPerHour, sixth.getPeriodsPerHour()); + + // find the next period + ProfilePeriod seventh = sixth.next(); + assertEquals(2016, seventh.getYear()); + assertEquals(238, seventh.getDayOfYear()); + assertEquals(14, seventh.getHour()); + assertEquals(0, seventh.getPeriod()); + assertEquals(periodsPerHour, seventh.getPeriodsPerHour()); + } + + @Test + public void testNextWith240PeriodsPerHour() { + final int periodsPerHour = 240; + + ProfilePeriod p = new ProfilePeriod(AUG2016, periodsPerHour); + assertEquals(2016, p.getYear()); + assertEquals(238, p.getDayOfYear()); + assertEquals(13, p.getHour()); + assertEquals(108, p.getPeriod()); + + int lastPeriod = p.getPeriod(); + for(int i=0; i<(periodsPerHour - 108); i++) { + p = p.next(); + + // validate the next period + assertEquals(2016, p.getYear()); + assertEquals(238, p.getDayOfYear()); + assertEquals(periodsPerHour, p.getPeriodsPerHour()); + + int nextPeriod = lastPeriod + 1; + boolean rolloverToNextHour = nextPeriod >= periodsPerHour; + if(!rolloverToNextHour) { + // still within the same hour + assertEquals(13, p.getHour()); + assertEquals(nextPeriod, p.getPeriod()); + + } else { + // rollover to next hour + assertEquals(14, p.getHour()); + assertEquals(0, p.getPeriod()); + break; + } + + lastPeriod = p.getPeriod(); + } + } + + /** + * With 2 periods per hour, 'Thu, Aug 25 2016 13:27:10 GMT' falls within the 1st period. + * Period starts at 'Thu, Aug 25 2016 13:00:00 000 GMT' ~ 1472130000000L + */ + @Test + public void testTimeInMillisWith2PeriodsPerHour() { + final ProfilePeriod period = new ProfilePeriod(AUG2016, 2); + assertEquals(1472130000000L, period.getTimeInMillis()); + } + + /** + * With 4 periods per hour, 'Thu, Aug 25 2016 13:27:10 GMT' falls within the 2nd period. + * Period starts at 'Thu, Aug 25 2016 13:15:00 000 GMT' ~ 1472130900000L + */ + @Test + public void testTimeInMillisWith4PeriodsPerHour() { + final ProfilePeriod period = new ProfilePeriod(AUG2016, 4); + assertEquals(1472130900000L, period.getTimeInMillis()); + } + + /** + * With 60 periods per hour, 'Thu, Aug 25 2016 13:27:10 GMT' falls within the 27th period. + * Period starts at 'Thu, Aug 25 2016 13:27:00 000 GMT' ~ 1472131620000L + */ + @Test + public void testTimeInMillisWith60PeriodsPerHour() { + final ProfilePeriod period = new ProfilePeriod(AUG2016, 60); + assertEquals(1472131620000L, period.getTimeInMillis()); + } + + /** + * With 240 periods per hour, 'Thu, Aug 25 2016 13:27:10 GMT' falls within the 108th period. + * Period starts at 'Thu, Aug 25 2016 13:27:00 000 GMT' ~ 1472131620000L + */ + @Test + public void testTimeInMillisWith240PeriodsPerHour() { + final ProfilePeriod period = new ProfilePeriod(AUG2016, 240); + assertEquals(1472131620000L, period.getTimeInMillis()); + } + + +} diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java index 5390bd5f58..df8b3358cc 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java @@ -164,6 +164,7 @@ private ProfileBuilderBolt createBolt() throws IOException { bolt.setCuratorFramework(client); bolt.setTreeCache(cache); bolt.setExecutor(new DefaultStellarExecutor()); + bolt.setPeriodsPerHour(4); bolt.prepare(new HashMap<>(), topologyContext, outputCollector); return bolt; diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java new file mode 100644 index 0000000000..2d28a65a93 --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java @@ -0,0 +1,114 @@ +/* + * + * 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.bolt; + +import backtype.storm.tuple.Tuple; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.metron.profiler.hbase.RowKeyBuilder; +import org.apache.metron.profiler.stellar.DefaultStellarExecutor; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; + +import java.util.Arrays; +import java.util.List; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests the ProfileHBaseMapper class. + */ +public class ProfileHBaseMapperTest { + + Tuple tuple; + ProfileHBaseMapper mapper; + ProfileMeasurement measurement; + DefaultStellarExecutor executor; + RowKeyBuilder rowKeyBuilder; + + @Before + public void setup() { + executor = new DefaultStellarExecutor(); + + rowKeyBuilder = mock(RowKeyBuilder.class); + + mapper = new ProfileHBaseMapper(); + mapper.setExecutor(executor); + mapper.setRowKeyBuilder(rowKeyBuilder); + + measurement = new ProfileMeasurement("profile", "entity", 20000, 4); + measurement.setValue(22); + + // the tuple will contain the original message + tuple = mock(Tuple.class); + when(tuple.getValueByField(eq("measurement"))).thenReturn(measurement); + } + + /** + * The mapper should execute the 'groupBy' Stellar expressions and use that to generate + * a row key. + */ + @Test + public void testExecuteGroupBy() throws Exception { + + // setup - expression that refers to the ProfileMeasurement.end + measurement.setGroupBy(Arrays.asList("2 + 2")); + + // execute + mapper.rowKey(tuple); + + // capture the ProfileMeasurement that should be emitted + ArgumentCaptor arg = ArgumentCaptor.forClass(List.class); + verify(rowKeyBuilder).rowKey(any(), arg.capture()); + + // validate + List actual = arg.getValue(); + Assert.assertEquals(4.0, actual.get(0)); + } + + /** + * The mapper should execute each 'groupBy' Stellar expression and use that to generate + * a row key. There can be multiple. + */ + @Test + public void testExecuteMultipleGroupBys() throws Exception { + + // setup - expression that refers to the ProfileMeasurement.end + measurement.setGroupBy(Arrays.asList("2 + 2", "4 + 4")); + + // execute + mapper.rowKey(tuple); + + // capture the ProfileMeasurement that should be emitted + ArgumentCaptor arg = ArgumentCaptor.forClass(List.class); + verify(rowKeyBuilder).rowKey(any(), arg.capture()); + + // validate + List actual = arg.getValue(); + Assert.assertEquals(4.0, actual.get(0)); + Assert.assertEquals(8.0, actual.get(1)); + } +} diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java new file mode 100644 index 0000000000..dce7757d90 --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/hbase/SaltyRowKeyBuilderTest.java @@ -0,0 +1,275 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler.hbase; + +import backtype.storm.tuple.Tuple; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.metron.profiler.ProfilePeriod; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Formatter; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.core.IsEqual.equalTo; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Tests the SaltyRowKeyBuilder. + */ +public class SaltyRowKeyBuilderTest { + + private static final int saltDivisor = 1000; + private static final int periodsPerHour = 4; + + private SaltyRowKeyBuilder rowKeyBuilder; + private ProfileMeasurement measurement; + private Tuple tuple; + + /** + * Thu, Aug 25 2016 09:27:10 EST + * Thu, Aug 25 2016 13:27:10 GMT + * + * 238th day of the year + */ + private long AUG2016 = 1472131630748L; + + @Before + public void setup() throws Exception { + + // a profile measurement + measurement = new ProfileMeasurement("profile", "entity", AUG2016, periodsPerHour); + measurement.setValue(22); + + // the tuple will contain the original message + tuple = mock(Tuple.class); + when(tuple.getValueByField(eq("measurement"))).thenReturn(measurement); + } + + /** + * Build a row key that includes only one group. + */ + @Test + public void testRowKeyWithOneGroup() throws Exception { + // setup + rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodsPerHour); + List groups = Arrays.asList("group1"); + + // the expected row key + ByteBuffer buffer = ByteBuffer + .allocate(100) + .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor)) + .put(measurement.getProfileName().getBytes()) + .put(measurement.getEntity().getBytes()) + .put("group1".getBytes()) + .putInt(2016) + .putInt(238) + .putInt(13) + .putInt(1); + + buffer.flip(); + final byte[] expected = new byte[buffer.limit()]; + buffer.get(expected, 0, buffer.limit()); + + // validate + byte[] actual = rowKeyBuilder.rowKey(measurement, groups); + Assert.assertTrue(Arrays.equals(expected, actual)); + } + + /** + * Build a row key that includes two groups. + */ + @Test + public void testRowKeyWithTwoGroups() throws Exception { + // setup + rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodsPerHour); + List groups = Arrays.asList("group1","group2"); + + // the expected row key + ByteBuffer buffer = ByteBuffer + .allocate(100) + .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor)) + .put(measurement.getProfileName().getBytes()) + .put(measurement.getEntity().getBytes()) + .put("group1".getBytes()) + .put("group2".getBytes()) + .putInt(2016) + .putInt(238) + .putInt(13) + .putInt(1); + + buffer.flip(); + final byte[] expected = new byte[buffer.limit()]; + buffer.get(expected, 0, buffer.limit()); + + // validate + byte[] actual = rowKeyBuilder.rowKey(measurement, groups); + Assert.assertTrue(Arrays.equals(expected, actual)); + } + + /** + * Build a row key that includes a single group that is an integer. + */ + @Test + public void testRowKeyWithOneIntegerGroup() throws Exception { + // setup + rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodsPerHour); + List groups = Arrays.asList(200); + + // the expected row key + ByteBuffer buffer = ByteBuffer + .allocate(100) + .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor)) + .put(measurement.getProfileName().getBytes()) + .put(measurement.getEntity().getBytes()) + .put("200".getBytes()) + .putInt(2016) + .putInt(238) + .putInt(13) + .putInt(1); + + buffer.flip(); + final byte[] expected = new byte[buffer.limit()]; + buffer.get(expected, 0, buffer.limit()); + + // validate + byte[] actual = rowKeyBuilder.rowKey(measurement, groups); + Assert.assertTrue(Arrays.equals(expected, actual)); + } + + /** + * Build a row key that includes a single group that is an integer. + */ + @Test + public void testRowKeyWithMixedGroups() throws Exception { + // setup + rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodsPerHour); + List groups = Arrays.asList(200, "group1"); + + // the expected row key + ByteBuffer buffer = ByteBuffer + .allocate(100) + .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor)) + .put(measurement.getProfileName().getBytes()) + .put(measurement.getEntity().getBytes()) + .put("200".getBytes()) + .put("group1".getBytes()) + .putInt(2016) + .putInt(238) + .putInt(13) + .putInt(1); + + buffer.flip(); + final byte[] expected = new byte[buffer.limit()]; + buffer.get(expected, 0, buffer.limit()); + + // validate + byte[] actual = rowKeyBuilder.rowKey(measurement, groups); + Assert.assertTrue(Arrays.equals(expected, actual)); + } + + /** + * Build a row key that does not include any groups. + */ + @Test + public void testRowKeyWithNoGroup() throws Exception { + // setup + rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodsPerHour); + List groups = Collections.emptyList(); + + // the expected row key + ByteBuffer buffer = ByteBuffer + .allocate(100) + .put(SaltyRowKeyBuilder.getSalt(measurement.getPeriod(), saltDivisor)) + .put(measurement.getProfileName().getBytes()) + .put(measurement.getEntity().getBytes()) + .putInt(2016) + .putInt(238) + .putInt(13) + .putInt(1); + + buffer.flip(); + final byte[] expected = new byte[buffer.limit()]; + buffer.get(expected, 0, buffer.limit()); + + // validate + byte[] actual = rowKeyBuilder.rowKey(measurement, groups); + Assert.assertTrue(Arrays.equals(expected, actual)); + } + + /** + * `rowKeys` should return all of the row keys needed to retrieve the profile values over a given time horizon. + */ + @Test + public void testRowKeys() throws Exception { + int hoursAgo = 1; + + // setup + List groups = Collections.emptyList(); + rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodsPerHour); + + // a dummy profile measurement + long oldest = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(hoursAgo); + ProfileMeasurement m = new ProfileMeasurement("profile", "entity", oldest, periodsPerHour); + m.setValue(22); + + // generate a list of expected keys + List expectedKeys = new ArrayList<>(); + for (int i=0; i<(hoursAgo * periodsPerHour)+1; i++) { + + // generate the expected key + byte[] rk = rowKeyBuilder.rowKey(m, groups); + expectedKeys.add(rk); + + // advance to the next period + ProfilePeriod next = m.getPeriod().next(); + m = new ProfileMeasurement("profile", "entity", next.getTimeInMillis(), periodsPerHour); + } + + // execute + List actualKeys = rowKeyBuilder.rowKeys(measurement.getProfileName(), measurement.getEntity(), groups, hoursAgo, TimeUnit.HOURS); + + // validate - expectedKeys == actualKeys + for(int i=0; imetron-maas-service metron-maas-common metron-profiler + metron-profiler-client diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java index f832e1d9e1..34dab139bf 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java @@ -18,7 +18,9 @@ package org.apache.metron.common.configuration.profiler; import java.io.Serializable; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; /** @@ -27,36 +29,56 @@ public class ProfileConfig implements Serializable { /** - * The name of the profile. + * A unique name identifying the profile. The field is treated as a string. */ private String profile; /** - * Stella code that when executed results in the name of the entity being profiled. A - * profile is created 'for-each' of these; hence the name. + * A separate profile is maintained for each of these. This is effectively the + * entity that the profile is describing. The field is expected to contain a + * Stellar expression whose result is the entity name. For example, if `ip_src_addr` + * then a separate profile would be maintained for each unique IP source address in + * the data; 10.0.0.1, 10.0.0.2, etc. */ private String foreach; /** - * Stella code that when executed determines whether a message should be included in this - * profile. + * An expression that determines if a message should be applied to the profile. A + * Stellar expression is expected that when executed returns a boolean. A message + * is only applied to a profile if this condition is true. This allows a profile + * to filter the messages that it receives. */ private String onlyif; /** - * Stella code that when executed results in a single measurement that is stored with the Profile. + * A set of expressions that is executed at the start of a window period. A map is + * expected where the key is the variable name and the value is a Stellar expression. + * The map can contain 0 or more variables/expressions. At the start of each window + * period the expression is executed once and stored in a variable with the given + * name. */ - private String result; + private Map init = new HashMap<>(); /** - * Defines how the state is initialized before any messages are received. + * A set of expressions that is executed when a message is applied to the profile. + * A map is expected where the key is the variable name and the value is a Stellar + * expression. The map can include 0 or more variables/expressions. */ - private Map init = new HashMap<>(); + private Map update = new HashMap<>(); /** - * Defines how the state is updated when a new message is received. + * A list of Stellar expressions that is executed in order and used to group the + * resulting profile data. */ - private Map update = new HashMap<>(); + private List groupBy = new ArrayList<>(); + + /** + * A Stellar expression that is executed when the window period expires. The + * expression is expected to in some way summarize the messages that were applied + * to the profile over the window period. The expression must result in a numeric + * value such as a Double, Long, Float, Short, or Integer. + */ + private String result; public String getProfile() { return profile; diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctions.java index ee468b8345..8ca90a85b8 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctions.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/StellarFunctions.java @@ -56,6 +56,7 @@ public enum StellarFunctions implements StellarFunction { TO_STRING(new StringFunctions.ToString()), TO_INTEGER(new ConversionFunctions.Cast<>(Integer.class)), TO_DOUBLE(new ConversionFunctions.Cast<>(Double.class)), + TO_LONG(new ConversionFunctions.Cast<>(Long.class)), // map functions MAP_GET(new MapFunctions.MapGet()), @@ -74,6 +75,13 @@ public enum StellarFunctions implements StellarFunction { // date functions TO_EPOCH_TIMESTAMP(new DateFunctions.ToTimestamp()), + YEAR(new DateFunctions.Year()), + MONTH(new DateFunctions.MonthOfYear()), + DAY_OF_MONTH(new DateFunctions.DayOfMonth()), + DAY_OF_WEEK(new DateFunctions.DayOfWeek()), + WEEK_OF_MONTH(new DateFunctions.WeekOfMonth()), + WEEK_OF_YEAR(new DateFunctions.WeekOfYear()), + DAY_OF_YEAR(new DateFunctions.DayOfYear()), // validation functions IS_EMPTY ( new DataStructureFunctions.IsEmpty()), diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DateFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DateFunctions.java index 6008cfc48b..41196c063b 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DateFunctions.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/DateFunctions.java @@ -22,20 +22,26 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import org.apache.metron.common.dsl.BaseStellarFunction; +import org.apache.metron.common.utils.ConversionUtils; import java.text.ParseException; import java.text.SimpleDateFormat; +import java.util.Calendar; import java.util.List; import java.util.Optional; import java.util.TimeZone; import java.util.concurrent.ExecutionException; -import java.util.function.Function; +/** + * Stellar data functions. + */ public class DateFunctions { private static class TimezonedFormat { + private String format; private Optional timezone; + public TimezonedFormat(String format, String timezone) { this.format = format; this.timezone = Optional.of(timezone); @@ -45,6 +51,7 @@ public TimezonedFormat(String format) { this.format = format; this.timezone = Optional.empty(); } + public SimpleDateFormat toDateFormat() { return createFormat(format, timezone); } @@ -58,7 +65,6 @@ public boolean equals(Object o) { if (format != null ? !format.equals(that.format) : that.format != null) return false; return timezone != null ? timezone.equals(that.timezone) : that.timezone == null; - } @Override @@ -69,19 +75,19 @@ public int hashCode() { } } - private static LoadingCache> formatCache - = CacheBuilder.newBuilder().build(new CacheLoader>() { - @Override - public ThreadLocal load(final TimezonedFormat format) throws Exception { - return new ThreadLocal() { - @Override - public SimpleDateFormat initialValue() { - return format.toDateFormat(); - } - }; - } - } - ); + private static LoadingCache> formatCache = + CacheBuilder.newBuilder().build( + new CacheLoader>() { + @Override + public ThreadLocal load(final TimezonedFormat format) throws Exception { + return new ThreadLocal() { + @Override + public SimpleDateFormat initialValue() { + return format.toDateFormat(); + } + }; + } + }); public static SimpleDateFormat createFormat(String format, Optional timezone) { SimpleDateFormat sdf = new SimpleDateFormat(format); @@ -90,19 +96,21 @@ public static SimpleDateFormat createFormat(String format, Optional time } return sdf; } + public static long getEpochTime(String date, String format, Optional timezone) throws ExecutionException, ParseException { - TimezonedFormat fmt = null; + TimezonedFormat fmt; if(timezone.isPresent()) { fmt = new TimezonedFormat(format, timezone.get()); - } - else { + } else { fmt = new TimezonedFormat(format); } SimpleDateFormat sdf = formatCache.get(fmt).get(); return sdf.parse(date).getTime(); } - + /** + * Stellar Function: TO_EPOCH_TIMESTAMP + */ public static class ToTimestamp extends BaseStellarFunction { @Override public Object apply(List objects) { @@ -114,17 +122,176 @@ public Object apply(List objects) { } if(dateObj != null && formatObj != null) { try { - return getEpochTime(dateObj.toString() - , formatObj.toString() - , tzObj == null?Optional.empty():Optional.of(tzObj.toString()) - ); - } catch (ExecutionException e) { - return null; - } catch (ParseException e) { + Optional tz = (tzObj == null) ? Optional.empty() : Optional.of(tzObj.toString()); + return getEpochTime(dateObj.toString(), formatObj.toString(), tz); + + } catch (ExecutionException | ParseException e) { return null; } } return null; } } + + /** + * Stellar Function: DAY_OF_WEEK + * + * The numbered day within the week. The first day of the week, Sunday, has a value of 1. + */ + public static class DayOfWeek extends BaseStellarFunction { + @Override + public Object apply(List args) { + + // expect epoch milliseconds + Long epochMillis = ConversionUtils.convert(args.get(0), Long.class); + if(epochMillis == null) { + return null; + } + + // create a calendar + Calendar calendar = Calendar.getInstance(); + calendar.setTimeInMillis(epochMillis); + + return calendar.get(Calendar.DAY_OF_WEEK); + } + } + + /** + * Stellar Function: DAY_OF_MONTH + * + * The day within the month. The first day within the month has a value of 1. + */ + public static class DayOfMonth extends BaseStellarFunction { + @Override + public Object apply(List args) { + + // expect epoch milliseconds + Long epochMillis = ConversionUtils.convert(args.get(0), Long.class); + if(epochMillis == null) { + return null; + } + + // create a calendar + Calendar calendar = Calendar.getInstance(); + calendar.setTimeInMillis(epochMillis); + + return calendar.get(Calendar.DAY_OF_MONTH); + } + } + + /** + * Stellar Function: WEEK_OF_MONTH + * + * The numbered week within the month. The first week has a value of 1. + */ + public static class WeekOfMonth extends BaseStellarFunction { + @Override + public Object apply(List args) { + + // expect epoch milliseconds + Long epochMillis = ConversionUtils.convert(args.get(0), Long.class); + if(epochMillis == null) { + return null; + } + + // create a calendar + Calendar calendar = Calendar.getInstance(); + calendar.setTimeInMillis(epochMillis); + + return calendar.get(Calendar.WEEK_OF_MONTH); + } + } + + /** + * Stellar Function: WEEK_OF_YEAR + * + * The numbered week within the year. The first week in the year has a value of 1. + */ + public static class WeekOfYear extends BaseStellarFunction { + @Override + public Object apply(List args) { + + // expect epoch milliseconds + Long epochMillis = ConversionUtils.convert(args.get(0), Long.class); + if(epochMillis == null) { + return null; + } + + // create a calendar + Calendar calendar = Calendar.getInstance(); + calendar.setTimeInMillis(epochMillis); + + return calendar.get(Calendar.WEEK_OF_YEAR); + } + } + + /** + * Stellar Function: MONTH + * + * A number representing the month. The first month, January, has a value of 0. + */ + public static class MonthOfYear extends BaseStellarFunction { + @Override + public Object apply(List args) { + + // expect epoch milliseconds + Long epochMillis = ConversionUtils.convert(args.get(0), Long.class); + if(epochMillis == null) { + return null; + } + + // create a calendar + Calendar calendar = Calendar.getInstance(); + calendar.setTimeInMillis(epochMillis); + + return calendar.get(Calendar.MONTH); + } + } + + /** + * Stellar Function: YEAR + * + * The calendar year. + */ + public static class Year extends BaseStellarFunction { + @Override + public Object apply(List args) { + + // expect epoch milliseconds + Long epochMillis = ConversionUtils.convert(args.get(0), Long.class); + if(epochMillis == null) { + return null; + } + + // create a calendar + Calendar calendar = Calendar.getInstance(); + calendar.setTimeInMillis(epochMillis); + + return calendar.get(Calendar.YEAR); + } + } + + /** + * Stellar Function: DAY_OF_YEAR + * + * The day number within the year. The first day of the year has value of 1. + */ + public static class DayOfYear extends BaseStellarFunction { + @Override + public Object apply(List args) { + + // expect epoch milliseconds + Long epochMillis = ConversionUtils.convert(args.get(0), Long.class); + if(epochMillis == null) { + return null; + } + + // create a calendar + Calendar calendar = Calendar.getInstance(); + calendar.setTimeInMillis(epochMillis); + + return calendar.get(Calendar.DAY_OF_YEAR); + } + } } + diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java index adf439bf17..325a2fff0b 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java @@ -20,9 +20,9 @@ package org.apache.metron.common.dsl.functions; +import org.apache.commons.math3.stat.descriptive.SummaryStatistics; import org.apache.metron.common.dsl.BaseStellarFunction; -import java.util.Collections; import java.util.List; import static org.apache.metron.common.utils.ConversionUtils.convert; @@ -33,63 +33,39 @@ */ public class StellarStatisticsFunctions { - /** - * Initializes the summary statistics. - * - * Initialization can occur from either STATS_INIT and STATS_ADD. - */ - private static StellarStatistics statsInit(List args) { - int windowSize = 0; - if(args.size() > 0 && args.get(0) instanceof Number) { - windowSize = convert(args.get(0), Integer.class); - } - - return new StellarStatistics(windowSize); - } - /** * Initialize the summary statistics. * * STATS_INIT (window_size) * - * window_size The number of input data values to maintain in a rolling window - * in memory. If equal to 0, then no rolling window is maintained. - * Using no rolling window is less memory intensive, but cannot - * calculate certain statistics like percentiles and kurtosis. + * @param window_size The number of input data values to maintain in a rolling window + * in memory. If equal to 0, then no rolling window is maintained. + * Using no rolling window is less memory intensive, but cannot + * calculate certain statistics like percentiles and kurtosis. */ public static class Init extends BaseStellarFunction { @Override public Object apply(List args) { - return statsInit(args); + int windowSize = (args.size() > 0 && args.get(0) instanceof Number) ? ((Number) args.get(0)).intValue() : 0; + return new StellarStatistics(windowSize); } } /** * Add an input value to those that are used to calculate the summary statistics. * - * STATS_ADD (stats, value [, value2, value3, ...]) + * STATS_ADD (value, stats) */ public static class Add extends BaseStellarFunction { @Override public Object apply(List args) { - - // initialize a stats object, if one does not already exist - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - if(stats == null) { - stats = statsInit(Collections.emptyList()); - } - - // add each of the numeric values - for(int i=1; i args) { public static class Mean extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getMean() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getMean(); } } @@ -109,8 +84,7 @@ public Object apply(List args) { public static class GeometricMean extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getGeometricMean() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getGeometricMean(); } } @@ -120,8 +94,7 @@ public Object apply(List args) { public static class Sum extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getSum() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getSum(); } } @@ -131,8 +104,7 @@ public Object apply(List args) { public static class Max extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getMax() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getMax(); } } @@ -142,8 +114,7 @@ public Object apply(List args) { public static class Min extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getMin() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getMin(); } } @@ -153,8 +124,7 @@ public Object apply(List args) { public static class Count extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? convert(stats.getCount(), Double.class) : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getCount(); } } @@ -164,8 +134,7 @@ public Object apply(List args) { public static class PopulationVariance extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getPopulationVariance() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getPopulationVariance(); } } @@ -175,8 +144,7 @@ public Object apply(List args) { public static class Variance extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getVariance() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getVariance(); } } @@ -186,8 +154,7 @@ public Object apply(List args) { public static class QuadraticMean extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getQuadraticMean() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getQuadraticMean(); } } @@ -197,8 +164,7 @@ public Object apply(List args) { public static class StandardDeviation extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getStandardDeviation() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getStandardDeviation(); } } @@ -208,8 +174,7 @@ public Object apply(List args) { public static class SumLogs extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getSumLogs() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getSumLogs(); } } @@ -219,8 +184,7 @@ public Object apply(List args) { public static class SumSquares extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getSumSquares() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getSumSquares(); } } @@ -230,8 +194,7 @@ public Object apply(List args) { public static class Kurtosis extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getKurtosis() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getKurtosis(); } } @@ -241,30 +204,18 @@ public Object apply(List args) { public static class Skewness extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - return (stats != null) ? stats.getSkewness() : Double.NaN; + return convert(args.get(0), StellarStatistics.class).getSkewness(); } } /** * Calculates the Pth percentile. - * - * STATS_PERCENTILE(stats, 0.90) */ public static class Percentile extends BaseStellarFunction { @Override public Object apply(List args) { - StellarStatistics stats = convert(args.get(0), StellarStatistics.class); - Double p = convert(args.get(1), Double.class); - - Double result; - if(stats == null || p == null) { - result = Double.NaN; - } else { - result = stats.getPercentile(p); - } - - return result; + double p = convert(args.get(0), Double.class); + return convert(args.get(1), StellarStatistics.class).getPercentile(p); } } } \ No newline at end of file diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/DateFunctionsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/DateFunctionsTest.java new file mode 100644 index 0000000000..caa690dea5 --- /dev/null +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/DateFunctionsTest.java @@ -0,0 +1,104 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.common.stellar; + +import org.apache.metron.common.dsl.Context; +import org.apache.metron.common.dsl.StellarFunctions; +import org.junit.Before; +import org.junit.Test; + +import java.util.Calendar; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static java.lang.String.format; + +/** + * Tests the DateFunctions class. + */ +public class DateFunctionsTest { + + private Map variables = new HashMap<>(); + + /** + * Runs a Stellar expression. + * @param expr The expression to run. + */ + private Object run(String expr) { + StellarProcessor processor = new StellarProcessor(); + assertTrue(processor.validate(expr)); + return processor.parse(expr, x -> variables.get(x), StellarFunctions.FUNCTION_RESOLVER(), Context.EMPTY_CONTEXT()); + } + + /** + * Thu Aug 25 2016 09:27:10 EST + */ + private long AUG2016 = 1472131630748L; + + @Before + public void setup() { + variables.put("epoch", AUG2016); + } + + @Test + public void testDayOfWeek() { + Object result = run(format("DAY_OF_WEEK(epoch)", AUG2016)); + assertEquals(Calendar.THURSDAY, result); + } + + @Test + public void testWeekOfMonth() { + Object result = run(format("WEEK_OF_MONTH(epoch)", AUG2016)); + assertEquals(4, result); + } + + @Test + public void testMonth() { + Object result = run(format("MONTH(epoch)", AUG2016)); + assertEquals(Calendar.AUGUST, result); + } + + @Test + public void testYear() { + Object result = run(format("YEAR(epoch)", AUG2016)); + assertEquals(2016, result); + } + + @Test + public void testDayOfMonth() { + Object result = run(format("DAY_OF_MONTH(epoch)", AUG2016)); + assertEquals(25, result); + } + + @Test + public void testWeekOfYear() { + Object result = run(format("WEEK_OF_YEAR(epoch)", AUG2016)); + assertEquals(35, result); + } + + @Test + public void testDayOfYear() { + Object result = run(format("DAY_OF_YEAR(epoch)", AUG2016)); + assertEquals(238, result); + } +} diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java index 8c7ff70b95..56b097bfe0 100644 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java @@ -38,8 +38,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static java.lang.String.format; /** * Tests the statistical summary functions of Stellar. @@ -96,40 +94,15 @@ private void statsInit(int windowSize) { assertNotNull(result); variables.put("stats", result); - // add some values - values = Arrays.asList(10.0, 20.0, 30.0, 40.0, 50.0, 60.0, 70.0); - values.stream().forEach(val -> run(format("STATS_ADD (stats, %f)", val), variables)); - } - - @Test - public void testAddManyIntegers() throws Exception { - statsInit(windowSize); - Object result = run("STATS_COUNT(stats)", variables); - double countAtStart = (double) result; - - run("STATS_ADD(stats, 10, 20, 30, 40, 50)", variables); - - Object actual = run("STATS_COUNT(stats)", variables); - assertEquals(countAtStart + 5.0, (double) actual, 0.1); - } - - @Test - public void testAddManyFloats() throws Exception { - statsInit(windowSize); - Object result = run("STATS_COUNT(stats)", variables); - double countAtStart = (double) result; - - run("STATS_ADD(stats, 10.0, 20.0, 30.0, 40.0, 50.0)", variables); - - Object actual = run("STATS_COUNT(stats)", variables); - assertEquals(countAtStart + 5.0, (double) actual, 0.1); + // add the test data + values.stream().forEach(val -> run("STATS_ADD(" + val + ",stats)", variables)); } @Test public void testCount() throws Exception { statsInit(windowSize); Object actual = run("STATS_COUNT(stats)", variables); - assertEquals(stats.getN(), (double) actual, 0.1); + assertEquals(stats.getN(), (Long) actual, 0.1); } @Test @@ -139,13 +112,6 @@ public void testMean() throws Exception { assertEquals(stats.getMean(), (Double) actual, 0.1); } - @Test - public void testGeometricMean() throws Exception { - statsInit(windowSize); - Object actual = run("STATS_GEOMETRIC_MEAN(stats)", variables); - assertEquals(stats.getGeometricMean(), (Double) actual, 0.1); - } - @Test public void testMax() throws Exception { statsInit(windowSize); @@ -181,6 +147,13 @@ public void testVariance() throws Exception { assertEquals(stats.getVariance(), (Double) actual, 0.1); } + @Test + public void testGeometricMean() throws Exception { + statsInit(windowSize); + Object actual = run("STATS_GEOMETRIC_MEAN(stats)", variables); + assertEquals(stats.getGeometricMean(), (Double) actual, 0.1); + } + @Test public void testPopulationVariance() throws Exception { statsInit(windowSize); @@ -195,6 +168,19 @@ public void testQuadraticMean() throws Exception { assertEquals(stats.getQuadraticMean(), (Double) actual, 0.1); } + @Test + public void testSecondMomentNoWindow() throws Exception { + statsInit(0); + Object actual = run("STATS_SECOND_MOMENT(stats)", variables); + assertEquals(summaryStats.getSecondMoment(), (Double) actual, 0.1); + } + + @Test(expected = ParseException.class) + public void testSecondMomentWithWindow() throws Exception { + statsInit(100); + Object actual = run("STATS_SECOND_MOMENT(stats)", variables); + } + @Test public void testSumLogsNoWindow() throws Exception { statsInit(0); @@ -245,26 +231,14 @@ public void testSkewnessWithWindow() throws Exception { public void testPercentileNoWindow() throws Exception { statsInit(0); final double percentile = 0.9; - Object actual = run(format("STATS_PERCENTILE(stats, %f)", percentile), variables); + Object actual = run("STATS_PERCENTILE(" + percentile + ", stats)", variables); } @Test public void testPercentileWithWindow() throws Exception { statsInit(100); final double percentile = 0.9; - Object actual = run(format("STATS_PERCENTILE(stats, %f)", percentile), variables); + Object actual = run("STATS_PERCENTILE(" + percentile + ", stats)", variables); assertEquals(stats.getPercentile(percentile), (Double) actual, 0.1); } - - @Test - public void testWithNull() throws Exception { - Object actual = run("STATS_MEAN(null)", variables); - assertTrue(((Double)actual).isNaN()); - - actual = run("STATS_COUNT(null)", variables); - assertTrue(((Double)actual).isNaN()); - - actual = run("STATS_VARIANCE(null)", variables); - assertTrue(((Double)actual).isNaN()); - } } \ No newline at end of file diff --git a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java index d19633ed52..925c63d614 100644 --- a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java +++ b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java @@ -45,9 +45,11 @@ import java.util.Arrays; import java.util.List; +import static org.hamcrest.core.IsCollectionContaining.hasItem; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.junit.Assert.assertEquals; /** * Tests the HBaseClient @@ -128,6 +130,10 @@ public void testWrite() throws Exception { Get get1 = client.constructGetRequests(rowKey1, criteria); Result[] results = client.batchGet(Arrays.asList(get1)); Assert.assertEquals(1, results.length); + + // validate + assertEquals(1, results.length); + assertEquals(widget1, toWidget(results[0])); } @Test @@ -152,6 +158,24 @@ public void testBatchWrite() throws Exception { Get get1 = client.constructGetRequests(rowKey1, criteria); Get get2 = client.constructGetRequests(rowKey2, criteria); Result[] results = client.batchGet(Arrays.asList(get1, get2)); - Assert.assertEquals(2, results.length); + + // validate + assertEquals(2, results.length); + List expected = Arrays.asList(widget1, widget2); + for(Result result : results) { + Widget widget = toWidget(result); + Assert.assertThat(expected, hasItem(widget)); + } + } + + /** + * Transforms the HBase Result to a Widget. + * @param result The HBase Result. + * @return The Widget. + */ + private Widget toWidget(Result result) { + int cost = Bytes.toInt(result.getValue(WidgetMapper.CF, WidgetMapper.QCOST)); + String name = Bytes.toString(result.getValue(WidgetMapper.CF, WidgetMapper.QNAME)); + return new Widget(name, cost); } } diff --git a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java index c522f8d792..723aa718e4 100644 --- a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java +++ b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java @@ -23,6 +23,7 @@ import com.google.protobuf.Service; import com.google.protobuf.ServiceException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; @@ -130,7 +131,7 @@ public TableName getName() { @Override public Configuration getConfiguration() { - throw new UnsupportedOperationException(); + return HBaseConfiguration.create(); } @Override From 87b1d20fb9e74244d2ea83203d77dcee2e955c3d Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 29 Aug 2016 21:23:03 -0400 Subject: [PATCH 02/10] METRON-389 Missed removal of old tests during merge --- .../stellar/StellarStatisticsFunctionsTest.java | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java index 56b097bfe0..ef3847606a 100644 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java @@ -168,19 +168,6 @@ public void testQuadraticMean() throws Exception { assertEquals(stats.getQuadraticMean(), (Double) actual, 0.1); } - @Test - public void testSecondMomentNoWindow() throws Exception { - statsInit(0); - Object actual = run("STATS_SECOND_MOMENT(stats)", variables); - assertEquals(summaryStats.getSecondMoment(), (Double) actual, 0.1); - } - - @Test(expected = ParseException.class) - public void testSecondMomentWithWindow() throws Exception { - statsInit(100); - Object actual = run("STATS_SECOND_MOMENT(stats)", variables); - } - @Test public void testSumLogsNoWindow() throws Exception { statsInit(0); From 6041acacedcc7b54bda51592ad5bf9161fa02e7e Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 29 Aug 2016 21:31:55 -0400 Subject: [PATCH 03/10] METRON-389 Change to Context was lost in merge --- .../org/apache/metron/common/dsl/Context.java | 40 +++++++++++-------- 1 file changed, 23 insertions(+), 17 deletions(-) diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Context.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Context.java index 8080363989..0731b00b9d 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Context.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/Context.java @@ -17,56 +17,62 @@ */ package org.apache.metron.common.dsl; +import java.io.Serializable; import java.util.HashMap; import java.util.Map; import java.util.Optional; -import java.util.function.Function; -public class Context { +public class Context implements Serializable { + public interface Capability { Object get(); } - public enum Capabilities { - HBASE_PROVIDER - , ZOOKEEPER_CLIENT - , SERVICE_DISCOVERER; + public enum Capabilities { + HBASE_PROVIDER, + ZOOKEEPER_CLIENT, + SERVICE_DISCOVERER } public static class Builder { + private Map capabilityMap = new HashMap<>(); public Builder with(String s, Capability capability) { capabilityMap.put(s, capability); return this; } + public Builder with(Enum s, Capability capability) { capabilityMap.put(s.toString(), capability); return this; } - public Context build() { + public Context build() { return new Context(capabilityMap); } } + public static Context EMPTY_CONTEXT() { return - new Context(new HashMap<>()){ - @Override - public Optional getCapability(String capability) { - return Optional.empty(); - } - }; + new Context(new HashMap<>()){ + @Override + public Optional getCapability(String capability) { + return Optional.empty(); + } + }; } + private Map capabilities; - private Context( Map capabilities - ) - { + + private Context( Map capabilities) { this.capabilities = capabilities; } + public Optional getCapability(Enum capability) { return getCapability(capability.toString()); } + public Optional getCapability(String capability) { Capability c = capabilities.get(capability); if(c == null) { @@ -82,4 +88,4 @@ public void addCapability(String s, Capability capability) { public void addCapability(Enum s, Capability capability) { this.capabilities.put(s.toString(), capability); } -} +} \ No newline at end of file From 24069577792c727e387d495b200402275ca41a2c Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 30 Aug 2016 09:23:56 -0400 Subject: [PATCH 04/10] METRON-389 Was missing changes from METRON-377 --- .../functions/StellarStatisticsFunctions.java | 104 +++++++++++++----- 1 file changed, 76 insertions(+), 28 deletions(-) diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java index 325a2fff0b..e32fdb55e7 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/StellarStatisticsFunctions.java @@ -20,9 +20,9 @@ package org.apache.metron.common.dsl.functions; -import org.apache.commons.math3.stat.descriptive.SummaryStatistics; import org.apache.metron.common.dsl.BaseStellarFunction; +import java.util.Collections; import java.util.List; import static org.apache.metron.common.utils.ConversionUtils.convert; @@ -33,36 +33,59 @@ */ public class StellarStatisticsFunctions { + /** + * Initializes the summary statistics. + * + * Initialization can occur from either STATS_INIT and STATS_ADD. + */ + private static StellarStatistics statsInit(List args) { + int windowSize = 0; + if(args.size() > 0 && args.get(0) instanceof Number) { + windowSize = convert(args.get(0), Integer.class); + } + + return new StellarStatistics(windowSize); + } + /** * Initialize the summary statistics. * * STATS_INIT (window_size) * - * @param window_size The number of input data values to maintain in a rolling window - * in memory. If equal to 0, then no rolling window is maintained. - * Using no rolling window is less memory intensive, but cannot - * calculate certain statistics like percentiles and kurtosis. + * window_size The number of input data values to maintain in a rolling window + * in memory. If equal to 0, then no rolling window is maintained. + * Using no rolling window is less memory intensive, but cannot + * calculate certain statistics like percentiles and kurtosis. */ public static class Init extends BaseStellarFunction { @Override public Object apply(List args) { - int windowSize = (args.size() > 0 && args.get(0) instanceof Number) ? ((Number) args.get(0)).intValue() : 0; - return new StellarStatistics(windowSize); + return statsInit(args); } } /** * Add an input value to those that are used to calculate the summary statistics. * - * STATS_ADD (value, stats) + * STATS_ADD (stats, value [, value2, value3, ...]) */ public static class Add extends BaseStellarFunction { @Override public Object apply(List args) { - double value = convert(args.get(0), Double.class); - StellarStatistics stats = convert(args.get(1), StellarStatistics.class); - stats.addValue(value); - return null; + + // initialize a stats object, if one does not already exist + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + if(stats == null) { + stats = statsInit(Collections.emptyList()); + } + + // add each of the numeric values + for(int i=1; i args) { public static class Mean extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getMean(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getMean() : Double.NaN; } } @@ -84,7 +108,8 @@ public Object apply(List args) { public static class GeometricMean extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getGeometricMean(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getGeometricMean() : Double.NaN; } } @@ -94,7 +119,8 @@ public Object apply(List args) { public static class Sum extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getSum(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getSum() : Double.NaN; } } @@ -104,7 +130,8 @@ public Object apply(List args) { public static class Max extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getMax(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getMax() : Double.NaN; } } @@ -114,7 +141,8 @@ public Object apply(List args) { public static class Min extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getMin(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getMin() : Double.NaN; } } @@ -124,7 +152,8 @@ public Object apply(List args) { public static class Count extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getCount(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? convert(stats.getCount(), Double.class) : Double.NaN; } } @@ -134,7 +163,8 @@ public Object apply(List args) { public static class PopulationVariance extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getPopulationVariance(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getPopulationVariance() : Double.NaN; } } @@ -144,7 +174,8 @@ public Object apply(List args) { public static class Variance extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getVariance(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getVariance() : Double.NaN; } } @@ -154,7 +185,8 @@ public Object apply(List args) { public static class QuadraticMean extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getQuadraticMean(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getQuadraticMean() : Double.NaN; } } @@ -164,7 +196,8 @@ public Object apply(List args) { public static class StandardDeviation extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getStandardDeviation(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getStandardDeviation() : Double.NaN; } } @@ -174,7 +207,8 @@ public Object apply(List args) { public static class SumLogs extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getSumLogs(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getSumLogs() : Double.NaN; } } @@ -184,7 +218,8 @@ public Object apply(List args) { public static class SumSquares extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getSumSquares(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getSumSquares() : Double.NaN; } } @@ -194,7 +229,8 @@ public Object apply(List args) { public static class Kurtosis extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getKurtosis(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getKurtosis() : Double.NaN; } } @@ -204,18 +240,30 @@ public Object apply(List args) { public static class Skewness extends BaseStellarFunction { @Override public Object apply(List args) { - return convert(args.get(0), StellarStatistics.class).getSkewness(); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + return (stats != null) ? stats.getSkewness() : Double.NaN; } } /** * Calculates the Pth percentile. + * + * STATS_PERCENTILE(stats, 0.90) */ public static class Percentile extends BaseStellarFunction { @Override public Object apply(List args) { - double p = convert(args.get(0), Double.class); - return convert(args.get(1), StellarStatistics.class).getPercentile(p); + StellarStatistics stats = convert(args.get(0), StellarStatistics.class); + Double p = convert(args.get(1), Double.class); + + Double result; + if(stats == null || p == null) { + result = Double.NaN; + } else { + result = stats.getPercentile(p); + } + + return result; } } } \ No newline at end of file From d9ae8cd3379c561e4c9d7f1fc5e9110c226e94fb Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 30 Aug 2016 09:27:36 -0400 Subject: [PATCH 05/10] METRON-389 Removed unnecessary commented-out code --- metron-analytics/metron-profiler/pom.xml | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/metron-analytics/metron-profiler/pom.xml b/metron-analytics/metron-profiler/pom.xml index ebb51a2f9e..51aa64ac4c 100644 --- a/metron-analytics/metron-profiler/pom.xml +++ b/metron-analytics/metron-profiler/pom.xml @@ -133,16 +133,6 @@ storm-hdfs ${global_storm_version} provided - - - - - - - - - - org.apache.kafka From 8eecb8b1a8529d9dde937f147804eaf876199953 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 30 Aug 2016 09:46:10 -0400 Subject: [PATCH 06/10] METRON-389 Made sure the docs are consistent; periodsPerHour can be a multiple or divisor of 60. --- .../main/java/org/apache/metron/profiler/ProfilePeriod.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java index 91b8dd1567..648eb280c7 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfilePeriod.java @@ -53,7 +53,8 @@ public class ProfilePeriod { private int period; /** - * The number of periods per hour. This value must be a divisor of 60; 1, 2, 3, 4, 6, 10, etc. + * The number of periods per hour. This value must be a divisor or multiple + * of 60; 1, 2, 4, 6, 240, etc. */ private int periodsPerHour; @@ -65,7 +66,8 @@ public class ProfilePeriod { /** * @param epochMillis A timestamp contained somewhere within the profile period. - * @param periodsPerHour The number of periods per hour. Must be an divisor or multiple of 60; 1, 2, 3, 4, 6, 240, etc. + * @param periodsPerHour The number of periods per hour. Must be a divisor or multiple + * of 60; 1, 2, 4, 6, 240, etc. */ public ProfilePeriod(long epochMillis, int periodsPerHour) { From 0dece4ee33ff764b1995d2104fb861f48d325b32 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 30 Aug 2016 10:01:20 -0400 Subject: [PATCH 07/10] METRON-389 Was missing updated test from METRON-377 --- .../StellarStatisticsFunctionsTest.java | 63 +++++++++++++++---- 1 file changed, 51 insertions(+), 12 deletions(-) diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java index ef3847606a..8c7ff70b95 100644 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarStatisticsFunctionsTest.java @@ -38,6 +38,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static java.lang.String.format; /** * Tests the statistical summary functions of Stellar. @@ -94,15 +96,40 @@ private void statsInit(int windowSize) { assertNotNull(result); variables.put("stats", result); - // add the test data - values.stream().forEach(val -> run("STATS_ADD(" + val + ",stats)", variables)); + // add some values + values = Arrays.asList(10.0, 20.0, 30.0, 40.0, 50.0, 60.0, 70.0); + values.stream().forEach(val -> run(format("STATS_ADD (stats, %f)", val), variables)); + } + + @Test + public void testAddManyIntegers() throws Exception { + statsInit(windowSize); + Object result = run("STATS_COUNT(stats)", variables); + double countAtStart = (double) result; + + run("STATS_ADD(stats, 10, 20, 30, 40, 50)", variables); + + Object actual = run("STATS_COUNT(stats)", variables); + assertEquals(countAtStart + 5.0, (double) actual, 0.1); + } + + @Test + public void testAddManyFloats() throws Exception { + statsInit(windowSize); + Object result = run("STATS_COUNT(stats)", variables); + double countAtStart = (double) result; + + run("STATS_ADD(stats, 10.0, 20.0, 30.0, 40.0, 50.0)", variables); + + Object actual = run("STATS_COUNT(stats)", variables); + assertEquals(countAtStart + 5.0, (double) actual, 0.1); } @Test public void testCount() throws Exception { statsInit(windowSize); Object actual = run("STATS_COUNT(stats)", variables); - assertEquals(stats.getN(), (Long) actual, 0.1); + assertEquals(stats.getN(), (double) actual, 0.1); } @Test @@ -112,6 +139,13 @@ public void testMean() throws Exception { assertEquals(stats.getMean(), (Double) actual, 0.1); } + @Test + public void testGeometricMean() throws Exception { + statsInit(windowSize); + Object actual = run("STATS_GEOMETRIC_MEAN(stats)", variables); + assertEquals(stats.getGeometricMean(), (Double) actual, 0.1); + } + @Test public void testMax() throws Exception { statsInit(windowSize); @@ -147,13 +181,6 @@ public void testVariance() throws Exception { assertEquals(stats.getVariance(), (Double) actual, 0.1); } - @Test - public void testGeometricMean() throws Exception { - statsInit(windowSize); - Object actual = run("STATS_GEOMETRIC_MEAN(stats)", variables); - assertEquals(stats.getGeometricMean(), (Double) actual, 0.1); - } - @Test public void testPopulationVariance() throws Exception { statsInit(windowSize); @@ -218,14 +245,26 @@ public void testSkewnessWithWindow() throws Exception { public void testPercentileNoWindow() throws Exception { statsInit(0); final double percentile = 0.9; - Object actual = run("STATS_PERCENTILE(" + percentile + ", stats)", variables); + Object actual = run(format("STATS_PERCENTILE(stats, %f)", percentile), variables); } @Test public void testPercentileWithWindow() throws Exception { statsInit(100); final double percentile = 0.9; - Object actual = run("STATS_PERCENTILE(" + percentile + ", stats)", variables); + Object actual = run(format("STATS_PERCENTILE(stats, %f)", percentile), variables); assertEquals(stats.getPercentile(percentile), (Double) actual, 0.1); } + + @Test + public void testWithNull() throws Exception { + Object actual = run("STATS_MEAN(null)", variables); + assertTrue(((Double)actual).isNaN()); + + actual = run("STATS_COUNT(null)", variables); + assertTrue(((Double)actual).isNaN()); + + actual = run("STATS_VARIANCE(null)", variables); + assertTrue(((Double)actual).isNaN()); + } } \ No newline at end of file From 0b32d5496f124f3adcf82cc7fa39683297f63e68 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 2 Sep 2016 14:21:00 -0400 Subject: [PATCH 08/10] METRON-389 Removed static dependencies between the HBaseProfileClient and ColumnBuilder --- .../{ => client}/HBaseProfilerClient.java | 27 +++++++++---------- .../profiler/{ => client}/ProfilerClient.java | 4 +-- .../{ => client}/HBaseProfilerClientTest.java | 8 +++--- .../metron/profiler/hbase/ColumnBuilder.java | 19 ++++++++----- .../hbase/ValueOnlyColumnBuilder.java | 19 +++++++++++-- .../integration/ProfilerIntegrationTest.java | 19 +++++++------ 6 files changed, 60 insertions(+), 36 deletions(-) rename metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/{ => client}/HBaseProfilerClient.java (82%) rename metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/{ => client}/ProfilerClient.java (95%) rename metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/{ => client}/HBaseProfilerClientTest.java (97%) diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/HBaseProfilerClient.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/HBaseProfilerClient.java similarity index 82% rename from metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/HBaseProfilerClient.java rename to metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/HBaseProfilerClient.java index 7140d3f9fb..b4e52b5643 100644 --- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/HBaseProfilerClient.java +++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/HBaseProfilerClient.java @@ -18,7 +18,7 @@ * */ -package org.apache.metron.profiler; +package org.apache.metron.profiler.client; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.HTableInterface; @@ -51,16 +51,14 @@ public class HBaseProfilerClient implements ProfilerClient { private RowKeyBuilder rowKeyBuilder; /** - * The column family used to store the profile data. + * Knows how profiles are organized in HBase. */ - private String columnFamily; + private ColumnBuilder columnBuilder; - private byte[] columnFamilyBytes; - - public HBaseProfilerClient(HTableInterface table, RowKeyBuilder rowKeyBuilder, String columnFamily) { + public HBaseProfilerClient(HTableInterface table, RowKeyBuilder rowKeyBuilder, ColumnBuilder columnBuilder) { setTable(table); setRowKeyBuilder(rowKeyBuilder); - setColumnFamily(columnFamily); + setColumnBuilder(columnBuilder); } /** @@ -79,11 +77,13 @@ public List fetch(String profile, String entity, long durationAgo, TimeUn // find all the row keys that satisfy this fetch List keysToFetch = rowKeyBuilder.rowKeys(profile, entity, groups, durationAgo, unit); + byte[] columnFamilyBytes = Bytes.toBytes(columnBuilder.getColumnFamily()); + byte[] columnQualifier = columnBuilder.getColumnQualifier("value"); // create a Get for each of the row keys List gets = keysToFetch .stream() - .map(k -> new Get(k).addColumn(Bytes.toBytes(columnFamily), ColumnBuilder.QVALUE)) + .map(k -> new Get(k).addColumn(columnFamilyBytes, columnQualifier)) .collect(Collectors.toList()); // submit the gets to HBase @@ -92,8 +92,8 @@ public List fetch(String profile, String entity, long durationAgo, TimeUn Result[] results = table.get(gets); Arrays.stream(results) - .filter(r -> r.containsColumn(columnFamilyBytes, ColumnBuilder.QVALUE)) - .map(r -> r.getValue(columnFamilyBytes, ColumnBuilder.QVALUE)) + .filter(r -> r.containsColumn(columnFamilyBytes, columnQualifier)) + .map(r -> r.getValue(columnFamilyBytes, columnQualifier)) .forEach(val -> values.add(Serializer.fromBytes(val, clazz))); return values; @@ -111,8 +111,7 @@ public void setRowKeyBuilder(RowKeyBuilder rowKeyBuilder) { this.rowKeyBuilder = rowKeyBuilder; } - public void setColumnFamily(String columnFamily) { - this.columnFamily = columnFamily; - this.columnFamilyBytes = Bytes.toBytes(columnFamily); + public void setColumnBuilder(ColumnBuilder columnBuilder) { + this.columnBuilder = columnBuilder; } -} +} \ No newline at end of file diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/ProfilerClient.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClient.java similarity index 95% rename from metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/ProfilerClient.java rename to metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClient.java index 72b591db7c..9cae0e90ad 100644 --- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/ProfilerClient.java +++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/ProfilerClient.java @@ -18,7 +18,7 @@ * */ -package org.apache.metron.profiler; +package org.apache.metron.profiler.client; import java.util.List; import java.util.concurrent.TimeUnit; @@ -38,7 +38,7 @@ public interface ProfilerClient { * @param clazz The type of values stored by the profile. * @param groups The groups used to sort the profile data. * @param The type of values stored by the Profile. - * @return An array of profile values. + * @return A list of values. */ List fetch(String profile, String entity, long durationAgo, TimeUnit unit, Class clazz, List groups); diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/HBaseProfilerClientTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java similarity index 97% rename from metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/HBaseProfilerClientTest.java rename to metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java index 95273d9c11..7e13f3b97f 100644 --- a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/HBaseProfilerClientTest.java +++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java @@ -18,7 +18,7 @@ * */ -package org.apache.metron.profiler; +package org.apache.metron.profiler.client; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -28,6 +28,8 @@ import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.util.Bytes; 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.SaltyRowKeyBuilder; import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder; @@ -91,10 +93,10 @@ public void setup() throws Exception { hbaseClient = new HBaseClient((c,t) -> table, table.getConfiguration(), tableName); executor = new DefaultStellarExecutor(); rowKeyBuilder = new SaltyRowKeyBuilder(); - columnBuilder = new ValueOnlyColumnBuilder(); + columnBuilder = new ValueOnlyColumnBuilder(columnFamily); // what we're actually testing - client = new HBaseProfilerClient(table, rowKeyBuilder, columnFamily); + client = new HBaseProfilerClient(table, rowKeyBuilder, columnBuilder); } /** diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java index 20b71ffb3c..44bf1294b4 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ColumnBuilder.java @@ -20,7 +20,6 @@ package org.apache.metron.profiler.hbase; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.metron.profiler.ProfileMeasurement; import org.apache.storm.hbase.common.ColumnList; @@ -37,9 +36,15 @@ public interface ColumnBuilder extends Serializable { */ ColumnList columns(ProfileMeasurement measurement); - byte[] QPROFILE = Bytes.toBytes("profile"); - byte[] QENTITY = Bytes.toBytes("entity"); - byte[] QSTART = Bytes.toBytes("start"); - byte[] QEND = Bytes.toBytes("end"); - byte[] QVALUE = Bytes.toBytes("value"); -} + /** + * Returns the column family used to store the ProfileMeasurement values. + * @return + */ + String getColumnFamily(); + + /** + * Returns the column qualifiers for the given field of a ProfileMeasurement. + * @return The column qualifier used to store a ProfileMeasurement's field in HBase. + */ + byte[] getColumnQualifier(String fieldName); +} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java index a75e382e0f..aeda3179b1 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/hbase/ValueOnlyColumnBuilder.java @@ -48,13 +48,28 @@ public ValueOnlyColumnBuilder(String columnFamily) { public ColumnList columns(ProfileMeasurement measurement) { ColumnList cols = new ColumnList(); - cols.addColumn(columnFamilyBytes, QVALUE, Serializer.toBytes(measurement.getValue())); + cols.addColumn(columnFamilyBytes, getColumnQualifier("value"), Serializer.toBytes(measurement.getValue())); return cols; } + @Override + public String getColumnFamily() { + return this.columnFamily; + } + public void setColumnFamily(String columnFamily) { this.columnFamily = columnFamily; this.columnFamilyBytes = Bytes.toBytes(columnFamily); } -} + + @Override + public byte[] getColumnQualifier(String fieldName) { + + if("value".equals(fieldName)) { + return Bytes.toBytes("value"); + } + + throw new IllegalArgumentException(("unexpected field name: " + fieldName)); + } +} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java index 3b77c17ebb..e9a5509fed 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java @@ -34,6 +34,7 @@ import org.apache.metron.integration.components.FluxTopologyComponent; import org.apache.metron.integration.components.KafkaWithZKComponent; import org.apache.metron.profiler.hbase.ColumnBuilder; +import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder; import org.apache.metron.test.mock.MockHTable; import org.junit.After; import org.junit.Assert; @@ -94,6 +95,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { @Multiline private String message3; + private ColumnBuilder columnBuilder; private FluxTopologyComponent fluxComponent; private KafkaWithZKComponent kafkaComponent; private List input; @@ -133,7 +135,7 @@ public void testExample1() throws Exception { timeout(seconds(90))); // verify - there are 5 'HTTP' each with 390 bytes - double actual = readDouble(ColumnBuilder.QVALUE); + double actual = readDouble(columnBuilder.getColumnQualifier("value")); Assert.assertEquals(390.0 * 5, actual, 0.01); } @@ -154,7 +156,7 @@ public void testExample2() throws Exception { timeout(seconds(90))); // verify - there are 5 'HTTP' and 5 'DNS' messages thus 5/5 = 1 - double actual = readDouble(ColumnBuilder.QVALUE); + double actual = readDouble(columnBuilder.getColumnQualifier("value")); Assert.assertEquals(5.0 / 5.0, actual, 0.01); } @@ -175,7 +177,7 @@ public void testExample3() throws Exception { timeout(seconds(90))); // verify - there are 5 'HTTP' messages each with a length of 20, thus the average should be 20 - double actual = readDouble(ColumnBuilder.QVALUE); + double actual = readDouble(columnBuilder.getColumnQualifier("value")); Assert.assertEquals(20.0, actual, 0.01); } @@ -193,7 +195,7 @@ public void testWriteInteger() throws Exception { timeout(seconds(90))); // verify - there are 5 'HTTP' messages each with a length of 20, thus the average should be 20 - double actual = readInteger(ColumnBuilder.QVALUE); + double actual = readInteger(columnBuilder.getColumnQualifier("value")); Assert.assertEquals(10.0, actual, 0.01); } @@ -211,7 +213,7 @@ public void testPercentiles() throws Exception { timeout(seconds(90))); // verify - the 70th percentile of 5 x 20s = 20.0 - double actual = readDouble(ColumnBuilder.QVALUE); + double actual = readDouble(columnBuilder.getColumnQualifier("value")); Assert.assertEquals(20.0, actual, 0.01); } @@ -224,7 +226,7 @@ private Double readDouble(byte[] columnQual) throws IOException { ResultScanner scanner = profilerTable.getScanner(cf, columnQual); for (Result result : scanner) { - byte[] raw = result.getValue(cf, ColumnBuilder.QVALUE); + byte[] raw = result.getValue(cf, columnQual); return Bytes.toDouble(raw); } @@ -240,7 +242,7 @@ private Integer readInteger(byte[] columnQual) throws IOException { ResultScanner scanner = profilerTable.getScanner(cf, columnQual); for (Result result : scanner) { - byte[] raw = result.getValue(cf, ColumnBuilder.QVALUE); + byte[] raw = result.getValue(cf, columnQual); return Bytes.toInt(raw); } @@ -248,6 +250,7 @@ private Integer readInteger(byte[] columnQual) throws IOException { } public void setup(String pathToConfig) throws Exception { + columnBuilder = new ValueOnlyColumnBuilder(columnFamily); // create input messages for the profiler to consume input = Stream.of(message1, message2, message3) @@ -308,4 +311,4 @@ public void tearDown() throws Exception { runner.stop(); } } -} +} \ No newline at end of file From d0befa8519c2f5d21c98e95cf31cb3365ff2ef54 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 6 Sep 2016 16:16:52 -0400 Subject: [PATCH 09/10] Still need to call initialize which was lost in the merge --- .../org/apache/metron/profiler/bolt/ProfileBuilderBolt.java | 2 ++ .../apache/metron/profiler/bolt/ProfileSplitterBolt.java | 6 ++---- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java index 017892be36..b92d188dce 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java @@ -32,6 +32,7 @@ import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.common.dsl.Context; import org.apache.metron.common.dsl.ParseException; +import org.apache.metron.common.dsl.StellarFunctions; import org.apache.metron.profiler.ProfileMeasurement; import org.apache.metron.profiler.stellar.StellarExecutor; import org.json.simple.JSONObject; @@ -112,6 +113,7 @@ protected void initializeStellar() { Context context = new Context.Builder() .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client) .build(); + StellarFunctions.initialize(context); executor.setContext(context); } diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java index cbd7423a5d..d389410951 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java @@ -31,6 +31,7 @@ import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.common.configuration.profiler.ProfilerConfig; import org.apache.metron.common.dsl.Context; +import org.apache.metron.common.dsl.StellarFunctions; import org.apache.metron.profiler.stellar.StellarExecutor; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; @@ -80,13 +81,10 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll } protected void initializeStellar() { - /* - * the context provides global configuration values that Stellar may need - * during initialization to connect to external resources like Zookeeper or HBase. - */ Context context = new Context.Builder() .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client) .build(); + StellarFunctions.initialize(context); executor.setContext(context); } From bc192e38a2d80b92bc587f899a90894967e3a4cb Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 12 Sep 2016 09:27:07 -0400 Subject: [PATCH 10/10] METRON-389 Fixed comment; s/bold/bolt --- .../org/apache/metron/profiler/bolt/ProfileBuilderBolt.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java index b92d188dce..79c046a056 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java @@ -102,7 +102,7 @@ public ProfileBuilderBolt(String zookeeperUrl) { public Map getComponentConfiguration() { Config conf = new Config(); - // how frequently should the bold receive tick tuples? + // how frequently should the bolt receive tick tuples? long freqInSeconds = ((60 * 60) / periodsPerHour); conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, freqInSeconds);