You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@metron.apache.org by ottobackwards <gi...@git.apache.org> on 2017/11/30 14:31:01 UTC

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

GitHub user ottobackwards opened a pull request:

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

    METRON-1339 Stellar Shell functionality to verify stored stellar statements 

    This will allow users to check their deployed statements, say after upgrade, when they are at rest ( and would fail on use ).
    In other words, they were valid when stored, but are not now because of stellar changes, such as new keywords.
    
    The interface `StellarConfiguredStatementReporter`, which is `@IndexSubclasses` ( ClassIndex) marked, allows the shell to discover reporters that can provide statements for validation.  This discovery allows de-coupling of stellar and 'hosts' that know about the location of the stored statements, and the configuration structure details.
    
    > We do mention the configurations in the shell output at this time.
    
    `metron-common` implements this interface, and can run through visiting all the configurations.
    
    A new magic keyword was added ` %validate_configured_expressions`
    When executed, the shell 
    
    - discovers the reporters through class index 
    - visits the reports, with callbacks for visits or errors
    - per visit ( which is called for a specific stellar statement ) the statement is compiled and errors reported
    - if the entire config fails ( threat triage stellar errors fail on deserialize so we don't get to do ANY enrichment visits in that case ) the error callback handles that
    
    I'm getting this out there, still a couple of things todo:
    
    1. full dev run. I have been testing with stellar external to full dev iteratively
    2. readme
    3. steps to test
    4. unit test
    
    
    
    
    ### 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:
    - [ ] Have you included steps to reproduce the behavior or problem that is being changed or addressed?
    - [ ] 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 
      ```
    
    - [ ] Have you written or updated unit tests and or integration tests to verify your changes?
    - [ ] 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)? 
    - [ ] 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:
    - [ ] 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
      ```


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

    $ git pull https://github.com/ottobackwards/metron stellar_verify_deployed_shell

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

    https://github.com/apache/metron/pull/856.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 #856
    
----
commit a5087f3a170eeda6ee778397c919d9eddd5597e2
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-11-30T14:15:40Z

    Stellar shell functionality to verify stellar statements.
    
    This will allow users to check their deployed statements, say after upgrade, when they are at rest ( and would fail on use ).
    In other words, they were valid when stored, but are not now because of stellar changes, such as new keywords.
    
    The interface StellarConfiguredStatementReporter, which is @IndexSubclasses marked, allows the shell to discover
    reporters that can provide statements for validation.  This discovery allows de-coupling of stellar and 'hosts' that
    know about the location of the stored statements, and the configuration structure details.
    
    We do mention the configurations in the shell output at this time.
    
    metron-common implements this interface, and can run through visiting all the configurations.

----


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r161236049
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    > You mean couple it to metron? We don't want to do that anymore.
    
    I don't see your justification.  Maybe another reviewer will understand this better.  As is, I am a +0 on this.  


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154111104
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    When I say track pojo's I mean the reporter would still 'know' about the Configuration object, but would discover what fields where stellar.  Know the object but not the fields.


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    ![stellar_validate](https://user-images.githubusercontent.com/1111551/33501682-73bf156a-d6ab-11e7-81c2-e827c9e62650.png)



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154200025
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    I think we eventually want to get to that point, with a consistent, comprehensive mechanism to validate stellar statements ( short of compiling ;) ). 



---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Per the conversation above, i'm going to take a stab at the attributed approach.
    I think the Stellar Functions should be a separate Jira.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588342
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationProvider.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +import static org.apache.metron.common.configuration.ConfigurationType.PROFILER;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.profiler.ProfileConfig;
    +import org.apache.metron.common.configuration.profiler.ProfilerConfig;
    +import org.apache.metron.common.utils.JSONUtils;
    +import org.apache.metron.stellar.common.utils.validation.ExpressionConfigurationHolder;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfigurationProvider;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfiguredStatementContainer.ErrorConsumer;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +
    +/**
    + * {@code ConfigurationProvider} is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class ConfigurationProvider implements StellarConfigurationProvider {
    +
    +  /**
    +   * Default constructor.
    +   */
    +  public ConfigurationProvider() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public List<ExpressionConfigurationHolder> provideConfigurations(CuratorFramework client,
    +      ErrorConsumer errorConsumer) {
    +    List<ExpressionConfigurationHolder> holders = new LinkedList<>();
    +    visitParserConfigs(client, holders, errorConsumer);
    +    visitEnrichmentConfigs(client, holders, errorConsumer);
    +    visitProfilerConfigs(client, holders, errorConsumer);
    --- End diff --
    
    There is no stellar in indexing is there?  It is not in the readme


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Any chance we can add a `VALIDATE(str, type)` function to the stellar management functions where str is the json blob string for the config and the type is the type of config?  Generally the goal is to disallow invalid stellar to get pushed to zookeeper via `zk_load_utils.sh`, so I suspect a function would be more useful in the situation where you're constructing a config in the REPL via management functions and want to validate it before pushing it.



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r160042250
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    They would automatically be picked up.
    
    I'm confused.  This PR as stated is to help validation of stellar configurations, where ever they may be.  it is not for the user to hunt and peck and pick a configuration.  So that is kind of the point.
    
    Although there have been comments about doing configuration validation with specific objects or files passed in, that is not the thrust of this pr, even if this pr enables it.



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588519
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarConfiguredStatementContainer.java ---
    @@ -0,0 +1,38 @@
    +/**
    + * 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.stellar.common.utils.validation;
    +
    +/**
    + * StellarConfiguredStatementProviders are used provide stellar statements
    + * and the context around those statements to the caller
    + */
    +public interface StellarConfiguredStatementContainer {
    --- End diff --
    
    The issue is not using the interface, I'll address that.  


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    So, the scenario here is checking things that *were* valid when uploaded, but have been invalidated by external changes ( language changes ).  I would like to keep the magic specific.
    
    I think the functionality for the management functions is valid, but can we do that as a separate Jira/PR?  I'll do it, I just want to keep this tight.  If you create the jira and assign it to me that would be super.
    
    I would do the files on disk using the management functions as well.
    
    So we just need to think of the stellar interface for calling
    `VALIDATE`  with a string, and with a file path.  Also saying what configuration type it is.
    
    Does that make sense?


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154171805
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    > you know what would work better maybe? if we could attribute the JSON.
    
    Maybe not?  Users/admins of the system create the JSON.  I don't want them to annotate which field is Stellar. I want the developers, the creators of the extension points, to define which fields are Stellar.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

Posted by ottobackwards <gi...@git.apache.org>.
GitHub user ottobackwards reopened a pull request:

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

    METRON-1339 Stellar Shell functionality to verify stored stellar statements 

    This will allow users to check their deployed statements, say after upgrade, when they are at rest ( and would fail on use ).
    In other words, they were valid when stored, but are not now because of stellar changes, such as new keywords.
    
    The interface `StellarConfiguredStatementReporter`, which is `@IndexSubclasses` ( ClassIndex) marked, allows the shell to discover reporters that can provide statements for validation.  This discovery allows de-coupling of stellar and 'hosts' that know about the location of the stored statements, and the configuration structure details.
    
    > We do mention the configurations in the shell output at this time.
    
    `metron-common` implements this interface, and can run through visiting all the configurations.
    
    A new magic keyword was added ` %validate_configured_expressions`
    When executed, the shell 
    
    - discovers the reporters through class index 
    - visits the reports, with callbacks for visits or errors
    - per visit ( which is called for a specific stellar statement ) the statement is compiled and errors reported
    - if the entire config fails ( threat triage stellar errors fail on deserialize so we don't get to do ANY enrichment visits in that case ) the error callback handles that
    
    I'm getting this out there, still a couple of things todo:
    
    [x] ~~full dev run. I have been testing with stellar external to full dev iteratively~~
    [x] ~~readme~~
    [x] ~~steps to test~~
    [x] ~~unit test~~
    [x] ~~ThreatTriage Rule Reason~~
    
    
    ## Testing
    - deploy full dev
    - edit the squid parser transformation(s) such that the stellar would not compile, such as adding a dangling  `=` in zookeeper
    ```json
    { 
    "parserClassName": "org.apache.metron.parsers.GrokParser", 
    "sensorTopic": "squid", 
    "parserConfig": { 
    "grokPath": "/patterns/squid", 
    "patternLabel": "SQUID_DELIMITED", 
    "timestampField": "timestamp" 
    }, 
    "fieldTransformations" : [ 
    { 
    "transformation" : "STELLAR" 
    ,"output" : [ "full_hostname", "domain_without_subdomains" ] 
    ,"config" : { 
    "full_hostname" : "URL_TO_HOST(url) =" 
    ,"domain_without_subdomains" : "DOMAIN_REMOVE_SUBDOMAINS(full_hostname)" 
    } 
    } 
    ] 
    }
    
    ```
    
    - edit the snort threat triage rules in it's enrichment config in zookeeper ( here with an extra `)` )
    
    ```json
    { 
    "enrichment" : { 
    "fieldMap": 
    { 
    "geo": ["ip_dst_addr", "ip_src_addr"], 
    "host": ["host"] 
    } 
    }, 
    "threatIntel" : { 
    "fieldMap": 
    { 
    "hbaseThreatIntel": ["ip_src_addr", "ip_dst_addr"] 
    }, 
    "fieldToTypeMap": 
    { 
    "ip_src_addr" : ["malicious_ip"], 
    "ip_dst_addr" : ["malicious_ip"] 
    }, 
    "triageConfig" : { 
    "riskLevelRules" : [ 
    { 
    "rule" : "not(IN_SUBNET(ip_dst_addr, '192.168.0.0/24')) )", 
    "score" : 10 
    } 
    ], 
    "aggregator" : "MAX" 
    } 
    } 
    } 
    ```
    
    ## Working with zookeeper
    I am not a zk cli maestro, so I took the easy way out and used [ZK-WEB](https://github.com/qiuxiafei/zk-web).
    Following the readme instructions it was very simple to clone, edit the config for full dev, and run from source.  If you log in with the creds in the config you can edit the nodes.
    
    
    ## Results
    When you run the magic command, it will report the failed stellar statements, and the failed enrichment config:
    
    ```bash
    [Stellar]>>> %validate_configured_expressions
    Discovered 1 reporters
    Visiting all configurations.  ThreatTriage rules are checked when loading the configuration, thus an invalid ThreatTriage rule will fail the entire Enrichement Configuration.
    Apache Metron
    Visiting Apache Metron
    
    
    ==================================================
    
    
    validating Apache Metron->PARSER->squid->full_hostname
    [!] Error Visiting Apache Metron->PARSER->squid->full_hostname
    Syntax error @ 1:17 token recognition error at: '='
    --
    [!] : URL_TO_HOST(url) =
    
    
    ==================================================
    
    
    
    
    ==================================================
    
    
    validating Apache Metron->PARSER->squid->domain_without_subdomains
    
    
    ==================================================
    
    
    [!] Configuration Apache Metron->ENRICHMENT->snort is not valid, please review
    
    Done validation
    [Stellar]>>>
    ```
    
    ### 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:
    - [ ] 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 
      ```
    
    - [ ] Have you written or updated unit tests and or integration tests to verify your changes?
    - [ ] 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:
    - [ ] 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
      ```

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

    $ git pull https://github.com/ottobackwards/metron stellar_verify_deployed_shell

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

    https://github.com/apache/metron/pull/856.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 #856
    
----
commit a5087f3a170eeda6ee778397c919d9eddd5597e2
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-11-30T14:15:40Z

    Stellar shell functionality to verify stellar statements.
    
    This will allow users to check their deployed statements, say after upgrade, when they are at rest ( and would fail on use ).
    In other words, they were valid when stored, but are not now because of stellar changes, such as new keywords.
    
    The interface StellarConfiguredStatementReporter, which is @IndexSubclasses marked, allows the shell to discover
    reporters that can provide statements for validation.  This discovery allows de-coupling of stellar and 'hosts' that
    know about the location of the stored statements, and the configuration structure details.
    
    We do mention the configurations in the shell output at this time.
    
    metron-common implements this interface, and can run through visiting all the configurations.

commit 96df802318d74bf8dfcd3bcae9208f63c3d034f0
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-11-30T16:25:58Z

    add readme and remove some newlines

commit dcd55e8f4a72e5c3e694807e13c7eebc53d1860f
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-11-30T23:34:38Z

    add tests for StellarStatementReporter

commit c0315b8291557de94dcf701d8def16d0b2866798
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-01T01:18:45Z

    refactor to utility classes, first step in major refactor

commit 65278a67a07f1c4c23ab2d95ebb6de92e1cac731
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-01T16:46:57Z

    Refactor based on review and inspiration from review.
    Although the original implementation was functional, it required maintainence to keep current.
    The suggested 'best state' was to have it be possible, maybe through annotations, for the validation
    system to be able to handle any config, regarless or composition using annotations.
    That would leave it up to the implementor to propertly annotate thier configurations, and allow for support of new fields.
    
    This is an implementation of that.
    
    I have refactored the implemenations and details, but kept the discovery and mechanics ( loading and visitation ) somewhat the same.
    Hopefully keeping the good and reworking to a more sustainable solution.
    
    Several annotations where created to marks ceratin stellar configruation objects or scenarios.
    A holder object, to hold the configuration object, but knows how to process the annotations and run the visitation was added.
    This holder object and the annotations have parameters and handling for several special scenarios, such as 2x nested maps.
    
    This implementation should facilitate follow on work to validate files and streams and blobs by using implementing the StellarValidator interface
    and re-using the holder concept ( replacing the providers )

commit 70de632ee583b45c028b23d8305d76e4b6bc70c5
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-01T16:59:39Z

    fix imports

commit 8726a15a3db35bf24408e723ec069a391df16820
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-01T19:53:52Z

    small refactor and javadoc work

commit a6a9a4e5d558209175a8e5d2fa532c845efa830d
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-03T13:22:50Z

    format and javadoc

commit 3f12c2dace1157ffd4e870df39864b61b71c1270
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-03T20:31:30Z

    refactor name and tests

commit 5516bad34573ef11dc40eb9ed23b241e7d84c75f
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-04T00:54:08Z

    fix for exception change

commit b3e7cfb8ac76d618d35da97895169b9069e7fba0
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-04T01:17:56Z

    fix regression after fixing mapping in prior commit

commit c067c9b8e39b91556790645bdae7b3f55d89d6eb
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-04T16:18:30Z

    Merge remote-tracking branch 'apache/master' into stellar_verify_deployed_shell

commit a814a0e0e497bbb1e45b4694b78d585136c756c8
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-05T16:12:48Z

    Merge remote-tracking branch 'apache/master' into stellar_verify_deployed_shell

commit 7b28be6e2da9756a3cb4f3234308118e99a2e17c
Author: Otto Fowler <ot...@gmail.com>
Date:   2017-12-07T11:55:05Z

    Merge remote-tracking branch 'apache/master' into stellar_verify_deployed_shell

----


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588556
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    --- End diff --
    
    Sorry, I was going to take this out, this is from the prior shell based integration


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154123254
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    does profiler have stellar?  ( sorry to have to ask ).


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r160001387
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    I will try to think of a different way to put it



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r160002156
  
    --- Diff: metron-platform/metron-management/README.md ---
    @@ -276,6 +278,13 @@ The functions are split roughly into a few sections:
         * aggregatorConfig - Optional config for aggregator
       * Returns: The String representation of the enrichment config
     
    +### Validation Functions
    +* `VALIDATE_STELLAR_RULE_CONFIGS`
    --- End diff --
    
    There is reference to "rules" because that was the first use for Stellar.  @cestella called them rules when he first implemented.  But that is ancient history.  We use Stellar everywhere now.  Its long ago outgrown that.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154155977
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    @nickwallen you know what would work better maybe?  if we could attribute the JSON.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r161240685
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    It is true that Stellar should exist, as much as possible, independent from Metron; that was the aim of the effort to move stellar out of metron-common and into its own top level component in the project.  I'll look closer at this PR and the (rather long, but seemingly coherent...so congrats ;) comment thread and weigh in later.


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    The justification that you mentioned just doesn't seem strong enough to me.  Unless there is more that I am missing.
    
    IMHO We should only use magic commands for things that can't be accomplished in the language using the preferred extension mechanism; aka defining Stellar functions.


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    I don't think it should be in management necessarily though.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158584492
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    --- End diff --
    
    Why do we need a 'writer'?  It doesn't make sense to me why we need it nor do we use it.  I think it could be removed from the method signature completely in `StellarValidator`.


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Closing to test build in travis


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154185591
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    Nick, when the ThreatTriage is loaded from json, the call to setRiskLevelRules() gets called by jackson, which validates the stellar, which will throw the exception at load time, before I can do the check myself.
    
    So I have code to verify the rules, but the fact is if they are bad stellar, they will throw on load before that code will execute.



---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Should I close this?  


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588194
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationProvider.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +import static org.apache.metron.common.configuration.ConfigurationType.PROFILER;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.profiler.ProfileConfig;
    +import org.apache.metron.common.configuration.profiler.ProfilerConfig;
    +import org.apache.metron.common.utils.JSONUtils;
    +import org.apache.metron.stellar.common.utils.validation.ExpressionConfigurationHolder;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfigurationProvider;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfiguredStatementContainer.ErrorConsumer;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +
    +/**
    + * {@code ConfigurationProvider} is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class ConfigurationProvider implements StellarConfigurationProvider {
    --- End diff --
    
    That is a good idea


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154109552
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    I need to think about this, but I will add the reason check


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Bump?


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    @nickwallen any feedback, does the annotated approach match what you imagined?


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    ![stellar_validate](https://user-images.githubusercontent.com/1111551/33501768-cc853dd2-d6ab-11e7-9a1a-ace77a469441.png)



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154171511
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    > because the rules are checked on deserialize, if they are wrong, the whole config fails ( the entire Enrichment )
    
      Only the syntax is checked on deserialization now.  If I add a field that we don't expect, we will catch that.  But if I use invalid Stellar; example use a variable named 'default', we will NOT catch that.  That's what your work comes in.  
    
    I am not sure that I am understanding your suggestion though.  Please correct me if this is a miss.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154133309
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    > does profiler have stellar? ( sorry to have to ask ).
    
    No problem. :)  Yes, it is mostly Stellar.  The docs are pretty good on that one, but I can outline specifically which ones if that helps.


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    If that question is to me too, yes, I feel strongly it should be a function.  A function that is part of the management functions. That was also suggested previously [here](https://github.com/apache/metron/pull/856#issuecomment-348208274).


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154118407
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    > We would only need to worry about tracking the pojos but not the fields ( we need to keep the context ). Am I understanding you correctly?
    
    Yes, exactly.  You can use the functionality in `ConfigurationUtils` and just do something like....
    ``` 
    config = getProfilerConfig(...)
    validateStellar(config)
    
    config = getSensorConfig(...)
    validateStellar(config)
    
    config = getEnrichmentConfig(..)
    validateStellar(config)
    
    ...
    ```
    
    Or something to that effect.



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154189837
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    Sure, I see in that case validation does exist.  And this highlights an alternative approach.  Add custom code like that to each of the POJOs to validate any Stellar fields.  Personally, I like the annotation based approach.  
    
    But whichever way we do it, we'd probably want to see it done the same across the board.  There are many other places where no validation of Stellar fields happen, right?


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    In other words, a stellar function that called the stellar compilation stuff, did not seem correct.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154124236
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    True, but FieldUtils does a class, I don't think it recurses.
    So we would still need to do some work


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154143988
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    Gotcha.  Understand your point now.  Seems like we should be able to handle that in a generic way for any configuration object.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158583014
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationProvider.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +import static org.apache.metron.common.configuration.ConfigurationType.PROFILER;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.profiler.ProfileConfig;
    +import org.apache.metron.common.configuration.profiler.ProfilerConfig;
    +import org.apache.metron.common.utils.JSONUtils;
    +import org.apache.metron.stellar.common.utils.validation.ExpressionConfigurationHolder;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfigurationProvider;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfiguredStatementContainer.ErrorConsumer;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +
    +/**
    + * {@code ConfigurationProvider} is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class ConfigurationProvider implements StellarConfigurationProvider {
    +
    +  /**
    +   * Default constructor.
    +   */
    +  public ConfigurationProvider() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public List<ExpressionConfigurationHolder> provideConfigurations(CuratorFramework client,
    +      ErrorConsumer errorConsumer) {
    +    List<ExpressionConfigurationHolder> holders = new LinkedList<>();
    +    visitParserConfigs(client, holders, errorConsumer);
    +    visitEnrichmentConfigs(client, holders, errorConsumer);
    +    visitProfilerConfigs(client, holders, errorConsumer);
    +    return holders;
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client,
    +      List<ExpressionConfigurationHolder> holders, ErrorConsumer errorConsumer) {
    +    List<String> children = null;
    +
    +    try {
    +      children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    } catch (Exception nne) {
    +      return;
    +    }
    +    for (String child : children) {
    +      try {
    +        byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        ExpressionConfigurationHolder holder = new ExpressionConfigurationHolder(
    +            String.format("%s/%s", getName(), PARSER.toString()), parserConfig.getSensorTopic(),
    +            parserConfig);
    +        holders.add(holder);
    +      } catch (Exception e) {
    +        errorConsumer.consume(String.format("%s/%s/%s", getName(), PARSER.toString(), child), e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client,
    +      List<ExpressionConfigurationHolder> holders, ErrorConsumer errorConsumer) {
    +    List<String> children = null;
    +
    +    try {
    +      children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    } catch (Exception nne) {
    +      return;
    --- End diff --
    
    Need to log and comment here.  We are silently eating the exception.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588450
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationProvider.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +import static org.apache.metron.common.configuration.ConfigurationType.PROFILER;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.profiler.ProfileConfig;
    +import org.apache.metron.common.configuration.profiler.ProfilerConfig;
    +import org.apache.metron.common.utils.JSONUtils;
    +import org.apache.metron.stellar.common.utils.validation.ExpressionConfigurationHolder;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfigurationProvider;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfiguredStatementContainer.ErrorConsumer;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +
    +/**
    + * {@code ConfigurationProvider} is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class ConfigurationProvider implements StellarConfigurationProvider {
    +
    +  /**
    +   * Default constructor.
    +   */
    +  public ConfigurationProvider() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public List<ExpressionConfigurationHolder> provideConfigurations(CuratorFramework client,
    +      ErrorConsumer errorConsumer) {
    +    List<ExpressionConfigurationHolder> holders = new LinkedList<>();
    +    visitParserConfigs(client, holders, errorConsumer);
    +    visitEnrichmentConfigs(client, holders, errorConsumer);
    +    visitProfilerConfigs(client, holders, errorConsumer);
    --- End diff --
    
    I *think* you're right.  Would suggest just a comment to clarify that point and maybe help prompt us should that change
    ```
    // indexing contains no stellar to validate
    ```


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158583416
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    What is the following code block doing?  Why do we need to discover all of the StellarConfigurationProvider classes?


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Refactored based on feedback for some things, based on making what I was trying for more correct in others.



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154135697
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    right, for SensorEnrichmentConfig, I would have to know about the 'inner' config objects and do them to, just as I do now.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588357
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationProvider.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +import static org.apache.metron.common.configuration.ConfigurationType.PROFILER;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.profiler.ProfileConfig;
    +import org.apache.metron.common.configuration.profiler.ProfilerConfig;
    +import org.apache.metron.common.utils.JSONUtils;
    +import org.apache.metron.stellar.common.utils.validation.ExpressionConfigurationHolder;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfigurationProvider;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfiguredStatementContainer.ErrorConsumer;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +
    +/**
    + * {@code ConfigurationProvider} is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class ConfigurationProvider implements StellarConfigurationProvider {
    +
    +  /**
    +   * Default constructor.
    +   */
    +  public ConfigurationProvider() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public List<ExpressionConfigurationHolder> provideConfigurations(CuratorFramework client,
    +      ErrorConsumer errorConsumer) {
    +    List<ExpressionConfigurationHolder> holders = new LinkedList<>();
    +    visitParserConfigs(client, holders, errorConsumer);
    +    visitEnrichmentConfigs(client, holders, errorConsumer);
    +    visitProfilerConfigs(client, holders, errorConsumer);
    +    return holders;
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client,
    +      List<ExpressionConfigurationHolder> holders, ErrorConsumer errorConsumer) {
    +    List<String> children = null;
    +
    +    try {
    +      children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    } catch (Exception nne) {
    +      return;
    --- End diff --
    
    Will do


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r160026668
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    OK, I think I understand where you are coming from.  In the end, there will be more functions for validation.   Those which take in a passed in object, like CONFIG_GET() -> obj, EDIT -> obj, VALIDATE_CONFIG(obj).  Those functions will be user driven, ie.  __**Validate this thing I gave you__**.
    
    These functions are __**Validate everything the system knows about**.  And are born out of the introduction of possible errors through language changes in already deployed stellar rules.
    
    That is why we need discovery.  The "job" of this function at the moment is to find everything and make sure it is still OK.



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158584184
  
    --- Diff: metron-platform/metron-management/README.md ---
    @@ -276,6 +278,13 @@ The functions are split roughly into a few sections:
         * aggregatorConfig - Optional config for aggregator
       * Returns: The String representation of the enrichment config
     
    +### Validation Functions
    +* `VALIDATE_STELLAR_RULE_CONFIGS`
    --- End diff --
    
    This name confuses me, why Rules?  `VALIDATE_STELLAR` seems simple and to the point to me.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588382
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationProvider.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +import static org.apache.metron.common.configuration.ConfigurationType.PROFILER;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.profiler.ProfileConfig;
    +import org.apache.metron.common.configuration.profiler.ProfilerConfig;
    +import org.apache.metron.common.utils.JSONUtils;
    +import org.apache.metron.stellar.common.utils.validation.ExpressionConfigurationHolder;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfigurationProvider;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfiguredStatementContainer.ErrorConsumer;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +
    +/**
    + * {@code ConfigurationProvider} is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class ConfigurationProvider implements StellarConfigurationProvider {
    +
    +  /**
    +   * Default constructor.
    +   */
    +  public ConfigurationProvider() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public List<ExpressionConfigurationHolder> provideConfigurations(CuratorFramework client,
    +      ErrorConsumer errorConsumer) {
    +    List<ExpressionConfigurationHolder> holders = new LinkedList<>();
    +    visitParserConfigs(client, holders, errorConsumer);
    +    visitEnrichmentConfigs(client, holders, errorConsumer);
    +    visitProfilerConfigs(client, holders, errorConsumer);
    +    return holders;
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client,
    +      List<ExpressionConfigurationHolder> holders, ErrorConsumer errorConsumer) {
    +    List<String> children = null;
    +
    +    try {
    +      children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    } catch (Exception nne) {
    +      return;
    +    }
    +    for (String child : children) {
    +      try {
    +        byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        ExpressionConfigurationHolder holder = new ExpressionConfigurationHolder(
    +            String.format("%s/%s", getName(), PARSER.toString()), parserConfig.getSensorTopic(),
    +            parserConfig);
    +        holders.add(holder);
    +      } catch (Exception e) {
    +        errorConsumer.consume(String.format("%s/%s/%s", getName(), PARSER.toString(), child), e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client,
    +      List<ExpressionConfigurationHolder> holders, ErrorConsumer errorConsumer) {
    +    List<String> children = null;
    +
    +    try {
    +      children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    } catch (Exception nne) {
    +      return;
    --- End diff --
    
    Will do


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    @cestella I would say that proposed validate function has to be very much in a namespace. It feels like a name that would be much more useful for a function replacing our current approach to global validation in the future than config validation, other than that it sounds like a good idea.


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Resolved conflicts, in the event we ever get around to this


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Do you feel strongly that this should be a Function?  @cestella ?  I'm not opposed to changing it if you are.  I would like to here some more feedback


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    @nickwallen I have refactored to a function.
    see updated PR description


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588419
  
    --- Diff: metron-platform/metron-management/README.md ---
    @@ -276,6 +278,13 @@ The functions are split roughly into a few sections:
         * aggregatorConfig - Optional config for aggregator
       * Returns: The String representation of the enrichment config
     
    +### Validation Functions
    +* `VALIDATE_STELLAR_RULE_CONFIGS`
    --- End diff --
    
    Is you search the tree for 'rules' you will see that we call them rule or rules in various places.  It 'seemed like the thing to do'.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158583497
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationProvider.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +import static org.apache.metron.common.configuration.ConfigurationType.PROFILER;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.profiler.ProfileConfig;
    +import org.apache.metron.common.configuration.profiler.ProfilerConfig;
    +import org.apache.metron.common.utils.JSONUtils;
    +import org.apache.metron.stellar.common.utils.validation.ExpressionConfigurationHolder;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfigurationProvider;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfiguredStatementContainer.ErrorConsumer;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +
    +/**
    + * {@code ConfigurationProvider} is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class ConfigurationProvider implements StellarConfigurationProvider {
    --- End diff --
    
    This implementation of a StellarConfigurationProvider is one that talks to Zookeeper, right?  It retrieves Stellar configuration from Zookeeper. 
    
    Should we call it `ZookeeperConfigurationProvider` or something more descriptive?  


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    @ottobackwards Yes, definitely.  I like it at a 50k foot level.  The only thing that struck me was the need for the different annotation types.  But I haven't had a chance to dig into it yet.


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    https://github.com/apache/metron/pull/856/commits/65278a67a07f1c4c23ab2d95ebb6de92e1cac731 introduces conceptually what @nickwallen  and I have been discussing.
    I need to think about reworking the description.
    
    From the commit ->
    ----------------
    Refactor based on review and inspiration from review.
    Although the original implementation was functional, it required maintainence to keep current.
    The suggested 'best state' was to have it be possible, maybe through annotations, for the validation
    system to be able to handle any config, regarless or composition using annotations.
    That would leave it up to the implementor to propertly annotate thier configurations, and allow for support of new fields.
    
    This is an implementation of that.
    
    I have refactored the implemenations and details, but kept the discovery and mechanics ( loading and visitation ) somewhat the same.
    Hopefully keeping the good and reworking to a more sustainable solution.
    
    Several annotations where created to marks ceratin stellar configruation objects or scenarios.
    A holder object, to hold the configuration object, but knows how to process the annotations and run the visitation was added.
    This holder object and the annotations have parameters and handling for several special scenarios, such as 2x nested maps.
    
    This implementation should facilitate follow on work to validate files and streams and blobs by using implementing the StellarValidator interface
    and re-using the holder concept ( replacing the providers )


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    The next attempt at this, if there is one, should start off with some sort of consensus first. And some agreement on initial scope.  This PR would have been smaller and less ambitious, if that were true.  Or at least everyone would have been on the same page as to the 'why' of certain things.
    Lessons to learn :)



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154375781
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    but if you complain about scope or the review size @nickwallen I will be very cross


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154133974
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    > True, but FieldUtils does a class, I don't think it recurses.  So we would still need to do some work
    
    I think the logic would be something like...
    * get the configuration object
    * find what class the configuration object is
    * use `FieldUtils..getFieldsListWithAnnotation` to get all the fields that are Stellar
    * then for those fields that are Stellar, get the value from the configuration object 
    * that value is the stellar expression.  validate it.



---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    @simonellistonball, yes, the namespace should be part of the jira and interface design


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    It has been long enough that **I** don't even like this PR any more.
    
    -1


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r160511065
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    But there is no need to discover those, right?  Why have the extra discovery logic when we don't need it?  
    
    It is also more confusing from a user's perspective.  What I have on my class path decides what is going to get validated.  It is often very hard to control what gets added to a classpath.  That might lead to unexpected behavior.
    
    If you just removed the discovery logic and instantiated the `StellarConfigurationProvider`'s that you want to use directly...
    
    1. the functionality would work the same way, all the time, no matter what is on my class path.
    1. the implementation would be simpler.



---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    I am glad for the interest in this PR, and that it seems to have sparked some great ideas for continuing on.  
    
    What I would like to do is line it up as follows
    
    1. This PR with it's current scope and focus
    2. New Jira and PR(s) for the Stellar Functions/Namespace that @cestella  and @simonellistonball  mentioned
    3. Some research and possible prototyping of the Attributed approach @nickwallen has suggested ( which I agree with )
    
    Over the course of that work, and other work identified through working and reviewing it, I will iteratively refactor to a common code and reusable approach.
    



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154155446
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    @nickwallen, checking the reason field....  Because the rules are checked on deserialize, if they are wrong, the whole config fails ( the entire Enrichment ).  You *could* say that the checks that I do have are more than I need, and that rather than add reason, I should just get rid of the rules stuff all together.  What do you think?


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154114714
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    you would either have to 'know' about fields that were other nested types or do a recursive desert through the pojo though


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Why is this validation process driven by a %magic command?  
    
    Magics were made for functionality that cannot be implemented directly within a Stellar execution environment.  Often for answering questions 'about' the execution environment itself.  For example `%vars` or `%functions` or `%globals` all tell us about the Stellar execution environment.
    
    This logic doesn't seem like a good fit for a %magic, unless there is a limitation that I am not understanding.  This could be implemented, I would argue more simply, as a regular Stellar function.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r160030197
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    I think you're not understanding my point.   More specifically... What happens if another implementation gets added to the classpath?  
    
    > Based on your discovery logic here, just having a FilesystemConfigurationProvider on the classpath (or any other implementation) will cause the configuration in the file system to get validated. We don't want that to happen. We want the user to control that behavior.


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    ![stellar_validate](https://user-images.githubusercontent.com/1111551/33501645-54ef5320-d6ab-11e7-9e2b-7f8b3731974d.png)



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154116726
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    > you would either have to 'know' about fields that were other nested types or do a recursive desert through the pojo though
    
    Apache Commons has a `FieldUtils.getFieldsListWithAnnotation(...)`.  Once you get a 'configuration' class, then you just use that to get all the fields that have been marked with the annotation.
    



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158584449
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    If I understand this correctly, it seems that the `StellarConfigurationProvider` interface allows us to extend where configuration values get pulled in from.  In your current default implementation `ConfigurationProvider` you reach out to Zookeeper to pull in the config.
    
    If I wanted to validate configuration located on a file system, I would just create a `FilesystemConfigurationProvider` implementation of this interface.
    
    The decision as to whether I want to validate the config in Zookeeper, on the file system or both, needs to be user driven.  A user should make that decision based on how they call your new Stellar function.  
    
    Based on your discovery logic here, just having a `FilesystemConfigurationProvider` on the classpath (or any other implementation) will cause the configuration in the file system to get validated.  We don't want that to happen.  We want the user to control that behavior.
    
    So I don't think we really need this discovery logic, which nicely simplifies things.  I think we could just alter the `StellarValidater` interface to make this relationship simpler and more straight forward.  The `StellarConfigurationProvider` just gets passed in.
    
    ```
    StellarValidator {
       ...
       Iterable<ValidationResult> validate(StellarConfigurationProvider provider);
       ...
    }
    ```



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158583012
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationProvider.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +import static org.apache.metron.common.configuration.ConfigurationType.PROFILER;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.profiler.ProfileConfig;
    +import org.apache.metron.common.configuration.profiler.ProfilerConfig;
    +import org.apache.metron.common.utils.JSONUtils;
    +import org.apache.metron.stellar.common.utils.validation.ExpressionConfigurationHolder;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfigurationProvider;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfiguredStatementContainer.ErrorConsumer;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +
    +/**
    + * {@code ConfigurationProvider} is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class ConfigurationProvider implements StellarConfigurationProvider {
    +
    +  /**
    +   * Default constructor.
    +   */
    +  public ConfigurationProvider() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public List<ExpressionConfigurationHolder> provideConfigurations(CuratorFramework client,
    +      ErrorConsumer errorConsumer) {
    +    List<ExpressionConfigurationHolder> holders = new LinkedList<>();
    +    visitParserConfigs(client, holders, errorConsumer);
    +    visitEnrichmentConfigs(client, holders, errorConsumer);
    +    visitProfilerConfigs(client, holders, errorConsumer);
    +    return holders;
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client,
    +      List<ExpressionConfigurationHolder> holders, ErrorConsumer errorConsumer) {
    +    List<String> children = null;
    +
    +    try {
    +      children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    } catch (Exception nne) {
    +      return;
    --- End diff --
    
    Need to log and comment here.  We are silently eating the exception.  Seems especially problematic because of the overly generic `Exception` declaration that the Curator library gives us.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154241894
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    I *think* I know how to do it and not have it be terrible.  The question is if it should be in this PR or after


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154110355
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    This also makes it simple to validate the configuration, no matter where it rests.  
    
    To validate the configuration in Zk, you just read all the configs in from Zk, deserialize into POJOs, iterate all the fields looking for the annotation, if you find the annotation, then run your validation logic on the expression.
    
    To validate the configuration on the file system, you just read it all in from the FS, then repeat the same steps I described in the Zk scenario.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r161239454
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    https://issues.apache.org/jira/browse/METRON-989
    https://issues.apache.org/jira/browse/METRON-876
    
    @mattf-horton @cestella 
    
    I have implemented this as to not increase the amount of tie-in between stellar and metron, and support future non-metron configuration sources. 
    
    Thus, the sources of the configuration are discovered and not coupled.  I believe this is in the spirit if not the letter of the design discussions that we have had.
    
    Can you take a look?
    



---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    @nickwallen yeah, we need to cover a bunch a scenarios


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

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


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    @nickwallen @mattf-horton I think we can use the annotation approach to resolve [METRON-989](https://issues.apache.org/jira/browse/METRON-989) as well.
    
    Thoughts?


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Ok, I'll change it.  Feels a little crossing the streams, but we'll see


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158583046
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationProvider.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +import static org.apache.metron.common.configuration.ConfigurationType.PROFILER;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.profiler.ProfileConfig;
    +import org.apache.metron.common.configuration.profiler.ProfilerConfig;
    +import org.apache.metron.common.utils.JSONUtils;
    +import org.apache.metron.stellar.common.utils.validation.ExpressionConfigurationHolder;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfigurationProvider;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfiguredStatementContainer.ErrorConsumer;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +
    +/**
    + * {@code ConfigurationProvider} is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class ConfigurationProvider implements StellarConfigurationProvider {
    +
    +  /**
    +   * Default constructor.
    +   */
    +  public ConfigurationProvider() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public List<ExpressionConfigurationHolder> provideConfigurations(CuratorFramework client,
    +      ErrorConsumer errorConsumer) {
    +    List<ExpressionConfigurationHolder> holders = new LinkedList<>();
    +    visitParserConfigs(client, holders, errorConsumer);
    +    visitEnrichmentConfigs(client, holders, errorConsumer);
    +    visitProfilerConfigs(client, holders, errorConsumer);
    --- End diff --
    
    What about indexing?


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

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


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588377
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigurationProvider.java ---
    @@ -0,0 +1,144 @@
    +/*
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +import static org.apache.metron.common.configuration.ConfigurationType.PROFILER;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.profiler.ProfileConfig;
    +import org.apache.metron.common.configuration.profiler.ProfilerConfig;
    +import org.apache.metron.common.utils.JSONUtils;
    +import org.apache.metron.stellar.common.utils.validation.ExpressionConfigurationHolder;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfigurationProvider;
    +import org.apache.metron.stellar.common.utils.validation.StellarConfiguredStatementContainer.ErrorConsumer;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +
    +/**
    + * {@code ConfigurationProvider} is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class ConfigurationProvider implements StellarConfigurationProvider {
    +
    +  /**
    +   * Default constructor.
    +   */
    +  public ConfigurationProvider() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public List<ExpressionConfigurationHolder> provideConfigurations(CuratorFramework client,
    +      ErrorConsumer errorConsumer) {
    +    List<ExpressionConfigurationHolder> holders = new LinkedList<>();
    +    visitParserConfigs(client, holders, errorConsumer);
    +    visitEnrichmentConfigs(client, holders, errorConsumer);
    +    visitProfilerConfigs(client, holders, errorConsumer);
    +    return holders;
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client,
    +      List<ExpressionConfigurationHolder> holders, ErrorConsumer errorConsumer) {
    +    List<String> children = null;
    +
    +    try {
    +      children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    } catch (Exception nne) {
    +      return;
    --- End diff --
    
    will do


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158584054
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarConfiguredStatementContainer.java ---
    @@ -0,0 +1,38 @@
    +/**
    + * 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.stellar.common.utils.validation;
    +
    +/**
    + * StellarConfiguredStatementProviders are used provide stellar statements
    + * and the context around those statements to the caller
    + */
    +public interface StellarConfiguredStatementContainer {
    --- End diff --
    
    `ExpressionConfigurationHolder` is an implementation of this interface. That being said, I don't understand the point of this interface.  
    
    In all the code that I see, you use the implementation class `ExpressionConfigurationHolder` rather than this interface.  For example, in `StellarZookeeperBasedValidator` and other places.  
    
    We should either use the interface or get rid of it.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r160529767
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    You mean couple it to metron?  We don't want to do that anymore.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158582738
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/annotations/StellarExpressionMap.java ---
    @@ -0,0 +1,83 @@
    +/**
    + * 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.stellar.common.utils.validation.annotations;
    +
    +import java.lang.annotation.Documented;
    +import java.lang.annotation.ElementType;
    +import java.lang.annotation.Retention;
    +import java.lang.annotation.RetentionPolicy;
    +import java.lang.annotation.Target;
    +
    +/**
    + * {@code StellarExpressionMap} is applied to
    + * a {@code Map} which contains Stellar expressions as the values, such
    + * that calling .toString() on a value of this map yields a Stellar expression.
    + *
    + * The key is used as the name the expression.
    + *
    + * It is possible for a {@code Map} to contain other maps or complex objects,
    + * thus this annotation contains properties that give information on evaluation of the {@code Map}
    + * should it be complex and contain nested maps.
    + *
    + */
    +@Documented
    +@Retention(RetentionPolicy.RUNTIME)
    +@Target({ElementType.FIELD,ElementType.TYPE})
    +public @interface StellarExpressionMap {
    +
    +  /**
    +   * The Name to give to the map
    +   *
    +   * @return String of the name
    +   */
    +  String name() default "default";
    +
    +  /**
    +   * A map may be a StellarExpressionMap based on the type
    +   * of another field, this is that field's name.
    +   *
    +   * {@code} qualify_with_field_type} is the type of that field
    +   *
    +   * @return Field Name or empty String
    +   */
    +  String qualify_with_field() default "";
    --- End diff --
    
    These variables_go_against_the_style convention, no?  qualify_with_field, qualify_with_field_type, inner_map_keys


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    Also, it might be useful for `%validate_configured_expressions` to take a file path so you can validate a set of configs on disk (again, if it gets to zookeeper, zk_load_utils.sh should fail if it's invalid)


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154245631
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    I think I can do it such that the stellar-common holds the code for doing the traversals as well as the interfaces and callbacks, and the *reporter* just provides the configs.
    
    I'm thinking it through, but now that I'm thinking of it I will be miserable until I do it so I'll do it.


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    This has gone from a small thing to at least 'say' we have a way to check if we broke all your stellar stuff after upgrade, to stretching it based on feedback which was a mistake, to a run around to abandoned review status.
    
    I'm going shelve this



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588712
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/annotations/StellarExpressionMap.java ---
    @@ -0,0 +1,83 @@
    +/**
    + * 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.stellar.common.utils.validation.annotations;
    +
    +import java.lang.annotation.Documented;
    +import java.lang.annotation.ElementType;
    +import java.lang.annotation.Retention;
    +import java.lang.annotation.RetentionPolicy;
    +import java.lang.annotation.Target;
    +
    +/**
    + * {@code StellarExpressionMap} is applied to
    + * a {@code Map} which contains Stellar expressions as the values, such
    + * that calling .toString() on a value of this map yields a Stellar expression.
    + *
    + * The key is used as the name the expression.
    + *
    + * It is possible for a {@code Map} to contain other maps or complex objects,
    + * thus this annotation contains properties that give information on evaluation of the {@code Map}
    + * should it be complex and contain nested maps.
    + *
    + */
    +@Documented
    +@Retention(RetentionPolicy.RUNTIME)
    +@Target({ElementType.FIELD,ElementType.TYPE})
    +public @interface StellarExpressionMap {
    +
    +  /**
    +   * The Name to give to the map
    +   *
    +   * @return String of the name
    +   */
    +  String name() default "default";
    +
    +  /**
    +   * A map may be a StellarExpressionMap based on the type
    +   * of another field, this is that field's name.
    +   *
    +   * {@code} qualify_with_field_type} is the type of that field
    +   *
    +   * @return Field Name or empty String
    +   */
    +  String qualify_with_field() default "";
    --- End diff --
    
    oops


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    It did not seem appropriate to me for this to be a stellar function, and %magic is the other way to execute things from the shell.  At the time at least.



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r165048687
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    @cestella ping


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154102327
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    I like what you are doing with this PR.
    
    There is just one downside that bugs me. How do we ensure that as our configuration evolves over time that we also update this to ensure it gets validated?  
    
    We have Stellar expressions as configurations all over the place; parsing, enrichment, triage, indexing, and profiler.  I feel like any/all of these will evolve and change over time.  I would definitely forget to come here and update this to make sure it gets validated.
    
    As a motivating example, right here you are only validating the `rule` field.  But triage also has the `reason` field that is a Stellar expression.  That would need to be validated also.  That was something we added later and likely the scenario where I would forget to add that expression to your validation logic.
    



---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154375558
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    I think you are really going to like this @nickwallen 


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r160530273
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    Maybe that is where we are missing each other? 


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    @simonellistonball Agree to the namespace idea.  My bad :)


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r158588686
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    The idea is that not only can we not know the details of the classes that hold the rules, but also, that stellar may be hosted by other things than metron, that 'know' how to provide those configurations.  
    
    The problem with this isn't the discovery per se, but in that it is not correct given it's purpose and the implementation


---

[GitHub] metron issue #856: METRON-1339 Stellar Shell functionality to verify stored ...

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

    https://github.com/apache/metron/pull/856
  
    @cestella @simonellistonball 
    With the new implementation, doing the blob or file check should be a piece of cake.... would you prefer it as part of this or as a new issue?


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154110765
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    So - if I'm understanding you correctly, as it applies to what I am doing ->
    when visiting configs, instead of explicitly validating fields, we would want to 'visit' all the members per pojo by attribute.  We would only need to worry about tracking the pojos but not the fields ( we need to keep the context ).  Am I understanding you correctly?


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r154105765
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/StellarStatementReporter.java ---
    @@ -0,0 +1,166 @@
    +/**
    + * 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;
    +
    +import static org.apache.metron.common.configuration.ConfigurationType.ENRICHMENT;
    +import static org.apache.metron.common.configuration.ConfigurationType.PARSER;
    +
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.common.configuration.enrichment.EnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
    +import org.apache.metron.common.configuration.enrichment.threatintel.RiskLevelRule;
    +import org.apache.metron.common.configuration.enrichment.threatintel.ThreatTriageConfig;
    +import org.apache.metron.common.field.transformation.FieldTransformation;
    +import org.apache.metron.common.field.transformation.StellarTransformation;
    +import org.apache.metron.common.utils.StringUtils;
    +import org.apache.metron.stellar.common.StellarConfiguredStatementReporter;
    +
    +/**
    + * StellarStatementReporter is used to report all of the configured / deployed Stellar statements in
    + * the system.
    + */
    +public class StellarStatementReporter implements StellarConfiguredStatementReporter {
    +
    +  public enum Type {
    +    ENRICHMENT, THREAT_INTEL;
    +  }
    +
    +  public StellarStatementReporter() {
    +  }
    +
    +  @Override
    +  public String getName() {
    +    return "Apache Metron";
    +  }
    +
    +  @Override
    +  public void vist(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    visitParserConfigs(client, visitor, errorConsumer);
    +    visitEnrichmentConfigs(client, visitor, errorConsumer);
    +  }
    +
    +  private void visitParserConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(PARSER.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(PARSER.getZookeeperRoot() + "/" + child);
    +      try {
    +        SensorParserConfig parserConfig = SensorParserConfig.fromBytes(data);
    +        List<FieldTransformer> transformations = parserConfig.getFieldTransformations();
    +        transformations.forEach((f) -> {
    +          if (StellarTransformation.class.isAssignableFrom(f.getFieldTransformation().getClass())) {
    +            FieldTransformation transformation = f.getFieldTransformation();
    +            f.getConfig().forEach((k, v) -> {
    +              List<String> names = Arrays
    +                  .asList(getName(), PARSER.toString(), parserConfig.getSensorTopic(), k);
    +              visitor.visit(names, v.toString());
    +            });
    +          }
    +        });
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), PARSER.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentConfigs(CuratorFramework client, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    List<String> children = client.getChildren().forPath(ENRICHMENT.getZookeeperRoot());
    +    for (String child : children) {
    +      byte[] data = client.getData().forPath(ENRICHMENT.getZookeeperRoot() + "/" + child);
    +      try {
    +        final SensorEnrichmentConfig sensorEnrichmentConfig = SensorEnrichmentConfig
    +            .fromBytes(data);
    +
    +        EnrichmentConfig enrichmentConfig = null;
    +        enrichmentConfig = sensorEnrichmentConfig.getEnrichment();
    +        visitEnrichementConfig(child, Type.ENRICHMENT, enrichmentConfig, visitor, errorConsumer);
    +        enrichmentConfig = sensorEnrichmentConfig.getThreatIntel();
    +        visitEnrichementConfig(child, Type.THREAT_INTEL, enrichmentConfig, visitor, errorConsumer);
    +        ThreatTriageConfig threatTriageConfig = sensorEnrichmentConfig.getThreatIntel()
    +            .getTriageConfig();
    +        visitEnrichmentThreatTriageConfigs(child, threatTriageConfig, visitor, errorConsumer);
    +      } catch (Exception e) {
    +        List<String> names = Arrays.asList(getName(), ENRICHMENT.toString(), child);
    +        errorConsumer.consume(names, e);
    +      }
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichementConfig(String topicName, Type type,
    +      EnrichmentConfig enrichmentConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +
    +    Map<String, Object> enrichmentStellarMap = (Map<String, Object>) enrichmentConfig.getFieldMap()
    +        .getOrDefault("stellar", new HashMap<>());
    +    Map<String, Object> transforms = (Map<String, Object>) enrichmentStellarMap
    +        .getOrDefault("config", new HashMap<>());
    +    try {
    +      for (Map.Entry<String, Object> kv : transforms.entrySet()) {
    +        // we can have a group or an entry
    +        if (kv.getValue() instanceof Map) {
    +          Map<String, String> groupMap = (Map<String, String>) kv.getValue();
    +          for (Map.Entry<String, String> groupKv : groupMap.entrySet()) {
    +            List<String> names = Arrays
    +                .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), kv.getKey(),
    +                    groupKv.getKey());
    +            visitor.visit(names, groupKv.getValue());
    +          }
    +        } else {
    +          List<String> names = Arrays
    +              .asList(getName(), ENRICHMENT.toString(), topicName, type.toString(), "(default)",
    +                  kv.getKey(), kv.getKey());
    +          visitor.visit(names, kv.getValue().toString());
    +        }
    +      }
    +    } catch (Exception e) {
    +      List<String> names = Arrays
    +          .asList(getName(), ENRICHMENT.toString(), topicName, type.toString());
    +      errorConsumer.consume(names, e);
    +    }
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  private void visitEnrichmentThreatTriageConfigs(String topicName,
    +      ThreatTriageConfig threatTriageConfig, StatementReportVisitor visitor,
    +      ConfigReportErrorConsumer errorConsumer) throws Exception {
    +    try {
    +      List<RiskLevelRule> riskLevelRules = threatTriageConfig.getRiskLevelRules();
    +      riskLevelRules.forEach((r) -> {
    +        String name = r.getName();
    +        if (org.apache.commons.lang.StringUtils.isEmpty(name)) {
    +          name = "(default)";
    +        }
    +        List<String> names = Arrays
    +            .asList(getName(), ENRICHMENT.toString(), topicName, "THREAT_TRIAGE", name);
    +        visitor.visit(names, r.getRule());
    --- End diff --
    
    And let me follow-on with one proposal that might help address this.  There are probably other (better?) ways to solve this, but here is one approach to chew on.
    
    All of our configuration gets deserialized from JSON into POJOs; EnrichmentConfig, ProfilerConfig, etc.  What if we had an annotation that we marked the fields that are required to be valid Stellar expressions?  Based on the annotation we can then validate each of those fields.
    
    Since the annotation is directly in the configuration classes, it is less likely I'm going to forget that annotation.  And it is also remains decoupled, which is a good benefit of your current approach.


---

[GitHub] metron pull request #856: METRON-1339 Stellar Shell functionality to verify ...

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

    https://github.com/apache/metron/pull/856#discussion_r159999157
  
    --- Diff: metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/utils/validation/StellarZookeeperBasedValidator.java ---
    @@ -0,0 +1,106 @@
    +/*
    + *
    + *  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.stellar.common.utils.validation;
    +
    +import static org.apache.metron.stellar.common.shell.StellarShell.ERROR_PROMPT;
    +
    +import java.lang.invoke.MethodHandles;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import org.apache.commons.lang.NullArgumentException;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.metron.stellar.common.StellarProcessor;
    +import org.atteo.classindex.ClassIndex;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class StellarZookeeperBasedValidator implements StellarValidator {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
    +  private static final String FAILED_COMPILE = "Failed to compile";
    +  private CuratorFramework client;
    +
    +  public StellarZookeeperBasedValidator(CuratorFramework client) throws NullArgumentException {
    +    if (client == null) {
    +      throw new NullArgumentException("client");
    +    }
    +    this.client = client;
    +  }
    +
    +
    +  @Override
    +  public Iterable<ValidationResult> validate(Optional<LineWriter> writer) {
    +    // discover all the StellarConfigurationProvider
    +    Set<StellarConfigurationProvider> providerSet = new HashSet<>();
    +
    +    for (Class<?> c : ClassIndex.getSubclasses(StellarConfigurationProvider.class,
    --- End diff --
    
    I am not following your explanation of why we need the discovery logic.  Can you try to explain it again?  In the latest commits, I still see the discovery logic in StellarSimpleValidator.


---