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

[GitHub] incubator-metron pull request #207: METRON-362: Unify the Stellar Languages

GitHub user cestella opened a pull request:

    https://github.com/apache/incubator-metron/pull/207

    METRON-362: Unify the Stellar Languages

    At the moment, stellar has some architectural issues:
    * The query and transformation languages are distinct despite sharing the same transformation functions
    * The transformation language does not have any notion of the comparison operations or the logical (aka boolean) functions from the query language
    * Neither language has the ability to do arithmetic or even represent negative numbers as constants
    * Neither language has the ability to do simple if/then/else constructs
    
    We should unify the languages and correct the deficiencies as they have come up in multiple situations. This should be done in a backwards compatible way so that deployed stellar statements as part of field transformations or threat triage do not have to change.
    
    This is completely tested via the unit test `StellarTest` in `metron-common` as well as the existing integration tests around FieldTransformations and Threat Triage.

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

    $ git pull https://github.com/cestella/incubator-metron METRON-362

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

    https://github.com/apache/incubator-metron/pull/207.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 #207
    
----
commit 261e393e91c6972b7da7a33a76e46e095892d592
Author: cstella <ce...@gmail.com>
Date:   2016-08-08T18:46:23Z

    METRON-362: Unify the Stellar Languages

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207#discussion_r74420102
  
    --- Diff: metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarTest.java ---
    @@ -0,0 +1,422 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.metron.common.stellar;
    +
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.metron.common.dsl.MapVariableResolver;
    +import org.apache.metron.common.dsl.ParseException;
    +import org.apache.metron.common.dsl.VariableResolver;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +public class StellarTest {
    +  @Test
    +  public void testIfThenElse() {
    +    {
    +      String query = "if 1 < 2 then 'one' else 'two'";
    +      Assert.assertEquals("one", run(query, ImmutableMap.of("foo", "casey ")));
    --- End diff --
    
    What does adding the variable foo='casey' have to do with these tests?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    Understand your reasoning on DSL package. Personally, I would collapse 'org.apache.metron.common.dsl' under the stellar package.  If we're going to use it a different way in the future then we can make that change in the future.  But right now, it just confuses the simplicity of having a single parent stellar package.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207#discussion_r74425090
  
    --- Diff: metron-platform/metron-common/src/main/java/org/apache/metron/common/stellar/StellarCompiler.java ---
    @@ -16,276 +16,224 @@
      * limitations under the License.
      */
     
    -package org.apache.metron.common.query;
    +package org.apache.metron.common.stellar;
     
     import com.google.common.base.Joiner;
     import com.google.common.collect.ImmutableSet;
     import org.apache.metron.common.dsl.*;
    -import org.apache.metron.common.query.generated.PredicateBaseListener;
    -import org.apache.metron.common.query.generated.PredicateParser;
    +import org.apache.metron.common.stellar.generated.StellarBaseListener;
    +import org.apache.metron.common.stellar.generated.StellarParser;
    +import org.apache.metron.common.utils.ConversionUtils;
     
     import java.util.*;
     import java.util.function.Function;
    -import java.util.function.Predicate;
     
    -class QueryCompiler extends PredicateBaseListener {
    +public class StellarCompiler extends StellarBaseListener {
       private VariableResolver resolver = null;
       private Stack<Token> tokenStack = new Stack<>();
    -
    -  public QueryCompiler(VariableResolver resolver) {
    +  public StellarCompiler(VariableResolver resolver) {
         this.resolver = resolver;
       }
     
       @Override
    -  public void enterSingle_rule(org.apache.metron.common.query.generated.PredicateParser.Single_ruleContext ctx) {
    +  public void enterTransformation(StellarParser.TransformationContext ctx) {
         tokenStack.clear();
       }
     
    -  @Override
    -  public void exitSingle_rule(org.apache.metron.common.query.generated.PredicateParser.Single_ruleContext ctx) {
    -  }
    +  private boolean handleIn(Token<?> left, Token<?> right) {
    +    Object key = null;
     
    -  @Override
    -  public void exitLogicalExpressionAnd(PredicateParser.LogicalExpressionAndContext ctx) {
    -    Token<?> left = popStack();
    -    Token<?> right = popStack();
    -    tokenStack.push(new Token<>(booleanOp(left, right, (l, r) -> l && r, "&&"), Boolean.class));
    -  }
    +    Set<Object> set = null;
    +    if(left.getValue() instanceof Collection) {
    +      set = new HashSet<>((List<Object>) left.getValue());
    +    }
    +    else if(left.getValue() != null) {
    +      set = ImmutableSet.of(left.getValue());
    +    }
    +    else {
    +      set = new HashSet<>();
    +    }
     
    -  @Override
    -  public void exitLogicalExpressionOr(PredicateParser.LogicalExpressionOrContext ctx) {
    -    Token<?> left = popStack();
    -    Token<?> right = popStack();
     
    -    tokenStack.push(new Token<>(booleanOp(left, right, (l, r) -> l || r, "||"), Boolean.class));
    +    key = right.getValue();
    +    if(key == null || set.isEmpty()) {
    +      return false;
    +    }
    +    return set.contains(key);
       }
     
    -  private boolean booleanOp(Token<?> left, Token<?> right, BooleanOp op, String opName)
    -  {
    -    if(left.getUnderlyingType().equals(right.getUnderlyingType()) && left.getUnderlyingType().equals(Boolean.class)) {
    -      Boolean l = (Boolean) left.getValue();
    -      Boolean r = (Boolean) right.getValue();
    -      if(l == null || r == null) {
    -        throw new ParseException("Unable to operate on " + left.getValue()  + " " + opName + " " + right.getValue() + ", null value");
    -      }
    -      return op.op(l, r);
    +  private Double getDouble(Token<?> token) {
    +    Number n = (Number)token.getValue();
    +    if(n == null) {
    +      return 0d;
         }
         else {
    -      throw new ParseException("Unable to operate on " + left.getValue()  + " " + opName + " " + right.getValue() + ", bad types");
    +      return n.doubleValue();
         }
       }
     
     
    -  @Override
    -  public void exitLogicalConst(PredicateParser.LogicalConstContext ctx) {
    -    Boolean b = null;
    -    switch(ctx.getText().toUpperCase()) {
    -      case "TRUE":
    -        b = true;
    -        break;
    -      case "FALSE":
    -        b = false;
    -        break;
    -      default:
    -        throw new ParseException("Unable to process " + ctx.getText() + " as a boolean constant");
    -    }
    -    tokenStack.push(new Token<>(b, Boolean.class));
    +
    +
    +  /*@Override
    +  public void enterArithExpr_plus(StellarParser.ArithExpr_plusContext ctx) {
    --- End diff --
    
    Assuming we don't need this since it is commented out?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    I will defend the org.apache.metron.common.dsl package.  That is intended to be utilities for domain specific languages (like variable resolvers, etc).  My intention was to open up the possibility to reuse across *other* DSLs that we might need going forward.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    @nickwallen There is no longer an org.apache.metron.common.{transformation,query} and there is, as part of this PR an org.apache.metron.common.stellar.  Not sure where you're seeing these packages continuing to exist.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    Oops, maybe i checked out the wrong PR.  That would explain a lot.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    +1 via inspection 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron pull request #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207#discussion_r74420961
  
    --- Diff: metron-platform/metron-common/src/test/java/org/apache/metron/common/stellar/StellarTest.java ---
    @@ -0,0 +1,422 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.metron.common.stellar;
    +
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.metron.common.dsl.MapVariableResolver;
    +import org.apache.metron.common.dsl.ParseException;
    +import org.apache.metron.common.dsl.VariableResolver;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +public class StellarTest {
    +  @Test
    +  public void testIfThenElse() {
    +    {
    +      String query = "if 1 < 2 then 'one' else 'two'";
    +      Assert.assertEquals("one", run(query, ImmutableMap.of("foo", "casey ")));
    +    }
    +    {
    +      String query = "if 1 + 1 < 2 then 'one' else 'two'";
    +      Assert.assertEquals("two", run(query, ImmutableMap.of("foo", "casey ")));
    +    }
    +    {
    +      String query = "1 < 2 ? 'one' : 'two'";
    +      Assert.assertEquals("one", run(query, ImmutableMap.of("foo", "casey ")));
    +    }
    +    {
    +      String query = "if not(1 < 2) then 'one' else 'two'";
    +      Assert.assertEquals("two", run(query, ImmutableMap.of("foo", "casey ")));
    +    }
    +    {
    +      String query = "if 1 == 1.000001 then 'one' else 'two'";
    +      Assert.assertEquals("one", run(query, ImmutableMap.of("foo", "casey ")));
    +    }
    +    {
    +      String query = "if one < two then 'one' else 'two'";
    +      Assert.assertEquals("one", run(query, ImmutableMap.of("one", 1, "two", 2)));
    +    }
    +    {
    +      String query = "if one == very_nearly_one then 'one' else 'two'";
    +      Assert.assertEquals("one", run(query, ImmutableMap.of("one", 1, "very_nearly_one", 1.000001)));
    +    }
    +  }
    +
    +  @Test
    +  public void testNumericOperations() {
    +    {
    +      String query = "TO_INTEGER(1 + 2*2 + 3 - 4 - 0.5)";
    +      Assert.assertEquals(3, (Integer)run(query, ImmutableMap.of("one", 1, "very_nearly_one", 1.000001)), 1e-6);
    +    }
    +    {
    +      String query = "1 + 2*2 + 3 - 4 - 0.5";
    +      Assert.assertEquals(3.5, (Double)run(query, ImmutableMap.of("one", 1, "very_nearly_one", 1.000001)), 1e-6);
    +    }
    +    {
    +      String query = "2*one*(1 + 2*2 + 3 - 4)";
    +      Assert.assertEquals(8, (Double)run(query, ImmutableMap.of("one", 1, "very_nearly_one", 1.000001)), 1e-6);
    +    }
    +    {
    +      String query = "2*(1 + 2 + 3 - 4)";
    +      Assert.assertEquals(4, (Double)run(query, ImmutableMap.of("one", 1, "very_nearly_one", 1.000001)), 1e-6);
    +    }
    +    {
    +      String query = "1 + 2 + 3 - 4 - 2";
    +      Assert.assertEquals(0, (Double)run(query, ImmutableMap.of("one", 1, "very_nearly_one", 1.000001)), 1e-6);
    +    }
    +    {
    +      String query = "1 + 2 + 3 + 4";
    +      Assert.assertEquals(10, (Double)run(query, ImmutableMap.of("one", 1, "very_nearly_one", 1.000001)), 1e-6);
    +    }
    +    {
    +      String query = "(one + 2)*3";
    +      Assert.assertEquals(9, (Double)run(query, ImmutableMap.of("one", 1, "very_nearly_one", 1.000001)), 1e-6);
    +    }
    +    {
    +      String query = "TO_INTEGER((one + 2)*3.5)";
    +      Assert.assertEquals(10, (Integer)run(query, ImmutableMap.of("one", 1, "very_nearly_one", 1.000001)), 1e-6);
    +    }
    +    {
    +      String query = "1 + 2*3";
    +      Assert.assertEquals(7, (Double)run(query, ImmutableMap.of("one", 1, "very_nearly_one", 1.000001)), 1e-6);
    +    }
    +
    +  }
    +
    +
    +  @Test
    +  public void testHappyPath() {
    +    String query = "TO_UPPER(TRIM(foo))";
    +    Assert.assertEquals("CASEY", run(query, ImmutableMap.of("foo", "casey ")));
    +  }
    +
    +  @Test
    +  public void testJoin() {
    +    String query = "JOIN( [ TO_UPPER(TRIM(foo)), 'bar' ], ',')";
    +    Assert.assertEquals("CASEY,bar", run(query, ImmutableMap.of("foo", "casey ")));
    +  }
    +
    +  @Test
    +  public void testSplit() {
    +    String query = "JOIN( SPLIT(foo, ':'), ',')";
    +    Assert.assertEquals("casey,bar", run(query, ImmutableMap.of("foo", "casey:bar")));
    +  }
    +
    +  @Test
    +  public void testMapGet() {
    +    String query = "MAP_GET(dc, dc2tz, 'UTC')";
    +    Assert.assertEquals("UTC"
    +                       , run(query, ImmutableMap.of("dc", "nyc"
    +                                                   ,"dc2tz", ImmutableMap.of("la", "PST")
    +                                                   )
    +                            )
    +                       );
    +    Assert.assertEquals("EST"
    +                       , run(query, ImmutableMap.of("dc", "nyc"
    +                                                   ,"dc2tz", ImmutableMap.of("nyc", "EST")
    +                                                   )
    +                            )
    +                       );
    +  }
    +  @Test
    +  public void testTLDExtraction() {
    +    String query = "DOMAIN_TO_TLD(foo)";
    +    Assert.assertEquals("co.uk", run(query, ImmutableMap.of("foo", "www.google.co.uk")));
    +  }
    +
    +  @Test
    +  public void testTLDRemoval() {
    +    String query = "DOMAIN_REMOVE_TLD(foo)";
    +    Assert.assertEquals("www.google", run(query, ImmutableMap.of("foo", "www.google.co.uk")));
    +  }
    +
    +  @Test
    +  public void testSubdomainRemoval() {
    +    String query = "DOMAIN_REMOVE_SUBDOMAINS(foo)";
    +    Assert.assertEquals("google.co.uk", run(query, ImmutableMap.of("foo", "www.google.co.uk")));
    +    Assert.assertEquals("google.com", run(query, ImmutableMap.of("foo", "www.google.com")));
    +  }
    +  @Test
    +  public void testURLToHost() {
    +    String query = "URL_TO_HOST(foo)";
    +    Assert.assertEquals("www.google.co.uk", run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path")));
    +  }
    +
    +  @Test
    +  public void testURLToPort() {
    +    String query = "URL_TO_PORT(foo)";
    +    Assert.assertEquals(80, run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path")));
    +  }
    +
    +  @Test
    +  public void testURLToProtocol() {
    +    String query = "URL_TO_PROTOCOL(foo)";
    +    Assert.assertEquals("http", run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path")));
    +  }
    +
    +  @Test
    +  public void testURLToPath() {
    +    String query = "URL_TO_PATH(foo)";
    +    Assert.assertEquals("/my/path", run(query, ImmutableMap.of("foo", "http://www.google.co.uk/my/path")));
    +  }
    +
    +  @Test
    +  public void testProtocolToName() {
    +    String query = "PROTOCOL_TO_NAME(protocol)";
    +    Assert.assertEquals("TCP", run(query, ImmutableMap.of("protocol", "6")));
    +    Assert.assertEquals("TCP", run(query, ImmutableMap.of("protocol", 6)));
    +    Assert.assertEquals(null, run(query, ImmutableMap.of("foo", 6)));
    +    Assert.assertEquals("chicken", run(query, ImmutableMap.of("protocol", "chicken")));
    +  }
    +  @Test
    +  public void testDateConversion() {
    +    long expected =1452013350000L;
    +    {
    +      String query = "TO_EPOCH_TIMESTAMP(foo, 'yyyy-MM-dd HH:mm:ss', 'UTC')";
    +      Assert.assertEquals(expected, run(query, ImmutableMap.of("foo", "2016-01-05 17:02:30")));
    +    }
    +    {
    +      String query = "TO_EPOCH_TIMESTAMP(foo, 'yyyy-MM-dd HH:mm:ss')";
    +      Long ts = (Long) run(query, ImmutableMap.of("foo", "2016-01-05 17:02:30"));
    +      //is it within 24 hours of the UTC?
    +      Assert.assertTrue(Math.abs(ts - expected) < 8.64e+7);
    +    }
    +  }
    +
    +  @Test
    +  public void testToString() {
    +    Assert.assertEquals("5", run("TO_STRING(foo)", ImmutableMap.of("foo", 5)));
    +  }
    +
    +  @Test
    +  public void testToInteger() {
    +    Assert.assertEquals(5, run("TO_INTEGER(foo)", ImmutableMap.of("foo", "5")));
    +    Assert.assertEquals(5, run("TO_INTEGER(foo)", ImmutableMap.of("foo", 5)));
    +  }
    +
    +  @Test
    +  public void testToDouble() {
    +    Assert.assertEquals(new Double(5.1), run("TO_DOUBLE(foo)", ImmutableMap.of("foo", 5.1d)));
    +    Assert.assertEquals(new Double(5.1), run("TO_DOUBLE(foo)", ImmutableMap.of("foo", "5.1")));
    +  }
    +  @Test
    +  public void testGet() {
    +    Map<String, Object> variables = ImmutableMap.of("foo", "www.google.co.uk");
    +    Assert.assertEquals("www", run("GET_FIRST(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'))", variables));
    +    Assert.assertEquals("www", run("GET(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'), 0)", variables));
    +    Assert.assertEquals("google", run("GET_LAST(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'))", variables));
    +    Assert.assertEquals("google", run("GET(SPLIT(DOMAIN_REMOVE_TLD(foo), '.'), 1)", variables));
    +  }
    +  private static Object run(String rule, Map<String, Object> variables) {
    +    StellarProcessor processor = new StellarProcessor();
    +    Assert.assertTrue(rule + " not valid.", processor.validate(rule));
    +    return processor.parse(rule, x -> variables.get(x));
    +  }@Test
    --- End diff --
    
    Easy enough to have Intellij fix the spacing?  Sometimes we have spaces between methods, sometimes not, here there is no line break at all.  All simple, but add up to making it more difficult to read than it might need to be.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    Why is there still both a PredicateProcessor and a TransformationProcessor?  Can't we unify these? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    +1 by inspection


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    It would be good to add Javadocs to the StellarProcessor and related classes that describe your reasoning.  When am I supposed to use one or the other?  Right now there are no comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    Ran it up in quick dev. Worked as expected. +1.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    What is the reasoning to have Stellar not share a common parent package?  We have org.apache.metron.common.dsl, also org.apache.metron.common.query, org.apache.metron.common.transformation.  Aren't all these packages focused on Stellar?  Maybe org.apache.metron.common.stellar?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-metron issue #207: METRON-362: Unify the Stellar Languages

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

    https://github.com/apache/incubator-metron/pull/207
  
    @nickwallen There is still a PredicateProcessor because having a stellar function which acts as a predicate is a sensible differentiation from a stellar function which returns an arbitrary function.  Just as there is a java.util.function.Function and a java.util.function.Predicate and they are distinct.  They both reuse a base class, however.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---