You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@metron.apache.org by cestella <gi...@git.apache.org> on 2017/10/04 18:40:06 UTC

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

GitHub user cestella opened a pull request:

    https://github.com/apache/metron/pull/785

    METRON-1230: As a stopgap prior to METRON-777, add more simplistic sideloading of custom Parsers

    ## Contributor Comments
    Until we get METRON-777 in, it'd be nice to have a simple ability using the normal storm functionality to enable users to provide custom parsers without forking Metron. This should be done via simply creating a jar with their code (and bundled dependencies) and have it picked up and available to the REST Service (and consequently the management UI) as well as the start_parser_topology.sh command.
    
    This should be minimal movement as we have a more robust solution coming with METRON-777.
    
    To test this, please look at and execute the examples in the 3rdPartyParsers.md document included as part of this PR.
    
    ## Pull Request Checklist
    
    Thank you for submitting a contribution to Apache Metron.  
    Please refer to our [Development Guidelines](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=61332235) for the complete guide to follow for contributions.  
    Please refer also to our [Build Verification Guidelines](https://cwiki.apache.org/confluence/display/METRON/Verifying+Builds?show-miniview) for complete smoke testing guides.  
    
    
    In order to streamline the review of the contribution we ask you follow these guidelines and ask you to double check the following:
    
    ### For all changes:
    - [x] Is there a JIRA ticket associated with this PR? If not one needs to be created at [Metron Jira](https://issues.apache.org/jira/browse/METRON/?selectedTab=com.atlassian.jira.jira-projects-plugin:summary-panel). 
    - [x] Does your PR title start with METRON-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    - [x] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    
    ### For code changes:
    - [x] Have you included steps to reproduce the behavior or problem that is being changed or addressed?
    - [x] Have you included steps or a guide to how the change may be verified and tested manually?
    - [x] Have you ensured that the full suite of tests and checks have been executed in the root metron folder via:
      ```
      mvn -q clean integration-test install && build_utils/verify_licenses.sh 
      ```
    
    - [x] Have you written or updated unit tests and or integration tests to verify your changes?
    - [x] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
    - [x] Have you verified the basic functionality of the build by building and running locally with Vagrant full-dev environment or the equivalent?
    
    ### For documentation related changes:
    - [x] Have you ensured that format looks appropriate for the output in which it is rendered by building and verifying the site-book? If not then run the following commands and the verify changes via `site-book/target/site/index.html`:
    
      ```
      cd site-book
      mvn site
      ```
    
    #### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.
    It is also recommended that [travis-ci](https://travis-ci.org) is set up for your personal repository such that your branches are built there before submitting a pull request.
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/cestella/incubator-metron 3rd_party_parsers

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/metron/pull/785.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #785
    
----

----


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/metron/pull/785


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143547079
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/MergeAndShadeTransformer.java ---
    @@ -0,0 +1,90 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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 com.google.common.base.Splitter;
    +import org.apache.storm.daemon.JarTransformer;
    +import org.apache.storm.hack.StormShadeTransformer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.*;
    +import java.lang.invoke.MethodHandles;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.jar.JarEntry;
    +import java.util.jar.JarInputStream;
    +import java.util.jar.JarOutputStream;
    +
    +/**
    + * This is a storm jar transformer that will add in additional jars pulled from an
    + * environment variable.  The jars will be merged with the main uber jar and then
    + * the resulting jar will be shaded and relocated according to the StormShadeTransformer.
    + *
    + */
    +public class MergeAndShadeTransformer implements JarTransformer {
    +  public static final String EXTRA_JARS_ENV = "EXTRA_JARS";
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  StormShadeTransformer _underlyingTransformer = new StormShadeTransformer();
    --- End diff --
    
    Small nit, but is there a specific reason we're adding an underscore and default visibility for this member variable?


---

[GitHub] metron issue #785: METRON-1230: As a stopgap prior to METRON-777, add more s...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on the issue:

    https://github.com/apache/metron/pull/785
  
    The only real change here is the addition of a Transformer that hooks into Storm's `storm jar` command and will merge the dependent jars.  This was done so that we could avoid touching the Parser infrastructure and have as minimal impact on the effort around 777 as possible.  In order to remove this functionality, we would need only revert the shell scripts.
    
    The remainder of the functionality is cleaning up a bit in the REST code.


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143552937
  
    --- Diff: metron-interface/metron-rest/pom.xml ---
    @@ -163,6 +163,7 @@
                 <groupId>org.apache.metron</groupId>
                 <artifactId>metron-parsers</artifactId>
                 <version>${project.parent.version}</version>
    +            <scope>provided</scope>
    --- End diff --
    
    Yep, there's no reason to do that, the REST API can load dependencies from the classpath.


---

[GitHub] metron issue #785: METRON-1230: As a stopgap prior to METRON-777, add more s...

Posted by ottobackwards <gi...@git.apache.org>.
Github user ottobackwards commented on the issue:

    https://github.com/apache/metron/pull/785
  
    So, when 777 hits, this is going to be removed?


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r155033012
  
    --- Diff: metron-platform/metron-parsers/3rdPartyParser.md ---
    @@ -0,0 +1,306 @@
    +# Custom Metron Parsers
    +
    +We have many stock parsers for normal operations.  Some of these are
    +networking and cybersecurity focused (e.g. the ASA Parser), some of
    +these are general purpose (e.g. the CSVParser), but inevitably users
    +will want to extend the system to process their own data formats.  To
    +enable this, this is a walkthrough of how to create and use a custom
    +parser within Metron.
    +
    +# Writing A Custom Parser
    +Before we can use a parser, we will need to create a custom parser.  The
    +parser is the workhorse of Metron ingest.  It provides the mapping
    +between the raw data coming in via the Kafka value and a `JSONObject`,
    +the internal data structure provided.
    +
    +## Implementation
    +
    +In order to do create a custom parser, we need to do one of the following:
    +* Write a class which conforms to the `org.apache.metron.parsers.interfaces.MessageParser<JSONObject>` and `java.util.Serializable` interfaces
    +  * Implement `init()`, `validate(JSONObject message)`, and `List<JSONObject> parse(byte[] rawMessage)`
    +* Write a class which extends `org.apache.metron.parsers.BasicParser`
    +  * Provides convenience implementations to `validate` which ensures `timestamp` and `original_string` fields exist.
    +
    +## Example
    +
    +In order to illustrate how this might be done, let's create a very
    +simple parser that takes a comma separated pair and creates a couple of
    +fields:
    +* `original_string` -- the raw data
    +* `timestamp` -- the current time
    +* `first` -- the first field of the comma separated pair
    +* `last` -- the last field of the comma separated pair
    +
    +For this demonstration, let's create a maven project to compile our
    +project.  We'll call it `extra_parsers`, so in your workspace, let's set
    +up the maven project:
    +* Create the maven infrastructure for `extra_parsers` via
    +```
    +mkdir -p extra_parsers/src/{main,test}/java
    +```
    +* Create a pom file indicating how we should build our parsers by
    +  editing `extra_parsers/pom.xml` with the following content:
    +```
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <groupId>com.3rdparty</groupId>
    +  <artifactId>extra-parsers</artifactId>
    +  <packaging>jar</packaging>
    +  <version>1.0-SNAPSHOT</version>
    +  <name>extra-parsers</name>
    +  <url>http://thirdpartysoftware.org</url>
    +  <properties>
    +    <!-- The java version to conform to.  Metron works all the way to 1.8 -->
    +    <java_version>1.8</java_version>
    +    <!-- The version of Metron that we'll be targetting. -->
    +    <metron_version>0.4.1</metron_version>
    +    <!-- To complete the simulation, we'll depend on a common dependency -->
    +    <guava_version>19.0</guava_version>
    +    <!-- We will shade our dependencies to create a single jar at the end -->
    +    <shade_version>2.4.3</shade_version>
    +  </properties>
    +  <dependencies>
    +    <!--
    +    We want to depend on Metron, but ensure that the scope is "provided"
    +    as we do not want to include it in our bundle.
    +    -->
    +    <dependency>
    +      <groupId>org.apache.metron</groupId>
    +      <artifactId>metron-parsers</artifactId>
    +      <version>${metron_version}</version>
    +      <scope>provided</scope>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.google.guava</groupId>
    +      <artifactId>guava</artifactId>
    +      <version>${guava_version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>junit</groupId>
    +      <artifactId>junit</artifactId>
    +      <version>3.8.1</version>
    +      <scope>test</scope>
    +    </dependency>
    +  </dependencies>
    +  <build>
    +    <plugins>
    +     <!-- We will set up the shade plugin to create a single jar at the
    +           end of the build lifecycle.  We will exclude some things and
    +           relocate others to simulate a real situation.
    +           
    +           One thing to note is that it's a good practice to shade and
    +           relocate common libraries that may be dependencies in Metron.
    +           Your jar will be merged with the parsers jar, so the metron
    +           version will be included for all overlapping classes.
    +           So, shade and relocate to ensure that YOUR version of the library is used.
    +      -->
    +
    +      <plugin>
    +        <groupId>org.apache.maven.plugins</groupId>
    +        <artifactId>maven-shade-plugin</artifactId>
    +        <version>${shade_version}</version>
    +        <configuration>
    +          <createDependencyReducedPom>true</createDependencyReducedPom>
    +          <artifactSet>
    +            <excludes>
    +              <!-- Exclude slf4j for no reason other than to illustrate how to exclude dependencies.
    +                   The metron team has nothing against slf4j. :-)
    +               -->
    +              <exclude>*slf4j*</exclude>
    +            </excludes>
    +          </artifactSet>
    +        </configuration>
    +        <executions>
    +          <execution>
    +            <phase>package</phase>
    +            <goals>
    +              <goal>shade</goal>
    +            </goals>
    +            <configuration>
    +              <shadedArtifactAttached>true</shadedArtifactAttached>
    +              <shadedClassifierName>uber</shadedClassifierName>
    +              <filters>
    +                <filter>
    +                  <!-- Sometimes these get added and confuse the uber jar out of shade -->
    +                  <artifact>*:*</artifact>
    +                  <excludes>
    +                    <exclude>META-INF/*.SF</exclude>
    +                    <exclude>META-INF/*.DSA</exclude>
    +                    <exclude>META-INF/*.RSA</exclude>
    +                  </excludes>
    +                </filter>
    +              </filters>
    +              <relocations>
    +                <!-- Relocate guava as it's used in Metron and I really want 0.19 -->
    +                <relocation>
    +                  <pattern>com.google</pattern>
    +                  <shadedPattern>com.thirdparty.guava</shadedPattern>
    +                </relocation>
    +              </relocations>
    +              <artifactSet>
    +                <excludes>
    +                  <!-- We can also exclude by artifactId and groupId -->
    +                  <exclude>storm:storm-core:*</exclude>
    +                  <exclude>storm:storm-lib:*</exclude>
    +                  <exclude>org.slf4j.impl*</exclude>
    +                  <exclude>org.slf4j:slf4j-log4j*</exclude>
    +                </excludes>
    +              </artifactSet>
    +            </configuration>
    +          </execution>
    +        </executions>
    +      </plugin>
    +      <!--
    +      We want to make sure we compile using java 1.8.
    +      -->
    +      <plugin>
    +        <groupId>org.apache.maven.plugins</groupId>
    +        <artifactId>maven-compiler-plugin</artifactId>
    +        <version>3.5.1</version>
    +        <configuration>
    +          <forceJavacCompilerUse>true</forceJavacCompilerUse>
    +          <source>${java_version}</source>
    +          <compilerArgument>-Xlint:unchecked</compilerArgument>
    +          <target>${java_version}</target>
    +          <showWarnings>true</showWarnings>
    +        </configuration>
    +      </plugin>
    +    </plugins>
    +  </build>
    +</project>
    +```
    +* Now let's create our parser  `com.thirdparty.SimpleParser` by creating the file `extra-parsers/src/main/java/com/thirdparty/SimpleParser.java` with the following content:
    +```
    +package com.thirdparty;
    +
    +import com.google.common.base.Splitter;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Iterables;
    +import org.apache.metron.parsers.BasicParser;
    +import org.json.simple.JSONObject;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class SimpleParser extends BasicParser {
    +  @Override
    +  public void init() {
    +
    +  }
    +
    +  @Override
    +  public List<JSONObject> parse(byte[] bytes) {
    +    String input = new String(bytes);
    +    Iterable<String> it = Splitter.on(",").split(input);
    +    JSONObject ret = new JSONObject();
    +    ret.put("original_string", input);
    +    ret.put("timestamp", System.currentTimeMillis());
    +    ret.put("first", Iterables.getFirst(it, "missing"));
    +    ret.put("last", Iterables.getLast(it, "missing"));
    +    return ImmutableList.of(ret);
    +  }
    +
    +  @Override
    +  public void configure(Map<String, Object> map) {
    +
    +  }
    +}
    +```
    +* Compile the parser via `mvn clean package` in `extra_parsers`
    +
    +This will create a jar containing your parser and its dependencies (sans Metron dependencies) in `extra-parsers/target/extra-parsers-1.0-SNAPSHOT-uber.jar`
    +
    +# Deploying Your Custom Parser
    +
    +In order to deploy your newly built custom parser, you would place the jar file above in the `$METRON_HOME/parser_contrib` directory on the Metron host (i.e. any host you would start parsers from or, alternatively, where the Metron REST is hosted).
    +
    +## Example
    +
    +Let's work through deploying the example above.
    +
    +### Preliminaries
    +
    +We assume that the following environment variables are set:
    +* `METRON_HOME` - the home directory for metron
    +* `ZOOKEEPER` - The zookeeper quorum (comma separated with port specified: e.g. `node1:2181` for full-dev)
    +* `BROKERLIST` - The Kafka broker list (comma separated with port specified: e.g. `node1:6667` for full-dev)
    +* `ES_HOST` - The elasticsearch master (and port) e.g. `node1:9200` for full-dev.
    +
    +Also, this does not assume that you are using a kerberized cluster.  If you are, then the parser start command will adjust slightly to include the security protocol.
    +
    +### Copy the jar file up
    +
    +Copy the jar file located in `extra-parsers/target/extra-parsers-1.0-SNAPSHOT-uber.jar` to `$METRON_HOME/parser_contrib` and ensure the permissions are such that the `metron` user can read and execute.
    +
    +### Restart the REST service in Ambari
    +
    +In order for new parsers to be picked up, the REST service must be restarted.  You can do that from within Ambari by restarting the `Metron REST` service.
    +
    +### Create a Kafka Topic
    +
    +Create a kafka topic, let's call it `test` via:
    +`/usr/hdp/current/kafka-broker/bin/kafka-topics.sh --zookeeper $ZOOKEEPER --create --topic test --partitions 1 --replication-factor 1`
    +
    +Note, in a real deployment, that topic would be named something more descriptive and would have replication factor and partitions set to something less trivial.
    +
    +### Configure Test Parser
    +
    +Create the a file called `$METRON_HOME/config/zookeeper/parsers/test.json` with the following content:
    +```
    +{
    +  "parserClassName":"com.thirdparty.SimpleParser",
    +  "sensorTopic":"test"
    +}
    +```
    +
    +### Start Parser
    --- End diff --
    
    Need to push config to Zookeeper before starting the parser.
    
    ```
    Now push the config to Zookeeper with the following command:
    $METRON_HOME/bin/zk_load_configs.sh -m PUSH -i $METRON_HOME/config/zookeeper/ -z $ZOOKEEPER
    
    ```


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143546668
  
    --- Diff: metron-platform/metron-parsers/pom.xml ---
    @@ -125,6 +125,12 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.storm</groupId>
    +            <artifactId>storm-rename-hack</artifactId>
    --- End diff --
    
    Haha, I love that this is the actual artifact name.


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143544318
  
    --- Diff: metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SensorParserConfigServiceImpl.java ---
    @@ -122,29 +123,15 @@ public boolean delete(String name) throws RestException {
     
       @Override
       public Map<String, String> getAvailableParsers() {
    -    if (availableParsers == null) {
    -      availableParsers = new HashMap<>();
    -      Set<Class<? extends MessageParser>> parserClasses = getParserClasses();
    -      parserClasses.forEach(parserClass -> {
    -        if (!"BasicParser".equals(parserClass.getSimpleName())) {
    -          availableParsers.put(parserClass.getSimpleName().replaceAll("Basic|Parser", ""),
    -              parserClass.getName());
    -        }
    -      });
    -    }
    -    return availableParsers;
    +    return ParserIndex.INSTANCE.getIndex();
    --- End diff --
    
    Nice


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143550076
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/MergeAndShadeTransformer.java ---
    @@ -0,0 +1,90 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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 com.google.common.base.Splitter;
    +import org.apache.storm.daemon.JarTransformer;
    +import org.apache.storm.hack.StormShadeTransformer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.*;
    +import java.lang.invoke.MethodHandles;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.jar.JarEntry;
    +import java.util.jar.JarInputStream;
    +import java.util.jar.JarOutputStream;
    +
    +/**
    + * This is a storm jar transformer that will add in additional jars pulled from an
    + * environment variable.  The jars will be merged with the main uber jar and then
    + * the resulting jar will be shaded and relocated according to the StormShadeTransformer.
    + *
    + */
    +public class MergeAndShadeTransformer implements JarTransformer {
    +  public static final String EXTRA_JARS_ENV = "EXTRA_JARS";
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  StormShadeTransformer _underlyingTransformer = new StormShadeTransformer();
    +  @Override
    +  public void transform(InputStream input, OutputStream output) throws IOException {
    +    String extraJars = System.getenv().get(EXTRA_JARS_ENV);
    +    if(extraJars == null || extraJars.length() == 0) {
    +      _underlyingTransformer.transform(input, output);
    +      return;
    +    }
    +    File tmpFile = File.createTempFile("metron", "jar");
    +    tmpFile.deleteOnExit();
    +    Set<String> entries = new HashSet<>();
    +    try (JarOutputStream jout = new JarOutputStream(new BufferedOutputStream(new FileOutputStream(tmpFile)))) {
    +      try (JarInputStream jin = new JarInputStream(new BufferedInputStream(input))){
    +        copy(jin, jout, entries);
    +      }
    +      for (String fileStr : Splitter.on(",").split(extraJars)) {
    +        File f = new File(fileStr);
    +        if (!f.exists()) {
    +          continue;
    +        }
    +        LOG.info("Merging jar {} from {}", f.getName(), f.getAbsolutePath());
    +        try (JarInputStream jin = new JarInputStream(new BufferedInputStream(new FileInputStream(f)))) {
    +          copy(jin, jout, entries);
    --- End diff --
    
    Same comment as above re copy/entries


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143553954
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java ---
    @@ -285,7 +285,14 @@ public static Options getOptions() {
       }
     
       private static CommandLine parse(Options options, String[] args) {
    -    CommandLineParser parser = new PosixParser();
    +    CommandLineParser parser = new PosixParser() {
    +      @Override
    +      protected void processOption(String arg, ListIterator iter) throws ParseException {
    --- End diff --
    
    Ah, yes, so this does bear a comment. Not sure if it requires a code comment or not, but the general gist is that in order to pass args to `storm jar` (in this case storm configs to override the transformation class), we have to disregard options that we don't know about in the CLI.  Storm will ignore our args, we have to do the same.  Do you think that should be a code comment?


---

[GitHub] metron issue #785: METRON-1230: As a stopgap prior to METRON-777, add more s...

Posted by ottobackwards <gi...@git.apache.org>.
Github user ottobackwards commented on the issue:

    https://github.com/apache/metron/pull/785
  
    well then.
    
    In a world where 777 didn't exist I would be a +1.
    In world where 777 does exist, i'm a +0, since I feel the effort would have been better spent reviewing 777, but think this is technically good.  



---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r160243119
  
    --- Diff: metron-platform/metron-parsers/3rdPartyParser.md ---
    @@ -0,0 +1,306 @@
    +# Custom Metron Parsers
    +
    +We have many stock parsers for normal operations.  Some of these are
    +networking and cybersecurity focused (e.g. the ASA Parser), some of
    +these are general purpose (e.g. the CSVParser), but inevitably users
    +will want to extend the system to process their own data formats.  To
    +enable this, this is a walkthrough of how to create and use a custom
    +parser within Metron.
    +
    +# Writing A Custom Parser
    +Before we can use a parser, we will need to create a custom parser.  The
    +parser is the workhorse of Metron ingest.  It provides the mapping
    +between the raw data coming in via the Kafka value and a `JSONObject`,
    +the internal data structure provided.
    +
    +## Implementation
    +
    +In order to do create a custom parser, we need to do one of the following:
    +* Write a class which conforms to the `org.apache.metron.parsers.interfaces.MessageParser<JSONObject>` and `java.util.Serializable` interfaces
    +  * Implement `init()`, `validate(JSONObject message)`, and `List<JSONObject> parse(byte[] rawMessage)`
    +* Write a class which extends `org.apache.metron.parsers.BasicParser`
    +  * Provides convenience implementations to `validate` which ensures `timestamp` and `original_string` fields exist.
    +
    +## Example
    +
    +In order to illustrate how this might be done, let's create a very
    +simple parser that takes a comma separated pair and creates a couple of
    +fields:
    +* `original_string` -- the raw data
    +* `timestamp` -- the current time
    +* `first` -- the first field of the comma separated pair
    +* `last` -- the last field of the comma separated pair
    +
    +For this demonstration, let's create a maven project to compile our
    +project.  We'll call it `extra_parsers`, so in your workspace, let's set
    +up the maven project:
    +* Create the maven infrastructure for `extra_parsers` via
    +```
    +mkdir -p extra_parsers/src/{main,test}/java
    +```
    +* Create a pom file indicating how we should build our parsers by
    +  editing `extra_parsers/pom.xml` with the following content:
    +```
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +  <groupId>com.3rdparty</groupId>
    +  <artifactId>extra-parsers</artifactId>
    +  <packaging>jar</packaging>
    +  <version>1.0-SNAPSHOT</version>
    +  <name>extra-parsers</name>
    +  <url>http://thirdpartysoftware.org</url>
    +  <properties>
    +    <!-- The java version to conform to.  Metron works all the way to 1.8 -->
    +    <java_version>1.8</java_version>
    +    <!-- The version of Metron that we'll be targetting. -->
    +    <metron_version>0.4.1</metron_version>
    +    <!-- To complete the simulation, we'll depend on a common dependency -->
    +    <guava_version>19.0</guava_version>
    +    <!-- We will shade our dependencies to create a single jar at the end -->
    +    <shade_version>2.4.3</shade_version>
    +  </properties>
    +  <dependencies>
    +    <!--
    +    We want to depend on Metron, but ensure that the scope is "provided"
    +    as we do not want to include it in our bundle.
    +    -->
    +    <dependency>
    +      <groupId>org.apache.metron</groupId>
    +      <artifactId>metron-parsers</artifactId>
    +      <version>${metron_version}</version>
    +      <scope>provided</scope>
    +    </dependency>
    +    <dependency>
    +      <groupId>com.google.guava</groupId>
    +      <artifactId>guava</artifactId>
    +      <version>${guava_version}</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>junit</groupId>
    +      <artifactId>junit</artifactId>
    +      <version>3.8.1</version>
    +      <scope>test</scope>
    +    </dependency>
    +  </dependencies>
    +  <build>
    +    <plugins>
    +     <!-- We will set up the shade plugin to create a single jar at the
    +           end of the build lifecycle.  We will exclude some things and
    +           relocate others to simulate a real situation.
    +           
    +           One thing to note is that it's a good practice to shade and
    +           relocate common libraries that may be dependencies in Metron.
    +           Your jar will be merged with the parsers jar, so the metron
    +           version will be included for all overlapping classes.
    +           So, shade and relocate to ensure that YOUR version of the library is used.
    +      -->
    +
    +      <plugin>
    +        <groupId>org.apache.maven.plugins</groupId>
    +        <artifactId>maven-shade-plugin</artifactId>
    +        <version>${shade_version}</version>
    +        <configuration>
    +          <createDependencyReducedPom>true</createDependencyReducedPom>
    +          <artifactSet>
    +            <excludes>
    +              <!-- Exclude slf4j for no reason other than to illustrate how to exclude dependencies.
    +                   The metron team has nothing against slf4j. :-)
    +               -->
    +              <exclude>*slf4j*</exclude>
    +            </excludes>
    +          </artifactSet>
    +        </configuration>
    +        <executions>
    +          <execution>
    +            <phase>package</phase>
    +            <goals>
    +              <goal>shade</goal>
    +            </goals>
    +            <configuration>
    +              <shadedArtifactAttached>true</shadedArtifactAttached>
    +              <shadedClassifierName>uber</shadedClassifierName>
    +              <filters>
    +                <filter>
    +                  <!-- Sometimes these get added and confuse the uber jar out of shade -->
    +                  <artifact>*:*</artifact>
    +                  <excludes>
    +                    <exclude>META-INF/*.SF</exclude>
    +                    <exclude>META-INF/*.DSA</exclude>
    +                    <exclude>META-INF/*.RSA</exclude>
    +                  </excludes>
    +                </filter>
    +              </filters>
    +              <relocations>
    +                <!-- Relocate guava as it's used in Metron and I really want 0.19 -->
    +                <relocation>
    +                  <pattern>com.google</pattern>
    +                  <shadedPattern>com.thirdparty.guava</shadedPattern>
    +                </relocation>
    +              </relocations>
    +              <artifactSet>
    +                <excludes>
    +                  <!-- We can also exclude by artifactId and groupId -->
    +                  <exclude>storm:storm-core:*</exclude>
    +                  <exclude>storm:storm-lib:*</exclude>
    +                  <exclude>org.slf4j.impl*</exclude>
    +                  <exclude>org.slf4j:slf4j-log4j*</exclude>
    +                </excludes>
    +              </artifactSet>
    +            </configuration>
    +          </execution>
    +        </executions>
    +      </plugin>
    +      <!--
    +      We want to make sure we compile using java 1.8.
    +      -->
    +      <plugin>
    +        <groupId>org.apache.maven.plugins</groupId>
    +        <artifactId>maven-compiler-plugin</artifactId>
    +        <version>3.5.1</version>
    +        <configuration>
    +          <forceJavacCompilerUse>true</forceJavacCompilerUse>
    +          <source>${java_version}</source>
    +          <compilerArgument>-Xlint:unchecked</compilerArgument>
    +          <target>${java_version}</target>
    +          <showWarnings>true</showWarnings>
    +        </configuration>
    +      </plugin>
    +    </plugins>
    +  </build>
    +</project>
    +```
    +* Now let's create our parser  `com.thirdparty.SimpleParser` by creating the file `extra-parsers/src/main/java/com/thirdparty/SimpleParser.java` with the following content:
    +```
    +package com.thirdparty;
    +
    +import com.google.common.base.Splitter;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.Iterables;
    +import org.apache.metron.parsers.BasicParser;
    +import org.json.simple.JSONObject;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class SimpleParser extends BasicParser {
    +  @Override
    +  public void init() {
    +
    +  }
    +
    +  @Override
    +  public List<JSONObject> parse(byte[] bytes) {
    +    String input = new String(bytes);
    +    Iterable<String> it = Splitter.on(",").split(input);
    +    JSONObject ret = new JSONObject();
    +    ret.put("original_string", input);
    +    ret.put("timestamp", System.currentTimeMillis());
    +    ret.put("first", Iterables.getFirst(it, "missing"));
    +    ret.put("last", Iterables.getLast(it, "missing"));
    +    return ImmutableList.of(ret);
    +  }
    +
    +  @Override
    +  public void configure(Map<String, Object> map) {
    +
    +  }
    +}
    +```
    +* Compile the parser via `mvn clean package` in `extra_parsers`
    +
    +This will create a jar containing your parser and its dependencies (sans Metron dependencies) in `extra-parsers/target/extra-parsers-1.0-SNAPSHOT-uber.jar`
    +
    +# Deploying Your Custom Parser
    +
    +In order to deploy your newly built custom parser, you would place the jar file above in the `$METRON_HOME/parser_contrib` directory on the Metron host (i.e. any host you would start parsers from or, alternatively, where the Metron REST is hosted).
    +
    +## Example
    +
    +Let's work through deploying the example above.
    +
    +### Preliminaries
    +
    +We assume that the following environment variables are set:
    +* `METRON_HOME` - the home directory for metron
    +* `ZOOKEEPER` - The zookeeper quorum (comma separated with port specified: e.g. `node1:2181` for full-dev)
    +* `BROKERLIST` - The Kafka broker list (comma separated with port specified: e.g. `node1:6667` for full-dev)
    +* `ES_HOST` - The elasticsearch master (and port) e.g. `node1:9200` for full-dev.
    +
    +Also, this does not assume that you are using a kerberized cluster.  If you are, then the parser start command will adjust slightly to include the security protocol.
    +
    +### Copy the jar file up
    +
    +Copy the jar file located in `extra-parsers/target/extra-parsers-1.0-SNAPSHOT-uber.jar` to `$METRON_HOME/parser_contrib` and ensure the permissions are such that the `metron` user can read and execute.
    +
    +### Restart the REST service in Ambari
    +
    +In order for new parsers to be picked up, the REST service must be restarted.  You can do that from within Ambari by restarting the `Metron REST` service.
    +
    +### Create a Kafka Topic
    +
    +Create a kafka topic, let's call it `test` via:
    +`/usr/hdp/current/kafka-broker/bin/kafka-topics.sh --zookeeper $ZOOKEEPER --create --topic test --partitions 1 --replication-factor 1`
    +
    +Note, in a real deployment, that topic would be named something more descriptive and would have replication factor and partitions set to something less trivial.
    +
    +### Configure Test Parser
    +
    +Create the a file called `$METRON_HOME/config/zookeeper/parsers/test.json` with the following content:
    +```
    +{
    +  "parserClassName":"com.thirdparty.SimpleParser",
    +  "sensorTopic":"test"
    +}
    +```
    +
    +### Start Parser
    --- End diff --
    
    WHoops, missed that one.  Thanks, mike!


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143554015
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/MergeAndShadeTransformer.java ---
    @@ -0,0 +1,90 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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 com.google.common.base.Splitter;
    +import org.apache.storm.daemon.JarTransformer;
    +import org.apache.storm.hack.StormShadeTransformer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.*;
    +import java.lang.invoke.MethodHandles;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.jar.JarEntry;
    +import java.util.jar.JarInputStream;
    +import java.util.jar.JarOutputStream;
    +
    +/**
    + * This is a storm jar transformer that will add in additional jars pulled from an
    + * environment variable.  The jars will be merged with the main uber jar and then
    + * the resulting jar will be shaded and relocated according to the StormShadeTransformer.
    + *
    + */
    +public class MergeAndShadeTransformer implements JarTransformer {
    +  public static final String EXTRA_JARS_ENV = "EXTRA_JARS";
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  StormShadeTransformer _underlyingTransformer = new StormShadeTransformer();
    +  @Override
    +  public void transform(InputStream input, OutputStream output) throws IOException {
    --- End diff --
    
    It is.


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143554494
  
    --- Diff: metron-interface/metron-rest/src/main/scripts/metron-rest.sh ---
    @@ -47,6 +55,15 @@ rest_jar_pattern="${METRON_HOME}/lib/metron-rest*.jar"
     rest_files=( ${rest_jar_pattern} )
     echo "Default metron-rest jar is: ${rest_files[0]}"
     METRON_REST_CLASSPATH+=":${rest_files[0]}"
    +METRON_REST_CLASSPATH+=":$PARSER_LIB"
    +
    +if [ -d "$PARSER_CONTRIB" ]; then
    +  contrib_jar_pattern="${PARSER_CONTRIB}/*.jar"
    +  contrib_list=( $contrib_jar_pattern ) # expand the glob to a list
    --- End diff --
    
    It's definitely not cross-shell.  The shell scripts we have currently all shbang to bash on the first line for that reason.


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143546323
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java ---
    @@ -285,7 +285,14 @@ public static Options getOptions() {
       }
     
       private static CommandLine parse(Options options, String[] args) {
    -    CommandLineParser parser = new PosixParser();
    +    CommandLineParser parser = new PosixParser() {
    +      @Override
    +      protected void processOption(String arg, ListIterator iter) throws ParseException {
    --- End diff --
    
    Is this to ignore options passed that we don't provide support for, and can you make a small comment about it?


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143549913
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/MergeAndShadeTransformer.java ---
    @@ -0,0 +1,90 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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 com.google.common.base.Splitter;
    +import org.apache.storm.daemon.JarTransformer;
    +import org.apache.storm.hack.StormShadeTransformer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.*;
    +import java.lang.invoke.MethodHandles;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.jar.JarEntry;
    +import java.util.jar.JarInputStream;
    +import java.util.jar.JarOutputStream;
    +
    +/**
    + * This is a storm jar transformer that will add in additional jars pulled from an
    + * environment variable.  The jars will be merged with the main uber jar and then
    + * the resulting jar will be shaded and relocated according to the StormShadeTransformer.
    + *
    + */
    +public class MergeAndShadeTransformer implements JarTransformer {
    +  public static final String EXTRA_JARS_ENV = "EXTRA_JARS";
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  StormShadeTransformer _underlyingTransformer = new StormShadeTransformer();
    +  @Override
    +  public void transform(InputStream input, OutputStream output) throws IOException {
    +    String extraJars = System.getenv().get(EXTRA_JARS_ENV);
    +    if(extraJars == null || extraJars.length() == 0) {
    +      _underlyingTransformer.transform(input, output);
    +      return;
    +    }
    +    File tmpFile = File.createTempFile("metron", "jar");
    +    tmpFile.deleteOnExit();
    +    Set<String> entries = new HashSet<>();
    +    try (JarOutputStream jout = new JarOutputStream(new BufferedOutputStream(new FileOutputStream(tmpFile)))) {
    +      try (JarInputStream jin = new JarInputStream(new BufferedInputStream(input))){
    +        copy(jin, jout, entries);
    --- End diff --
    
    Can you document that this method has side effects on the set "entries", or modify the method to return a modified `Set<String>` to make this more explicit?


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143552344
  
    --- Diff: metron-interface/metron-rest/src/main/scripts/metron-rest.sh ---
    @@ -47,6 +55,15 @@ rest_jar_pattern="${METRON_HOME}/lib/metron-rest*.jar"
     rest_files=( ${rest_jar_pattern} )
     echo "Default metron-rest jar is: ${rest_files[0]}"
     METRON_REST_CLASSPATH+=":${rest_files[0]}"
    +METRON_REST_CLASSPATH+=":$PARSER_LIB"
    +
    +if [ -d "$PARSER_CONTRIB" ]; then
    +  contrib_jar_pattern="${PARSER_CONTRIB}/*.jar"
    +  contrib_list=( $contrib_jar_pattern ) # expand the glob to a list
    --- End diff --
    
    This is probably ok per our supported OS's and shells, but I don't think this is cross platform. It does not work in ZSH, for instance.


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143555784
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/MergeAndShadeTransformer.java ---
    @@ -0,0 +1,90 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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 com.google.common.base.Splitter;
    +import org.apache.storm.daemon.JarTransformer;
    +import org.apache.storm.hack.StormShadeTransformer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.*;
    +import java.lang.invoke.MethodHandles;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.jar.JarEntry;
    +import java.util.jar.JarInputStream;
    +import java.util.jar.JarOutputStream;
    +
    +/**
    + * This is a storm jar transformer that will add in additional jars pulled from an
    + * environment variable.  The jars will be merged with the main uber jar and then
    + * the resulting jar will be shaded and relocated according to the StormShadeTransformer.
    + *
    + */
    +public class MergeAndShadeTransformer implements JarTransformer {
    +  public static final String EXTRA_JARS_ENV = "EXTRA_JARS";
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  StormShadeTransformer _underlyingTransformer = new StormShadeTransformer();
    +  @Override
    +  public void transform(InputStream input, OutputStream output) throws IOException {
    +    String extraJars = System.getenv().get(EXTRA_JARS_ENV);
    +    if(extraJars == null || extraJars.length() == 0) {
    +      _underlyingTransformer.transform(input, output);
    +      return;
    +    }
    +    File tmpFile = File.createTempFile("metron", "jar");
    +    tmpFile.deleteOnExit();
    +    Set<String> entries = new HashSet<>();
    +    try (JarOutputStream jout = new JarOutputStream(new BufferedOutputStream(new FileOutputStream(tmpFile)))) {
    +      try (JarInputStream jin = new JarInputStream(new BufferedInputStream(input))){
    +        copy(jin, jout, entries);
    +      }
    +      for (String fileStr : Splitter.on(",").split(extraJars)) {
    +        File f = new File(fileStr);
    +        if (!f.exists()) {
    +          continue;
    +        }
    +        LOG.info("Merging jar {} from {}", f.getName(), f.getAbsolutePath());
    +        try (JarInputStream jin = new JarInputStream(new BufferedInputStream(new FileInputStream(f)))) {
    +          copy(jin, jout, entries);
    --- End diff --
    
    Gotcha


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143555806
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/MergeAndShadeTransformer.java ---
    @@ -0,0 +1,90 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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 com.google.common.base.Splitter;
    +import org.apache.storm.daemon.JarTransformer;
    +import org.apache.storm.hack.StormShadeTransformer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.*;
    +import java.lang.invoke.MethodHandles;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.jar.JarEntry;
    +import java.util.jar.JarInputStream;
    +import java.util.jar.JarOutputStream;
    +
    +/**
    + * This is a storm jar transformer that will add in additional jars pulled from an
    + * environment variable.  The jars will be merged with the main uber jar and then
    + * the resulting jar will be shaded and relocated according to the StormShadeTransformer.
    + *
    + */
    +public class MergeAndShadeTransformer implements JarTransformer {
    +  public static final String EXTRA_JARS_ENV = "EXTRA_JARS";
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  StormShadeTransformer _underlyingTransformer = new StormShadeTransformer();
    +  @Override
    +  public void transform(InputStream input, OutputStream output) throws IOException {
    +    String extraJars = System.getenv().get(EXTRA_JARS_ENV);
    +    if(extraJars == null || extraJars.length() == 0) {
    +      _underlyingTransformer.transform(input, output);
    +      return;
    +    }
    +    File tmpFile = File.createTempFile("metron", "jar");
    +    tmpFile.deleteOnExit();
    +    Set<String> entries = new HashSet<>();
    +    try (JarOutputStream jout = new JarOutputStream(new BufferedOutputStream(new FileOutputStream(tmpFile)))) {
    +      try (JarInputStream jin = new JarInputStream(new BufferedInputStream(input))){
    +        copy(jin, jout, entries);
    +      }
    +      for (String fileStr : Splitter.on(",").split(extraJars)) {
    +        File f = new File(fileStr);
    +        if (!f.exists()) {
    +          continue;
    +        }
    +        LOG.info("Merging jar {} from {}", f.getName(), f.getAbsolutePath());
    +        try (JarInputStream jin = new JarInputStream(new BufferedInputStream(new FileInputStream(f)))) {
    +          copy(jin, jout, entries);
    +        }
    +      }
    +    }
    +    _underlyingTransformer.transform(new BufferedInputStream(new FileInputStream(tmpFile)), output);
    +  }
    +
    +  private void copy(JarInputStream jin, JarOutputStream jout, Set<String> entries) throws IOException {
    +    byte[] buffer = new byte[1024];
    +    for(JarEntry entry = jin.getNextJarEntry(); entry != null; entry = jin.getNextJarEntry()) {
    +      if(entries.contains(entry.getName())) {
    +        continue;
    +      }
    +      if(LOG.isDebugEnabled()) {
    --- End diff --
    
    Yup


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143555770
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/MergeAndShadeTransformer.java ---
    @@ -0,0 +1,90 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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 com.google.common.base.Splitter;
    +import org.apache.storm.daemon.JarTransformer;
    +import org.apache.storm.hack.StormShadeTransformer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.*;
    +import java.lang.invoke.MethodHandles;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.jar.JarEntry;
    +import java.util.jar.JarInputStream;
    +import java.util.jar.JarOutputStream;
    +
    +/**
    + * This is a storm jar transformer that will add in additional jars pulled from an
    + * environment variable.  The jars will be merged with the main uber jar and then
    + * the resulting jar will be shaded and relocated according to the StormShadeTransformer.
    + *
    + */
    +public class MergeAndShadeTransformer implements JarTransformer {
    +  public static final String EXTRA_JARS_ENV = "EXTRA_JARS";
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  StormShadeTransformer _underlyingTransformer = new StormShadeTransformer();
    +  @Override
    +  public void transform(InputStream input, OutputStream output) throws IOException {
    +    String extraJars = System.getenv().get(EXTRA_JARS_ENV);
    +    if(extraJars == null || extraJars.length() == 0) {
    +      _underlyingTransformer.transform(input, output);
    +      return;
    +    }
    +    File tmpFile = File.createTempFile("metron", "jar");
    +    tmpFile.deleteOnExit();
    +    Set<String> entries = new HashSet<>();
    +    try (JarOutputStream jout = new JarOutputStream(new BufferedOutputStream(new FileOutputStream(tmpFile)))) {
    +      try (JarInputStream jin = new JarInputStream(new BufferedInputStream(input))){
    +        copy(jin, jout, entries);
    --- End diff --
    
    Gotcha


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r145423853
  
    --- Diff: metron-interface/metron-rest/src/main/java/org/apache/metron/rest/util/ParserIndex.java ---
    @@ -0,0 +1,92 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.metron.rest.util;
    +
    +import org.apache.metron.parsers.interfaces.MessageParser;
    +import org.reflections.Reflections;
    +import org.reflections.util.ClasspathHelper;
    +import org.reflections.util.ConfigurationBuilder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.net.URL;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Index the parsers.  Analyzing the classpath is a costly operation, so caching it makes sense.
    + * Eventually, we will probably want to have a timer that periodically reindexes so that new parsers show up.
    + */
    +public enum ParserIndex {
    +  INSTANCE;
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static Set<Class<? extends MessageParser>> index;
    +  private static Map<String, String> availableParsers ;
    +
    +  static {
    +    load();
    +  }
    +
    +  public synchronized Map<String, String> getIndex() {
    +    if(availableParsers == null) {
    +      load();
    +    }
    +    return availableParsers;
    +  }
    +
    +  public synchronized Set<Class<? extends MessageParser>> getClasses() {
    +    if(index == null) {
    +      load();
    +    }
    +    return index;
    +  }
    +
    +  public static void reload() {
    +    load();
    +  }
    +
    +  /**
    +   * To handle the situation where classpath is specified in the manifest of the jar, we have to augment the URLs.
    +   * This happens as part of the surefire plugin as well as elsewhere in the wild.
    +   * @param classLoaders
    +   * @return
    --- End diff --
    
    Agreed.  I corrected it here.  Thanks for pointing it out, mike.


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143542088
  
    --- Diff: metron-interface/metron-rest/pom.xml ---
    @@ -163,6 +163,7 @@
                 <groupId>org.apache.metron</groupId>
                 <artifactId>metron-parsers</artifactId>
                 <version>${project.parent.version}</version>
    +            <scope>provided</scope>
    --- End diff --
    
    Just so I'm clear, are these additions of "provided" scope along with the "createDependencyReducedPom=false" flag done to remove the embedding of metron-hbase and metron-parsers within the rest API?


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143555004
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java ---
    @@ -285,7 +285,14 @@ public static Options getOptions() {
       }
     
       private static CommandLine parse(Options options, String[] args) {
    -    CommandLineParser parser = new PosixParser();
    +    CommandLineParser parser = new PosixParser() {
    +      @Override
    +      protected void processOption(String arg, ListIterator iter) throws ParseException {
    --- End diff --
    
    nvm, if there's a question, then the answer is that we need a comment here.


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143549304
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/MergeAndShadeTransformer.java ---
    @@ -0,0 +1,90 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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 com.google.common.base.Splitter;
    +import org.apache.storm.daemon.JarTransformer;
    +import org.apache.storm.hack.StormShadeTransformer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.*;
    +import java.lang.invoke.MethodHandles;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.jar.JarEntry;
    +import java.util.jar.JarInputStream;
    +import java.util.jar.JarOutputStream;
    +
    +/**
    + * This is a storm jar transformer that will add in additional jars pulled from an
    + * environment variable.  The jars will be merged with the main uber jar and then
    + * the resulting jar will be shaded and relocated according to the StormShadeTransformer.
    + *
    + */
    +public class MergeAndShadeTransformer implements JarTransformer {
    +  public static final String EXTRA_JARS_ENV = "EXTRA_JARS";
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  StormShadeTransformer _underlyingTransformer = new StormShadeTransformer();
    +  @Override
    +  public void transform(InputStream input, OutputStream output) throws IOException {
    --- End diff --
    
    Is input the existing uber jar?


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143550601
  
    --- Diff: metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/MergeAndShadeTransformer.java ---
    @@ -0,0 +1,90 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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 com.google.common.base.Splitter;
    +import org.apache.storm.daemon.JarTransformer;
    +import org.apache.storm.hack.StormShadeTransformer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.*;
    +import java.lang.invoke.MethodHandles;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.jar.JarEntry;
    +import java.util.jar.JarInputStream;
    +import java.util.jar.JarOutputStream;
    +
    +/**
    + * This is a storm jar transformer that will add in additional jars pulled from an
    + * environment variable.  The jars will be merged with the main uber jar and then
    + * the resulting jar will be shaded and relocated according to the StormShadeTransformer.
    + *
    + */
    +public class MergeAndShadeTransformer implements JarTransformer {
    +  public static final String EXTRA_JARS_ENV = "EXTRA_JARS";
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +
    +  StormShadeTransformer _underlyingTransformer = new StormShadeTransformer();
    +  @Override
    +  public void transform(InputStream input, OutputStream output) throws IOException {
    +    String extraJars = System.getenv().get(EXTRA_JARS_ENV);
    +    if(extraJars == null || extraJars.length() == 0) {
    +      _underlyingTransformer.transform(input, output);
    +      return;
    +    }
    +    File tmpFile = File.createTempFile("metron", "jar");
    +    tmpFile.deleteOnExit();
    +    Set<String> entries = new HashSet<>();
    +    try (JarOutputStream jout = new JarOutputStream(new BufferedOutputStream(new FileOutputStream(tmpFile)))) {
    +      try (JarInputStream jin = new JarInputStream(new BufferedInputStream(input))){
    +        copy(jin, jout, entries);
    +      }
    +      for (String fileStr : Splitter.on(",").split(extraJars)) {
    +        File f = new File(fileStr);
    +        if (!f.exists()) {
    +          continue;
    +        }
    +        LOG.info("Merging jar {} from {}", f.getName(), f.getAbsolutePath());
    +        try (JarInputStream jin = new JarInputStream(new BufferedInputStream(new FileInputStream(f)))) {
    +          copy(jin, jout, entries);
    +        }
    +      }
    +    }
    +    _underlyingTransformer.transform(new BufferedInputStream(new FileInputStream(tmpFile)), output);
    +  }
    +
    +  private void copy(JarInputStream jin, JarOutputStream jout, Set<String> entries) throws IOException {
    +    byte[] buffer = new byte[1024];
    +    for(JarEntry entry = jin.getNextJarEntry(); entry != null; entry = jin.getNextJarEntry()) {
    +      if(entries.contains(entry.getName())) {
    +        continue;
    +      }
    +      if(LOG.isDebugEnabled()) {
    --- End diff --
    
    isDebugEnabled is no longer necessary with SLF4J - the token replacement is disabled under the hood if debugging is not enabled. 


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by justinleet <gi...@git.apache.org>.
Github user justinleet commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r159122619
  
    --- Diff: metron-platform/metron-parsers/3rdPartyParser.md ---
    @@ -0,0 +1,306 @@
    +# Custom Metron Parsers
    --- End diff --
    
    Not sure what the state of this is, but if we're going to merge this as a stop-gap to 777, could you add the license header here?
    
    ```
    <!--
    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.
    -->
    ```


---

[GitHub] metron pull request #785: METRON-1230: As a stopgap prior to METRON-777, add...

Posted by mmiklavc <gi...@git.apache.org>.
Github user mmiklavc commented on a diff in the pull request:

    https://github.com/apache/metron/pull/785#discussion_r143585182
  
    --- Diff: metron-interface/metron-rest/src/main/java/org/apache/metron/rest/util/ParserIndex.java ---
    @@ -0,0 +1,92 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.metron.rest.util;
    +
    +import org.apache.metron.parsers.interfaces.MessageParser;
    +import org.reflections.Reflections;
    +import org.reflections.util.ClasspathHelper;
    +import org.reflections.util.ConfigurationBuilder;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.net.URL;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * Index the parsers.  Analyzing the classpath is a costly operation, so caching it makes sense.
    + * Eventually, we will probably want to have a timer that periodically reindexes so that new parsers show up.
    + */
    +public enum ParserIndex {
    +  INSTANCE;
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static Set<Class<? extends MessageParser>> index;
    +  private static Map<String, String> availableParsers ;
    +
    +  static {
    +    load();
    +  }
    +
    +  public synchronized Map<String, String> getIndex() {
    +    if(availableParsers == null) {
    +      load();
    +    }
    +    return availableParsers;
    +  }
    +
    +  public synchronized Set<Class<? extends MessageParser>> getClasses() {
    +    if(index == null) {
    +      load();
    +    }
    +    return index;
    +  }
    +
    +  public static void reload() {
    +    load();
    +  }
    +
    +  /**
    +   * To handle the situation where classpath is specified in the manifest of the jar, we have to augment the URLs.
    +   * This happens as part of the surefire plugin as well as elsewhere in the wild.
    +   * @param classLoaders
    +   * @return
    --- End diff --
    
    I figure it's worth mentioning now that we have community contributions correcting warnings and the like, this is going to throw WARNINGs when running javadoc - https://github.com/apache/metron/pull/792


---

[GitHub] metron issue #785: METRON-1230: As a stopgap prior to METRON-777, add more s...

Posted by cestella <gi...@git.apache.org>.
Github user cestella commented on the issue:

    https://github.com/apache/metron/pull/785
  
    @ottobackwards  One of the reasons why I did it the way that I did (hook into the way the storm accumulates the jar to submit) was that:
    1. while it's useful for sideloading parsers, it's also useful for sideloading *other* things (i.e. indexers)
    2. When 777 comes in, the user will just adjust to that scheme and delete the custom sideloaded jars when they decide it makes sense to do so.
    
    To recap, this is just intended to give rudimentary capabilities for the moment and the sideloading code is intended to be useful beyond sideloading parsers.


---