From 11ef29bb73c8363b1c905f597be496929b77888f Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 29 Jul 2016 16:53:49 -0400 Subject: [PATCH 01/10] METRON-309 Create a normalcy profiler --- metron-analytics/metron-profiler/README.md | 195 ++++++++++ metron-analytics/metron-profiler/pom.xml | 332 ++++++++++++++++++ .../src/main/assembly/assembly.xml | 72 ++++ .../src/main/config/profiler.properties | 36 ++ .../src/main/flux/profiler/remote.yaml | 114 ++++++ .../metron/profiler/ProfileMeasurement.java | 133 +++++++ .../profiler/bolt/ProfileBuilderBolt.java | 265 ++++++++++++++ .../profiler/bolt/ProfileHBaseMapper.java | 136 +++++++ .../profiler/bolt/ProfileSplitterBolt.java | 158 +++++++++ .../stellar/DefaultStellarExecutor.java | 164 +++++++++ .../profiler/stellar/StellarExecutor.java | 64 ++++ .../main/scripts/start_profiler_topology.sh | 22 ++ .../src/test/config/zookeeper/profiler.json | 13 + .../profiler/bolt/ProfileBuilderBoltTest.java | 204 +++++++++++ .../bolt/ProfileSplitterBoltTest.java | 212 +++++++++++ .../integration/ConfigUploadComponent.java | 119 +++++++ .../integration/ProfilerIntegrationTest.java | 197 +++++++++++ .../util/DefaultStellarExecutorTest.java | 148 ++++++++ metron-analytics/pom.xml | 3 +- .../roles/metron_streaming/defaults/main.yml | 8 + .../metron_streaming/tasks/copy_bundles.yml | 6 +- .../common/bolt/ConfiguredEnrichmentBolt.java | 1 - .../common/bolt/ConfiguredProfilerBolt.java | 75 ++++ .../configuration/ConfigurationType.java | 34 +- .../configuration/ConfigurationsUtils.java | 105 ++++-- .../configuration/profiler/ProfileConfig.java | 145 ++++++++ .../profiler/ProfilerConfig.java | 73 ++++ .../profiler/ProfilerConfigurations.java | 53 +++ .../common/dsl/MapVariableResolver.java | 2 + .../common/dsl/TransformationFunctions.java | 79 +++++ .../dsl/functions/ControlFlowFunctions.java | 58 +++ .../common/dsl/functions/MathFunctions.java | 180 ++++++++++ .../common/stellar/BaseStellarProcessor.java | 1 + .../ControlFlowFunctionsTest.java | 76 ++++ .../transformation/MathFunctionsTest.java | 85 +++++ .../transformation/TransformationTest.java | 161 +++++++++ .../metron/enrichment/bolt/HBaseBolt.java | 184 ---------- .../components/ConfigUploadComponent.java | 32 +- metron-platform/metron-hbase/pom.xml | 128 ++++++- .../apache/metron/hbase/bolt/BatchHelper.java | 127 +++++++ .../apache/metron/hbase/bolt/HBaseBolt.java | 202 +++++++++++ .../metron/hbase/client/HBaseClient.java | 165 +++++++++ .../java/org/apache/metron/hbase/Widget.java | 77 ++++ .../org/apache/metron/hbase/WidgetMapper.java | 55 +++ .../metron/hbase/bolt/HBaseBoltTest.java | 156 ++++++++ .../metron/hbase/client/HBaseClientTest.java | 149 ++++++++ .../components/FluxTopologyComponent.java | 173 ++++----- .../apache/metron/test/mock/MockHTable.java | 5 +- metron-platform/pom.xml | 1 - pom.xml | 1 + 50 files changed, 4861 insertions(+), 323 deletions(-) create mode 100644 metron-analytics/metron-profiler/README.md create mode 100644 metron-analytics/metron-profiler/pom.xml create mode 100644 metron-analytics/metron-profiler/src/main/assembly/assembly.xml create mode 100644 metron-analytics/metron-profiler/src/main/config/profiler.properties create mode 100644 metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java create mode 100644 metron-analytics/metron-profiler/src/main/scripts/start_profiler_topology.sh create mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileSplitterBoltTest.java create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/util/DefaultStellarExecutorTest.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredProfilerBolt.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfig.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfigurations.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/TransformationFunctions.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ControlFlowFunctions.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MathFunctions.java create mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/ControlFlowFunctionsTest.java create mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/MathFunctionsTest.java create mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/TransformationTest.java delete mode 100644 metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/HBaseBolt.java create mode 100644 metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/BatchHelper.java create mode 100644 metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java create mode 100644 metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java create mode 100644 metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/Widget.java create mode 100644 metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/WidgetMapper.java create mode 100644 metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/bolt/HBaseBoltTest.java create mode 100644 metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md new file mode 100644 index 0000000000..76303fa769 --- /dev/null +++ b/metron-analytics/metron-profiler/README.md @@ -0,0 +1,195 @@ +# Metron Profiler + +The Profiler is a feature extraction mechanism that can generate a profile describing the behavior of an entity on a network. An entity might be a server, user, subnet or application. Once a profile has been generated defining what normal behavior looks-like, models can be built that identify anomalous behavior. + +This is achieved by summarizing the streaming telemetry data consumed by Metron over sliding windows. A summary statistic is applied to the data received within a given window. Collecting this summary across many windows results in a time series that is useful for analysis. + +## Usage + +Any field contained within a message can be used to generate a profile. A profile can even be produced from combining fields that originate in different data sources. A user has considerable power to transform the data used in a profile by leveraging the Stellar language. A user only need configure the desired profiles in Zookeeper and ensure that the Profiler topology is running. + +### Configuration + +The Profiler configuration requires a JSON-formatted set of elements, many of which can contain Stellar code. The configuration contains the following elements. + +* `profile` A unique name identifying the profile. +* `foreach` A separate profile is maintained for each of these. This is effectively the entity that the profile is describing. For example, if `ip_src_addr` then a separate profile would be maintained for each unique IP source address. +* `onlyif` A message is only applied to a profile if this condition is true. This allows the incoming messages to be filtered. +* `init` A set of variables and Stellar code that describes how they are initialized at the beginning of each window period. +* `update` A set of variables along with Stellar code that describes how those variables are updated by each message. +* `result` Stellar code that is executed at the end of a window period. This field must result in a Long that becomes part of the profile. + +### Examples + +Examples of the types of profiles that can be collected include the following. Each shows the configuration that would be required to produce the profile. + +### Example 1 + +The total number of bytes received for each host. The following configuration would be used to generate this profile. + +``` +{ "profiler": [ + { + "profile": "sum_bytes_in", + "foreach": "ip_src_addr", + "onlyif": "EXISTS(is_local)", + “init”: { “sum”: 0 }, + "update": { "sum": "sum + bytes_in" }, + "result": "sum" + } +]} +``` + +This creates a profile... + * Named ‘sum_bytes_in’ + * That for each IP source address + * Only if it is on the local network + * Initializes a counter ‘sum’ to 0 + * Updates ‘sum’ by adding the value of ‘bytes_in’ from each message + * After the window expires, ‘sum’ becomes the result + +### Example 2 + +The ratio of DNS traffic to HTTP traffic for each host. The following configuration would be used to generate this profile. + +``` +{ "profiler": [ + { + "profile": "ratio_dns_to_http", + "foreach": "ip_src_addr", + "onlyif": "protocol == 'DNS' or protocol == ‘HTTP’ + “init”: { + “num_dns”: 0, + “num_http”: 0 + }, +   "update": { +     "num_dns": "num_dns + IF_THEN_ELSE(protocol == ‘DNS’, 1, 0)", + "num_http": "num_http + IF_THEN_ELSE(protocol == ‘HTTP’, 1, 0)" +   }, +   "result": "num_dns / num_http" + } +]} +``` + +This creates a profile... + * Named ‘ratio_dns_to_http’ + * That for each IP source address + * Only if the message is either DNS or HTTP + * Accumulates the number of DNS requests + * Accumulates the number of HTTP requests + * After the window expires, the ratio of these is the result + +### Example 3 + +The average response body length of HTTP traffic. The following configuration would be used to generate this profile. + +``` +{ "profiler": [ + { + "profile": "http_mean_resp_body_len", + "foreach": "ip_src_addr", + "onlyif": "protocol == 'HTTP'", + “init”: { + “sum”: 0, + “cnt”: 0 + }, + "update": { + "sum": "sum + resp_body_len", + "cnt": "cnt + 1" + }, + "result": "sum / cnt", + } + ]} +``` + +This creates a profile... + * Named ‘http_mean_resp_body_len’ + * That for each IP source address + * That is either HTTP or HTTPS + * Accumulates the sum of response body length + * Accumulates the count of messages + * After the window period expires, the average is calculated + +## Getting Started + +This section will describe the steps required to get your first profile running. + +1. Launch the 'Quick Dev' environment. + ``` + $ cd metron-deployment/vagrant/quick-dev-platform/ + $ ./run.sh + ``` + +2. After the environment has been deployed, then login to the host. + ``` + $ vagrant ssh + $ sudo su - + $ cd /usr/metron/0.2.0BETA/ + ``` + +3. Create a table within HBase that will store the profile data. The table name and column family must match the Profiler topology configuration stored at `/usr/metron/0.2.0BETA/config/profiler.properties`. + ``` + $ /usr/hdp/current/hbase-client/bin/hbase shell + hbase(main):001:0> create 'profiler1', 'cfProfile' + ``` + +4. Shorten the flush intervals to more immediately see results. Edit the Profiler topology properties located at `/usr/metron/0.2.0BETA/config/profiler.properties`. Alter the following two properties. + ``` + profiler.flush.interval.seconds=30 + profiler.hbase.flush.interval.seconds=30 + ``` + +5. Create the Profiler definition in a file located at `/usr/metron/0.2.0BETA/config/zookeeper/profiler.json`. The following JSON will create a profile that simply counts the number of messages. + ``` + { + "inputTopic": "indexing", + "profiles": [ + { + "profile": "test", + "foreach": "ip_src_addr", + "onlyif": "true", + "init": { "sum": 0 }, + "update": { "sum": "ADD(sum,1)" }, + "result": "TO_LONG(sum)" + } + ] + } + ``` + +6. Upload the Profiler definition to Zookeeper. + ``` + $ bin/zk_load_configs.sh -m PUSH -i config/zookeeper/ -z node1:2181 + ``` + +7. Start the Profiler topology. + ``` + bin/start_profiler_topology.sh + ``` + +8. Ensure that test messages are being sent to the Profiler's input topic in Kafka. The Profiler will consume messages from the `inputTopic` in the Profiler definition. + +9. Check the HBase table to validate that the Profiler is working. + ``` + $ /usr/hdp/current/hbase-client/bin/hbase shell + hbase(main):001:0> count 'profiler1' + ``` + +## Design + +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. + +### 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. + +### 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 + +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 new file mode 100644 index 0000000000..e3b4b4fdfc --- /dev/null +++ b/metron-analytics/metron-profiler/pom.xml @@ -0,0 +1,332 @@ + + + + + 4.0.0 + + org.apache.metron + metron-analytics + 0.2.0BETA + + metron-profiler + + UTF-8 + UTF-8 + + + + org.apache.metron + metron-hbase + ${project.parent.version} + + + org.apache.metron + metron-integration-test + ${project.parent.version} + test + + + org.apache.metron + metron-test-utilities + ${project.parent.version} + test + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + com.google.guava + guava + + + + + 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.adrianwalker + multiline-string + 0.1.2 + test + + + tools + sun.jdk + + + + + org.apache.storm + storm-core + ${global_storm_version} + + + servlet-api + javax.servlet + + + log4j-over-slf4j + org.slf4j + + + + + org.apache.storm + storm-hdfs + ${global_storm_version} + + + org.apache.storm + storm-core + + + org.apache.hadoop + hadoop-client + + + + + org.apache.storm + storm-hbase + ${global_storm_version} + + + org.apache.kafka + kafka_2.9.2 + ${global_kafka_version} + + + log4j + log4j + + + + + 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 + ${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.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 + + + + + + + diff --git a/metron-analytics/metron-profiler/src/main/assembly/assembly.xml b/metron-analytics/metron-profiler/src/main/assembly/assembly.xml new file mode 100644 index 0000000000..a5f1643e39 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/assembly/assembly.xml @@ -0,0 +1,72 @@ + + + + archive + + tar.gz + + false + + + ${project.basedir}/src/main/config + /config + true + + **/*.formatted + **/*.filtered + + 0644 + unix + true + + + ${project.basedir}/src/main/scripts + /bin + true + + **/*.formatted + **/*.filtered + + 0755 + unix + true + + + ${project.basedir}/src/main/flux + /flux + true + + **/*.formatted + **/*.filtered + + 0644 + unix + + + ${project.basedir}/target + + ${project.artifactId}-${project.version}.jar + + /lib + true + + + diff --git a/metron-analytics/metron-profiler/src/main/config/profiler.properties b/metron-analytics/metron-profiler/src/main/config/profiler.properties new file mode 100644 index 0000000000..e8516e9b06 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/config/profiler.properties @@ -0,0 +1,36 @@ +# +# +# 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. +# +# + +##### Storm ##### + +profiler.workers=1 +profiler.executors=0 +profiler.input.topic=indexing +profiler.flush.interval.seconds=900 +profiler.hbase.salt.divisor=10 +profiler.hbase.table=profiler +profiler.hbase.batch=10 +profiler.hbase.flush.interval.seconds=30 + +##### Kafka ##### + +kafka.zk=node1:2181 +kafka.broker=node1:6667 +kafka.start=WHERE_I_LEFT_OFF diff --git a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml new file mode 100644 index 0000000000..d99510a036 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml @@ -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. + +name: "profiler" + +config: + topology.workers: ${profiler.workers} + topology.acker.executors: ${profiler.executors} + +components: + + - id: "defaultExecutor" + className: "org.apache.metron.profiler.stellar.DefaultStellarExecutor" + + - id: "hbaseMapper" + className: "org.apache.metron.profiler.bolt.ProfileHBaseMapper" + properties: + - name: "saltDivisor" + value: ${profiler.hbase.salt.divisor} + + - id: "zkHosts" + className: "storm.kafka.ZkHosts" + constructorArgs: + - "${kafka.zk}" + + - id: "kafkaConfig" + className: "org.apache.metron.common.spout.kafka.SpoutConfig" + constructorArgs: + # zookeeper hosts + - ref: "zkHosts" + # topic name + - "${profiler.input.topic}" + # zk root + - "" + # id + - "indexing" + configMethods: + - name: "from" + args: + - "${kafka.start}" + +spouts: + + - id: "kafkaSpout" + className: "storm.kafka.KafkaSpout" + constructorArgs: + - ref: "kafkaConfig" + +bolts: + + - id: "splitterBolt" + className: "org.apache.metron.profiler.bolt.ProfileSplitterBolt" + constructorArgs: + - "${kafka.zk}" + properties: + - name: "executor" + ref: "defaultExecutor" + + - id: "builderBolt" + className: "org.apache.metron.profiler.bolt.ProfileBuilderBolt" + constructorArgs: + - "${kafka.zk}" + properties: + - name: "executor" + ref: "defaultExecutor" + - name: "flushFrequency" + value: ${profiler.flush.interval.seconds} + + - id: "hbaseBolt" + className: "org.apache.metron.hbase.bolt.HBaseBolt" + constructorArgs: + - "${profiler.hbase.table}" + - ref: "hbaseMapper" + configMethods: + - name: "withTableProvider" + args: ["${hbase.provider.impl}"] + - name: "withBatchSize" + args: [${profiler.hbase.batch}] + - name: "withFlushIntervalSecs" + args: [${profiler.hbase.flush.interval.seconds}] + +streams: + + - name: "spout -> splitter" + from: "kafkaSpout" + to: "splitterBolt" + grouping: + type: SHUFFLE + + - name: "splitter -> builder" + from: "splitterBolt" + to: "builderBolt" + grouping: + type: FIELDS + args: ["entity", "profile", "message"] + + - name: "builder -> hbase" + from: "builderBolt" + to: "hbaseBolt" + grouping: + type: SHUFFLE \ No newline at end of file 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 new file mode 100644 index 0000000000..e240567991 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java @@ -0,0 +1,133 @@ +/* + * + * 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; + +/** + * Represents a single data point within a Profile. + * + * A Profile is effectively a time series. To this end a Profile is composed + * of many ProfileMeasurement values which in aggregate form a time series. + */ +public class ProfileMeasurement { + + /** + * The name of the profile that this measurement is associated with. + */ + private String profileName; + + /** + * The name of the entity being profiled. + */ + private String entity; + + /** + * When the measurement window was started in milliseconds since the epoch. + */ + private long start; + + /** + * When the measurement window closed in milliseconds since the epoch. + */ + private long end; + + /** + * The actual measurement itself. + */ + private double value; + + public String getProfileName() { + return profileName; + } + + public void setProfileName(String profileName) { + this.profileName = 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 long getEnd() { + return end; + } + + public void setEnd(long end) { + this.end = end; + } + + public double getValue() { + return value; + } + + public void setValue(double value) { + this.value = value; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ProfileMeasurement that = (ProfileMeasurement) o; + + if (start != that.start) return false; + if (end != that.end) return false; + if (Double.compare(that.value, value) != 0) return false; + if (profileName != null ? !profileName.equals(that.profileName) : that.profileName != null) return false; + return entity != null ? entity.equals(that.entity) : that.entity == null; + } + + @Override + public int hashCode() { + int result; + long temp; + 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)); + temp = Double.doubleToLongBits(value); + result = 31 * result + (int) (temp ^ (temp >>> 32)); + return result; + } + + @Override + public String toString() { + return "ProfileMeasurement{" + + "profileName='" + profileName + '\'' + + ", entity='" + entity + '\'' + + ", start=" + start + + ", end=" + end + + ", value=" + value + + '}'; + } +} 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 new file mode 100644 index 0000000000..6d03f6e4a5 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java @@ -0,0 +1,265 @@ +/* + * + * 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.Config; +import backtype.storm.Constants; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import org.apache.metron.common.bolt.ConfiguredProfilerBolt; +import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.metron.profiler.stellar.StellarExecutor; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; + +/** + * A bolt that is responsible for building a Profile. + * + * This bolt maintains 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. + */ +public class ProfileBuilderBolt extends ConfiguredProfilerBolt { + + protected static final Logger LOG = LoggerFactory.getLogger(ProfileBuilderBolt.class); + + /** + * Executes Stellar code and maintains state across multiple invocations. + */ + private StellarExecutor executor; + + /** + * The number of seconds between when the Profile is flushed. + */ + private int flushFrequency; + + /** + * A ProfileMeasurement is created and emitted each window period. A Profile + * itself is composed of many ProfileMeasurements. + */ + private transient ProfileMeasurement measurement; + + /** + * The definition of the Profile that the bolt is building. + */ + private transient ProfileConfig profileConfig; + + /** + * Parses JSON messages. + */ + private transient JSONParser parser; + + private OutputCollector collector; + + /** + * @param zookeeperUrl The Zookeeper URL that contains the configuration data. + */ + public ProfileBuilderBolt(String zookeeperUrl) { + super(zookeeperUrl); + } + + /** + * Defines the frequency at which the bolt will receive tick tuples. Tick tuples are + * used to control how often a profile is flushed. + */ + @Override + public Map getComponentConfiguration() { + Config conf = new Config(); + conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, getFlushFrequency()); + return conf; + } + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + super.prepare(stormConf, context, collector); + this.collector = collector; + this.parser = new JSONParser(); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + // once the time window expires, a complete ProfileMeasurement is emitted + declarer.declare(new Fields("measurement")); + } + + @Override + public void execute(Tuple input) { + try { + doExecute(input); + collector.ack(input); + + } catch (IOException e) { + LOG.error("exception: {}", e); + collector.reportError(e); + } + } + + /** + * Update the execution environment based on data contained in the + * message. If the tuple is a tick tuple, then flush the profile + * and reset the execution environment. + * @param input The tuple to execute. + */ + private void doExecute(Tuple input) throws IOException { + + 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); + } + } + } + + /** + * Initialize the bolt. Occurs when the first tuple is received at the start + * of each window period. + * @param input The input tuple + */ + private void init(Tuple input) throws IOException { + + // save the profile definition - needed later during a flush + 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()); + + // execute the 'init' expression + JSONObject message = (JSONObject) input.getValueByField("message"); + Map expressions = profileConfig.getInit(); + expressions.forEach((var, expr) -> executor.assign(var, expr, message)); + } + + /** + * Update the Profile based on data contained in a new message. + * @param input The tuple containing a new message. + */ + private void update(Tuple input) throws IOException { + JSONObject message = (JSONObject) input.getValueByField("message"); + + // execute each of the 'update' expressions + Map expressions = profileConfig.getUpdate(); + expressions.forEach((var, expr) -> executor.assign(var, expr, message)); + } + + /** + * Flush the Profile. + * + * Executed on a fixed time period when a tick tuple is received. Completes + * and emits the ProfileMeasurement. Clears all state in preparation for + * the next window period. + */ + private void flush(Tuple tickTuple) throws IOException { + LOG.info(String.format("Flushing profile: profile=%s, entity=%s, start=%d", + measurement.getProfileName(), measurement.getEntity(), measurement.getStart())); + + // execute the 'result' expression + String resultExpr = profileConfig.getResult(); + Long result = executor.execute(resultExpr, new JSONObject(), Long.class); + + // emit the completed profile measurement + measurement.setEnd(getTimestamp()); + measurement.setValue(result); + emit(measurement, tickTuple); + + // clear the execution state to prepare for the next window + executor.clearState(); + + // reset measurement - used as a flag to indicate if initialized + measurement = null; + } + + /** + * Emits a message containing a ProfileMeasurement. + * @param measurement The completed ProfileMeasurement. + */ + private void emit(ProfileMeasurement measurement, Tuple anchor) { + collector.emit(anchor, new Values(measurement)); + collector.ack(anchor); + } + + /** + * Returns a value that can be used as the current timestamp. Allows subclasses + * to override, if necessary. + */ + private long getTimestamp() { + return System.currentTimeMillis(); + } + + /** + * Has the Stellar execution environment already been initialized + * @return True, it it has been initialized. + */ + private boolean isInitialized() { + return measurement != null; + } + + /** + * Is this a tick tuple? + * @param tuple The tuple + */ + protected static boolean isTickTuple(Tuple tuple) { + return Constants.SYSTEM_COMPONENT_ID.equals(tuple.getSourceComponent()) && + Constants.SYSTEM_TICK_STREAM_ID.equals(tuple.getSourceStreamId()); + } + + public StellarExecutor getExecutor() { + return executor; + } + + public void setExecutor(StellarExecutor executor) { + this.executor = executor; + } + + public int getFlushFrequency() { + return flushFrequency; + } + + 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 new file mode 100644 index 0000000000..693ca28b0e --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileHBaseMapper.java @@ -0,0 +1,136 @@ +/* + * + * 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.hadoop.hbase.util.Bytes; +import org.apache.metron.profiler.ProfileMeasurement; +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; + +/** + * An HbaseMapper that defines how a ProfileMeasurement is persisted within an HBase table. + */ +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. + */ + private int saltDivisor = 10; + + /** + * The name of the column family. + */ + private String columnFamily; + + public ProfileHBaseMapper() { + setColumnFamily("cfProfile"); + } + + /** + * Define the row key for a ProfileMeasurement. + * @param tuple The tuple containing a ProfileMeasurement. + * @return The Hbase row key. + */ + @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()); + } + + /** + * Defines how the fields within a ProfileMeasurement are mapped to HBase. + * @param tuple The tuple containing the ProfileMeasurement. + */ + @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, Bytes.toBytes(measurement.getValue())); + + return cols; + } + + /** + * 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 int getSaltDivisor() { + return saltDivisor; + } + + public void setSaltDivisor(int saltDivisor) { + this.saltDivisor = saltDivisor; + } + + 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 new file mode 100644 index 0000000000..196809fe15 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java @@ -0,0 +1,158 @@ +/* + * + * 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.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import org.apache.metron.common.bolt.ConfiguredProfilerBolt; +import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.apache.metron.profiler.stellar.StellarExecutor; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.json.simple.parser.ParseException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.UnsupportedEncodingException; +import java.util.Map; + +/** + * The bolt responsible for filtering incoming messages and directing + * each to the one or more bolts responsible for building a Profile. Each + * message may be needed by 0, 1 or even many Profiles. + */ +public class ProfileSplitterBolt extends ConfiguredProfilerBolt { + + protected static final Logger LOG = LoggerFactory.getLogger(ProfileSplitterBolt.class); + + private OutputCollector collector; + + /** + * JSON parser. + */ + private transient JSONParser parser; + + /** + * Executes Stellar code. + */ + private StellarExecutor executor; + + /** + * @param zookeeperUrl The Zookeeper URL that contains the configuration for this bolt. + */ + public ProfileSplitterBolt(String zookeeperUrl) { + super(zookeeperUrl); + } + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + super.prepare(stormConf, context, collector); + this.collector = collector; + this.parser = new JSONParser(); + } + + @Override + public void execute(Tuple input) { + try { + doExecute(input); + + } catch (IllegalArgumentException | ParseException | UnsupportedEncodingException e) { + LOG.error("Unexpected exception", e); + collector.reportError(e); + } + } + + private void doExecute(Tuple input) throws ParseException, UnsupportedEncodingException { + + // retrieve the input message + byte[] data = input.getBinary(0); + JSONObject message = (JSONObject) parser.parse(new String(data, "UTF8")); + + // ensure there is a valid profiler configuration + ProfilerConfig config = getProfilerConfig(); + if(config == null) { + throw new IllegalArgumentException("Fatal: Unable to find valid profiler definition"); + } + + // apply the message to each of the profile definitions + for (ProfileConfig profile: config.getProfiles()) { + applyProfile(profile, input, message); + } + + collector.ack(input); + } + + /** + * Applies a message to a Profile definition. + * @param profile The profile definition. + * @param input The input tuple that delivered the message. + * @param message The message that may be needed by the profile. + */ + private void applyProfile(ProfileConfig profile, Tuple input, JSONObject message) throws ParseException, UnsupportedEncodingException { + + // is this message needed by this profile? + String onlyIf = profile.getOnlyif(); + if (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); + + // emit a message for the bolt responsible for building this profile + collector.emit(input, new Values(entity, profile, copyMessage(input))); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + + // each emitted tuple contains the 'resolved' entity, the profile definition, and the input message + declarer.declare(new Fields("entity", "profile", "message")); + } + + /** + * Creates a deep copy of the original JSON message. + * @param input The input tuple. + */ + private JSONObject copyMessage(Tuple input) throws ParseException, UnsupportedEncodingException { + + // create a new message + byte[] data = input.getBinary(0); + JSONObject message = (JSONObject) parser.parse(new String(data, "UTF8")); + + // stamp the message with a timestamp + message.put(getClass().getSimpleName().toLowerCase() + ".splitter.begin.ts", "" + System.currentTimeMillis()); + + return message; + } + + public StellarExecutor getExecutor() { + return executor; + } + + public void setExecutor(StellarExecutor executor) { + this.executor = executor; + } +} 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 new file mode 100644 index 0000000000..9b3dd18613 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/DefaultStellarExecutor.java @@ -0,0 +1,164 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler.stellar; + +import org.apache.metron.common.dsl.MapVariableResolver; +import org.apache.metron.common.dsl.VariableResolver; +import org.apache.metron.common.query.PredicateProcessor; +import org.apache.metron.common.transformation.TransformationProcessor; +import org.json.simple.JSONObject; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +/** + * The default implementation of a StellarExecutor. + */ +public class DefaultStellarExecutor implements StellarExecutor, Serializable { + + /** + * The current state of the Stellar execution environment. + */ + private Map state; + + public DefaultStellarExecutor() { + clearState(); + } + + /** + * @param initialState Initial state loaded into the execution environment. + */ + public DefaultStellarExecutor(Map initialState) { + this.state = new HashMap<>(initialState); + } + + @Override + public Map getState() { + return new HashMap<>(state); + } + + /** + * Execute an expression and assign the result to a variable. + * + * Stellar does not directly support assignment currently. This makes + * life easier until it does. + * + * @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. + */ + @Override + public void assign(String variable, String expression, JSONObject message) { + + Object result = execute(expression, message); + state.put(variable, result); + } + + /** + * Execute a Stellar expression and returns the result. + * + * @param expression The expression to execute. + * @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. + */ + @Override + public T execute(String expression, JSONObject message, Class clazz) { + + Object result = execute(expression, message); + if(clazz.isAssignableFrom(result.getClass())) { + return (T) result; + + } else { + throw new RuntimeException(String.format("Unexpected type; expected=%s, actual=%s, expression=%s", + clazz.getSimpleName(), result.getClass().getSimpleName(), expression)); + } + } + + @Override + public void clearState() { + this.state = new HashMap<>(); + } + + /** + * Execute a Stellar expression. + * + * There are two sets of functions in Stellar. One can be executed with + * a PredicateProcessor and the other a TransformationProcessor. This method + * uses the TransformationProcessor. + * + * @param expression The expression to execute. + * @param message The message that is accessible when Stellar is executed. + */ + private Object execute(String expression, JSONObject message) { + + Object result = null; + try { + result = executeTransformation(expression, message); + + } finally { + // maybe the expression is a predicate, not a transformation + if(result == null) { + result = executePredicate(expression, message); + } + + return result; + } + } + + /** + * Executes Stella predicates using the TransformationProcessor. There are two sets + * of functions in Stellar currently. One can be executed with a + * PredicateProcessor and the other a TransformationProcessor. + * @param expression The expression to execute. + * @param message The message that is accessible when Stellar is executed. + */ + private Object executeTransformation(String expression, JSONObject message) { + + // vars can be resolved from the execution state or the current message + VariableResolver resolver = new MapVariableResolver(state, message); + + TransformationProcessor processor = new TransformationProcessor(); + return processor.parse(expression, resolver); + } + + /** + * Executes Stella predicates using the PredicateProcessor. There are two sets + * of functions in Stellar currently. One can be executed with a + * PredicateProcessor and the other a TransformationProcessor. + * @param expression The expression to execute. + * @param message The message that is accessible when Stellar is executed. + */ + private boolean executePredicate(String expression, JSONObject message) { + + // vars can be resolved from the execution state or the input message + VariableResolver resolver = new MapVariableResolver(state, message); + + PredicateProcessor processor = new PredicateProcessor(); + boolean valid = processor.validate(expression); + if(!valid) { + throw new RuntimeException(String.format("Invalid predicate expression; expression=%s", expression)); + } + + return processor.parse(expression, resolver); + } +} 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 new file mode 100644 index 0000000000..9005e00b1e --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/stellar/StellarExecutor.java @@ -0,0 +1,64 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler.stellar; + +import org.json.simple.JSONObject; + +import java.util.Map; + +/** + * Executes Stellar expressions and maintains state across multiple invocations. + * + * There are two sets of functions in Stellar currently. One can be executed with + * a PredicateProcessor and the other a TransformationProcessor. This interface + * abstracts away that complication. + */ +public interface StellarExecutor { + + /** + * Execute an expression and assign the result to a variable. + * + * @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. + */ + void assign(String variable, String expression, JSONObject 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 clazz The expected class of the expression's result. + * @param The expected class of the expression's result. + */ + T execute(String expression, JSONObject message, Class clazz); + + /** + * The current state of the Stellar execution environment. + */ + Map getState(); + + /** + * Removes all state from the execution environment. + */ + void clearState(); +} diff --git a/metron-analytics/metron-profiler/src/main/scripts/start_profiler_topology.sh b/metron-analytics/metron-profiler/src/main/scripts/start_profiler_topology.sh new file mode 100644 index 0000000000..6f78471293 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/scripts/start_profiler_topology.sh @@ -0,0 +1,22 @@ +#!/bin/bash +# +# 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. +# +METRON_VERSION=${project.version} +METRON_HOME=/usr/metron/$METRON_VERSION +TOPOLOGY_JAR=${project.artifactId}-$METRON_VERSION.jar +storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.storm.flux.Flux --remote $METRON_HOME/flux/profiler/remote.yaml --filter $METRON_HOME/config/profiler.properties diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json new file mode 100644 index 0000000000..eecdc2105c --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json @@ -0,0 +1,13 @@ +{ + "inputTopic": "indexing", + "profiles": [ + { + "profile": "profiler-integration-test", + "foreach": "ip_src_addr", + "onlyif": "STARTS_WITH(protocol, 'HTTP')", + "init": { "cnt": 0 }, + "update": { "cnt": "ADD(cnt, 1)" }, + "result": "TO_LONG(cnt)" + } + ] +} \ No newline at end of file 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 new file mode 100644 index 0000000000..7bd4771091 --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java @@ -0,0 +1,204 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler.bolt; + +import backtype.storm.Constants; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import org.adrianwalker.multilinestring.Multiline; +import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.metron.profiler.stellar.DefaultStellarExecutor; +import org.apache.metron.test.bolt.BaseBoltTest; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; + +import java.io.IOException; +import java.util.HashMap; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.*; + +/** + * Tests the ProfileBuilderBolt. + */ +public class ProfileBuilderBoltTest extends BaseBoltTest { + + /** + * { + * "ip_src_addr": "10.0.0.1", + * "ip_dst_addr": "10.0.0.20" + * } + */ + @Multiline + private String input; + + /** + * { + * "profile": "test", + * "foreach": "ip_src_addr", + * "onlyif": "true", + * "init": { + * "x": "10", + * "y": "20" + * }, + * "update": { + * "x": "ADD(x, 10)", + * "y": "ADD(y, 20)" + * }, + * "result": "TO_LONG(ADD(x, y))" + * } + */ + @Multiline + private String profile; + + private JSONObject message; + + public static Tuple mockTickTuple() { + return mockTuple(Constants.SYSTEM_COMPONENT_ID, Constants.SYSTEM_TICK_STREAM_ID); + } + + public static Tuple mockTuple(String componentId, String streamId) { + Tuple tuple = mock(Tuple.class); + when(tuple.getSourceComponent()).thenReturn(componentId); + when(tuple.getSourceStreamId()).thenReturn(streamId); + return tuple; + } + + @Before + public void setup() throws Exception { + + // parse the input message + JSONParser parser = new JSONParser(); + message = (JSONObject) parser.parse(input); + + // the tuple will contain the original message + when(tuple.getValueByField(eq("message"))).thenReturn(message); + + // the tuple will contain the 'fully resolved' name of the entity + when(tuple.getStringByField(eq("entity"))).thenReturn("10.0.0.1"); + + // the tuple will contain the profile definition + ProfileConfig profileConfig = JSONUtils.INSTANCE.load(profile, ProfileConfig.class); + when(tuple.getValueByField(eq("profile"))).thenReturn(profileConfig); + } + + /** + * Create a ProfileBuilderBolt to test + */ + private ProfileBuilderBolt createBolt() throws IOException { + + ProfileBuilderBolt bolt = new ProfileBuilderBolt("zookeeperURL"); + bolt.setCuratorFramework(client); + bolt.setTreeCache(cache); + bolt.setExecutor(new DefaultStellarExecutor()); + + bolt.prepare(new HashMap<>(), topologyContext, outputCollector); + return bolt; + } + + /** + * Ensure that the bolt can update a profile based on new messages that it receives. + */ + @Test + public void testUpdateProfile() throws Exception { + + ProfileBuilderBolt bolt = createBolt(); + bolt.execute(tuple); + bolt.execute(tuple); + + // validate that x=10+10+10 y=20+20+20 + assertThat(bolt.getExecutor().getState().get("x"), equalTo(10+10+10)); + assertThat(bolt.getExecutor().getState().get("y"), equalTo(20+20+20)); + } + + /** + * Ensure that the bolt can flush the profile when a tick tuple is received. + */ + @Test + public void testFlushProfile() throws Exception { + + // setup + ProfileBuilderBolt bolt = createBolt(); + bolt.execute(tuple); + bolt.execute(tuple); + + // execute - the tick tuple triggers a flush of the profile + bolt.execute(mockTickTuple()); + + // capture the ProfileMeasurement that should be emitted + ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); + verify(outputCollector, times(1)).emit(refEq(tuple), arg.capture()); + + Values actual = arg.getValue(); + ProfileMeasurement measurement = (ProfileMeasurement) actual.get(0); + + // verify + assertThat(measurement.getValue(), equalTo(90.0)); + assertThat(measurement.getEntity(), equalTo("10.0.0.1")); + assertThat(measurement.getProfileName(), equalTo("test")); + } + + /** + * What happens if we try to flush, but have yet to receive any messages to + * apply to the profile? + * + * The ProfileBuilderBolt will not have received the data necessary from the + * ProfileSplitterBolt, like the entity and profile name, that is required + * to perform the flush. The flush has to be skipped until this information + * is received from the Splitter. + */ + @Test + public void testFlushProfileWithNoMessages() throws Exception { + + ProfileBuilderBolt bolt = createBolt(); + + // no messages have been received before a flush occurs + bolt.execute(mockTickTuple()); + bolt.execute(mockTickTuple()); + bolt.execute(mockTickTuple()); + + // no ProfileMeasurement should be written to the ProfileStore + verify(outputCollector, times(0)).emit(any(Values.class)); + } + + /** + * The executor's state should be cleared after a flush. + */ + @Test + public void testStateClearedAfterFlush() throws Exception { + + ProfileBuilderBolt bolt = createBolt(); + bolt.execute(tuple); + bolt.execute(tuple); + + // execute - should clear state from previous tuples + bolt.execute(mockTickTuple()); + + assertThat(bolt.getExecutor().getState().size(), equalTo(0)); + } +} diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileSplitterBoltTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileSplitterBoltTest.java new file mode 100644 index 0000000000..13f81710bd --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileSplitterBoltTest.java @@ -0,0 +1,212 @@ +/* + * + * 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 backtype.storm.tuple.Values; +import org.adrianwalker.multilinestring.Multiline; +import org.apache.metron.profiler.stellar.DefaultStellarExecutor; +import org.apache.metron.test.bolt.BaseBoltTest; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.json.simple.parser.ParseException; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashMap; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.refEq; +import static org.mockito.Mockito.*; + +/** + * Tests the ProfileSplitterBolt. + */ +public class ProfileSplitterBoltTest extends BaseBoltTest { + + /** + * { + * "ip_src_addr": "10.0.0.1", + * "ip_dst_addr": "10.0.0.20", + * "protocol": "HTTP" + * } + */ + @Multiline + private String input; + + /** + * { + * "inputTopic": "enrichment", + * "profiles": [ + * { + * "profile": "test", + * "foreach": "ip_src_addr", + * "onlyif": "protocol == 'HTTP'", + * "init": {}, + * "update": {}, + * "result": 2 + * } + * ] + * } + */ + @Multiline + private String onlyIfTrue; + + /** + * { + * "inputTopic": "enrichment", + * "profiles": [ + * { + * "profile": "test", + * "foreach": "ip_src_addr", + * "onlyif": "false", + * "init": {}, + * "update": {}, + * "result": 2 + * } + * ] + * } + */ + @Multiline + private String onlyIfFalse; + + /** + * { + * "inputTopic": "enrichment", + * "profiles": [ + * { + * "profile": "test", + * "foreach": "ip_src_addr", + * "onlyif": "NOT-VALID", + * "init": {}, + * "update": {}, + * "result": 2 + * } + * ] + * } + */ + @Multiline + private String invalidOnlyIf; + + private JSONObject message; + + @Before + public void setup() throws ParseException { + + // parse the input message + JSONParser parser = new JSONParser(); + message = (JSONObject) parser.parse(input); + + // ensure the tuple returns the expected json message + when(tuple.getBinary(0)).thenReturn(input.getBytes()); + } + + /** + * Create a ProfileSplitterBolt to test + */ + private ProfileSplitterBolt createBolt(String profilerConfig) throws IOException { + + ProfileSplitterBolt bolt = new ProfileSplitterBolt("zookeeperURL"); + bolt.setCuratorFramework(client); + bolt.setTreeCache(cache); + bolt.getConfigurations().updateProfilerConfig(profilerConfig.getBytes("UTF-8")); + bolt.setExecutor(new DefaultStellarExecutor()); + + bolt.prepare(new HashMap<>(), topologyContext, outputCollector); + return bolt; + } + + /** + * What happens when a message is received that is needed by a profile? + * + * This occurs when a profile's 'onlyif' expression is true. + */ + @Test + public void testMessageNeededByProfile() throws Exception { + + // setup + ProfileSplitterBolt bolt = createBolt(onlyIfTrue); + + // execute + bolt.execute(tuple); + + // a tuple should be emitted for the downstream profile builder + verify(outputCollector, times(1)).emit(refEq(tuple), any(Values.class)); + + // the original tuple should be ack'd + verify(outputCollector, times(1)).ack(tuple); + } + + /** + * What happens when a message is received that is NOT needed by a profile? + * + * This occurs when a profile's 'onlyif' expression is false + */ + @Test + public void testMessageNotNeededByProfile() throws Exception { + + // setup + ProfileSplitterBolt bolt = createBolt(onlyIfFalse); + + // execute + bolt.execute(tuple); + + // a tuple should NOT be emitted for the downstream profile builder + verify(outputCollector, times(0)).emit(any(Values.class)); + + // the original tuple should be ack'd + verify(outputCollector, times(1)).ack(tuple); + } + + /** + * The entity associated with a ProfileMeasurement can be defined using a variable that is resolved + * via Stella. In this case the entity is defined as 'ip_src_addr' which is resolved to + * '10.0.0.1' based on the data contained within the message. + */ + @Test + public void testResolveEntityName() throws Exception { + + // setup + ProfileSplitterBolt bolt = createBolt(onlyIfTrue); + + // execute + bolt.execute(tuple); + + // verify - the entity name comes from variable resolution in stella + String expectedEntity = "10.0.0.1"; + verify(outputCollector, times(1)).emit(any(Tuple.class), refEq(new Values(expectedEntity, onlyIfTrue, message))); + } + + /** + * What happens when invalid Stella code is used for 'onlyif'? + */ + @Test(expected = org.apache.metron.common.dsl.ParseException.class) + public void testOnlyIfInvalid() throws Exception { + + // setup + ProfileSplitterBolt bolt = createBolt(invalidOnlyIf); + + // execute + bolt.execute(tuple); + } +} diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java new file mode 100644 index 0000000000..f096d51214 --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java @@ -0,0 +1,119 @@ +/* + * + * 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.integration; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.metron.common.configuration.SensorParserConfig; +import org.apache.metron.integration.InMemoryComponent; +import org.apache.metron.integration.UnableToStartException; +import org.apache.metron.integration.components.KafkaWithZKComponent; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; + +import static org.apache.metron.common.configuration.ConfigurationsUtils.*; + +public class ConfigUploadComponent implements InMemoryComponent { + + private Properties topologyProperties; + private String globalConfigPath; + private String parserConfigsPath; + private String enrichmentConfigsPath; + private String profilerConfigPath; + private Optional globalConfig = Optional.empty(); + private Map parserSensorConfigs = new HashMap<>(); + + public ConfigUploadComponent withTopologyProperties(Properties topologyProperties) { + this.topologyProperties = topologyProperties; + return this; + } + + public ConfigUploadComponent withGlobalConfigsPath(String globalConfigPath) { + this.globalConfigPath = globalConfigPath; + return this; + } + + public ConfigUploadComponent withParserConfigsPath(String parserConfigsPath) { + this.parserConfigsPath = parserConfigsPath; + return this; + } + public ConfigUploadComponent withEnrichmentConfigsPath(String enrichmentConfigsPath) { + this.enrichmentConfigsPath = enrichmentConfigsPath; + return this; + } + + public ConfigUploadComponent withProfilerConfigsPath(String profilerConfigsPath) { + this.profilerConfigPath = profilerConfigsPath; + return this; + } + + public ConfigUploadComponent withParserSensorConfig(String name, SensorParserConfig config) { + parserSensorConfigs.put(name, config); + return this; + } + + public ConfigUploadComponent withGlobalConfig(String globalConfig) { + this.globalConfig = Optional.ofNullable(globalConfig); + return this; + } + + @Override + public void start() throws UnableToStartException { + try { + final String zookeeperUrl = topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY); + + if(globalConfigPath != null) { + uploadConfigsToZookeeper(globalConfigPath, parserConfigsPath, enrichmentConfigsPath, profilerConfigPath, zookeeperUrl); + } + + for(Map.Entry kv : parserSensorConfigs.entrySet()) { + writeSensorParserConfigToZookeeper(kv.getKey(), kv.getValue(), zookeeperUrl); + } + + if(globalConfig.isPresent()) { + writeGlobalConfigToZookeeper(globalConfig.get().getBytes(), zookeeperUrl); + } + + } catch (Exception e) { + throw new UnableToStartException(e.getMessage(), e); + } + } + + public SensorParserConfig getSensorParserConfig(String sensorType) { + SensorParserConfig sensorParserConfig = new SensorParserConfig(); + CuratorFramework client = getClient(topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY)); + client.start(); + try { + sensorParserConfig = readSensorParserConfigFromZookeeper(sensorType, client); + } catch (Exception e) { + e.printStackTrace(); + } finally { + client.close(); + } + return sensorParserConfig; + } + + @Override + public void stop() { + + } +} 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 new file mode 100644 index 0000000000..a898b9b45e --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.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.integration; + +import org.adrianwalker.multilinestring.Multiline; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.metron.common.Constants; +import org.apache.metron.common.spout.kafka.SpoutConfig; +import org.apache.metron.hbase.TableProvider; +import org.apache.metron.integration.BaseIntegrationTest; +import org.apache.metron.integration.ComponentRunner; +import org.apache.metron.integration.components.FluxTopologyComponent; +import org.apache.metron.integration.components.KafkaWithZKComponent; +import org.apache.metron.test.mock.MockHTable; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.Assert.assertTrue; + +/** + * An integration test of the Profiler topology. + */ +public class ProfilerIntegrationTest extends BaseIntegrationTest { + + private static final String FLUX_PATH = "../metron-profiler/src/main/flux/profiler/remote.yaml"; + + /** + * { + * "ip_src_addr": "10.0.0.1", + * "protocol": "HTTPS", + * "length": "10" + * } + */ + @Multiline + private String message1; + + /** + * { + * "ip_src_addr": "10.0.0.2", + * "protocol": "HTTP", + * "length": "20" + * } + */ + @Multiline + private String message2; + + private FluxTopologyComponent fluxComponent; + private KafkaWithZKComponent kafkaComponent; + private List input; + private ComponentRunner runner; + private MockHTable profilerTable; + + private static final String tableName = "profiler"; + private static final String columnFamily = "cfProfile"; + + public static class MockTableProvider implements TableProvider, Serializable { + + MockHTable.Provider provider = new MockHTable.Provider(); + + @Override + public HTableInterface getTable(Configuration config, String tableName) throws IOException { + return provider.getTable(config, tableName); + } + } + + @Before + public void setup() throws Exception { + + // create input messages for the profiler to consume + input = Stream.of(message1, message2) + .map(Bytes::toBytes) + .map(m -> Collections.nCopies(10, m)) + .flatMap(l -> l.stream()) + .collect(Collectors.toList()); + + // storm topology properties + final Properties topologyProperties = new Properties() {{ + setProperty("kafka.start", SpoutConfig.Offset.BEGINNING.name()); + setProperty("profiler.workers", "1"); + setProperty("profiler.executors", "0"); + setProperty("profiler.input.topic", Constants.INDEXING_TOPIC); + setProperty("profiler.flush.interval.seconds", "15"); + setProperty("profiler.hbase.salt.divisor", "10"); + setProperty("profiler.hbase.table", tableName); + setProperty("profiler.hbase.batch", "10"); + setProperty("profiler.hbase.flush.interval.seconds", "2"); + setProperty("hbase.provider.impl", "" + MockTableProvider.class.getName()); + }}; + + // create the mock table + profilerTable = (MockHTable) MockHTable.Provider.addToCache(tableName, columnFamily); + + // create the input topic + kafkaComponent = getKafkaComponent(topologyProperties, + Arrays.asList(new KafkaWithZKComponent.Topic(Constants.INDEXING_TOPIC, 1))); + + // upload profiler configuration to zookeeper + ConfigUploadComponent configUploadComponent = new ConfigUploadComponent() + .withTopologyProperties(topologyProperties) + .withGlobalConfigsPath("../../metron-analytics/metron-profiler/src/test/config/zookeeper") + .withProfilerConfigsPath("../../metron-analytics/metron-profiler/src/test/config/zookeeper"); + + // load flux definition for the profiler topology + fluxComponent = new FluxTopologyComponent.Builder() + .withTopologyLocation(new File(FLUX_PATH)) + .withTopologyName("profiler") + .withTopologyProperties(topologyProperties) + .build(); + + // start all components + runner = new ComponentRunner.Builder() + .withComponent("kafka", kafkaComponent) + .withComponent("config", configUploadComponent) + .withComponent("storm", fluxComponent) + .withMillisecondsBetweenAttempts(15000) + .withNumRetries(10) + .build(); + runner.start(); + } + + @After + public void tearDown() throws Exception { + if (runner != null) { + runner.stop(); + } + } + + /** + * Tests the Profiler topology by ensuring that at least one ProfileMeasurement is persisted + * within a mock HBase table. + */ + @Test + public void testProfiler() throws Exception { + + // start the topology + fluxComponent.submitTopology(); + + // write test messages to the input topic + kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); + + + // keep trying to verify until we timeout - a bit ugly, wish JUnit had a mechanism for this + int retry = 0; + int maxRetry = 60; + while(true) { + try { + // verify - ensure that at least one profile measurement was persisted + List puts = profilerTable.getPutLog(); + assertTrue(puts.size() > 0); + + // success! + break; + + } catch (AssertionError e) { + TimeUnit.SECONDS.sleep(3); + + // if too many retries, give up the ghost + if(retry++ >= maxRetry) { + throw new Exception((retry-1) + " retry attempts failed", e); + } + } + } + } +} diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/util/DefaultStellarExecutorTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/util/DefaultStellarExecutorTest.java new file mode 100644 index 0000000000..3e8ca671f9 --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/util/DefaultStellarExecutorTest.java @@ -0,0 +1,148 @@ +/* + * + * 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.util; + +import org.adrianwalker.multilinestring.Multiline; +import org.apache.metron.profiler.stellar.DefaultStellarExecutor; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.json.simple.parser.ParseException; +import org.junit.Before; +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.junit.Assert.assertThat; + +/** + * Tests the DefaultStellarExecutor. + */ +public class DefaultStellarExecutorTest { + + /** + * { + * "ip_src_addr": "10.0.0.1", + * "ip_dst_addr": "10.0.0.20" + * } + */ + @Multiline + private String input; + + private JSONObject message; + private DefaultStellarExecutor executor; + + @Before + public void setup() throws ParseException { + + // parse the input message + JSONParser parser = new JSONParser(); + message = (JSONObject) parser.parse(input); + + // create the executor to test + executor = new DefaultStellarExecutor(); + } + + /** + * Ensure that a value can be assigned to a variable. + */ + @Test + public void testAssign() { + executor.assign("foo", "2", message); + + // verify + Object var = executor.getState().get("foo"); + assertThat(var, instanceOf(Integer.class)); + assertThat(var, equalTo(2)); + } + + /** + * Ensure that a variable can be resolved from a message field. + */ + @Test + public void testAssignWithVariableResolution() { + executor.assign("foo", "ip_src_addr", message); + + // verify + Object var = executor.getState().get("foo"); + assertThat(var, instanceOf(String.class)); + assertThat(var, equalTo("10.0.0.1")); + } + + /** + * Ensure that state is maintained correctly in the execution environment. + */ + @Test + public void testState() { + executor.assign("two", "2", message); + executor.assign("four", "4", message); + executor.assign("sum", "ADD(two, four)", message); + + // verify + Object var = executor.getState().get("sum"); + assertThat(var, equalTo(6)); + } + + /** + * Ensure that state is maintained correctly in the execution environment. + */ + @Test + public void testClearState() { + executor.assign("two", "2", message); + executor.clearState(); + + // verify + assertThat(executor.getState().containsKey("two"), equalTo(false)); + } + + /** + * Ensure that a Transformation function can be executed. + * + * There are two sets of functions in Stellar currently. One can be executed with + * a PredicateProcessor and the other a TransformationProcessor. The StellarExecutor + * abstracts away that complication. + */ + @Test + public void testExecuteTransformation() { + String actual = executor.execute("TO_UPPER('lowercase')", message, String.class); + assertThat(actual, equalTo("LOWERCASE")); + } + + /** + * Ensure that a Predicate function can be executed. + * + * There are two sets of functions in Stellar currently. One can be executed with + * a PredicateProcessor and the other a TransformationProcessor. The StellarExecutor + * abstracts away that complication. + */ + @Test + public void testExecutePredicate() { + boolean actual = executor.execute("IS_INTEGER(2)", message, Boolean.class); + assertThat(actual, equalTo(true)); + } + + /** + * An exception is expected if an expression results in an unexpected type. + */ + @Test(expected = RuntimeException.class) + public void testWrongType() { + executor.execute("ADD(2,2)", message, Boolean.class); + } +} diff --git a/metron-analytics/pom.xml b/metron-analytics/pom.xml index 8930f5fdef..3b4f5bdd48 100644 --- a/metron-analytics/pom.xml +++ b/metron-analytics/pom.xml @@ -32,7 +32,6 @@ HEAD https://git-wip-us.apache.org/repos/asf/incubator-metron - The Apache Software License, Version 2.0 @@ -43,6 +42,7 @@ metron-maas-service metron-maas-common + metron-profiler @@ -60,7 +60,6 @@ - diff --git a/metron-deployment/roles/metron_streaming/defaults/main.yml b/metron-deployment/roles/metron_streaming/defaults/main.yml index 593c0a8021..f125d70221 100644 --- a/metron-deployment/roles/metron_streaming/defaults/main.yml +++ b/metron-deployment/roles/metron_streaming/defaults/main.yml @@ -21,7 +21,9 @@ metron_elasticsearch_jar_name: metron-elasticsearch-{{ metron_version }}.jar metron_pcap_jar_name: metron-pcap-backend-{{ metron_version }}.jar metron_parsers_jar_name: metron-parsers-{{ metron_version }}.jar metron_enrichment_jar_name: metron-enrichment-{{ metron_version }}-uber.jar +metron_profiler_jar_name: metron-profiler-{{ metron_version }}.jar +# bundle names metron_common_bundle_name: metron-common-{{ metron_version }}-archive.tar.gz metron_pcap_bundle_name: metron-pcap-backend-{{ metron_version }}-archive.tar.gz metron_data_management_bundle_name: metron-data-management-{{ metron_version }}-archive.tar.gz @@ -31,6 +33,9 @@ metron_solr_bundle_name: metron-solr-{{ metron_version }}-archive.tar.gz metron_elasticsearch_bundle_name: metron-elasticsearch-{{ metron_version }}-archive.tar.gz metron_parsers_bundle_name: metron-parsers-{{ metron_version }}-archive.tar.gz metron_maas_bundle_name: metron-maas-service-{{ metron_version }}-archive.tar.gz +metron_profiler_bundle_name: metron-profiler-{{ metron_version }}-archive.tar.gz + +# bundle paths metron_pcap_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-pcap-backend/target/{{ metron_pcap_bundle_name }}" metron_common_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-common/target/{{ metron_common_bundle_name }}" metron_data_management_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-data-management/target/{{ metron_data_management_bundle_name }}" @@ -40,7 +45,9 @@ metron_solr_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-solr/t metron_elasticsearch_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-elasticsearch/target/{{ metron_elasticsearch_bundle_name }}" metron_parsers_bundle_path: "{{ playbook_dir }}/../../metron-platform/metron-parsers/target/{{ metron_parsers_bundle_name }}" metron_maas_bundle_path: "{{ playbook_dir }}/../../metron-analytics/metron-maas-service/target/{{ metron_maas_bundle_name }}" +metron_profiler_bundle_path: "{{ playbook_dir }}/../../metron-analytics/metron-profiler/target/{{ metron_profiler_bundle_name }}" +# configuration paths config_path: "{{ metron_directory }}/config" zookeeper_config_path: "{{ config_path }}/zookeeper" zookeeper_global_config_path: "{{ zookeeper_config_path }}/global.json" @@ -49,6 +56,7 @@ metron_solr_properties_config_path: "{{ metron_directory }}/config/solr.properti metron_elasticsearch_properties_config_path: "{{ metron_directory }}/config/elasticsearch.properties" metron_enrichment_properties_config_path: "{{ metron_directory }}/config/enrichment.properties" metron_parsers_properties_config_path: "{{ metron_directory }}/config/parsers.properties" + hbase_config_path: "/etc/hbase/conf" hdfs_config_path: "/etc/hadoop/conf" pcap_hdfs_path: "/apps/metron/pcap" diff --git a/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml b/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml index 12dd411bf2..300f9ccfd0 100644 --- a/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml +++ b/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml @@ -35,7 +35,6 @@ src: "{{ metron_indexing_bundle_path }}" dest: "{{ metron_directory }}" - - name: Copy Metron Parsers bundle copy: src: "{{ metron_parsers_bundle_path }}" @@ -59,6 +58,10 @@ - name: Copy Metron MaaS bundle copy: src: "{{ metron_maas_bundle_path }}" + +- name: Copy Metron Profiler bundle + copy: + src: "{{ metron_profiler_bundle_path }}" dest: "{{ metron_directory }}" - name: Unbundle Metron bundles @@ -75,4 +78,5 @@ - tar xzvf metron-common*.tar.gz - tar xzvf metron-pcap-backend*.tar.gz - tar xzvf metron-maas-service*.tar.gz + - tar xzvf metron-profiler*.tar.gz - rm *.tar.gz diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java index 6fed7d4a1a..dedf681fe5 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredEnrichmentBolt.java @@ -18,7 +18,6 @@ package org.apache.metron.common.bolt; import org.apache.log4j.Logger; -import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.ConfigurationType; import org.apache.metron.common.configuration.ConfigurationsUtils; import org.apache.metron.common.configuration.EnrichmentConfigurations; diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredProfilerBolt.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredProfilerBolt.java new file mode 100644 index 0000000000..bc1e5d6778 --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredProfilerBolt.java @@ -0,0 +1,75 @@ +/** + * 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.bolt; + +import org.apache.log4j.Logger; +import org.apache.metron.common.configuration.ConfigurationType; +import org.apache.metron.common.configuration.ConfigurationsUtils; +import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.apache.metron.common.configuration.profiler.ProfilerConfigurations; + +import java.io.IOException; + +/** + * A bolt used in the Profiler topology that is configured with values stored in Zookeeper. + */ +public abstract class ConfiguredProfilerBolt extends ConfiguredBolt { + + private static final Logger LOG = Logger.getLogger(ConfiguredProfilerBolt.class); + protected final ProfilerConfigurations configurations = new ProfilerConfigurations(); + + public ConfiguredProfilerBolt(String zookeeperUrl) { + super(zookeeperUrl); + } + + protected ProfilerConfig getProfilerConfig() { + return getConfigurations().getProfilerConfig(); + } + + @Override + protected ProfilerConfigurations defaultConfigurations() { + return new ProfilerConfigurations(); + } + + @Override + public void loadConfig() { + try { + ConfigurationsUtils.updateProfilerConfigsFromZookeeper(getConfigurations(), client); + } catch (Exception e) { + LOG.warn("Unable to load configs from zookeeper, but the cache should load lazily..."); + } + } + + @Override + public void updateConfig(String path, byte[] data) throws IOException { + if (data.length != 0) { + String name = path.substring(path.lastIndexOf("/") + 1); + + // update the profiler configuration from zookeeper + if (path.startsWith(ConfigurationType.PROFILER.getZookeeperRoot())) { + getConfigurations().updateProfilerConfig(data); + reloadCallback(name, ConfigurationType.PROFILER); + + // update the global configuration from zookeeper + } else if (ConfigurationType.GLOBAL.getZookeeperRoot().equals(path)) { + getConfigurations().updateGlobalConfig(data); + reloadCallback(name, ConfigurationType.GLOBAL); + } + } + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java index 9469bf6f82..27478d497a 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationType.java @@ -22,15 +22,15 @@ import com.google.common.base.Function; import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig; +import org.apache.metron.common.configuration.profiler.ProfilerConfig; import org.apache.metron.common.utils.JSONUtils; import java.io.IOException; import java.util.Map; public enum ConfigurationType implements Function { - GLOBAL("global" - ,"." - , s -> { + + GLOBAL("global",".", s -> { try { return JSONUtils.INSTANCE.load(s, new TypeReference>() { }); @@ -38,28 +38,36 @@ public enum ConfigurationType implements Function { throw new RuntimeException("Unable to load " + s, e); } }), - PARSER("parsers" - ,"parsers" - , s -> { + + PARSER("parsers","parsers", s -> { try { return JSONUtils.INSTANCE.load(s, SensorParserConfig.class); } catch (IOException e) { throw new RuntimeException("Unable to load " + s, e); } }), - ENRICHMENT("enrichments" - ,"enrichments" - , s -> { + + ENRICHMENT("enrichments","enrichments", s -> { try { return JSONUtils.INSTANCE.load(s, SensorEnrichmentConfig.class); } catch (IOException e) { throw new RuntimeException("Unable to load " + s, e); } + }), + + PROFILER("profiler","profiler", s -> { + try { + return JSONUtils.INSTANCE.load(s, ProfilerConfig.class); + } catch (IOException e) { + throw new RuntimeException("Unable to load " + s, e); + } }); + String name; String directory; String zookeeperRoot; Function deserializer; + ConfigurationType(String name, String directory, Function deserializer) { this.name = name; this.directory = directory; @@ -67,16 +75,18 @@ public enum ConfigurationType implements Function { this.deserializer = deserializer; } - public String getName() { return name; } + public String getName() { + return name; + } public String getDirectory() { return directory; } - public Object deserialize(String s) - { + public Object deserialize(String s) { return deserializer.apply(s); } + @Override public Object apply(String s) { return deserialize(s); diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java index 56ac9d9fbd..1411f9c2fa 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationsUtils.java @@ -24,6 +24,7 @@ import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig; +import org.apache.metron.common.configuration.profiler.ProfilerConfigurations; import org.apache.metron.common.utils.JSONUtils; import org.apache.zookeeper.KeeperException; @@ -36,6 +37,8 @@ import java.util.List; import java.util.Map; +import static org.apache.metron.common.configuration.ConfigurationType.*; + public class ConfigurationsUtils { public static CuratorFramework getClient(String zookeeperUrl) { @@ -65,8 +68,13 @@ public static void writeGlobalConfigToZookeeper(byte[] globalConfig, String zook } public static void writeGlobalConfigToZookeeper(byte[] globalConfig, CuratorFramework client) throws Exception { - ConfigurationType.GLOBAL.deserialize(new String(globalConfig)); - writeToZookeeper(ConfigurationType.GLOBAL.getZookeeperRoot(), globalConfig, client); + GLOBAL.deserialize(new String(globalConfig)); + writeToZookeeper(GLOBAL.getZookeeperRoot(), globalConfig, client); + } + + public static void writeProfilerConfigToZookeeper(byte[] config, CuratorFramework client) throws Exception { + PROFILER.deserialize(new String(config)); + writeToZookeeper(PROFILER.getZookeeperRoot(), config, client); } public static void writeSensorParserConfigToZookeeper(String sensorType, SensorParserConfig sensorParserConfig, String zookeeperUrl) throws Exception { @@ -85,7 +93,7 @@ public static void writeSensorParserConfigToZookeeper(String sensorType, byte[] } public static void writeSensorParserConfigToZookeeper(String sensorType, byte[] configData, CuratorFramework client) throws Exception { - writeToZookeeper(ConfigurationType.PARSER.getZookeeperRoot() + "/" + sensorType, configData, client); + writeToZookeeper(PARSER.getZookeeperRoot() + "/" + sensorType, configData, client); } public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, SensorEnrichmentConfig sensorEnrichmentConfig, String zookeeperUrl) throws Exception { @@ -104,7 +112,7 @@ public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, byt } public static void writeSensorEnrichmentConfigToZookeeper(String sensorType, byte[] configData, CuratorFramework client) throws Exception { - writeToZookeeper(ConfigurationType.ENRICHMENT.getZookeeperRoot() + "/" + sensorType, configData, client); + writeToZookeeper(ENRICHMENT.getZookeeperRoot() + "/" + sensorType, configData, client); } public static void writeConfigToZookeeper(String name, Map config, String zookeeperUrl) throws Exception { @@ -136,7 +144,7 @@ public static void updateConfigsFromZookeeper(Configurations configurations, Cur public static void updateParserConfigsFromZookeeper(ParserConfigurations configurations, CuratorFramework client) throws Exception { updateConfigsFromZookeeper(configurations, client); - List sensorTypes = client.getChildren().forPath(ConfigurationType.PARSER.getZookeeperRoot()); + List sensorTypes = client.getChildren().forPath(PARSER.getZookeeperRoot()); for(String sensorType: sensorTypes) { configurations.updateSensorParserConfig(sensorType, readSensorParserConfigBytesFromZookeeper(sensorType, client)); } @@ -144,30 +152,38 @@ public static void updateParserConfigsFromZookeeper(ParserConfigurations configu public static void updateEnrichmentConfigsFromZookeeper(EnrichmentConfigurations configurations, CuratorFramework client) throws Exception { updateConfigsFromZookeeper(configurations, client); - List sensorTypes = client.getChildren().forPath(ConfigurationType.ENRICHMENT.getZookeeperRoot()); + List sensorTypes = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot()); for(String sensorType: sensorTypes) { configurations.updateSensorEnrichmentConfig(sensorType, readSensorEnrichmentConfigBytesFromZookeeper(sensorType, client)); } } + public static void updateProfilerConfigsFromZookeeper(ProfilerConfigurations configurations, CuratorFramework client) throws Exception { + updateConfigsFromZookeeper(configurations, client); + } + public static SensorEnrichmentConfig readSensorEnrichmentConfigFromZookeeper(String sensorType, CuratorFramework client) throws Exception { - return JSONUtils.INSTANCE.load(new ByteArrayInputStream(readFromZookeeper(ConfigurationType.ENRICHMENT.getZookeeperRoot() + "/" + sensorType, client)), SensorEnrichmentConfig.class); + return JSONUtils.INSTANCE.load(new ByteArrayInputStream(readFromZookeeper(ENRICHMENT.getZookeeperRoot() + "/" + sensorType, client)), SensorEnrichmentConfig.class); } public static SensorParserConfig readSensorParserConfigFromZookeeper(String sensorType, CuratorFramework client) throws Exception { - return JSONUtils.INSTANCE.load(new ByteArrayInputStream(readFromZookeeper(ConfigurationType.PARSER.getZookeeperRoot() + "/" + sensorType, client)), SensorParserConfig.class); + return JSONUtils.INSTANCE.load(new ByteArrayInputStream(readFromZookeeper(PARSER.getZookeeperRoot() + "/" + sensorType, client)), SensorParserConfig.class); } public static byte[] readGlobalConfigBytesFromZookeeper(CuratorFramework client) throws Exception { - return readFromZookeeper(ConfigurationType.GLOBAL.getZookeeperRoot(), client); + return readFromZookeeper(GLOBAL.getZookeeperRoot(), client); + } + + public static byte[] readProfilerConfigBytesFromZookeeper(CuratorFramework client) throws Exception { + return readFromZookeeper(PROFILER.getZookeeperRoot(), client); } public static byte[] readSensorParserConfigBytesFromZookeeper(String sensorType, CuratorFramework client) throws Exception { - return readFromZookeeper(ConfigurationType.PARSER.getZookeeperRoot() + "/" + sensorType, client); + return readFromZookeeper(PARSER.getZookeeperRoot() + "/" + sensorType, client); } public static byte[] readSensorEnrichmentConfigBytesFromZookeeper(String sensorType, CuratorFramework client) throws Exception { - return readFromZookeeper(ConfigurationType.ENRICHMENT.getZookeeperRoot() + "/" + sensorType, client); + return readFromZookeeper(ENRICHMENT.getZookeeperRoot() + "/" + sensorType, client); } public static byte[] readConfigBytesFromZookeeper(String name, CuratorFramework client) throws Exception { @@ -178,41 +194,63 @@ public static byte[] readFromZookeeper(String path, CuratorFramework client) thr return client.getData().forPath(path); } - public static void uploadConfigsToZookeeper(String globalConfigPath, String parsersConfigPath, String enrichmentsConfigPath, String zookeeperUrl) throws Exception { + public static void uploadConfigsToZookeeper(String globalConfigPath, + String parsersConfigPath, + String enrichmentsConfigPath, + String profilerConfigPath, + String zookeeperUrl) throws Exception { try(CuratorFramework client = getClient(zookeeperUrl)) { client.start(); - uploadConfigsToZookeeper(globalConfigPath, parsersConfigPath, enrichmentsConfigPath, client); + uploadConfigsToZookeeper(globalConfigPath, parsersConfigPath, enrichmentsConfigPath, profilerConfigPath, client); } } public static void uploadConfigsToZookeeper(String rootFilePath, CuratorFramework client) throws Exception { - uploadConfigsToZookeeper(rootFilePath, rootFilePath, rootFilePath, client); + uploadConfigsToZookeeper(rootFilePath, rootFilePath, rootFilePath, rootFilePath, client); } - public static void uploadConfigsToZookeeper(String globalConfigPath, String parsersConfigPath, String enrichmentsConfigPath, CuratorFramework client) throws Exception { + public static void uploadConfigsToZookeeper(String globalConfigPath, + String parsersConfigPath, + String enrichmentsConfigPath, + String profilerConfigPath, + CuratorFramework client) throws Exception { + + // global if (globalConfigPath != null) { byte[] globalConfig = readGlobalConfigFromFile(globalConfigPath); if (globalConfig.length > 0) { ConfigurationsUtils.writeGlobalConfigToZookeeper(readGlobalConfigFromFile(globalConfigPath), client); } } + + // parsers if (parsersConfigPath != null) { Map sensorParserConfigs = readSensorParserConfigsFromFile(parsersConfigPath); for (String sensorType : sensorParserConfigs.keySet()) { ConfigurationsUtils.writeSensorParserConfigToZookeeper(sensorType, sensorParserConfigs.get(sensorType), client); } } + + // enrichments if (enrichmentsConfigPath != null) { Map sensorEnrichmentConfigs = readSensorEnrichmentConfigsFromFile(enrichmentsConfigPath); for (String sensorType : sensorEnrichmentConfigs.keySet()) { ConfigurationsUtils.writeSensorEnrichmentConfigToZookeeper(sensorType, sensorEnrichmentConfigs.get(sensorType), client); } } + + // profiler + if (profilerConfigPath != null) { + byte[] globalConfig = readProfilerConfigFromFile(profilerConfigPath); + if (globalConfig.length > 0) { + ConfigurationsUtils.writeProfilerConfigToZookeeper(readProfilerConfigFromFile(profilerConfigPath), client); + } + } } public static byte[] readGlobalConfigFromFile(String rootPath) throws IOException { byte[] globalConfig = new byte[0]; - File configPath = new File(rootPath, ConfigurationType.GLOBAL.getName() + ".json"); + File configPath = new File(rootPath, GLOBAL.getName() + ".json"); if (configPath.exists()) { globalConfig = Files.readAllBytes(configPath.toPath()); } @@ -220,11 +258,27 @@ public static byte[] readGlobalConfigFromFile(String rootPath) throws IOExceptio } public static Map readSensorParserConfigsFromFile(String rootPath) throws IOException { - return readSensorConfigsFromFile(rootPath, ConfigurationType.PARSER); + return readSensorConfigsFromFile(rootPath, PARSER); } public static Map readSensorEnrichmentConfigsFromFile(String rootPath) throws IOException { - return readSensorConfigsFromFile(rootPath, ConfigurationType.ENRICHMENT); + return readSensorConfigsFromFile(rootPath, ENRICHMENT); + } + + + /** + * Read the Profiler configuration from a file. There is only a single profiler configuration. + * @param rootPath Path to the Profiler configuration. + */ + public static byte[] readProfilerConfigFromFile(String rootPath) throws IOException { + + byte[] config = new byte[0]; + File configPath = new File(rootPath, PROFILER.getName() + ".json"); + if (configPath.exists()) { + config = Files.readAllBytes(configPath.toPath()); + } + + return config; } public static Map readSensorConfigsFromFile(String rootPath, ConfigurationType configType) throws IOException { @@ -247,19 +301,24 @@ public interface ConfigurationVisitor{ } public static void visitConfigs(CuratorFramework client, ConfigurationVisitor callback) throws Exception { - visitConfigs(client, callback, ConfigurationType.GLOBAL); - visitConfigs(client, callback, ConfigurationType.PARSER); - visitConfigs(client, callback, ConfigurationType.ENRICHMENT); + visitConfigs(client, callback, GLOBAL); + visitConfigs(client, callback, PARSER); + visitConfigs(client, callback, ENRICHMENT); + visitConfigs(client, callback, PROFILER); } public static void visitConfigs(CuratorFramework client, ConfigurationVisitor callback, ConfigurationType configType) throws Exception { + if (client.checkExists().forPath(configType.getZookeeperRoot()) != null) { - if (configType.equals(ConfigurationType.GLOBAL)) { + + if (configType.equals(GLOBAL)) { byte[] globalConfigData = client.getData().forPath(configType.getZookeeperRoot()); callback.visit(configType, "global", new String(globalConfigData)); - } else if (configType.equals(ConfigurationType.PARSER) || configType.equals(ConfigurationType.ENRICHMENT)) { + + } else if (configType.equals(PARSER) || configType.equals(ENRICHMENT) || configType.equals(PROFILER)) { List children = client.getChildren().forPath(configType.getZookeeperRoot()); for (String child : children) { + byte[] data = client.getData().forPath(configType.getZookeeperRoot() + "/" + child); callback.visit(configType, child, new String(data)); } 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 new file mode 100644 index 0000000000..f832e1d9e1 --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java @@ -0,0 +1,145 @@ +/** + * 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.configuration.profiler; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +/** + * The user defined configuration values required to generate a Profile. + */ +public class ProfileConfig implements Serializable { + + /** + * The name of the profile. + */ + 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. + */ + private String foreach; + + /** + * Stella code that when executed determines whether a message should be included in this + * profile. + */ + private String onlyif; + + /** + * Stella code that when executed results in a single measurement that is stored with the Profile. + */ + private String result; + + /** + * Defines how the state is initialized before any messages are received. + */ + private Map init = new HashMap<>(); + + /** + * Defines how the state is updated when a new message is received. + */ + private Map update = new HashMap<>(); + + public String getProfile() { + return profile; + } + + public void setProfile(String profile) { + this.profile = profile; + } + + public String getForeach() { + return foreach; + } + + public void setForeach(String foreach) { + this.foreach = foreach; + } + + public String getOnlyif() { + return onlyif; + } + + public void setOnlyif(String onlyif) { + this.onlyif = onlyif; + } + + public String getResult() { + return result; + } + + public void setResult(String result) { + this.result = result; + } + + public Map getInit() { + return init; + } + + public void setInit(Map init) { + this.init = init; + } + + public Map getUpdate() { + return update; + } + + public void setUpdate(Map update) { + this.update = update; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ProfileConfig that = (ProfileConfig) o; + if (profile != null ? !profile.equals(that.profile) : that.profile != null) return false; + if (foreach != null ? !foreach.equals(that.foreach) : that.foreach != null) return false; + if (onlyif != null ? !onlyif.equals(that.onlyif) : that.onlyif != null) return false; + if (result != null ? !result.equals(that.result) : that.result != null) return false; + if (init != null ? !init.equals(that.init) : that.init != null) return false; + return update != null ? update.equals(that.update) : that.update == null; + } + + @Override + public int hashCode() { + int result1 = profile != null ? profile.hashCode() : 0; + result1 = 31 * result1 + (foreach != null ? foreach.hashCode() : 0); + result1 = 31 * result1 + (onlyif != null ? onlyif.hashCode() : 0); + result1 = 31 * result1 + (result != null ? result.hashCode() : 0); + result1 = 31 * result1 + (init != null ? init.hashCode() : 0); + result1 = 31 * result1 + (update != null ? update.hashCode() : 0); + return result1; + } + + @Override + public String toString() { + return "ProfileConfig{" + + "profile='" + profile + '\'' + + ", foreach='" + foreach + '\'' + + ", onlyif='" + onlyif + '\'' + + ", result='" + result + '\'' + + ", init=" + init + + ", update=" + update + + '}'; + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfig.java new file mode 100644 index 0000000000..8263d39bbf --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfig.java @@ -0,0 +1,73 @@ +/** + * 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.configuration.profiler; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +/** + * The user defined configuration values required for the Profiler. + */ +public class ProfilerConfig implements Serializable { + + /** + * The input topic from which messages will be read. + */ + private String inputTopic; + + /** + * One or more profile definitions. + */ + private List profiles = new ArrayList<>(); + + public String getInputTopic() { + return inputTopic; + } + + public void setInputTopic(String inputTopic) { + this.inputTopic = inputTopic; + } + + public List getProfiles() { + return profiles; + } + + public void setProfiles(List profiles) { + this.profiles = profiles; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ProfilerConfig that = (ProfilerConfig) o; + + if (inputTopic != null ? !inputTopic.equals(that.inputTopic) : that.inputTopic != null) return false; + return profiles != null ? profiles.equals(that.profiles) : that.profiles == null; + + } + + @Override + public int hashCode() { + int result = inputTopic != null ? inputTopic.hashCode() : 0; + result = 31 * result + (profiles != null ? profiles.hashCode() : 0); + return result; + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfigurations.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfigurations.java new file mode 100644 index 0000000000..19b6a5a5b2 --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfigurations.java @@ -0,0 +1,53 @@ +/** + * 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.configuration.profiler; + +import org.apache.metron.common.configuration.ConfigurationType; +import org.apache.metron.common.configuration.Configurations; +import org.apache.metron.common.utils.JSONUtils; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; + +/** + * Created by nallen on 7/28/16. + */ +public class ProfilerConfigurations extends Configurations { + + public ProfilerConfig getProfilerConfig() { + return (ProfilerConfig) configurations.get(getKey()); + } + + public void updateProfilerConfig(byte[] data) throws IOException { + updateProfilerConfig(new ByteArrayInputStream(data)); + } + + public void updateProfilerConfig(InputStream io) throws IOException { + ProfilerConfig config = JSONUtils.INSTANCE.load(io, ProfilerConfig.class); + updateProfilerConfig(config); + } + + public void updateProfilerConfig(ProfilerConfig config) { + configurations.put(getKey(), config); + } + + private String getKey() { + return ConfigurationType.PROFILER.getName(); + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/MapVariableResolver.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/MapVariableResolver.java index b03c66f51a..a1788510b0 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/MapVariableResolver.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/MapVariableResolver.java @@ -23,7 +23,9 @@ import java.util.Map; public class MapVariableResolver implements VariableResolver { + List variableMappings = new ArrayList<>(); + public MapVariableResolver(Map variableMappingOne, Map... variableMapping) { variableMappings.add(variableMappingOne); for(Map m : variableMapping) { diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/TransformationFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/TransformationFunctions.java new file mode 100644 index 0000000000..68b75b4d79 --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/TransformationFunctions.java @@ -0,0 +1,79 @@ +/** + * 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.dsl; + +import org.apache.metron.common.dsl.functions.*; +import org.apache.metron.common.field.transformation.IPProtocolTransformation; +import org.apache.metron.common.utils.ConversionUtils; + +import java.util.List; +import java.util.function.Function; + +public enum TransformationFunctions implements Function, Object> { + + // string functions + TO_LOWER(strings -> strings.get(0)==null?null:strings.get(0).toString().toLowerCase()), + TO_UPPER(strings -> strings.get(0) == null?null:strings.get(0).toString().toUpperCase()), + TO_STRING(strings -> strings.get(0) == null?null:strings.get(0).toString()), + TO_INTEGER(strings -> strings.get(0) == null?null: ConversionUtils.convert(strings.get(0), Integer.class)), + TO_DOUBLE(strings -> strings.get(0) == null?null: ConversionUtils.convert(strings.get(0), Double.class)), + TO_LONG(strings -> strings.get(0) == null?null: ConversionUtils.convert(strings.get(0), Long.class)), + TRIM(strings -> strings.get(0) == null?null:strings.get(0).toString().trim()), + JOIN(new StringFunctions.JoinFunction()), + SPLIT(new StringFunctions.SplitFunction()), + GET_FIRST(new StringFunctions.GetFirst()), + GET_LAST(new StringFunctions.GetLast()), + GET(new StringFunctions.Get()), + + // map functions + MAP_GET(new MapFunctions.MapGet()), + + // network functions + DOMAIN_TO_TLD(new NetworkFunctions.ExtractTLD()), + DOMAIN_REMOVE_TLD(new NetworkFunctions.RemoveTLD()), + DOMAIN_REMOVE_SUBDOMAINS(new NetworkFunctions.RemoveSubdomains()), + URL_TO_HOST(new NetworkFunctions.URLToHost()), + URL_TO_PORT(new NetworkFunctions.URLToPort()), + URL_TO_PATH(new NetworkFunctions.URLToPath()), + URL_TO_PROTOCOL(new NetworkFunctions.URLToProtocol()), + PROTOCOL_TO_NAME(new IPProtocolTransformation()), + + // timestamp functions + TO_EPOCH_TIMESTAMP(new DateFunctions.ToTimestamp()), + + // math functions + ADD(new MathFunctions.AddFunction()), + SUB(new MathFunctions.SubtractFunction()), + MULT(new MathFunctions.MultiplyFunction()), + DIV(new MathFunctions.DivideFunction()), + + // control flow functions + IF_THEN_ELSE(new ControlFlowFunctions.IfThenElseFunction()), + EQUAL(new ControlFlowFunctions.EqualsFunction()); + + Function, Object> func; + TransformationFunctions(Function, Object> func) { + this.func = func; + } + + @Override + public Object apply(List input) { + return func.apply(input); + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ControlFlowFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ControlFlowFunctions.java new file mode 100644 index 0000000000..b20dfdd79a --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ControlFlowFunctions.java @@ -0,0 +1,58 @@ +/* + * + * 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.dsl.functions; + +import org.apache.metron.common.utils.ConversionUtils; + +import java.util.List; +import java.util.function.Function; + +/** + * Functions that provide functionality related to control flow. + */ +public class ControlFlowFunctions { + + public static class IfThenElseFunction implements Function, Object> { + @Override + public Object apply(List args) { + + Object ifExpression = args.get(0); + Object thenExpression = args.get(1); + Object elseExpression = args.get(2); + + Boolean ifCondition = ConversionUtils.convert(ifExpression, Boolean.class); + if(ifCondition) { + return thenExpression; + } else { + return elseExpression; + } + } + } + + public static class EqualsFunction implements Function, Object> { + @Override + public Object apply(List args) { + Object left = args.get(0); + Object right = args.get(1); + return left.equals(right); + } + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MathFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MathFunctions.java new file mode 100644 index 0000000000..8be7c2078d --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MathFunctions.java @@ -0,0 +1,180 @@ +/* + * + * 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.dsl.functions; + +import org.apache.metron.common.utils.ConversionUtils; + +import java.util.List; +import java.util.function.Function; + +/** + * Implements basic math functions. + */ +public class MathFunctions { + + public static class AddFunction implements Function, Object> { + @Override + public Object apply(List args) { + + if(args.get(0) instanceof Double) { + Double left = ConversionUtils.convert(args.get(0), Double.class); + Double right = ConversionUtils.convert(args.get(1), Double.class); + return left + right; + + } else if(args.get(0) instanceof Float) { + Float left = ConversionUtils.convert(args.get(0), Float.class); + Float right = ConversionUtils.convert(args.get(1), Float.class); + return left + right; + + } else if(args.get(0) instanceof Long) { + Long left = ConversionUtils.convert(args.get(0), Long.class); + Long right = ConversionUtils.convert(args.get(1), Long.class); + return left + right; + + } else if(args.get(0) instanceof Integer) { + Integer left = ConversionUtils.convert(args.get(0), Integer.class); + Integer right = ConversionUtils.convert(args.get(1), Integer.class); + return left + right; + + } else if(args.get(0) instanceof Short) { + Short left = ConversionUtils.convert(args.get(0), Short.class); + Short right = ConversionUtils.convert(args.get(1), Short.class); + return left + right; + + } else { + throw new RuntimeException(String.format( + "Cannot apply AddFunction to '%s'", + args.get(0).getClass().getSimpleName())); + } + } + } + + public static class SubtractFunction implements Function, Object> { + @Override + public Object apply(List args) { + + if(args.get(0) instanceof Double) { + Double left = ConversionUtils.convert(args.get(0), Double.class); + Double right = ConversionUtils.convert(args.get(1), Double.class); + return left - right; + + } else if(args.get(0) instanceof Float) { + Float left = ConversionUtils.convert(args.get(0), Float.class); + Float right = ConversionUtils.convert(args.get(1), Float.class); + return left - right; + + } else if(args.get(0) instanceof Long) { + Long left = ConversionUtils.convert(args.get(0), Long.class); + Long right = ConversionUtils.convert(args.get(1), Long.class); + return left - right; + + } else if(args.get(0) instanceof Integer) { + Integer left = ConversionUtils.convert(args.get(0), Integer.class); + Integer right = ConversionUtils.convert(args.get(1), Integer.class); + return left - right; + + } else if(args.get(0) instanceof Short) { + Short left = ConversionUtils.convert(args.get(0), Short.class); + Short right = ConversionUtils.convert(args.get(1), Short.class); + return left - right; + + } else { + throw new RuntimeException(String.format( + "Cannot apply SubtractFunction to '%s'", + args.get(0).getClass().getSimpleName())); + } + } + } + + public static class MultiplyFunction implements Function, Object> { + @Override + public Object apply(List args) { + + if(args.get(0) instanceof Double) { + Double left = ConversionUtils.convert(args.get(0), Double.class); + Double right = ConversionUtils.convert(args.get(1), Double.class); + return left * right; + + } else if(args.get(0) instanceof Float) { + Float left = ConversionUtils.convert(args.get(0), Float.class); + Float right = ConversionUtils.convert(args.get(1), Float.class); + return left * right; + + } else if(args.get(0) instanceof Long) { + Long left = ConversionUtils.convert(args.get(0), Long.class); + Long right = ConversionUtils.convert(args.get(1), Long.class); + return left * right; + + } else if(args.get(0) instanceof Integer) { + Integer left = ConversionUtils.convert(args.get(0), Integer.class); + Integer right = ConversionUtils.convert(args.get(1), Integer.class); + return left * right; + + } else if(args.get(0) instanceof Short) { + Short left = ConversionUtils.convert(args.get(0), Short.class); + Short right = ConversionUtils.convert(args.get(1), Short.class); + return left * right; + + } else { + throw new RuntimeException(String.format( + "Cannot apply MultiplyFunction to '%s'", + args.get(0).getClass().getSimpleName())); + } + } + } + + public static class DivideFunction implements Function, Object> { + @Override + public Object apply(List args) { + + if(args.get(0) instanceof Double) { + Double left = ConversionUtils.convert(args.get(0), Double.class); + Double right = ConversionUtils.convert(args.get(1), Double.class); + return left / right; + + } else if(args.get(0) instanceof Float) { + Float left = ConversionUtils.convert(args.get(0), Float.class); + Float right = ConversionUtils.convert(args.get(1), Float.class); + return left / right; + + } else if(args.get(0) instanceof Long) { + Long left = ConversionUtils.convert(args.get(0), Long.class); + Long right = ConversionUtils.convert(args.get(1), Long.class); + return left / right; + + } else if(args.get(0) instanceof Integer) { + Integer left = ConversionUtils.convert(args.get(0), Integer.class); + Integer right = ConversionUtils.convert(args.get(1), Integer.class); + return left / right; + + } else if(args.get(0) instanceof Short) { + Short left = ConversionUtils.convert(args.get(0), Short.class); + Short right = ConversionUtils.convert(args.get(1), Short.class); + return left / right; + + } else { + throw new RuntimeException(String.format( + "Cannot apply DivideFunction to '%s'", + args.get(0).getClass().getSimpleName())); + } + } + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/BaseStellarProcessor.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/BaseStellarProcessor.java index ffd4b960fe..fac81855e8 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/BaseStellarProcessor.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/BaseStellarProcessor.java @@ -56,6 +56,7 @@ public T parse(String rule, VariableResolver resolver) { public boolean validate(String rule) throws ParseException { return validate(rule, true); } + public boolean validate(String rule, boolean throwException) throws ParseException { try { parse(rule, x -> null); diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/ControlFlowFunctionsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/ControlFlowFunctionsTest.java new file mode 100644 index 0000000000..db94b3b509 --- /dev/null +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/ControlFlowFunctionsTest.java @@ -0,0 +1,76 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.common.transformation; + +import com.google.common.collect.ImmutableMap; +import org.apache.metron.common.dsl.MapVariableResolver; +import org.apache.metron.common.dsl.VariableResolver; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +/** + * Tests the ControlFlowFunctions. + */ +public class ControlFlowFunctionsTest { + + private static Object run(String expression, Map variables) { + VariableResolver resolver = new MapVariableResolver(variables); + TransformationProcessor processor = new TransformationProcessor(); + return processor.parse(expression, resolver); + } + + private static Object run(String expression) { + return run(expression, new HashMap<>()); + } + + @Test + public void testEqualStrings() { + Assert.assertEquals(true, run("EQUAL('foo', 'foo')")); + Assert.assertEquals(false, run("EQUAL('foo', 'bar')")); + } + + @Test + public void testEqualIntegers() { + Assert.assertEquals(true, run("EQUAL(2, 2)")); + Assert.assertEquals(false, run("EQUAL(2, 1)")); + } + + @Test + public void testEqualVariables() { + Assert.assertEquals(true, run("EQUAL(two, two)", ImmutableMap.of("two", 2, "four", 4))); + Assert.assertEquals(false, run("EQUAL(two, four)", ImmutableMap.of("two", 2, "four", 4))); + } + + @Test + public void testIfThenElse() { + Assert.assertEquals(1, run("IF_THEN_ELSE( EQUAL(2,2), 1, 0)")); + Assert.assertEquals(0, run("IF_THEN_ELSE( EQUAL(2,1), 1, 0)")); + } + + @Test + public void testIfThenElseStrings() { + Assert.assertEquals("equal", run("IF_THEN_ELSE( EQUAL(2,2), 'equal', 'not')")); + Assert.assertEquals("not", run("IF_THEN_ELSE( EQUAL(2,1), 'equal', 'not')")); + } +} diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/MathFunctionsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/MathFunctionsTest.java new file mode 100644 index 0000000000..6f725df985 --- /dev/null +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/MathFunctionsTest.java @@ -0,0 +1,85 @@ +/* + * + * 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.transformation; + +import com.google.common.collect.ImmutableMap; +import org.apache.metron.common.dsl.MapVariableResolver; +import org.apache.metron.common.dsl.VariableResolver; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +/** + * Tests the MathFunctions. + */ +public class MathFunctionsTest { + + private static Object run(String expression, Map variables) { + VariableResolver resolver = new MapVariableResolver(variables); + TransformationProcessor processor = new TransformationProcessor(); + return processor.parse(expression, resolver); + } + + @Test + public void testAdd() { + Assert.assertEquals(10.0d, run("ADD(foo, foo)", ImmutableMap.of("foo", 5.0d))); + Assert.assertEquals(10.0f, run("ADD(foo, foo)", ImmutableMap.of("foo", 5.0f))); + Assert.assertEquals(10L, run("ADD(foo, foo)", ImmutableMap.of("foo", 5L))); + Assert.assertEquals(10, run("ADD(foo, foo)", ImmutableMap.of("foo", 5))); + + // in java, short + short = integer + Assert.assertEquals(10, run("ADD(foo, foo)", ImmutableMap.of("foo", (short) 5))); + } + + @Test + public void testSubtract() { + Assert.assertEquals(0d, run("SUB(foo, foo)", ImmutableMap.of("foo", 5.0d))); + Assert.assertEquals(0f, run("SUB(foo, foo)", ImmutableMap.of("foo", 5.0f))); + Assert.assertEquals(0L, run("SUB(foo, foo)", ImmutableMap.of("foo", 5L))); + Assert.assertEquals(0, run("SUB(foo, foo)", ImmutableMap.of("foo", 5))); + + // per JVM specification, short - short = integer + Assert.assertEquals(0, run("SUB(foo, foo)", ImmutableMap.of("foo", (short) 5))); + } + + @Test + public void testDivide() { + Assert.assertEquals(1d, run("DIV(foo, foo)", ImmutableMap.of("foo", 5.0d))); + Assert.assertEquals(1f, run("DIV(foo, foo)", ImmutableMap.of("foo", 5.0f))); + Assert.assertEquals(1L, run("DIV(foo, foo)", ImmutableMap.of("foo", 5L))); + Assert.assertEquals(1, run("DIV(foo, foo)", ImmutableMap.of("foo", 5))); + + // per JVM specification, short / short = integer + Assert.assertEquals(1, run("DIV(foo, foo)", ImmutableMap.of("foo", (short) 5))); + } + + @Test + public void testMultiply() { + Assert.assertEquals(25.0d, run("MULT(foo, foo)", ImmutableMap.of("foo", 5.0d))); + Assert.assertEquals(25.0f, run("MULT(foo, foo)", ImmutableMap.of("foo", 5.0f))); + Assert.assertEquals(25L, run("MULT(foo, foo)", ImmutableMap.of("foo", 5L))); + Assert.assertEquals(25, run("MULT(foo, foo)", ImmutableMap.of("foo", 5))); + + // in java, short * short = integer + Assert.assertEquals(25, run("MULT(foo, foo)", ImmutableMap.of("foo", (short) 5))); + } +} diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/TransformationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/TransformationTest.java new file mode 100644 index 0000000000..a5793a8f11 --- /dev/null +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/TransformationTest.java @@ -0,0 +1,161 @@ +/** + * 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.transformation; + +import com.google.common.collect.ImmutableMap; +import org.apache.metron.common.utils.timestamp.TimestampConverters; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class TransformationTest { + + @Test + public void testHappyPath() { + String query = "TO_UPPER(TRIM(foo))"; + Assert.assertEquals("CASEY", run(query, ImmutableMap.of("foo", "casey "))); + } + + @Test + public void testJoin() { + String query = "JOIN( [ TO_UPPER(TRIM(foo)), 'bar' ], ',')"; + Assert.assertEquals("CASEY,bar", run(query, ImmutableMap.of("foo", "casey "))); + } + + @Test + public void testSplit() { + String query = "JOIN( SPLIT(foo, ':'), ',')"; + Assert.assertEquals("casey,bar", run(query, ImmutableMap.of("foo", "casey:bar"))); + } + + @Test + public void testMapGet() { + String query = "MAP_GET(dc, dc2tz, 'UTC')"; + Assert.assertEquals("UTC" + , run(query, ImmutableMap.of("dc", "nyc" + ,"dc2tz", ImmutableMap.of("la", "PST") + ) + ) + ); + Assert.assertEquals("EST" + , run(query, ImmutableMap.of("dc", "nyc" + ,"dc2tz", ImmutableMap.of("nyc", "EST") + ) + ) + ); + } + @Test + public void testTLDExtraction() { + String query = "DOMAIN_TO_TLD(foo)"; + Assert.assertEquals("co.uk", run(query, ImmutableMap.of("foo", "www.google.co.uk"))); + } + + @Test + public void testTLDRemoval() { + String query = "DOMAIN_REMOVE_TLD(foo)"; + Assert.assertEquals("www.google", run(query, ImmutableMap.of("foo", "www.google.co.uk"))); + } + + @Test + public void testSubdomainRemoval() { + String query = "DOMAIN_REMOVE_SUBDOMAINS(foo)"; + Assert.assertEquals("google.co.uk", run(query, ImmutableMap.of("foo", "www.google.co.uk"))); + Assert.assertEquals("google.com", run(query, ImmutableMap.of("foo", "www.google.com"))); + } + @Test + public void testURLToHost() { + String query = "URL_TO_HOST(foo)"; + Assert.assertEquals("www.google.co.uk", run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path"))); + } + + @Test + public void testURLToPort() { + String query = "URL_TO_PORT(foo)"; + Assert.assertEquals(80, run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path"))); + } + + @Test + public void testURLToProtocol() { + String query = "URL_TO_PROTOCOL(foo)"; + Assert.assertEquals("http", run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path"))); + } + + @Test + public void testURLToPath() { + String query = "URL_TO_PATH(foo)"; + Assert.assertEquals("/my/path", run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path"))); + } + + @Test + public void testProtocolToName() { + String query = "PROTOCOL_TO_NAME(protocol)"; + Assert.assertEquals("TCP", run(query, ImmutableMap.of("protocol", "6"))); + Assert.assertEquals("TCP", run(query, ImmutableMap.of("protocol", 6))); + Assert.assertEquals(null, run(query, ImmutableMap.of("foo", 6))); + Assert.assertEquals("chicken", run(query, ImmutableMap.of("protocol", "chicken"))); + } + @Test + public void testDateConversion() { + long expected =1452013350000L; + { + String query = "TO_EPOCH_TIMESTAMP(foo, 'yyyy-MM-dd HH:mm:ss', 'UTC')"; + Assert.assertEquals(expected, run(query, ImmutableMap.of("foo", "2016-01-05 17:02:30"))); + } + { + String query = "TO_EPOCH_TIMESTAMP(foo, 'yyyy-MM-dd HH:mm:ss')"; + Long ts = (Long) run(query, ImmutableMap.of("foo", "2016-01-05 17:02:30")); + //is it within 24 hours of the UTC? + Assert.assertTrue(Math.abs(ts - expected) < 8.64e+7); + } + } + + @Test + public void testToString() { + Assert.assertEquals("5", run("TO_STRING(foo)", ImmutableMap.of("foo", 5))); + } + + @Test + public void testToInteger() { + Assert.assertEquals(5, run("TO_INTEGER(foo)", ImmutableMap.of("foo", "5"))); + Assert.assertEquals(5, run("TO_INTEGER(foo)", ImmutableMap.of("foo", 5))); + } + + @Test + public void testToDouble() { + Assert.assertEquals(new Double(5.1), run("TO_DOUBLE(foo)", ImmutableMap.of("foo", 5.1d))); + Assert.assertEquals(new Double(5.1), run("TO_DOUBLE(foo)", ImmutableMap.of("foo", "5.1"))); + } + + @Test + public void testGet() { + Map variables = ImmutableMap.of("foo", "www.google.co.uk"); + Assert.assertEquals("www", run("GET_FIRST(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'))", variables)); + Assert.assertEquals("www", run("GET(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'), 0)", variables)); + Assert.assertEquals("google", run("GET_LAST(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'))", variables)); + Assert.assertEquals("google", run("GET(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'), 1)", variables)); + } + + private static Object run(String rule, Map variables) { + TransformationProcessor processor = new TransformationProcessor(); + Assert.assertTrue(rule + " not valid.", processor.validate(rule)); + return processor.parse(rule, x -> variables.get(x)); + } +} diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/HBaseBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/HBaseBolt.java deleted file mode 100644 index 1eff028416..0000000000 --- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/HBaseBolt.java +++ /dev/null @@ -1,184 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.metron.enrichment.bolt; - - - -import java.io.IOException; -import java.util.Map; - -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Splitter; -import com.google.common.collect.Iterables; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.hbase.client.Put; -import org.apache.log4j.Logger; -import org.apache.metron.hbase.Connector; -import org.apache.metron.hbase.HTableConnector; -import org.apache.metron.hbase.TupleTableConfig; -import org.json.simple.JSONObject; - -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; - -import org.apache.metron.common.utils.ErrorUtils; - -/** - * A Storm bolt for putting data into HBase. - *

- * By default works in batch mode by enabling HBase's client-side write buffer. Enabling batch mode - * is recommended for high throughput, but it can be disabled in {@link TupleTableConfig}. - *

- * The HBase configuration is picked up from the first hbase-site.xml encountered in the - * classpath - * @see TupleTableConfig - * @see HTableConnector - */ -@SuppressWarnings("serial") -public class HBaseBolt implements IRichBolt { - private static final Logger LOG = Logger.getLogger(HBaseBolt.class); - private static final String DEFAULT_ZK_PORT = "2181"; - - protected OutputCollector collector; - protected TupleTableConfig conf; - protected boolean autoAck = true; - protected Connector connector; - private String _quorum; - private String _port; - - public HBaseBolt(TupleTableConfig conf, String quorum, String port) { - this.conf = conf; - _quorum = quorum; - _port = port; - } - public HBaseBolt(final TupleTableConfig conf, String zkConnectString) throws IOException { - this(conf, zkConnectStringToHosts(zkConnectString), zkConnectStringToPort(zkConnectString)); - } - public static String zkConnectStringToHosts(String connString) { - Iterable hostPortPairs = Splitter.on(',').split(connString); - return Joiner.on(',').join(Iterables.transform(hostPortPairs, new Function() { - - @Override - public String apply(String hostPortPair) { - return Iterables.getFirst(Splitter.on(':').split(hostPortPair), ""); - } - })); - } - public static String zkConnectStringToPort(String connString) { - String hostPortPair = Iterables.getFirst(Splitter.on(",").split(connString), ""); - return Iterables.getLast(Splitter.on(":").split(hostPortPair),DEFAULT_ZK_PORT); - } - - - public Connector createConnector() throws IOException{ - initialize(); - return new HTableConnector(conf, _quorum, _port); - } - - public void initialize() { - TupleTableConfig hbaseBoltConfig = conf; - String allColumnFamiliesColumnQualifiers = conf.getFields(); - String[] tokenizedColumnFamiliesWithColumnQualifiers = StringUtils - .split(allColumnFamiliesColumnQualifiers, "\\|"); - for (String tokenizedColumnFamilyWithColumnQualifiers : tokenizedColumnFamiliesWithColumnQualifiers) { - String[] cfCqTokens = StringUtils.split( tokenizedColumnFamilyWithColumnQualifiers, ":"); - String columnFamily = cfCqTokens[0]; - String[] columnQualifiers = StringUtils.split(cfCqTokens[1], ","); - for (String columnQualifier : columnQualifiers) { - hbaseBoltConfig.addColumn(columnFamily, columnQualifier); - } - setAutoAck(true); - } - } - - /** {@inheritDoc} */ - @SuppressWarnings("rawtypes") - - public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { - this.collector = collector; - - try { - if(connector == null) { - this.connector = createConnector(); - } - - } catch (IOException e) { - throw new RuntimeException(e); - } - - LOG.info("Preparing HBaseBolt for table: " + this.conf.getTableName()); - } - - /** {@inheritDoc} */ - - public void execute(Tuple input) { - try { - Put p = conf.getPutFromTuple(input); - this.connector.put(p); - } catch (IOException ex) { - - JSONObject error = ErrorUtils.generateErrorMessage( - "Alerts problem: " + input.toString(), ex); - collector.emit("error", new Values(error)); - - throw new RuntimeException(ex); - } - - if (this.autoAck) { - this.collector.ack(input); - } - } - - /** {@inheritDoc} */ - - public void cleanup() { - this.connector.close(); - } - - /** {@inheritDoc} */ - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declareStream("error", new Fields("HBase")); - } - - /** {@inheritDoc} */ - - public Map getComponentConfiguration() { - return null; - } - - /** - * @return the autoAck - */ - public boolean isAutoAck() { - return autoAck; - } - - /** - * @param autoAck the autoAck to set - */ - public void setAutoAck(boolean autoAck) { - this.autoAck = autoAck; - } -} diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java index 5cc14ac2e8..97effac911 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java @@ -18,7 +18,6 @@ package org.apache.metron.enrichment.integration.components; import org.apache.curator.framework.CuratorFramework; -import org.apache.metron.common.configuration.ConfigurationsUtils; import org.apache.metron.common.configuration.SensorParserConfig; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; @@ -29,12 +28,15 @@ import java.util.Optional; import java.util.Properties; +import static org.apache.metron.common.configuration.ConfigurationsUtils.*; + public class ConfigUploadComponent implements InMemoryComponent { private Properties topologyProperties; private String globalConfigPath; private String parserConfigsPath; private String enrichmentConfigsPath; + private String profilerConfigPath; private Optional globalConfig = Optional.empty(); private Map parserSensorConfigs = new HashMap<>(); public ConfigUploadComponent withTopologyProperties(Properties topologyProperties) { @@ -56,6 +58,11 @@ public ConfigUploadComponent withEnrichmentConfigsPath(String enrichmentConfigsP return this; } + public ConfigUploadComponent withProfilerConfigsPath(String profilerConfigsPath) { + this.profilerConfigPath = profilerConfigsPath; + return this; + } + public ConfigUploadComponent withParserSensorConfig(String name, SensorParserConfig config) { parserSensorConfigs.put(name, config); return this; @@ -69,23 +76,18 @@ public ConfigUploadComponent withGlobalConfig(String globalConfig) { @Override public void start() throws UnableToStartException { try { + final String zookeeperUrl = topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY); + if(globalConfigPath != null) { - ConfigurationsUtils.uploadConfigsToZookeeper(globalConfigPath - , parserConfigsPath - , enrichmentConfigsPath - , topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY) - ); + uploadConfigsToZookeeper(globalConfigPath, parserConfigsPath, enrichmentConfigsPath, profilerConfigPath, zookeeperUrl); } + for(Map.Entry kv : parserSensorConfigs.entrySet()) { - ConfigurationsUtils.writeSensorParserConfigToZookeeper( kv.getKey() - , kv.getValue() - , topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY) - ); + writeSensorParserConfigToZookeeper(kv.getKey(), kv.getValue(), zookeeperUrl); } + if(globalConfig.isPresent()) { - ConfigurationsUtils.writeGlobalConfigToZookeeper(globalConfig.get().getBytes() - , topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY) - ); + writeGlobalConfigToZookeeper(globalConfig.get().getBytes(), zookeeperUrl); } } catch (Exception e) { @@ -95,10 +97,10 @@ public void start() throws UnableToStartException { public SensorParserConfig getSensorParserConfig(String sensorType) { SensorParserConfig sensorParserConfig = new SensorParserConfig(); - CuratorFramework client = ConfigurationsUtils.getClient(topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY)); + CuratorFramework client = getClient(topologyProperties.getProperty(KafkaWithZKComponent.ZOOKEEPER_PROPERTY)); client.start(); try { - sensorParserConfig = ConfigurationsUtils.readSensorParserConfigFromZookeeper(sensorType, client); + sensorParserConfig = readSensorParserConfigFromZookeeper(sensorType, client); } catch (Exception e) { e.printStackTrace(); } finally { diff --git a/metron-platform/metron-hbase/pom.xml b/metron-platform/metron-hbase/pom.xml index 7949f838e2..31f2960278 100644 --- a/metron-platform/metron-hbase/pom.xml +++ b/metron-platform/metron-hbase/pom.xml @@ -34,11 +34,50 @@ org.apache.metron metron-common ${project.parent.version} + + + com.google.guava + guava + + + + + org.apache.metron + metron-test-utilities + ${project.parent.version} + test + + + org.apache.hadoop + hadoop-auth + ${global_hadoop_version} + provided + + + org.apache.hadoop + hadoop-client + ${global_hadoop_version} + provided + + + org.apache.hadoop + hadoop-common + tests + ${global_hadoop_version} + provided + + + org.apache.hadoop + hadoop-hdfs + tests + ${global_hadoop_version} + provided org.apache.hbase hbase-client ${global_hbase_version} + provided org.slf4j @@ -48,8 +87,11 @@ log4j log4j + + com.google.guava + guava + - provided org.apache.storm @@ -65,8 +107,92 @@ 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.storm + storm-hdfs + ${global_storm_version} + provided + + + org.apache.storm + storm-core + + + org.apache.hadoop + hadoop-client + + + + + org.apache.storm + storm-hbase + ${global_storm_version} + provided + + + org.apache.storm + storm-core + + + org.apache.hadoop + hadoop-client + + + + + 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 + ${global_mockito_version} + test + log4j log4j diff --git a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/BatchHelper.java b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/BatchHelper.java new file mode 100644 index 0000000000..e7cf2cfd50 --- /dev/null +++ b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/BatchHelper.java @@ -0,0 +1,127 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.hbase.bolt; + +import backtype.storm.Constants; +import backtype.storm.task.OutputCollector; +import backtype.storm.tuple.Tuple; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; +import java.util.List; + +/** + * Provides functionality for handling batch writes to HBase. + */ +public class BatchHelper { + + private static final Logger LOG = LoggerFactory.getLogger(BatchHelper.class); + + /** + * The batch size. Defaults to 15,000. + */ + private int batchSize = 15000; + + /** + * A batch of tuples. + */ + private List tupleBatch; + + private boolean forceFlush = false; + private OutputCollector collector; + + /** + * @param batchSize The batch size. + * @param collector The output collector. + */ + public BatchHelper(int batchSize, OutputCollector collector) { + if (batchSize > 0) { + this.batchSize = batchSize; + } + this.collector = collector; + this.tupleBatch = new LinkedList<>(); + } + + /** + * Fail the batch. + * @param e The exception which caused the failure. + */ + public void fail(Exception e) { + collector.reportError(e); + for (Tuple t : tupleBatch) { + collector.fail(t); + } + tupleBatch.clear(); + forceFlush = false; + } + + /** + * Ack the batch. + */ + public void ack() { + for (Tuple t : tupleBatch) { + collector.ack(t); + } + tupleBatch.clear(); + forceFlush = false; + } + + public boolean shouldHandle(Tuple tuple) { + if (isTick(tuple)) { + LOG.debug("TICK received! current batch status [{}/{}]", tupleBatch.size(), batchSize); + forceFlush = true; + return false; + } else { + return true; + } + } + + /** + * Adds a tuple to the batch. + * @param tuple + */ + public void addBatch(Tuple tuple) { + tupleBatch.add(tuple); + if (tupleBatch.size() >= batchSize) { + forceFlush = true; + } + } + + public List getBatchTuples() { + return this.tupleBatch; + } + + public int getBatchSize() { + return this.batchSize; + } + + public boolean shouldFlush() { + return forceFlush && !tupleBatch.isEmpty(); + } + + public boolean isTick(Tuple tuple) { + return tuple != null && + Constants.SYSTEM_COMPONENT_ID.equals(tuple.getSourceComponent()) && + Constants.SYSTEM_TICK_STREAM_ID.equals(tuple.getSourceStreamId()); + } + +} diff --git a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java new file mode 100644 index 0000000000..4c70ae42e6 --- /dev/null +++ b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java @@ -0,0 +1,202 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.hbase.bolt; + +import backtype.storm.Config; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.tuple.Tuple; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.metron.hbase.HTableProvider; +import org.apache.metron.hbase.TableProvider; +import org.apache.metron.hbase.client.HBaseClient; +import org.apache.storm.hbase.bolt.mapper.HBaseMapper; +import org.apache.storm.hbase.common.ColumnList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.InvocationTargetException; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * A bolt that writes to HBase. + * + * Each bolt defined within a topology can interact with only a single HBase table. + */ +public class HBaseBolt extends BaseRichBolt { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseBolt.class); + + /** + * Should the write-ahead-log be used. + */ + private boolean writeToWAL = true; + + /** + * The interval in seconds at which the writes are flushed to Hbase. + */ + private int flushIntervalSecs = 1; + + /** + * The batch size. + */ + private int batchSize = 1000; + + /** + * The name of the HBase table. Each bolt communicates with a single HBase table. + */ + protected String tableName; + + /** + * The mapper which defines how tuple fields are mapped to HBase. + */ + protected HBaseMapper mapper; + + /** + * The name of the class that should be used as a table provider. + */ + protected String tableProvider = "org.apache.metron.hbase.HTableProvider"; + + private List batchMutations; + private BatchHelper batchHelper; + protected OutputCollector collector; + protected transient HBaseClient hbaseClient; + + public HBaseBolt(String tableName, HBaseMapper mapper) { + this.tableName = tableName; + this.mapper = mapper; + this.batchMutations = new LinkedList<>(); + } + + public HBaseBolt writeToWAL(boolean writeToWAL) { + this.writeToWAL = writeToWAL; + return this; + } + + public HBaseBolt withTableProvider(String tableProvider) { + this.tableProvider = tableProvider; + return this; + } + + public HBaseBolt withBatchSize(int batchSize) { + this.batchSize = batchSize; + return this; + } + + public HBaseBolt withFlushIntervalSecs(int flushIntervalSecs) { + this.flushIntervalSecs = flushIntervalSecs; + return this; + } + + public void setClient(HBaseClient hbaseClient) { + this.hbaseClient = hbaseClient; + } + + @Override + public Map getComponentConfiguration() { + Config conf = new Config(); + conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, flushIntervalSecs); + return conf; + } + + @Override + public void prepare(Map map, TopologyContext topologyContext, OutputCollector collector) { + this.collector = collector; + this.batchHelper = new BatchHelper(batchSize, collector); + + TableProvider provider = getTableProvider(tableProvider); + hbaseClient = new HBaseClient(provider, HBaseConfiguration.create(), tableName); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) { + // nothing emitted + } + + @Override + public void execute(Tuple tuple) { + try { + if (batchHelper.shouldHandle(tuple)) { + save(tuple); + } + + if (batchHelper.shouldFlush()) { + flush(); + } + + } catch (Exception e) { + batchHelper.fail(e); + batchMutations.clear(); + } + } + + /** + * Saves an operation for later. + * @param tuple Contains the data elements that need written to HBase. + */ + private void save(Tuple tuple) { + byte[] rowKey = this.mapper.rowKey(tuple); + ColumnList cols = this.mapper.columns(tuple); + Durability durability = writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL; + List mutations = hbaseClient.constructMutationReq(rowKey, cols, durability); + batchMutations.addAll(mutations); + batchHelper.addBatch(tuple); + } + + /** + * Flush all saved operations. + */ + private void flush() { + this.hbaseClient.batchMutate(batchMutations); + batchHelper.ack(); + batchMutations.clear(); + } + + /** + * + * @param connectorImpl + * @return + */ + private static TableProvider getTableProvider(String connectorImpl) { + + // if class name not defined, use a reasonable default + if(StringUtils.isEmpty(connectorImpl) || connectorImpl.charAt(0) == '$') { + return new HTableProvider(); + } + + // instantiate the table provider + try { + Class clazz = (Class) Class.forName(connectorImpl); + return clazz.getConstructor().newInstance(); + + } catch (InstantiationException | IllegalAccessException | IllegalStateException | + InvocationTargetException | NoSuchMethodException | ClassNotFoundException e) { + throw new IllegalStateException("Unable to instantiate connector", e); + } + } +} diff --git a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java new file mode 100644 index 0000000000..888e485464 --- /dev/null +++ b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java @@ -0,0 +1,165 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.hbase.client; + +import com.google.common.collect.Lists; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.security.UserProvider; +import org.apache.metron.hbase.TableProvider; +import org.apache.storm.hbase.bolt.mapper.HBaseProjectionCriteria; +import org.apache.storm.hbase.common.ColumnList; +import org.apache.storm.hbase.security.HBaseSecurityUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import java.util.List; +import java.util.Map; + +/** + * A client that interacts with HBase. + */ +public class HBaseClient implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(HBaseClient.class); + private HTableInterface table; + + public HBaseClient(TableProvider provider, final Configuration configuration, final String tableName) { + try { + this.table = provider.getTable(configuration, tableName); + + } catch(Exception e) { + throw new RuntimeException(e); + } + } + + public List constructMutationReq(byte[] rowKey, ColumnList cols, Durability durability) { + List mutations = Lists.newArrayList(); + + if (cols.hasColumns()) { + Put put = createPut(rowKey, cols, durability); + mutations.add(put); + } + + if (cols.hasCounters()) { + Increment inc = createIncrement(rowKey, cols, durability); + mutations.add(inc); + } + + if (mutations.isEmpty()) { + mutations.add(new Put(rowKey)); + } + + return mutations; + } + + public void batchMutate(List mutations) { + Object[] result = new Object[mutations.size()]; + try { + table.batch(mutations, result); + + } catch (InterruptedException | IOException e) { + LOG.warn("Error performing a mutation to HBase.", e); + throw new RuntimeException(e); + } + } + + public Get constructGetRequests(byte[] rowKey, HBaseProjectionCriteria projectionCriteria) { + Get get = new Get(rowKey); + + if (projectionCriteria != null) { + for (byte[] columnFamily : projectionCriteria.getColumnFamilies()) { + get.addFamily(columnFamily); + } + + for (HBaseProjectionCriteria.ColumnMetaData columnMetaData : projectionCriteria.getColumns()) { + get.addColumn(columnMetaData.getColumnFamily(), columnMetaData.getQualifier()); + } + } + + return get; + } + + public Result[] batchGet(List gets) { + try { + return table.get(gets); + + } catch (Exception e) { + LOG.warn("Could not perform HBase lookup.", e); + throw new RuntimeException(e); + } + } + + @Override + public void close() throws IOException { + table.close(); + } + + /** + * Creates an HBase Put. + * @param rowKey The row key. + * @param cols The columns to put. + * @param durability The durability of the put. + */ + private Put createPut(byte[] rowKey, ColumnList cols, Durability durability) { + Put put = new Put(rowKey); + put.setDurability(durability); + + for (ColumnList.Column col : cols.getColumns()) { + if (col.getTs() > 0) { + put.add(col.getFamily(), + col.getQualifier(), + col.getTs(), + col.getValue()); + + } else { + put.add(col.getFamily(), + col.getQualifier(), + col.getValue()); + } + } + + return put; + } + + /** + * Creates an HBase Increment for a counter. + * @param rowKey The row key. + * @param cols The columns to include. + * @param durability The durability of the increment. + */ + private Increment createIncrement(byte[] rowKey, ColumnList cols, Durability durability) { + Increment inc = new Increment(rowKey); + inc.setDurability(durability); + + for (ColumnList.Counter cnt : cols.getCounters()) { + inc.addColumn( + cnt.getFamily(), + cnt.getQualifier(), + cnt.getIncrement()); + } + + return inc; + } +} diff --git a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/Widget.java b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/Widget.java new file mode 100644 index 0000000000..d47e332c61 --- /dev/null +++ b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/Widget.java @@ -0,0 +1,77 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.hbase; + +/** + * A simple POJO used for testing. + */ +public class Widget { + + /** + * The name of the widget. + */ + private String name; + + /** + * The cost of the widget. + */ + private int cost; + + public Widget(String name, int cost) { + this.name = name; + this.cost = cost; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public int getCost() { + return cost; + } + + public void setCost(int cost) { + this.cost = cost; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Widget widget = (Widget) o; + + if (cost != widget.cost) return false; + return name != null ? name.equals(widget.name) : widget.name == null; + + } + + @Override + public int hashCode() { + int result = name != null ? name.hashCode() : 0; + result = 31 * result + cost; + return result; + } +} diff --git a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/WidgetMapper.java b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/WidgetMapper.java new file mode 100644 index 0000000000..c7d3fae7fd --- /dev/null +++ b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/WidgetMapper.java @@ -0,0 +1,55 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.hbase; + +import backtype.storm.tuple.Tuple; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.storm.hbase.bolt.mapper.HBaseMapper; +import org.apache.storm.hbase.common.ColumnList; + +import java.util.Calendar; + +/** + * Maps a Widget to HBase. Used only for testing. + */ +public class WidgetMapper implements HBaseMapper { + + @Override + public byte[] rowKey(Tuple tuple) { + Widget w = (Widget) tuple.getValueByField("widget"); + return Bytes.toBytes(w.getName()); + } + + @Override + public ColumnList columns(Tuple tuple) { + Widget w = (Widget) tuple.getValueByField("widget"); + + ColumnList cols = new ColumnList(); + cols.addColumn(CF, QNAME, Bytes.toBytes(w.getName())); + cols.addColumn(CF, QCOST, Bytes.toBytes(w.getCost())); + return cols; + } + + public static final String CF_STRING = "cfWidget"; + public static final byte[] CF = Bytes.toBytes(CF_STRING); + public static final byte[] QNAME = Bytes.toBytes("qName"); + public static final byte[] QCOST = Bytes.toBytes("qCost"); +} diff --git a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/bolt/HBaseBoltTest.java b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/bolt/HBaseBoltTest.java new file mode 100644 index 0000000000..d790d5153b --- /dev/null +++ b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/bolt/HBaseBoltTest.java @@ -0,0 +1,156 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.hbase.bolt; + +import backtype.storm.Constants; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.metron.hbase.Widget; +import org.apache.metron.hbase.WidgetMapper; +import org.apache.metron.hbase.client.HBaseClient; +import org.apache.metron.test.bolt.BaseBoltTest; +import org.apache.storm.hbase.bolt.mapper.HBaseMapper; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.Mock; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.*; + +/** + * Tests the HBaseBolt. + */ +public class HBaseBoltTest extends BaseBoltTest { + + private static final String tableName = "widgets"; + + @Mock + private HBaseClient client; + + @Mock + private Tuple tuple1; + + @Mock + private Tuple tuple2; + + private Widget widget1; + + private Widget widget2; + + private HBaseMapper mapper; + + + @Before + public void setupTuples() throws Exception { + + // setup the first tuple + widget1 = new Widget("widget1", 100); + when(tuple1.getValueByField(eq("widget"))).thenReturn(widget1); + + // setup the second tuple + widget2 = new Widget("widget2", 200); + when(tuple2.getValueByField(eq("widget"))).thenReturn(widget2); + } + + @Before + public void setup() throws Exception { + + // create a mapper + mapper = new WidgetMapper(); + + } + + /** + * Create a ProfileBuilderBolt to test + */ + private HBaseBolt createBolt() throws IOException { + HBaseBolt bolt = new HBaseBolt(tableName, mapper) + .withBatchSize(2); + + bolt.prepare(Collections.emptyMap(), topologyContext, outputCollector); + bolt.setClient(client); + return bolt; + } + + /** + * What happens if the batch is full? + * + * If the batch size is 2 and we have received 2 tuples the batch should be flushed. + */ + @Test + public void testBatchReady() throws Exception { + HBaseBolt bolt = createBolt(); + bolt.execute(tuple1); + bolt.execute(tuple2); + verify(client, times(1)).batchMutate(any(List.class)); + } + + /** + * What happens if the batch is not full? + * + * If the batch size is 2 and we have only received 2 tuple, the batch should not be flushed. + */ + @Test + public void testBatchNotReady() throws Exception { + HBaseBolt bolt = createBolt(); + bolt.execute(tuple1); + verify(client, times(0)).batchMutate(any(List.class)); + } + + /** + * What happens if the batch timeout is reached? + */ + @Test + public void testTimeFlush() throws Exception { + HBaseBolt bolt = createBolt(); + + // the batch is not ready to write + bolt.execute(tuple1); + verify(client, times(0)).batchMutate(any(List.class)); + + // the batch should be written after the tick tuple + bolt.execute(mockTickTuple()); + verify(client, times(1)).batchMutate(any(List.class)); + } + + private static Tuple mockTuple(String componentId, String streamId) { + Tuple tuple = mock(Tuple.class); + when(tuple.getSourceComponent()).thenReturn(componentId); + when(tuple.getSourceStreamId()).thenReturn(streamId); + return tuple; + } + + private static Tuple mockTickTuple() { + return mockTuple(Constants.SYSTEM_COMPONENT_ID, Constants.SYSTEM_TICK_STREAM_ID); + } +} 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 new file mode 100644 index 0000000000..f024420045 --- /dev/null +++ b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java @@ -0,0 +1,149 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.hbase.client; + +import backtype.storm.tuple.Tuple; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.metron.hbase.TableProvider; +import org.apache.metron.hbase.Widget; +import org.apache.metron.hbase.WidgetMapper; +import org.apache.storm.hbase.bolt.mapper.HBaseMapper; +import org.apache.storm.hbase.bolt.mapper.HBaseProjectionCriteria; +import org.apache.storm.hbase.common.ColumnList; +import org.junit.*; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.mockito.Mockito.*; + +/** + * Tests the HBaseClient + */ +public class HBaseClientTest { + + private static final String tableName = "widgets"; + + private static HBaseTestingUtility util; + private HBaseClient client; + private HTableInterface table; + + private Tuple tuple1; + private Tuple tuple2; + + private Widget widget1; + private Widget widget2; + + private HBaseMapper mapper; + + @BeforeClass + public static void startHBase() throws Exception { + util = new HBaseTestingUtility(); + util.startMiniCluster(); + } + + @AfterClass + public static void stopHBase() throws Exception { + util.shutdownMiniCluster(); + util.cleanupTestDir(); + } + + @Before + public void setupTuples() throws Exception { + + // setup the first tuple + widget1 = new Widget("widget1", 100); + tuple1 = mock(Tuple.class); + when(tuple1.getValueByField(eq("widget"))).thenReturn(widget1); + + // setup the second tuple + widget2 = new Widget("widget2", 200); + tuple2 = mock(Tuple.class); + when(tuple2.getValueByField(eq("widget"))).thenReturn(widget2); + } + + @Before + public void setup() throws Exception { + + // create a mapper + mapper = new WidgetMapper(); + + // create the table + table = util.createTable(Bytes.toBytes(tableName), WidgetMapper.CF); + + // setup the client + client = new HBaseClient((c,t) -> table, table.getConfiguration(), tableName); + } + + @After + public void tearDown() throws Exception { + util.deleteTable(tableName); + } + + @Test + public void testWrite() throws Exception { + + // add a tuple to the batch + byte[] rowKey1 = mapper.rowKey(tuple1); + ColumnList cols1 = mapper.columns(tuple1); + List mutations1 = client.constructMutationReq(rowKey1, cols1, Durability.SYNC_WAL); + client.batchMutate(mutations1); + + HBaseProjectionCriteria criteria = new HBaseProjectionCriteria(); + criteria.addColumnFamily(WidgetMapper.CF_STRING); + + // read back the tuple + Get get1 = client.constructGetRequests(rowKey1, criteria); + Result[] results = client.batchGet(Arrays.asList(get1)); + Assert.assertEquals(1, results.length); + } + + @Test + public void testBatchWrite() throws Exception { + + // add a tuple to the batch + byte[] rowKey1 = mapper.rowKey(tuple1); + ColumnList cols1 = mapper.columns(tuple1); + List mutations1 = client.constructMutationReq(rowKey1, cols1, Durability.SYNC_WAL); + client.batchMutate(mutations1); + + // add another tuple to the batch + byte[] rowKey2 = mapper.rowKey(tuple1); + ColumnList cols2 = mapper.columns(tuple1); + List mutations2 = client.constructMutationReq(rowKey2, cols2, Durability.SYNC_WAL); + client.batchMutate(mutations2); + + HBaseProjectionCriteria criteria = new HBaseProjectionCriteria(); + criteria.addColumnFamily(WidgetMapper.CF_STRING); + + // read back both tuples + 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); + } +} diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java index 39eca585c7..75be2d7d10 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java @@ -6,9 +6,9 @@ * 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 - * + *

+ * 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. @@ -20,6 +20,7 @@ import backtype.storm.Config; import backtype.storm.LocalCluster; import backtype.storm.generated.StormTopology; +import backtype.storm.generated.TopologyInfo; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; import org.apache.storm.flux.FluxBuilder; @@ -28,110 +29,122 @@ import org.apache.storm.flux.parser.FluxParser; import org.apache.thrift7.TException; import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.File; import java.io.FileWriter; import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.util.Properties; +import java.util.concurrent.TimeUnit; public class FluxTopologyComponent implements InMemoryComponent { - LocalCluster stormCluster; + + protected static final Logger LOG = LoggerFactory.getLogger(FluxTopologyComponent.class); + + LocalCluster stormCluster; + String topologyName; + File topologyLocation; + Properties topologyProperties; + + public static class Builder { + String topologyName; File topologyLocation; Properties topologyProperties; - public static class Builder { - String topologyName; - File topologyLocation; - Properties topologyProperties; - public Builder withTopologyName(String name) { - this.topologyName = name; - return this; - } - public Builder withTopologyLocation(File location) { - this.topologyLocation = location; - return this; - } - public Builder withTopologyProperties(Properties properties) { - this.topologyProperties = properties; - return this; - } - - public FluxTopologyComponent build() { - return new FluxTopologyComponent(topologyName, topologyLocation, topologyProperties); - } + public Builder withTopologyName(String name) { + this.topologyName = name; + return this; } - public FluxTopologyComponent(String topologyName, File topologyLocation, Properties topologyProperties) { - this.topologyName = topologyName; - this.topologyLocation = topologyLocation; - this.topologyProperties = topologyProperties; + public Builder withTopologyLocation(File location) { + this.topologyLocation = location; + return this; } - public LocalCluster getStormCluster() { - return stormCluster; + public Builder withTopologyProperties(Properties properties) { + this.topologyProperties = properties; + return this; } - public String getTopologyName() { - return topologyName; + public FluxTopologyComponent build() { + return new FluxTopologyComponent(topologyName, topologyLocation, topologyProperties); } + } - public File getTopologyLocation() { - return topologyLocation; - } + public FluxTopologyComponent(String topologyName, File topologyLocation, Properties topologyProperties) { + this.topologyName = topologyName; + this.topologyLocation = topologyLocation; + this.topologyProperties = topologyProperties; + } - public Properties getTopologyProperties() { - return topologyProperties; - } + public LocalCluster getStormCluster() { + return stormCluster; + } - public String getZookeeperConnectString() { - return "localhost:2000"; - } - public void start() throws UnableToStartException{ - try { - stormCluster = new LocalCluster(); - } catch (Exception e) { - throw new UnableToStartException("Unable to start flux topology: " + getTopologyLocation(), e); - } - } + public String getTopologyName() { + return topologyName; + } - public void stop() { - if(stormCluster != null) { - stormCluster.shutdown(); - } - } + public File getTopologyLocation() { + return topologyLocation; + } - public void submitTopology() throws NoSuchMethodException, IOException, InstantiationException, TException, IllegalAccessException, InvocationTargetException, ClassNotFoundException { - startTopology(getTopologyName(), getTopologyLocation(), getTopologyProperties()); - } - private void startTopology(String topologyName, File topologyLoc, Properties properties) throws IOException, ClassNotFoundException, NoSuchMethodException, InvocationTargetException, InstantiationException, IllegalAccessException, TException { - TopologyDef topologyDef = loadYaml(topologyName, topologyLoc, properties); - Config conf = FluxBuilder.buildConfig(topologyDef); - ExecutionContext context = new ExecutionContext(topologyDef, conf); - StormTopology topology = FluxBuilder.buildTopology(context); - Assert.assertNotNull(topology); - topology.validate(); - stormCluster.submitTopology(topologyName, conf, topology); + public Properties getTopologyProperties() { + return topologyProperties; + } + + public String getZookeeperConnectString() { + return "localhost:2000"; + } + + public void start() throws UnableToStartException { + try { + stormCluster = new LocalCluster(); + + } catch (Exception e) { + throw new UnableToStartException("Unable to start flux topology: " + getTopologyLocation(), e); } + } - private static TopologyDef loadYaml(String topologyName, File yamlFile, Properties properties) throws IOException { - File tmpFile = File.createTempFile(topologyName, "props"); - tmpFile.deleteOnExit(); - FileWriter propWriter = null; - try { - propWriter = new FileWriter(tmpFile); - properties.store(propWriter, topologyName + " properties"); - } - finally { - if(propWriter != null) { - propWriter.close(); - return FluxParser.parseFile(yamlFile.getAbsolutePath(), false, true, tmpFile.getAbsolutePath(), false); - } - - return null; - } + public void stop() { + if (stormCluster != null) { + stormCluster.shutdown(); + } + } + + public void submitTopology() throws NoSuchMethodException, IOException, InstantiationException, TException, IllegalAccessException, InvocationTargetException, ClassNotFoundException { + startTopology(getTopologyName(), getTopologyLocation(), getTopologyProperties()); + } + + private void startTopology(String topologyName, File topologyLoc, Properties properties) throws IOException, ClassNotFoundException, NoSuchMethodException, InvocationTargetException, InstantiationException, IllegalAccessException, TException { + TopologyDef topologyDef = loadYaml(topologyName, topologyLoc, properties); + Config conf = FluxBuilder.buildConfig(topologyDef); + ExecutionContext context = new ExecutionContext(topologyDef, conf); + StormTopology topology = FluxBuilder.buildTopology(context); + Assert.assertNotNull(topology); + topology.validate(); + stormCluster.submitTopology(topologyName, conf, topology); + } + + private static TopologyDef loadYaml(String topologyName, File yamlFile, Properties properties) throws IOException { + File tmpFile = File.createTempFile(topologyName, "props"); + tmpFile.deleteOnExit(); + FileWriter propWriter = null; + try { + propWriter = new FileWriter(tmpFile); + properties.store(propWriter, topologyName + " properties"); + } finally { + if (propWriter != null) { + propWriter.close(); + return FluxParser.parseFile(yamlFile.getAbsolutePath(), false, true, tmpFile.getAbsolutePath(), false); + } + + return null; } + } } 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 6606fdc01d..75d8104c04 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 @@ -118,7 +118,6 @@ public void addColumnFamily(String columnFamily) { } } - @Override public byte[] getTableName() { return Bytes.toBytes(tableName); @@ -200,8 +199,8 @@ public Boolean[] exists(List list) throws IOException { @Override public void batch(List list, Object[] objects) throws IOException, InterruptedException { - throw new UnsupportedOperationException(); - + Object[] results = batch(list); + System.arraycopy(results, 0, objects, 0, objects.length); } /** diff --git a/metron-platform/pom.xml b/metron-platform/pom.xml index 1f8d5f1fe7..018e7281ba 100644 --- a/metron-platform/pom.xml +++ b/metron-platform/pom.xml @@ -73,7 +73,6 @@ - diff --git a/pom.xml b/pom.xml index c138d2bead..fc91d859e5 100644 --- a/pom.xml +++ b/pom.xml @@ -27,6 +27,7 @@ metron-analytics metron-platform + metron-analytics @ApacheMetron From 28560b36273271c438c3309df610a4c4306ffdde Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 11 Aug 2016 16:46:00 -0400 Subject: [PATCH 02/10] METRON-309 Corrected a typo in the 'Getting Started' instructions --- metron-analytics/metron-profiler/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md index 76303fa769..825fec7f9c 100644 --- a/metron-analytics/metron-profiler/README.md +++ b/metron-analytics/metron-profiler/README.md @@ -130,7 +130,7 @@ This section will describe the steps required to get your first profile running. 3. Create a table within HBase that will store the profile data. The table name and column family must match the Profiler topology configuration stored at `/usr/metron/0.2.0BETA/config/profiler.properties`. ``` $ /usr/hdp/current/hbase-client/bin/hbase shell - hbase(main):001:0> create 'profiler1', 'cfProfile' + hbase(main):001:0> create 'profiler', 'cfProfile' ``` 4. Shorten the flush intervals to more immediately see results. Edit the Profiler topology properties located at `/usr/metron/0.2.0BETA/config/profiler.properties`. Alter the following two properties. @@ -171,7 +171,7 @@ This section will describe the steps required to get your first profile running. 9. Check the HBase table to validate that the Profiler is working. ``` $ /usr/hdp/current/hbase-client/bin/hbase shell - hbase(main):001:0> count 'profiler1' + hbase(main):001:0> count 'profiler' ``` ## Design From d55d32efeea38f52bd9d1095aa6f8817bc15f0a5 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 12 Aug 2016 09:16:19 -0400 Subject: [PATCH 03/10] METRON-309 Altered based on Stellar Unification changes --- metron-analytics/metron-profiler/README.md | 2 +- .../profiler/bolt/ProfileBuilderBolt.java | 2 +- .../stellar/DefaultStellarExecutor.java | 52 +----- .../profiler/bolt/ProfileBuilderBoltTest.java | 11 +- .../util/DefaultStellarExecutorTest.java | 7 +- .../ControlFlowFunctionsTest.java | 76 --------- .../transformation/MathFunctionsTest.java | 85 --------- .../transformation/TransformationTest.java | 161 ------------------ pom.xml | 1 - 9 files changed, 16 insertions(+), 381 deletions(-) delete mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/ControlFlowFunctionsTest.java delete mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/MathFunctionsTest.java delete mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/TransformationTest.java diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md index 825fec7f9c..6183d989c7 100644 --- a/metron-analytics/metron-profiler/README.md +++ b/metron-analytics/metron-profiler/README.md @@ -149,7 +149,7 @@ This section will describe the steps required to get your first profile running. "foreach": "ip_src_addr", "onlyif": "true", "init": { "sum": 0 }, - "update": { "sum": "ADD(sum,1)" }, + "update": { "sum": "sum + 1" }, "result": "TO_LONG(sum)" } ] 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 6d03f6e4a5..cedb163c89 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 @@ -199,7 +199,7 @@ private void flush(Tuple tickTuple) throws IOException { // execute the 'result' expression String resultExpr = profileConfig.getResult(); - Long result = executor.execute(resultExpr, new JSONObject(), Long.class); + Double result = executor.execute(resultExpr, new JSONObject(), Double.class); // emit the completed profile measurement measurement.setEnd(getTimestamp()); 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 9b3dd18613..760f618ef3 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 @@ -22,8 +22,8 @@ import org.apache.metron.common.dsl.MapVariableResolver; import org.apache.metron.common.dsl.VariableResolver; -import org.apache.metron.common.query.PredicateProcessor; -import org.apache.metron.common.transformation.TransformationProcessor; +import org.apache.metron.common.stellar.StellarPredicateProcessor; +import org.apache.metron.common.stellar.StellarProcessor; import org.json.simple.JSONObject; import java.io.Serializable; @@ -111,54 +111,10 @@ public void clearState() { */ private Object execute(String expression, JSONObject message) { - Object result = null; - try { - result = executeTransformation(expression, message); - - } finally { - // maybe the expression is a predicate, not a transformation - if(result == null) { - result = executePredicate(expression, message); - } - - return result; - } - } - - /** - * Executes Stella predicates using the TransformationProcessor. There are two sets - * of functions in Stellar currently. One can be executed with a - * PredicateProcessor and the other a TransformationProcessor. - * @param expression The expression to execute. - * @param message The message that is accessible when Stellar is executed. - */ - private Object executeTransformation(String expression, JSONObject message) { - - // vars can be resolved from the execution state or the current message - VariableResolver resolver = new MapVariableResolver(state, message); - - TransformationProcessor processor = new TransformationProcessor(); - return processor.parse(expression, resolver); - } - - /** - * Executes Stella predicates using the PredicateProcessor. There are two sets - * of functions in Stellar currently. One can be executed with a - * PredicateProcessor and the other a TransformationProcessor. - * @param expression The expression to execute. - * @param message The message that is accessible when Stellar is executed. - */ - private boolean executePredicate(String expression, JSONObject message) { - - // vars can be resolved from the execution state or the input message + // vartables can be resolved from the execution state or the current message VariableResolver resolver = new MapVariableResolver(state, message); - PredicateProcessor processor = new PredicateProcessor(); - boolean valid = processor.validate(expression); - if(!valid) { - throw new RuntimeException(String.format("Invalid predicate expression; expression=%s", expression)); - } - + StellarProcessor processor = new StellarProcessor(); return processor.parse(expression, resolver); } } 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 7bd4771091..7b6232e7c8 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 @@ -39,6 +39,7 @@ import java.util.HashMap; import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.*; @@ -67,10 +68,10 @@ public class ProfileBuilderBoltTest extends BaseBoltTest { * "y": "20" * }, * "update": { - * "x": "ADD(x, 10)", - * "y": "ADD(y, 20)" + * "x": "x + 10", + * "y": "y + 20" * }, - * "result": "TO_LONG(ADD(x, y))" + * "result": "x + y" * } */ @Multiline @@ -132,8 +133,8 @@ public void testUpdateProfile() throws Exception { bolt.execute(tuple); // validate that x=10+10+10 y=20+20+20 - assertThat(bolt.getExecutor().getState().get("x"), equalTo(10+10+10)); - assertThat(bolt.getExecutor().getState().get("y"), equalTo(20+20+20)); + assertEquals(10+10+10.0, bolt.getExecutor().getState().get("x")); + assertEquals(20+20+20.0, bolt.getExecutor().getState().get("y")); } /** diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/util/DefaultStellarExecutorTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/util/DefaultStellarExecutorTest.java index 3e8ca671f9..8046fe0fdd 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/util/DefaultStellarExecutorTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/util/DefaultStellarExecutorTest.java @@ -30,6 +30,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; /** @@ -93,11 +94,11 @@ public void testAssignWithVariableResolution() { public void testState() { executor.assign("two", "2", message); executor.assign("four", "4", message); - executor.assign("sum", "ADD(two, four)", message); + executor.assign("sum", "two + four", message); // verify Object var = executor.getState().get("sum"); - assertThat(var, equalTo(6)); + assertEquals(6.0, var); } /** @@ -143,6 +144,6 @@ public void testExecutePredicate() { */ @Test(expected = RuntimeException.class) public void testWrongType() { - executor.execute("ADD(2,2)", message, Boolean.class); + executor.execute("2 + 2", message, Boolean.class); } } diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/ControlFlowFunctionsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/ControlFlowFunctionsTest.java deleted file mode 100644 index db94b3b509..0000000000 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/ControlFlowFunctionsTest.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package org.apache.metron.common.transformation; - -import com.google.common.collect.ImmutableMap; -import org.apache.metron.common.dsl.MapVariableResolver; -import org.apache.metron.common.dsl.VariableResolver; -import org.junit.Assert; -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; - -/** - * Tests the ControlFlowFunctions. - */ -public class ControlFlowFunctionsTest { - - private static Object run(String expression, Map variables) { - VariableResolver resolver = new MapVariableResolver(variables); - TransformationProcessor processor = new TransformationProcessor(); - return processor.parse(expression, resolver); - } - - private static Object run(String expression) { - return run(expression, new HashMap<>()); - } - - @Test - public void testEqualStrings() { - Assert.assertEquals(true, run("EQUAL('foo', 'foo')")); - Assert.assertEquals(false, run("EQUAL('foo', 'bar')")); - } - - @Test - public void testEqualIntegers() { - Assert.assertEquals(true, run("EQUAL(2, 2)")); - Assert.assertEquals(false, run("EQUAL(2, 1)")); - } - - @Test - public void testEqualVariables() { - Assert.assertEquals(true, run("EQUAL(two, two)", ImmutableMap.of("two", 2, "four", 4))); - Assert.assertEquals(false, run("EQUAL(two, four)", ImmutableMap.of("two", 2, "four", 4))); - } - - @Test - public void testIfThenElse() { - Assert.assertEquals(1, run("IF_THEN_ELSE( EQUAL(2,2), 1, 0)")); - Assert.assertEquals(0, run("IF_THEN_ELSE( EQUAL(2,1), 1, 0)")); - } - - @Test - public void testIfThenElseStrings() { - Assert.assertEquals("equal", run("IF_THEN_ELSE( EQUAL(2,2), 'equal', 'not')")); - Assert.assertEquals("not", run("IF_THEN_ELSE( EQUAL(2,1), 'equal', 'not')")); - } -} diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/MathFunctionsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/MathFunctionsTest.java deleted file mode 100644 index 6f725df985..0000000000 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/MathFunctionsTest.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package org.apache.metron.common.transformation; - -import com.google.common.collect.ImmutableMap; -import org.apache.metron.common.dsl.MapVariableResolver; -import org.apache.metron.common.dsl.VariableResolver; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Map; - -/** - * Tests the MathFunctions. - */ -public class MathFunctionsTest { - - private static Object run(String expression, Map variables) { - VariableResolver resolver = new MapVariableResolver(variables); - TransformationProcessor processor = new TransformationProcessor(); - return processor.parse(expression, resolver); - } - - @Test - public void testAdd() { - Assert.assertEquals(10.0d, run("ADD(foo, foo)", ImmutableMap.of("foo", 5.0d))); - Assert.assertEquals(10.0f, run("ADD(foo, foo)", ImmutableMap.of("foo", 5.0f))); - Assert.assertEquals(10L, run("ADD(foo, foo)", ImmutableMap.of("foo", 5L))); - Assert.assertEquals(10, run("ADD(foo, foo)", ImmutableMap.of("foo", 5))); - - // in java, short + short = integer - Assert.assertEquals(10, run("ADD(foo, foo)", ImmutableMap.of("foo", (short) 5))); - } - - @Test - public void testSubtract() { - Assert.assertEquals(0d, run("SUB(foo, foo)", ImmutableMap.of("foo", 5.0d))); - Assert.assertEquals(0f, run("SUB(foo, foo)", ImmutableMap.of("foo", 5.0f))); - Assert.assertEquals(0L, run("SUB(foo, foo)", ImmutableMap.of("foo", 5L))); - Assert.assertEquals(0, run("SUB(foo, foo)", ImmutableMap.of("foo", 5))); - - // per JVM specification, short - short = integer - Assert.assertEquals(0, run("SUB(foo, foo)", ImmutableMap.of("foo", (short) 5))); - } - - @Test - public void testDivide() { - Assert.assertEquals(1d, run("DIV(foo, foo)", ImmutableMap.of("foo", 5.0d))); - Assert.assertEquals(1f, run("DIV(foo, foo)", ImmutableMap.of("foo", 5.0f))); - Assert.assertEquals(1L, run("DIV(foo, foo)", ImmutableMap.of("foo", 5L))); - Assert.assertEquals(1, run("DIV(foo, foo)", ImmutableMap.of("foo", 5))); - - // per JVM specification, short / short = integer - Assert.assertEquals(1, run("DIV(foo, foo)", ImmutableMap.of("foo", (short) 5))); - } - - @Test - public void testMultiply() { - Assert.assertEquals(25.0d, run("MULT(foo, foo)", ImmutableMap.of("foo", 5.0d))); - Assert.assertEquals(25.0f, run("MULT(foo, foo)", ImmutableMap.of("foo", 5.0f))); - Assert.assertEquals(25L, run("MULT(foo, foo)", ImmutableMap.of("foo", 5L))); - Assert.assertEquals(25, run("MULT(foo, foo)", ImmutableMap.of("foo", 5))); - - // in java, short * short = integer - Assert.assertEquals(25, run("MULT(foo, foo)", ImmutableMap.of("foo", (short) 5))); - } -} diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/TransformationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/TransformationTest.java deleted file mode 100644 index a5793a8f11..0000000000 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/transformation/TransformationTest.java +++ /dev/null @@ -1,161 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.metron.common.transformation; - -import com.google.common.collect.ImmutableMap; -import org.apache.metron.common.utils.timestamp.TimestampConverters; -import org.junit.Assert; -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; - -public class TransformationTest { - - @Test - public void testHappyPath() { - String query = "TO_UPPER(TRIM(foo))"; - Assert.assertEquals("CASEY", run(query, ImmutableMap.of("foo", "casey "))); - } - - @Test - public void testJoin() { - String query = "JOIN( [ TO_UPPER(TRIM(foo)), 'bar' ], ',')"; - Assert.assertEquals("CASEY,bar", run(query, ImmutableMap.of("foo", "casey "))); - } - - @Test - public void testSplit() { - String query = "JOIN( SPLIT(foo, ':'), ',')"; - Assert.assertEquals("casey,bar", run(query, ImmutableMap.of("foo", "casey:bar"))); - } - - @Test - public void testMapGet() { - String query = "MAP_GET(dc, dc2tz, 'UTC')"; - Assert.assertEquals("UTC" - , run(query, ImmutableMap.of("dc", "nyc" - ,"dc2tz", ImmutableMap.of("la", "PST") - ) - ) - ); - Assert.assertEquals("EST" - , run(query, ImmutableMap.of("dc", "nyc" - ,"dc2tz", ImmutableMap.of("nyc", "EST") - ) - ) - ); - } - @Test - public void testTLDExtraction() { - String query = "DOMAIN_TO_TLD(foo)"; - Assert.assertEquals("co.uk", run(query, ImmutableMap.of("foo", "www.google.co.uk"))); - } - - @Test - public void testTLDRemoval() { - String query = "DOMAIN_REMOVE_TLD(foo)"; - Assert.assertEquals("www.google", run(query, ImmutableMap.of("foo", "www.google.co.uk"))); - } - - @Test - public void testSubdomainRemoval() { - String query = "DOMAIN_REMOVE_SUBDOMAINS(foo)"; - Assert.assertEquals("google.co.uk", run(query, ImmutableMap.of("foo", "www.google.co.uk"))); - Assert.assertEquals("google.com", run(query, ImmutableMap.of("foo", "www.google.com"))); - } - @Test - public void testURLToHost() { - String query = "URL_TO_HOST(foo)"; - Assert.assertEquals("www.google.co.uk", run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path"))); - } - - @Test - public void testURLToPort() { - String query = "URL_TO_PORT(foo)"; - Assert.assertEquals(80, run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path"))); - } - - @Test - public void testURLToProtocol() { - String query = "URL_TO_PROTOCOL(foo)"; - Assert.assertEquals("http", run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path"))); - } - - @Test - public void testURLToPath() { - String query = "URL_TO_PATH(foo)"; - Assert.assertEquals("/my/path", run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path"))); - } - - @Test - public void testProtocolToName() { - String query = "PROTOCOL_TO_NAME(protocol)"; - Assert.assertEquals("TCP", run(query, ImmutableMap.of("protocol", "6"))); - Assert.assertEquals("TCP", run(query, ImmutableMap.of("protocol", 6))); - Assert.assertEquals(null, run(query, ImmutableMap.of("foo", 6))); - Assert.assertEquals("chicken", run(query, ImmutableMap.of("protocol", "chicken"))); - } - @Test - public void testDateConversion() { - long expected =1452013350000L; - { - String query = "TO_EPOCH_TIMESTAMP(foo, 'yyyy-MM-dd HH:mm:ss', 'UTC')"; - Assert.assertEquals(expected, run(query, ImmutableMap.of("foo", "2016-01-05 17:02:30"))); - } - { - String query = "TO_EPOCH_TIMESTAMP(foo, 'yyyy-MM-dd HH:mm:ss')"; - Long ts = (Long) run(query, ImmutableMap.of("foo", "2016-01-05 17:02:30")); - //is it within 24 hours of the UTC? - Assert.assertTrue(Math.abs(ts - expected) < 8.64e+7); - } - } - - @Test - public void testToString() { - Assert.assertEquals("5", run("TO_STRING(foo)", ImmutableMap.of("foo", 5))); - } - - @Test - public void testToInteger() { - Assert.assertEquals(5, run("TO_INTEGER(foo)", ImmutableMap.of("foo", "5"))); - Assert.assertEquals(5, run("TO_INTEGER(foo)", ImmutableMap.of("foo", 5))); - } - - @Test - public void testToDouble() { - Assert.assertEquals(new Double(5.1), run("TO_DOUBLE(foo)", ImmutableMap.of("foo", 5.1d))); - Assert.assertEquals(new Double(5.1), run("TO_DOUBLE(foo)", ImmutableMap.of("foo", "5.1"))); - } - - @Test - public void testGet() { - Map variables = ImmutableMap.of("foo", "www.google.co.uk"); - Assert.assertEquals("www", run("GET_FIRST(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'))", variables)); - Assert.assertEquals("www", run("GET(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'), 0)", variables)); - Assert.assertEquals("google", run("GET_LAST(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'))", variables)); - Assert.assertEquals("google", run("GET(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'), 1)", variables)); - } - - private static Object run(String rule, Map variables) { - TransformationProcessor processor = new TransformationProcessor(); - Assert.assertTrue(rule + " not valid.", processor.validate(rule)); - return processor.parse(rule, x -> variables.get(x)); - } -} diff --git a/pom.xml b/pom.xml index fc91d859e5..c138d2bead 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,6 @@ metron-analytics metron-platform - metron-analytics @ApacheMetron From ca81339e0f326dc04110eafd74fd0879dfe1a029 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 12 Aug 2016 09:23:06 -0400 Subject: [PATCH 04/10] METRON-309 Need to set the kafka broker in the Profiler topology properties --- .../roles/metron_streaming/defaults/main.yml | 1 + .../roles/metron_streaming/tasks/copy_bundles.yml | 1 + .../roles/metron_streaming/tasks/topologies.yml | 12 ++++++++++++ 3 files changed, 14 insertions(+) diff --git a/metron-deployment/roles/metron_streaming/defaults/main.yml b/metron-deployment/roles/metron_streaming/defaults/main.yml index f125d70221..b048bf04b3 100644 --- a/metron-deployment/roles/metron_streaming/defaults/main.yml +++ b/metron-deployment/roles/metron_streaming/defaults/main.yml @@ -56,6 +56,7 @@ metron_solr_properties_config_path: "{{ metron_directory }}/config/solr.properti metron_elasticsearch_properties_config_path: "{{ metron_directory }}/config/elasticsearch.properties" metron_enrichment_properties_config_path: "{{ metron_directory }}/config/enrichment.properties" metron_parsers_properties_config_path: "{{ metron_directory }}/config/parsers.properties" +metron_profiler_properties_config_path: "{{ metron_directory }}/config/profiler.properties" hbase_config_path: "/etc/hbase/conf" hdfs_config_path: "/etc/hadoop/conf" diff --git a/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml b/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml index 300f9ccfd0..ca972cb482 100644 --- a/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml +++ b/metron-deployment/roles/metron_streaming/tasks/copy_bundles.yml @@ -58,6 +58,7 @@ - name: Copy Metron MaaS bundle copy: src: "{{ metron_maas_bundle_path }}" + dest: "{{ metron_directory }}" - name: Copy Metron Profiler bundle copy: diff --git a/metron-deployment/roles/metron_streaming/tasks/topologies.yml b/metron-deployment/roles/metron_streaming/tasks/topologies.yml index 0ca0b3d2a3..7bb2bb1bcd 100644 --- a/metron-deployment/roles/metron_streaming/tasks/topologies.yml +++ b/metron-deployment/roles/metron_streaming/tasks/topologies.yml @@ -78,3 +78,15 @@ - { regexp: "bolt.hdfs.rotation.policy=", line: "bolt.hdfs.rotation.policy={{ metron_hdfs_rotation_policy }}" } - { regexp: "bolt.hdfs.rotation.policy.count=", line: "bolt.hdfs.rotation.policy.count={{ metron_hdfs_rotation_policy_count}}" } - { regexp: "bolt.hdfs.rotation.policy.units=", line: "bolt.hdfs.rotation.policy.units={{ metron_hdfs_rotation_policy_units }}" } + +- name: Configure Profiler topology + lineinfile: > + dest={{ metron_profiler_properties_config_path }} + regexp="{{ item.regexp }}" + line="{{ item.line }}" + with_items: + - { regexp: "kafka.zk=", line: "kafka.zk={{ zookeeper_url }}" } + - { regexp: "kafka.broker=", line: "kafka.broker={{ kafka_broker_url }}" } + + + From 712a1083c21eb8f7c0d81511d432c054792ebe9b Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 12 Aug 2016 11:17:47 -0400 Subject: [PATCH 05/10] METRON-309 Updated examples to use latest Stellar binary functions --- metron-analytics/metron-profiler/README.md | 10 +++++----- .../src/test/config/zookeeper/profiler.json | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md index 6183d989c7..2ab1f0aaf8 100644 --- a/metron-analytics/metron-profiler/README.md +++ b/metron-analytics/metron-profiler/README.md @@ -59,12 +59,12 @@ The ratio of DNS traffic to HTTP traffic for each host. The following configurat "foreach": "ip_src_addr", "onlyif": "protocol == 'DNS' or protocol == ‘HTTP’ “init”: { - “num_dns”: 0, - “num_http”: 0 + “num_dns”: 1, + “num_http”: 1 },   "update": { -     "num_dns": "num_dns + IF_THEN_ELSE(protocol == ‘DNS’, 1, 0)", - "num_http": "num_http + IF_THEN_ELSE(protocol == ‘HTTP’, 1, 0)" +     "num_dns": "num_dns + if protocol == ‘DNS’ then 1 else 0", + "num_http": "num_http + if protocol == ‘HTTP’ then 1 else 0"   },   "result": "num_dns / num_http" } @@ -150,7 +150,7 @@ This section will describe the steps required to get your first profile running. "onlyif": "true", "init": { "sum": 0 }, "update": { "sum": "sum + 1" }, - "result": "TO_LONG(sum)" + "result": "sum" } ] } diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json index eecdc2105c..d34b8a6f0e 100644 --- a/metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json +++ b/metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json @@ -6,8 +6,8 @@ "foreach": "ip_src_addr", "onlyif": "STARTS_WITH(protocol, 'HTTP')", "init": { "cnt": 0 }, - "update": { "cnt": "ADD(cnt, 1)" }, - "result": "TO_LONG(cnt)" + "update": { "cnt": "cnt + 1" }, + "result": "cnt" } ] } \ No newline at end of file From 01aa1f972324f97b231b7d9a26f122891a1b6d50 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 15 Aug 2016 14:03:49 -0400 Subject: [PATCH 06/10] METRON-309 Fixed the README examples and added each as an integration test. --- metron-analytics/metron-profiler/README.md | 128 +++++++++++------- metron-analytics/metron-profiler/pom.xml | 7 +- .../profiler/bolt/ProfileBuilderBolt.java | 34 +++-- .../stellar/DefaultStellarExecutor.java | 42 +++--- .../src/test/config/zookeeper/profiler.json | 13 -- .../zookeeper/readme-example-1/profiler.json | 17 +++ .../zookeeper/readme-example-2/profiler.json | 19 +++ .../zookeeper/readme-example-3/profiler.json | 20 +++ .../integration/ProfilerIntegrationTest.java | 105 ++++++++------ 9 files changed, 254 insertions(+), 131 deletions(-) delete mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json create mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json create mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-2/profiler.json create mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md index 2ab1f0aaf8..904446bbc8 100644 --- a/metron-analytics/metron-profiler/README.md +++ b/metron-analytics/metron-profiler/README.md @@ -21,94 +21,128 @@ The Profiler configuration requires a JSON-formatted set of elements, many of wh ### Examples -Examples of the types of profiles that can be collected include the following. Each shows the configuration that would be required to produce the profile. +Examples of the types of profiles that can be built include the following. Each shows the configuration that would be required to produce the profile. These examples assume a fictitious input messages that looks something like the following. + +``` +{ + "ip_src_addr": "10.0.0.1", + "protocol": "HTTPS", + "length": "10", + "bytes_in": "234" +}, +{ + "ip_src_addr": "10.0.0.2", + "protocol": "HTTP", + "length": "20", + "bytes_in": "390" +}, +{ + "ip_src_addr": "10.0.0.3", + "protocol": "DNS", + "length": "30", + "bytes_in": "560" +} +``` ### Example 1 -The total number of bytes received for each host. The following configuration would be used to generate this profile. +The total number of bytes of HTTP data for each host. The following configuration would be used to generate this profile. ``` -{ "profiler": [ - { - "profile": "sum_bytes_in", - "foreach": "ip_src_addr", - "onlyif": "EXISTS(is_local)", - “init”: { “sum”: 0 }, - "update": { "sum": "sum + bytes_in" }, - "result": "sum" - } -]} +{ + "inputTopic": "indexing", + "profiles": [ + { + "profile": "example1", + "foreach": "ip_src_addr", + "onlyif": "protocol == 'HTTP'", + "init": { + "total_bytes": 0.0 + }, + "update": { + "total_bytes": "total_bytes + bytes_in" + }, + "result": "total_bytes" + } + ] +} ``` This creates a profile... - * Named ‘sum_bytes_in’ + * Named ‘example1’ * That for each IP source address - * Only if it is on the local network - * Initializes a counter ‘sum’ to 0 - * Updates ‘sum’ by adding the value of ‘bytes_in’ from each message - * After the window expires, ‘sum’ becomes the result + * Only if the 'protocol' field equals 'HTTP' + * Initializes a counter ‘total_bytes’ to zero + * Adds to ‘total_bytes’ the value of the message's ‘bytes_in’ field + * Returns ‘total_bytes’ as the result ### Example 2 The ratio of DNS traffic to HTTP traffic for each host. The following configuration would be used to generate this profile. ``` -{ "profiler": [ - { - "profile": "ratio_dns_to_http", - "foreach": "ip_src_addr", - "onlyif": "protocol == 'DNS' or protocol == ‘HTTP’ - “init”: { - “num_dns”: 1, - “num_http”: 1 - }, -   "update": { -     "num_dns": "num_dns + if protocol == ‘DNS’ then 1 else 0", - "num_http": "num_http + if protocol == ‘HTTP’ then 1 else 0" -   }, -   "result": "num_dns / num_http" - } -]} +{ + "inputTopic": "indexing", + "profiles": [ + { + "profile": "example2", + "foreach": "ip_src_addr", + "onlyif": "protocol == 'DNS' or protocol == 'HTTP'", + "init": { + "num_dns": 1.0, + "num_http": 1.0 + }, + "update": { + "num_dns": "num_dns + (if protocol == 'DNS' then 1 else 0)", + "num_http": "num_http + (if protocol == 'HTTP' then 1 else 0)" + }, + "result": "num_dns / num_http" + } + ] +} ``` This creates a profile... - * Named ‘ratio_dns_to_http’ + * Named ‘example2’ * That for each IP source address - * Only if the message is either DNS or HTTP + * Only if the 'protocol' field equals 'HTTP' or 'DNS' * Accumulates the number of DNS requests * Accumulates the number of HTTP requests - * After the window expires, the ratio of these is the result + * Returns the ratio of these as the result ### Example 3 The average response body length of HTTP traffic. The following configuration would be used to generate this profile. ``` -{ "profiler": [ +{ + "inputTopic": "indexing", + "profiles": [ { - "profile": "http_mean_resp_body_len", + "profile": "example3", "foreach": "ip_src_addr", "onlyif": "protocol == 'HTTP'", - “init”: { - “sum”: 0, - “cnt”: 0 + "init": { + "sum": 0.0, + "cnt": 0.0 }, "update": { "sum": "sum + resp_body_len", "cnt": "cnt + 1" }, - "result": "sum / cnt", + "result": "sum / cnt" } - ]} + ] +} ``` This creates a profile... - * Named ‘http_mean_resp_body_len’ + * Named ‘example3’ * That for each IP source address - * That is either HTTP or HTTPS + * Only if the 'protocol' field equals 'HTTP' * Accumulates the sum of response body length - * Accumulates the count of messages - * After the window period expires, the average is calculated + * Accumulates the number of messages + * Calculates the average as the result ## Getting Started diff --git a/metron-analytics/metron-profiler/pom.xml b/metron-analytics/metron-profiler/pom.xml index e3b4b4fdfc..b31df67981 100644 --- a/metron-analytics/metron-profiler/pom.xml +++ b/metron-analytics/metron-profiler/pom.xml @@ -141,7 +141,6 @@ org.apache.hbase hbase-client ${global_hbase_version} - org.slf4j @@ -189,6 +188,12 @@ ${global_mockito_version} test + + com.google.code.tempus-fugit + tempus-fugit + 1.2-20140129.191141-5 + test + 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 cedb163c89..c5e166f82a 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 @@ -30,6 +30,7 @@ import backtype.storm.tuple.Values; import org.apache.metron.common.bolt.ConfiguredProfilerBolt; import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.common.dsl.ParseException; import org.apache.metron.profiler.ProfileMeasurement; import org.apache.metron.profiler.stellar.StellarExecutor; import org.json.simple.JSONObject; @@ -119,8 +120,8 @@ public void execute(Tuple input) { doExecute(input); collector.ack(input); - } catch (IOException e) { - LOG.error("exception: {}", e); + } catch (Throwable e) { + LOG.error("exception processing tuple: " + input, e); collector.reportError(e); } } @@ -169,9 +170,14 @@ private void init(Tuple input) throws IOException { measurement.setProfileName(profileConfig.getProfile()); // execute the 'init' expression - JSONObject message = (JSONObject) input.getValueByField("message"); - Map expressions = profileConfig.getInit(); - expressions.forEach((var, expr) -> executor.assign(var, expr, message)); + try { + JSONObject message = (JSONObject) input.getValueByField("message"); + Map expressions = profileConfig.getInit(); + expressions.forEach((var, expr) -> executor.assign(var, expr, message)); + + } catch(ParseException e) { + throw new ParseException("Bad 'init' expression", e); + } } /** @@ -182,8 +188,13 @@ private void update(Tuple input) throws IOException { JSONObject message = (JSONObject) input.getValueByField("message"); // execute each of the 'update' expressions - Map expressions = profileConfig.getUpdate(); - expressions.forEach((var, expr) -> executor.assign(var, expr, message)); + try { + Map expressions = profileConfig.getUpdate(); + expressions.forEach((var, expr) -> executor.assign(var, expr, message)); + + } catch(ParseException e) { + throw new ParseException("Bad 'update' expression", e); + } } /** @@ -198,8 +209,13 @@ private void flush(Tuple tickTuple) throws IOException { measurement.getProfileName(), measurement.getEntity(), measurement.getStart())); // execute the 'result' expression - String resultExpr = profileConfig.getResult(); - Double result = executor.execute(resultExpr, new JSONObject(), Double.class); + Double result; + try { + String resultExpr = profileConfig.getResult(); + result = executor.execute(resultExpr, new JSONObject(), Double.class); + } catch(ParseException e) { + throw new ParseException("Bad 'result' expression", e); + } // emit the completed profile measurement measurement.setEnd(getTimestamp()); 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 760f618ef3..1bc4c1efbf 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 @@ -21,6 +21,7 @@ package org.apache.metron.profiler.stellar; import org.apache.metron.common.dsl.MapVariableResolver; +import org.apache.metron.common.dsl.ParseException; import org.apache.metron.common.dsl.VariableResolver; import org.apache.metron.common.stellar.StellarPredicateProcessor; import org.apache.metron.common.stellar.StellarProcessor; @@ -59,16 +60,12 @@ public Map getState() { /** * Execute an expression and assign the result to a variable. * - * Stellar does not directly support assignment currently. This makes - * life easier until it does. - * * @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. */ @Override public void assign(String variable, String expression, JSONObject message) { - Object result = execute(expression, message); state.put(variable, result); } @@ -76,21 +73,22 @@ public void assign(String variable, String expression, JSONObject message) { /** * Execute a Stellar expression and returns the result. * - * @param expression The expression to execute. + * @param expr The expression to execute. * @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. */ @Override - public T execute(String expression, JSONObject message, Class clazz) { + public T execute(String expr, JSONObject message, Class clazz) { + Object result = execute(expr, message); - Object result = execute(expression, message); - if(clazz.isAssignableFrom(result.getClass())) { + // ensure the result type is as expected + if (clazz.isAssignableFrom(result.getClass())) { return (T) result; } else { - throw new RuntimeException(String.format("Unexpected type; expected=%s, actual=%s, expression=%s", - clazz.getSimpleName(), result.getClass().getSimpleName(), expression)); + throw new RuntimeException(String.format("Unexpected type: expected=%s, actual=%s, expression=%s", + clazz.getSimpleName(), result.getClass().getSimpleName(), expr)); } } @@ -102,19 +100,17 @@ public void clearState() { /** * Execute a Stellar expression. * - * There are two sets of functions in Stellar. One can be executed with - * a PredicateProcessor and the other a TransformationProcessor. This method - * uses the TransformationProcessor. - * - * @param expression The expression to execute. - * @param message The message that is accessible when Stellar is executed. + * @param expr The expression to execute. + * @param msg The message that is accessible when Stellar is executed. */ - private Object execute(String expression, JSONObject message) { - - // vartables can be resolved from the execution state or the current message - VariableResolver resolver = new MapVariableResolver(state, message); - - StellarProcessor processor = new StellarProcessor(); - return processor.parse(expression, resolver); + private Object execute(String expr, JSONObject msg) { + try { + VariableResolver resolver = new MapVariableResolver(state, msg); + StellarProcessor processor = new StellarProcessor(); + return processor.parse(expr, resolver); + + } 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/test/config/zookeeper/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json deleted file mode 100644 index d34b8a6f0e..0000000000 --- a/metron-analytics/metron-profiler/src/test/config/zookeeper/profiler.json +++ /dev/null @@ -1,13 +0,0 @@ -{ - "inputTopic": "indexing", - "profiles": [ - { - "profile": "profiler-integration-test", - "foreach": "ip_src_addr", - "onlyif": "STARTS_WITH(protocol, 'HTTP')", - "init": { "cnt": 0 }, - "update": { "cnt": "cnt + 1" }, - "result": "cnt" - } - ] -} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json new file mode 100644 index 0000000000..f811df71f1 --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json @@ -0,0 +1,17 @@ +{ + "inputTopic": "indexing", + "profiles": [ + { + "profile": "example1", + "foreach": "ip_src_addr", + "onlyif": "protocol == 'HTTP'", + "init": { + "total_bytes": 0.0 + }, + "update": { + "total_bytes": "total_bytes + bytes_in" + }, + "result": "total_bytes" + } + ] +} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-2/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-2/profiler.json new file mode 100644 index 0000000000..2c6a278ca6 --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-2/profiler.json @@ -0,0 +1,19 @@ +{ + "inputTopic": "indexing", + "profiles": [ + { + "profile": "example2", + "foreach": "ip_src_addr", + "onlyif": "protocol == 'DNS' or protocol == 'HTTP'", + "init": { + "num_dns": 1.0, + "num_http": 1.0 + }, + "update": { + "num_dns": "num_dns + (if protocol == 'DNS' then 1 else 0)", + "num_http": "num_http + (if protocol == 'HTTP' then 1 else 0)" + }, + "result": "num_dns / num_http" + } + ] +} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json new file mode 100644 index 0000000000..96c593525a --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json @@ -0,0 +1,20 @@ +{ + "inputTopic": "indexing", + "profiles": [ + { + "profile": "example3", + "foreach": "ip_src_addr", + "onlyif": "protocol == 'HTTP'", + "init": { + "sum": 0.0, + "cnt": 0.0 + }, + "update": { + "sum": "sum + resp_body_len", + "cnt": "cnt + 1" + }, + "result": "sum / cnt" + } + ] +} + 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 a898b9b45e..7b96a3e6e2 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 @@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.util.Bytes; import org.apache.metron.common.Constants; import org.apache.metron.common.spout.kafka.SpoutConfig; @@ -32,36 +33,36 @@ import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.components.FluxTopologyComponent; import org.apache.metron.integration.components.KafkaWithZKComponent; +import org.apache.metron.profiler.bolt.ProfileHBaseMapper; import org.apache.metron.test.mock.MockHTable; import org.junit.After; -import org.junit.Before; import org.junit.Test; import java.io.File; import java.io.IOException; import java.io.Serializable; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.TimeUnit; +import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.junit.Assert.assertTrue; +import static com.google.code.tempusfugit.temporal.Duration.seconds; +import static com.google.code.tempusfugit.temporal.Timeout.timeout; +import static com.google.code.tempusfugit.temporal.WaitFor.waitOrTimeout; /** * An integration test of the Profiler topology. */ public class ProfilerIntegrationTest extends BaseIntegrationTest { + private static final String TEST_RESOURCES = "../../metron-analytics/metron-profiler/src/test"; private static final String FLUX_PATH = "../metron-profiler/src/main/flux/profiler/remote.yaml"; /** * { * "ip_src_addr": "10.0.0.1", * "protocol": "HTTPS", - * "length": "10" + * "length": 10, + * "bytes_in": 234 * } */ @Multiline @@ -71,12 +72,24 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { * { * "ip_src_addr": "10.0.0.2", * "protocol": "HTTP", - * "length": "20" + * "length": 20, + * "bytes_in": 390 * } */ @Multiline private String message2; + /** + * { + * "ip_src_addr": "10.0.0.3", + * "protocol": "DNS", + * "length": 30, + * "bytes_in": 560 + * } + */ + @Multiline + private String message3; + private FluxTopologyComponent fluxComponent; private KafkaWithZKComponent kafkaComponent; private List input; @@ -86,6 +99,9 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { private static final String tableName = "profiler"; private static final String columnFamily = "cfProfile"; + /** + * A TableProvider that allows us to mock HBase. + */ public static class MockTableProvider implements TableProvider, Serializable { MockHTable.Provider provider = new MockHTable.Provider(); @@ -96,13 +112,12 @@ public HTableInterface getTable(Configuration config, String tableName) throws I } } - @Before - public void setup() throws Exception { + public void setup(String pathToConfig) throws Exception { // create input messages for the profiler to consume - input = Stream.of(message1, message2) + input = Stream.of(message1, message2, message3) .map(Bytes::toBytes) - .map(m -> Collections.nCopies(10, m)) + .map(m -> Collections.nCopies(5, m)) .flatMap(l -> l.stream()) .collect(Collectors.toList()); @@ -130,8 +145,8 @@ public void setup() throws Exception { // upload profiler configuration to zookeeper ConfigUploadComponent configUploadComponent = new ConfigUploadComponent() .withTopologyProperties(topologyProperties) - .withGlobalConfigsPath("../../metron-analytics/metron-profiler/src/test/config/zookeeper") - .withProfilerConfigsPath("../../metron-analytics/metron-profiler/src/test/config/zookeeper"); + .withGlobalConfigsPath(pathToConfig) + .withProfilerConfigsPath(pathToConfig); // load flux definition for the profiler topology fluxComponent = new FluxTopologyComponent.Builder() @@ -159,39 +174,53 @@ public void tearDown() throws Exception { } /** - * Tests the Profiler topology by ensuring that at least one ProfileMeasurement is persisted - * within a mock HBase table. + * Tests the first example contained within the README. */ @Test - public void testProfiler() throws Exception { + public void testExample1() throws Exception { + + setup(TEST_RESOURCES + "/config/zookeeper/readme-example-1"); - // start the topology + // start the topology and write test messages to kafka fluxComponent.submitTopology(); + kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); - // write test messages to the input topic + // verify - ensure the profile is being persisted + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, + timeout(seconds(90))); + } + + /** + * Tests the second example contained within the README. + */ + @Test + public void testExample2() throws Exception { + + setup(TEST_RESOURCES + "/config/zookeeper/readme-example-2"); + + // start the topology and write test messages to kafka + fluxComponent.submitTopology(); kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); + // verify - ensure the profile is being persisted + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, + timeout(seconds(90))); + } - // keep trying to verify until we timeout - a bit ugly, wish JUnit had a mechanism for this - int retry = 0; - int maxRetry = 60; - while(true) { - try { - // verify - ensure that at least one profile measurement was persisted - List puts = profilerTable.getPutLog(); - assertTrue(puts.size() > 0); + /** + * Tests the third example contained within the README. + */ + @Test + public void testExample3() throws Exception { - // success! - break; + setup(TEST_RESOURCES + "/config/zookeeper/readme-example-3"); - } catch (AssertionError e) { - TimeUnit.SECONDS.sleep(3); + // start the topology and write test messages to kafka + fluxComponent.submitTopology(); + kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); - // if too many retries, give up the ghost - if(retry++ >= maxRetry) { - throw new Exception((retry-1) + " retry attempts failed", e); - } - } - } + // verify - ensure the profile is being persisted + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, + timeout(seconds(90))); } } From d9e2c5e568292c4a08c6c6c314d3c2e8a9ea38be Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 15 Aug 2016 16:08:26 -0400 Subject: [PATCH 07/10] METRON-367 Enhance Profiler to Support Multiple Numeric Types --- metron-analytics/metron-profiler/README.md | 6 +- .../metron/profiler/ProfileMeasurement.java | 17 +- .../profiler/bolt/ProfileBuilderBolt.java | 5 +- .../profiler/bolt/ProfileHBaseMapper.java | 31 +++- .../zookeeper/readme-example-3/profiler.json | 2 +- .../zookeeper/write-integer/profiler.json | 13 ++ .../integration/ProfilerIntegrationTest.java | 172 ++++++++++++------ 7 files changed, 176 insertions(+), 70 deletions(-) create mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/write-integer/profiler.json diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md index 904446bbc8..799937dc9f 100644 --- a/metron-analytics/metron-profiler/README.md +++ b/metron-analytics/metron-profiler/README.md @@ -112,7 +112,7 @@ This creates a profile... ### Example 3 -The average response body length of HTTP traffic. The following configuration would be used to generate this profile. +The average of the `length` field of HTTP traffic. The following configuration would be used to generate this profile. ``` { @@ -127,7 +127,7 @@ The average response body length of HTTP traffic. The following configuration wo "cnt": 0.0 }, "update": { - "sum": "sum + resp_body_len", + "sum": "sum + length", "cnt": "cnt + 1" }, "result": "sum / cnt" @@ -140,7 +140,7 @@ This creates a profile... * Named ‘example3’ * That for each IP source address * Only if the 'protocol' field equals 'HTTP' - * Accumulates the sum of response body length + * Accumulates the sum of length * Accumulates the number of messages * Calculates the average as the result 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 e240567991..dc1c0af7a0 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 @@ -51,7 +51,7 @@ public class ProfileMeasurement { /** * The actual measurement itself. */ - private double value; + private Object value; public String getProfileName() { return profileName; @@ -85,11 +85,11 @@ public void setEnd(long end) { this.end = end; } - public double getValue() { + public Object getValue() { return value; } - public void setValue(double value) { + public void setValue(Object value) { this.value = value; } @@ -102,21 +102,18 @@ public boolean equals(Object o) { if (start != that.start) return false; if (end != that.end) return false; - if (Double.compare(that.value, value) != 0) return false; if (profileName != null ? !profileName.equals(that.profileName) : that.profileName != null) return false; - return entity != null ? entity.equals(that.entity) : that.entity == null; + if (entity != null ? !entity.equals(that.entity) : that.entity != null) return false; + return value != null ? value.equals(that.value) : that.value == null; } @Override public int hashCode() { - int result; - long temp; - result = profileName != null ? profileName.hashCode() : 0; + 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)); - temp = Double.doubleToLongBits(value); - result = 31 * result + (int) (temp ^ (temp >>> 32)); + result = 31 * result + (value != null ? value.hashCode() : 0); return result; } 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 c5e166f82a..8fe77fca77 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 @@ -209,10 +209,11 @@ private void flush(Tuple tickTuple) throws IOException { measurement.getProfileName(), measurement.getEntity(), measurement.getStart())); // execute the 'result' expression - Double result; + Object result; try { String resultExpr = profileConfig.getResult(); - result = executor.execute(resultExpr, new JSONObject(), Double.class); + result = executor.execute(resultExpr, new JSONObject(), Object.class); + } catch(ParseException e) { throw new ParseException("Bad 'result' expression", e); } 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 693ca28b0e..5ce2f5fb38 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 @@ -88,11 +88,40 @@ public ColumnList columns(Tuple tuple) { 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, Bytes.toBytes(measurement.getValue())); + cols.addColumn(cfBytes, QVALUE, toBytes(measurement.getValue())); return cols; } + /** + * Serialize a profile measurement's value. + * + * The profile's value could be any numeric data type depending on how it is defined + * by the user. This implementation allows the user some flexibility to choose the + * appropriate data type that best suits their use case. + * + * @param value The value to serialize. + */ + 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); + } + + return result; + } + /** * Calculates a salt value that is used as part of the row key. * diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json index 96c593525a..ce9d0e04c1 100644 --- a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json +++ b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json @@ -10,7 +10,7 @@ "cnt": 0.0 }, "update": { - "sum": "sum + resp_body_len", + "sum": "sum + length", "cnt": "cnt + 1" }, "result": "sum / cnt" diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/write-integer/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/write-integer/profiler.json new file mode 100644 index 0000000000..ae5e6c7057 --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/config/zookeeper/write-integer/profiler.json @@ -0,0 +1,13 @@ +{ + "inputTopic": "indexing", + "profiles": [ + { + "profile": "example1", + "foreach": "ip_src_addr", + "onlyif": "true", + "init": {}, + "update": {}, + "result": "TO_INTEGER(10.0)" + } + ] +} 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 7b96a3e6e2..081d4c1ca2 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 @@ -23,7 +23,7 @@ import org.adrianwalker.multilinestring.Multiline; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.HTableInterface; -import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.util.Bytes; import org.apache.metron.common.Constants; @@ -36,12 +36,16 @@ import org.apache.metron.profiler.bolt.ProfileHBaseMapper; import org.apache.metron.test.mock.MockHTable; import org.junit.After; +import org.junit.Assert; import org.junit.Test; import java.io.File; import java.io.IOException; import java.io.Serializable; -import java.util.*; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -112,6 +116,119 @@ public HTableInterface getTable(Configuration config, String tableName) throws I } } + /** + * Tests the first example contained within the README. + */ + @Test + public void testExample1() throws Exception { + + setup(TEST_RESOURCES + "/config/zookeeper/readme-example-1"); + + // start the topology and write test messages to kafka + fluxComponent.submitTopology(); + kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); + + // verify - ensure the profile is being persisted + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, + timeout(seconds(90))); + + // verify - there are 5 'HTTP' each with 390 bytes + double actual = readDouble(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + Assert.assertEquals(390.0 * 5, actual, 0.01); + } + + /** + * Tests the second example contained within the README. + */ + @Test + public void testExample2() throws Exception { + + setup(TEST_RESOURCES + "/config/zookeeper/readme-example-2"); + + // start the topology and write test messages to kafka + fluxComponent.submitTopology(); + kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); + + // verify - ensure the profile is being persisted + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, + timeout(seconds(90))); + + // verify - there are 5 'HTTP' and 5 'DNS' messages thus 5/5 = 1 + double actual = readDouble(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + Assert.assertEquals(5.0 / 5.0, actual, 0.01); + } + + /** + * Tests the third example contained within the README. + */ + @Test + public void testExample3() throws Exception { + + setup(TEST_RESOURCES + "/config/zookeeper/readme-example-3"); + + // start the topology and write test messages to kafka + fluxComponent.submitTopology(); + kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); + + // verify - ensure the profile is being persisted + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, + timeout(seconds(90))); + + // verify - there are 5 'HTTP' messages each with a length of 20, thus the average should be 20 + double actual = readDouble(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + Assert.assertEquals(20.0, actual, 0.01); + } + + @Test + public void testWriteInteger() throws Exception { + + setup(TEST_RESOURCES + "/config/zookeeper/write-integer"); + + // start the topology and write test messages to kafka + fluxComponent.submitTopology(); + kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); + + // verify - ensure the profile is being persisted + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, + timeout(seconds(90))); + + // verify - there are 5 'HTTP' messages each with a length of 20, thus the average should be 20 + double actual = readInteger(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + Assert.assertEquals(10.0, actual, 0.01); + } + + /** + * Reads a Double value written by the Profiler. + * @param columnFamily The column family. + * @param columnQual The column qualifier. + */ + private Double readDouble(byte[] columnFamily, byte[] columnQual) throws IOException { + ResultScanner scanner = profilerTable.getScanner(columnFamily, columnQual); + + for (Result result : scanner) { + byte[] raw = result.getValue(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + return Bytes.toDouble(raw); + } + + throw new IllegalStateException("No results found"); + } + + /** + * Reads an Integer value written by the Profiler. + * @param columnFamily The column family. + * @param columnQual The column qualifier. + */ + private Integer readInteger(byte[] columnFamily, byte[] columnQual) throws IOException { + ResultScanner scanner = profilerTable.getScanner(columnFamily, columnQual); + + for (Result result : scanner) { + byte[] raw = result.getValue(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + return Bytes.toInt(raw); + } + + throw new IllegalStateException("No results found"); + } + public void setup(String pathToConfig) throws Exception { // create input messages for the profiler to consume @@ -172,55 +289,4 @@ public void tearDown() throws Exception { runner.stop(); } } - - /** - * Tests the first example contained within the README. - */ - @Test - public void testExample1() throws Exception { - - setup(TEST_RESOURCES + "/config/zookeeper/readme-example-1"); - - // start the topology and write test messages to kafka - fluxComponent.submitTopology(); - kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); - - // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); - } - - /** - * Tests the second example contained within the README. - */ - @Test - public void testExample2() throws Exception { - - setup(TEST_RESOURCES + "/config/zookeeper/readme-example-2"); - - // start the topology and write test messages to kafka - fluxComponent.submitTopology(); - kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); - - // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); - } - - /** - * Tests the third example contained within the README. - */ - @Test - public void testExample3() throws Exception { - - setup(TEST_RESOURCES + "/config/zookeeper/readme-example-3"); - - // start the topology and write test messages to kafka - fluxComponent.submitTopology(); - kafkaComponent.writeMessages(Constants.INDEXING_TOPIC, input); - - // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); - } } From 93b59483f8a504cbd95b9bad7e9dd06548e74d21 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 16 Aug 2016 12:51:07 -0400 Subject: [PATCH 08/10] METRON-309 Removed legacy classes from Stellar Unification that are no longer needed --- .../common/dsl/TransformationFunctions.java | 79 -------- .../dsl/functions/ControlFlowFunctions.java | 58 ------ .../common/dsl/functions/MathFunctions.java | 180 ------------------ 3 files changed, 317 deletions(-) delete mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/TransformationFunctions.java delete mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ControlFlowFunctions.java delete mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MathFunctions.java diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/TransformationFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/TransformationFunctions.java deleted file mode 100644 index 68b75b4d79..0000000000 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/TransformationFunctions.java +++ /dev/null @@ -1,79 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.metron.common.dsl; - -import org.apache.metron.common.dsl.functions.*; -import org.apache.metron.common.field.transformation.IPProtocolTransformation; -import org.apache.metron.common.utils.ConversionUtils; - -import java.util.List; -import java.util.function.Function; - -public enum TransformationFunctions implements Function, Object> { - - // string functions - TO_LOWER(strings -> strings.get(0)==null?null:strings.get(0).toString().toLowerCase()), - TO_UPPER(strings -> strings.get(0) == null?null:strings.get(0).toString().toUpperCase()), - TO_STRING(strings -> strings.get(0) == null?null:strings.get(0).toString()), - TO_INTEGER(strings -> strings.get(0) == null?null: ConversionUtils.convert(strings.get(0), Integer.class)), - TO_DOUBLE(strings -> strings.get(0) == null?null: ConversionUtils.convert(strings.get(0), Double.class)), - TO_LONG(strings -> strings.get(0) == null?null: ConversionUtils.convert(strings.get(0), Long.class)), - TRIM(strings -> strings.get(0) == null?null:strings.get(0).toString().trim()), - JOIN(new StringFunctions.JoinFunction()), - SPLIT(new StringFunctions.SplitFunction()), - GET_FIRST(new StringFunctions.GetFirst()), - GET_LAST(new StringFunctions.GetLast()), - GET(new StringFunctions.Get()), - - // map functions - MAP_GET(new MapFunctions.MapGet()), - - // network functions - DOMAIN_TO_TLD(new NetworkFunctions.ExtractTLD()), - DOMAIN_REMOVE_TLD(new NetworkFunctions.RemoveTLD()), - DOMAIN_REMOVE_SUBDOMAINS(new NetworkFunctions.RemoveSubdomains()), - URL_TO_HOST(new NetworkFunctions.URLToHost()), - URL_TO_PORT(new NetworkFunctions.URLToPort()), - URL_TO_PATH(new NetworkFunctions.URLToPath()), - URL_TO_PROTOCOL(new NetworkFunctions.URLToProtocol()), - PROTOCOL_TO_NAME(new IPProtocolTransformation()), - - // timestamp functions - TO_EPOCH_TIMESTAMP(new DateFunctions.ToTimestamp()), - - // math functions - ADD(new MathFunctions.AddFunction()), - SUB(new MathFunctions.SubtractFunction()), - MULT(new MathFunctions.MultiplyFunction()), - DIV(new MathFunctions.DivideFunction()), - - // control flow functions - IF_THEN_ELSE(new ControlFlowFunctions.IfThenElseFunction()), - EQUAL(new ControlFlowFunctions.EqualsFunction()); - - Function, Object> func; - TransformationFunctions(Function, Object> func) { - this.func = func; - } - - @Override - public Object apply(List input) { - return func.apply(input); - } -} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ControlFlowFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ControlFlowFunctions.java deleted file mode 100644 index b20dfdd79a..0000000000 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/ControlFlowFunctions.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package org.apache.metron.common.dsl.functions; - -import org.apache.metron.common.utils.ConversionUtils; - -import java.util.List; -import java.util.function.Function; - -/** - * Functions that provide functionality related to control flow. - */ -public class ControlFlowFunctions { - - public static class IfThenElseFunction implements Function, Object> { - @Override - public Object apply(List args) { - - Object ifExpression = args.get(0); - Object thenExpression = args.get(1); - Object elseExpression = args.get(2); - - Boolean ifCondition = ConversionUtils.convert(ifExpression, Boolean.class); - if(ifCondition) { - return thenExpression; - } else { - return elseExpression; - } - } - } - - public static class EqualsFunction implements Function, Object> { - @Override - public Object apply(List args) { - Object left = args.get(0); - Object right = args.get(1); - return left.equals(right); - } - } -} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MathFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MathFunctions.java deleted file mode 100644 index 8be7c2078d..0000000000 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/dsl/functions/MathFunctions.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package org.apache.metron.common.dsl.functions; - -import org.apache.metron.common.utils.ConversionUtils; - -import java.util.List; -import java.util.function.Function; - -/** - * Implements basic math functions. - */ -public class MathFunctions { - - public static class AddFunction implements Function, Object> { - @Override - public Object apply(List args) { - - if(args.get(0) instanceof Double) { - Double left = ConversionUtils.convert(args.get(0), Double.class); - Double right = ConversionUtils.convert(args.get(1), Double.class); - return left + right; - - } else if(args.get(0) instanceof Float) { - Float left = ConversionUtils.convert(args.get(0), Float.class); - Float right = ConversionUtils.convert(args.get(1), Float.class); - return left + right; - - } else if(args.get(0) instanceof Long) { - Long left = ConversionUtils.convert(args.get(0), Long.class); - Long right = ConversionUtils.convert(args.get(1), Long.class); - return left + right; - - } else if(args.get(0) instanceof Integer) { - Integer left = ConversionUtils.convert(args.get(0), Integer.class); - Integer right = ConversionUtils.convert(args.get(1), Integer.class); - return left + right; - - } else if(args.get(0) instanceof Short) { - Short left = ConversionUtils.convert(args.get(0), Short.class); - Short right = ConversionUtils.convert(args.get(1), Short.class); - return left + right; - - } else { - throw new RuntimeException(String.format( - "Cannot apply AddFunction to '%s'", - args.get(0).getClass().getSimpleName())); - } - } - } - - public static class SubtractFunction implements Function, Object> { - @Override - public Object apply(List args) { - - if(args.get(0) instanceof Double) { - Double left = ConversionUtils.convert(args.get(0), Double.class); - Double right = ConversionUtils.convert(args.get(1), Double.class); - return left - right; - - } else if(args.get(0) instanceof Float) { - Float left = ConversionUtils.convert(args.get(0), Float.class); - Float right = ConversionUtils.convert(args.get(1), Float.class); - return left - right; - - } else if(args.get(0) instanceof Long) { - Long left = ConversionUtils.convert(args.get(0), Long.class); - Long right = ConversionUtils.convert(args.get(1), Long.class); - return left - right; - - } else if(args.get(0) instanceof Integer) { - Integer left = ConversionUtils.convert(args.get(0), Integer.class); - Integer right = ConversionUtils.convert(args.get(1), Integer.class); - return left - right; - - } else if(args.get(0) instanceof Short) { - Short left = ConversionUtils.convert(args.get(0), Short.class); - Short right = ConversionUtils.convert(args.get(1), Short.class); - return left - right; - - } else { - throw new RuntimeException(String.format( - "Cannot apply SubtractFunction to '%s'", - args.get(0).getClass().getSimpleName())); - } - } - } - - public static class MultiplyFunction implements Function, Object> { - @Override - public Object apply(List args) { - - if(args.get(0) instanceof Double) { - Double left = ConversionUtils.convert(args.get(0), Double.class); - Double right = ConversionUtils.convert(args.get(1), Double.class); - return left * right; - - } else if(args.get(0) instanceof Float) { - Float left = ConversionUtils.convert(args.get(0), Float.class); - Float right = ConversionUtils.convert(args.get(1), Float.class); - return left * right; - - } else if(args.get(0) instanceof Long) { - Long left = ConversionUtils.convert(args.get(0), Long.class); - Long right = ConversionUtils.convert(args.get(1), Long.class); - return left * right; - - } else if(args.get(0) instanceof Integer) { - Integer left = ConversionUtils.convert(args.get(0), Integer.class); - Integer right = ConversionUtils.convert(args.get(1), Integer.class); - return left * right; - - } else if(args.get(0) instanceof Short) { - Short left = ConversionUtils.convert(args.get(0), Short.class); - Short right = ConversionUtils.convert(args.get(1), Short.class); - return left * right; - - } else { - throw new RuntimeException(String.format( - "Cannot apply MultiplyFunction to '%s'", - args.get(0).getClass().getSimpleName())); - } - } - } - - public static class DivideFunction implements Function, Object> { - @Override - public Object apply(List args) { - - if(args.get(0) instanceof Double) { - Double left = ConversionUtils.convert(args.get(0), Double.class); - Double right = ConversionUtils.convert(args.get(1), Double.class); - return left / right; - - } else if(args.get(0) instanceof Float) { - Float left = ConversionUtils.convert(args.get(0), Float.class); - Float right = ConversionUtils.convert(args.get(1), Float.class); - return left / right; - - } else if(args.get(0) instanceof Long) { - Long left = ConversionUtils.convert(args.get(0), Long.class); - Long right = ConversionUtils.convert(args.get(1), Long.class); - return left / right; - - } else if(args.get(0) instanceof Integer) { - Integer left = ConversionUtils.convert(args.get(0), Integer.class); - Integer right = ConversionUtils.convert(args.get(1), Integer.class); - return left / right; - - } else if(args.get(0) instanceof Short) { - Short left = ConversionUtils.convert(args.get(0), Short.class); - Short right = ConversionUtils.convert(args.get(1), Short.class); - return left / right; - - } else { - throw new RuntimeException(String.format( - "Cannot apply DivideFunction to '%s'", - args.get(0).getClass().getSimpleName())); - } - } - } -} From 5ddd08e8bbb84353b06acff0dc1fb5d8521f08bc Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 22 Aug 2016 13:17:39 -0400 Subject: [PATCH 09/10] METRON-387 Fix Bad Merge with METRON-309 --- .../metron/profiler/stellar/DefaultStellarExecutor.java | 5 +++-- metron-analytics/pom.xml | 7 +++++++ 2 files changed, 10 insertions(+), 2 deletions(-) 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 d608222b6e..a45e4a79e4 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 @@ -20,10 +20,11 @@ package org.apache.metron.profiler.stellar; +import org.apache.metron.common.dsl.Context; import org.apache.metron.common.dsl.MapVariableResolver; import org.apache.metron.common.dsl.ParseException; +import org.apache.metron.common.dsl.StellarFunctions; import org.apache.metron.common.dsl.VariableResolver; -import org.apache.metron.common.stellar.StellarPredicateProcessor; import org.apache.metron.common.stellar.StellarProcessor; import org.apache.metron.common.utils.ConversionUtils; import org.json.simple.JSONObject; @@ -108,7 +109,7 @@ private Object execute(String expr, JSONObject msg) { try { VariableResolver resolver = new MapVariableResolver(state, msg); StellarProcessor processor = new StellarProcessor(); - return processor.parse(expr, resolver); + return processor.parse(expr, resolver, StellarFunctions.FUNCTION_RESOLVER(), Context.EMPTY_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/pom.xml b/metron-analytics/pom.xml index 3b4f5bdd48..ca20912c76 100644 --- a/metron-analytics/pom.xml +++ b/metron-analytics/pom.xml @@ -115,5 +115,12 @@ false + + sonatype.org + https://oss.sonatype.org/content/groups/public/ + + true + + From 8e3a14a78ebe069a14e7f0f8638584b7e8ba0911 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 22 Aug 2016 14:06:39 -0400 Subject: [PATCH 10/10] METRON-367 Fixed integration test based on changes in METRON-309 --- .../integration/ProfilerIntegrationTest.java | 22 +++++++++---------- 1 file changed, 10 insertions(+), 12 deletions(-) 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 3a1b53d08f..64d5bd8f87 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 @@ -133,7 +133,7 @@ public void testExample1() throws Exception { timeout(seconds(90))); // verify - there are 5 'HTTP' each with 390 bytes - double actual = readDouble(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + double actual = readDouble(ProfileHBaseMapper.QVALUE); Assert.assertEquals(390.0 * 5, actual, 0.01); } @@ -154,7 +154,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(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + double actual = readDouble(ProfileHBaseMapper.QVALUE); Assert.assertEquals(5.0 / 5.0, actual, 0.01); } @@ -175,7 +175,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(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + double actual = readDouble(ProfileHBaseMapper.QVALUE); Assert.assertEquals(20.0, actual, 0.01); } @@ -193,20 +193,19 @@ 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(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + double actual = readInteger(ProfileHBaseMapper.QVALUE); Assert.assertEquals(10.0, actual, 0.01); } /** * Reads a Double value written by the Profiler. - * @param columnFamily The column family. * @param columnQual The column qualifier. */ - private Double readDouble(byte[] columnFamily, byte[] columnQual) throws IOException { - ResultScanner scanner = profilerTable.getScanner(columnFamily, columnQual); + private Double readDouble(byte[] columnQual) throws IOException { + ResultScanner scanner = profilerTable.getScanner(Bytes.toBytes(columnFamily), columnQual); for (Result result : scanner) { - byte[] raw = result.getValue(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + byte[] raw = result.getValue(Bytes.toBytes(columnFamily), ProfileHBaseMapper.QVALUE); return Bytes.toDouble(raw); } @@ -215,14 +214,13 @@ private Double readDouble(byte[] columnFamily, byte[] columnQual) throws IOExcep /** * Reads an Integer value written by the Profiler. - * @param columnFamily The column family. * @param columnQual The column qualifier. */ - private Integer readInteger(byte[] columnFamily, byte[] columnQual) throws IOException { - ResultScanner scanner = profilerTable.getScanner(columnFamily, columnQual); + private Integer readInteger(byte[] columnQual) throws IOException { + ResultScanner scanner = profilerTable.getScanner(Bytes.toBytes(columnFamily), columnQual); for (Result result : scanner) { - byte[] raw = result.getValue(Bytes.toBytes("cfProfile"), ProfileHBaseMapper.QVALUE); + byte[] raw = result.getValue(Bytes.toBytes(columnFamily), ProfileHBaseMapper.QVALUE); return Bytes.toInt(raw); }