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

rmerriman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/metron.git


The following commit(s) were added to refs/heads/master by this push:
     new c453b1e  METRON-2065 Setting Parser Output Topic in Sensor Config is 
broken (merrimanr) closes apache/metron#1377
c453b1e is described below

commit c453b1edf27a4a2883e4c055246d939218908ebb
Author: merrimanr <[email protected]>
AuthorDate: Thu Apr 18 08:19:12 2019 -0500

    METRON-2065 Setting Parser Output Topic in Sensor Config is broken 
(merrimanr) closes apache/metron#1377
---
 metron-platform/metron-parsing/README.md           |  14 ++
 .../parsers/topology/ParserTopologyBuilder.java    |  23 ++-
 .../topology/ParserTopologyBuilderTest.java        | 169 +++++++++++++++++++++
 3 files changed, 202 insertions(+), 4 deletions(-)

diff --git a/metron-platform/metron-parsing/README.md 
b/metron-platform/metron-parsing/README.md
index e5368fe..085fb33 100644
--- a/metron-platform/metron-parsing/README.md
+++ b/metron-platform/metron-parsing/README.md
@@ -158,6 +158,19 @@ There are two general types types of parsers:
         ```
         "timestamp":"TO_EPOCH_TIMESTAMP(timestamp_str, timestamp_format, 
timezone_name )"
         ```
+        
+## Parser Message Routing
+
+Messages are routed to the Kafka `enrichments` topic by default.  The output 
topic can be changed with the `output_topic` 
+option when [Starting the Parser 
Topology](metron-parsing-storm/README.md#starting-the-parser-topology) or with 
the `outputTopic` 
+[Parser Configuration](#parser-configuration) setting.  The order of 
precedence from highest to lowest is as follows:
+
+1. Parser start script option
+2. Parser configuration setting
+3. Default `enrichments` topic
+
+A message can also be routed to other locations besides Kafka with the 
`writerClassName` [Parser Configuration](#parser-configuration) setting.
+Messages can be routed independently for each sensor type when configured with 
[Parser Configuration](#parser-configuration) settings.
 
 ## Parser Error Routing
 
@@ -278,6 +291,7 @@ The document is structured in the following way
         }
         ```
 
+* `writerClassName` : The class used to write messages after they have been 
parsed.  Defaults to `org.apache.metron.writer.kafka.KafkaWriter`.
 * `sensorTopic` : The kafka topic to that the parser will read messages from.  
If the topic is prefixed and suffixed by `/`
 then it is assumed to be a regex and will match any topic matching the pattern 
(e.g. `/bro.*/` would match `bro_cust0`, `bro_cust1` and `bro_cust2`)
 * `readMetadata` : Boolean indicating whether to read metadata or not (The 
default is raw message strategy dependent).  See below for a discussion about 
metadata.
diff --git 
a/metron-platform/metron-parsing/metron-parsing-storm/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java
 
b/metron-platform/metron-parsing/metron-parsing-storm/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java
index 9dc7b88..b0f9d7d 100644
--- 
a/metron-platform/metron-parsing/metron-parsing-storm/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java
+++ 
b/metron-platform/metron-parsing/metron-parsing-storm/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java
@@ -228,7 +228,7 @@ public class ParserTopologyBuilder {
    * @param securityProtocol An optional security protocol in use.
    * @return
    */
-  private static KafkaWriter createKafkaWriter(Optional<String> broker,
+  protected static KafkaWriter createKafkaWriter(Optional<String> broker,
                                                String zkQuorum,
                                                Optional<String> 
securityProtocol) {
     KafkaWriter writer = new KafkaWriter();
@@ -266,6 +266,21 @@ public class ParserTopologyBuilder {
                                               Optional<String> 
securityProtocol,
                                               ParserConfigurations configs,
                                               Optional<String> outputTopic) {
+    Map<String, WriterHandler> writerConfigs = 
createWriterConfigs(zookeeperUrl,
+            brokerUrl,
+            sensorTypeToParserConfig,
+            securityProtocol,
+            configs,
+            outputTopic);
+    return new ParserBolt(zookeeperUrl, new ParserRunnerImpl(new 
HashSet<>(sensorTypeToParserConfig.keySet())), writerConfigs);
+  }
+
+  protected static Map<String, WriterHandler> createWriterConfigs(String 
zookeeperUrl,
+                                                                  
Optional<String> brokerUrl,
+                                                                  Map<String, 
SensorParserConfig> sensorTypeToParserConfig,
+                                                                  
Optional<String> securityProtocol,
+                                                                  
ParserConfigurations configs,
+                                                                  
Optional<String> outputTopic) {
     Map<String, WriterHandler> writerConfigs = new HashMap<>();
     for( Entry<String, SensorParserConfig> entry : 
sensorTypeToParserConfig.entrySet()) {
       String sensorType = entry.getKey();
@@ -276,7 +291,8 @@ public class ParserTopologyBuilder {
       if (parserConfig.getWriterClassName() == null) {
         // if not configured, use a sensible default
         writer = createKafkaWriter(brokerUrl, zookeeperUrl, securityProtocol)
-            .withTopic(outputTopic.orElse(Constants.ENRICHMENT_TOPIC));
+                .withTopic(outputTopic.orElse(
+                        parserConfig.getOutputTopic() != null ? 
parserConfig.getOutputTopic() : Constants.ENRICHMENT_TOPIC));
 
       } else {
         writer = 
ReflectionUtils.createInstance(parserConfig.getWriterClassName());
@@ -289,8 +305,7 @@ public class ParserTopologyBuilder {
       WriterHandler writerHandler = createWriterHandler(writer);
       writerConfigs.put(sensorType, writerHandler);
     }
-
-    return new ParserBolt(zookeeperUrl, new ParserRunnerImpl(new 
HashSet<>(sensorTypeToParserConfig.keySet())), writerConfigs);
+    return writerConfigs;
   }
 
   /**
diff --git 
a/metron-platform/metron-parsing/metron-parsing-storm/src/test/java/org/apache/metron/parsers/topology/ParserTopologyBuilderTest.java
 
b/metron-platform/metron-parsing/metron-parsing-storm/src/test/java/org/apache/metron/parsers/topology/ParserTopologyBuilderTest.java
new file mode 100644
index 0000000..15ec266
--- /dev/null
+++ 
b/metron-platform/metron-parsing/metron-parsing-storm/src/test/java/org/apache/metron/parsers/topology/ParserTopologyBuilderTest.java
@@ -0,0 +1,169 @@
+/**
+ * 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.parsers.topology;
+
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.configuration.ParserConfigurations;
+import org.apache.metron.common.configuration.SensorParserConfig;
+import org.apache.metron.common.configuration.writer.ParserWriterConfiguration;
+import org.apache.metron.parsers.bolt.WriterHandler;
+import org.apache.metron.writer.NoopWriter;
+import org.apache.metron.writer.kafka.KafkaWriter;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.powermock.api.mockito.PowerMockito.spy;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(ParserTopologyBuilder.class)
+public class ParserTopologyBuilderTest {
+
+  @Mock
+  private ParserConfigurations configs;
+
+  @Mock
+  private KafkaWriter kafkaWriter;
+
+  @Before
+  public void setup() {
+    spy(ParserTopologyBuilder.class);
+    when(ParserTopologyBuilder.createKafkaWriter(Optional.of("brokerUrl"), 
"zookeeperUrl", Optional.of("securityProtocol")))
+            .thenReturn(kafkaWriter);
+  }
+
+  @Test
+  public void shouldCreateWriterConfig() {
+    SensorParserConfig broConfig = new SensorParserConfig();
+    broConfig.setSensorTopic("bro");
+    when(configs.getSensorParserConfig("bro")).thenReturn(broConfig);
+    KafkaWriter enrichmentWriter = mock(KafkaWriter.class);
+    
when(kafkaWriter.withTopic(Constants.ENRICHMENT_TOPIC)).thenReturn(enrichmentWriter);
+
+    Map<String, SensorParserConfig> sensorTypeToParserConfig = new 
HashMap<String, SensorParserConfig>() {{
+      put("bro", broConfig);
+    }};
+
+    Map<String, WriterHandler> writerConfigs = ParserTopologyBuilder
+            .createWriterConfigs("zookeeperUrl",
+                    Optional.of("brokerUrl"),
+                    sensorTypeToParserConfig,
+                    Optional.of("securityProtocol"),
+                    configs,
+                    Optional.empty());
+
+    assertEquals(1, writerConfigs.size());
+    assertEquals(enrichmentWriter, 
writerConfigs.get("bro").getBulkMessageWriter());
+    verify(enrichmentWriter, times(1)).configure(eq("bro"), 
any(ParserWriterConfiguration.class));
+    verifyNoMoreInteractions(enrichmentWriter);
+  }
+
+  @Test
+  public void shouldCreateWriterConfigWithSensorParserConfigOutputTopic() {
+    SensorParserConfig snortConfig = new SensorParserConfig();
+    snortConfig.setSensorTopic("snort");
+    snortConfig.setOutputTopic("snort_topic");
+    when(configs.getSensorParserConfig("snort")).thenReturn(snortConfig);
+    KafkaWriter snortTestWriter = mock(KafkaWriter.class);
+    when(kafkaWriter.withTopic("snort_topic")).thenReturn(snortTestWriter);
+
+    Map<String, SensorParserConfig> sensorTypeToParserConfig = new 
HashMap<String, SensorParserConfig>() {{
+      put("snort", snortConfig);
+    }};
+
+    Map<String, WriterHandler> writerConfigs = ParserTopologyBuilder
+            .createWriterConfigs("zookeeperUrl",
+                    Optional.of("brokerUrl"),
+                    sensorTypeToParserConfig,
+                    Optional.of("securityProtocol"),
+                    configs,
+                    Optional.empty());
+
+    assertEquals(1, writerConfigs.size());
+    assertEquals(snortTestWriter, 
writerConfigs.get("snort").getBulkMessageWriter());
+    verify(snortTestWriter, times(1)).configure(eq("snort"), 
any(ParserWriterConfiguration.class));
+    verifyNoMoreInteractions(snortTestWriter);
+  }
+
+  @Test
+  public void shouldCreateWriterConfigWithSuppliedOutputTopic() {
+    SensorParserConfig snortConfig = new SensorParserConfig();
+    snortConfig.setSensorTopic("snort");
+    when(configs.getSensorParserConfig("snort")).thenReturn(snortConfig);
+    KafkaWriter suppliedTopicWriter = mock(KafkaWriter.class);
+    
when(kafkaWriter.withTopic("supplied_topic")).thenReturn(suppliedTopicWriter);
+
+    Map<String, SensorParserConfig> sensorTypeToParserConfig = new 
HashMap<String, SensorParserConfig>() {{
+      put("snort", snortConfig);
+    }};
+
+    Map<String, WriterHandler> writerConfigs = ParserTopologyBuilder
+            .createWriterConfigs("zookeeperUrl",
+                    Optional.of("brokerUrl"),
+                    sensorTypeToParserConfig,
+                    Optional.of("securityProtocol"),
+                    configs,
+                    Optional.of("supplied_topic"));
+
+    assertEquals(1, writerConfigs.size());
+    assertEquals(suppliedTopicWriter, 
writerConfigs.get("snort").getBulkMessageWriter());
+    verify(suppliedTopicWriter, times(1)).configure(eq("snort"), 
any(ParserWriterConfiguration.class));
+    verifyNoMoreInteractions(suppliedTopicWriter);
+  }
+
+  @Test
+  public void shouldCreateWriterConfigWithWriterClassName() {
+    SensorParserConfig yafConfig = new SensorParserConfig();
+    yafConfig.setSensorTopic("yaf");
+    yafConfig.setWriterClassName("org.apache.metron.writer.NoopWriter");
+    when(configs.getSensorParserConfig("yaf")).thenReturn(yafConfig);
+
+    Map<String, SensorParserConfig> sensorTypeToParserConfig = new 
HashMap<String, SensorParserConfig>() {{
+      put("yaf", yafConfig);
+    }};
+
+    Map<String, WriterHandler> writerConfigs = ParserTopologyBuilder
+            .createWriterConfigs("zookeeperUrl",
+                    Optional.of("brokerUrl"),
+                    sensorTypeToParserConfig,
+                    Optional.of("securityProtocol"),
+                    configs,
+                    Optional.empty());
+
+    assertEquals(1, writerConfigs.size());
+    assertTrue(writerConfigs.get("yaf").getBulkMessageWriter() instanceof 
NoopWriter);
+  }
+
+}

Reply via email to