Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id A79FB200CC9 for ; Mon, 3 Jul 2017 00:43:32 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id A43D9160BFE; Sun, 2 Jul 2017 22:43:32 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 58C45160C02 for ; Mon, 3 Jul 2017 00:43:30 +0200 (CEST) Received: (qmail 54182 invoked by uid 500); 2 Jul 2017 22:43:29 -0000 Mailing-List: contact commits-help@metron.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@metron.apache.org Delivered-To: mailing list commits@metron.apache.org Received: (qmail 54110 invoked by uid 99); 2 Jul 2017 22:43:29 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 02 Jul 2017 22:43:29 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 976CAF54E3; Sun, 2 Jul 2017 22:43:28 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: mattf@apache.org To: commits@metron.apache.org Date: Sun, 02 Jul 2017 22:43:36 -0000 Message-Id: <456e508288d8480781fe433c2465f7f5@git.apache.org> In-Reply-To: <4ac0f94065ed47019bb1312d60aa6c29@git.apache.org> References: <4ac0f94065ed47019bb1312d60aa6c29@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [10/25] metron git commit: METRON-877 Extract core implementation and UDF support, create metron-stellar module (mattf-horton) closes apache/metron#616 archived-at: Sun, 02 Jul 2017 22:43:32 -0000 http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/configuration/ConfigurationsUtils.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/configuration/ConfigurationsUtils.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/configuration/ConfigurationsUtils.java new file mode 100644 index 0000000..1add3ab --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/configuration/ConfigurationsUtils.java @@ -0,0 +1,208 @@ +/** + * 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.configuration; + +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.metron.stellar.common.Constants; +import org.apache.metron.stellar.common.utils.JSONUtils; +import org.apache.metron.stellar.dsl.Context; +import org.apache.metron.stellar.dsl.StellarFunctions; +import org.apache.zookeeper.KeeperException; + +import java.io.File; +import java.io.IOException; +import java.io.PrintStream; +import java.nio.file.Files; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.apache.metron.stellar.common.configuration.ConfigurationType.GLOBAL; + +public class ConfigurationsUtils { + + public static CuratorFramework getClient(String zookeeperUrl) { + RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); + return CuratorFrameworkFactory.newClient(zookeeperUrl, retryPolicy); + } + + public static void writeGlobalConfigToZookeeper(Map globalConfig, String zookeeperUrl) throws Exception { + try(CuratorFramework client = getClient(zookeeperUrl)) { + client.start(); + writeGlobalConfigToZookeeper(globalConfig, client); + } + } + public static void writeGlobalConfigToZookeeper(Map globalConfig, CuratorFramework client) throws Exception { + writeGlobalConfigToZookeeper(JSONUtils.INSTANCE.toJSON(globalConfig), client); + } + + public static void writeGlobalConfigToZookeeper(byte[] globalConfig, String zookeeperUrl) throws Exception { + try(CuratorFramework client = getClient(zookeeperUrl)) { + client.start(); + writeGlobalConfigToZookeeper(globalConfig, client); + } + } + + public static void writeGlobalConfigToZookeeper(byte[] globalConfig, CuratorFramework client) throws Exception { + GLOBAL.deserialize(new String(globalConfig)); + writeToZookeeper(GLOBAL.getZookeeperRoot(), globalConfig, client); + } + + public static void writeConfigToZookeeper(String name, Map config, String zookeeperUrl) throws Exception { + writeConfigToZookeeper(name, JSONUtils.INSTANCE.toJSON(config), zookeeperUrl); + } + + public static void writeConfigToZookeeper(String name, byte[] config, String zookeeperUrl) throws Exception { + try(CuratorFramework client = getClient(zookeeperUrl)) { + client.start(); + writeToZookeeper(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + name, config, client); + } + } + + public static void writeToZookeeper(String path, byte[] configData, CuratorFramework client) throws Exception { + try { + client.setData().forPath(path, configData); + } catch (KeeperException.NoNodeException e) { + client.create().creatingParentsIfNeeded().forPath(path, configData); + } + } + + public static byte[] readGlobalConfigBytesFromZookeeper(CuratorFramework client) throws Exception { + return readFromZookeeper(GLOBAL.getZookeeperRoot(), client); + } + + public static byte[] readConfigBytesFromZookeeper(String name, CuratorFramework client) throws Exception { + return readFromZookeeper(Constants.ZOOKEEPER_TOPOLOGY_ROOT + "/" + name, client); + } + + public static byte[] readFromZookeeper(String path, CuratorFramework client) throws Exception { + if(client != null && client.getData() != null && path != null) { + return client.getData().forPath(path); + } + return new byte[]{}; + } + +/* public static void uploadConfigsToZookeeper(String globalConfigPath, + String parsersConfigPath, + String enrichmentsConfigPath, + String indexingConfigPath, + String profilerConfigPath, + String zookeeperUrl) throws Exception { + try(CuratorFramework client = getClient(zookeeperUrl)) { + client.start(); + uploadConfigsToZookeeper(globalConfigPath, parsersConfigPath, enrichmentsConfigPath, indexingConfigPath, profilerConfigPath, client); + } + } + + public static void uploadConfigsToZookeeper(String rootFilePath, CuratorFramework client) throws Exception { + uploadConfigsToZookeeper(rootFilePath, rootFilePath, rootFilePath, rootFilePath, rootFilePath, client); + } + + public static void uploadConfigsToZookeeper(String globalConfigPath, + String parsersConfigPath, + String enrichmentsConfigPath, + String indexingConfigPath, + String profilerConfigPath, + CuratorFramework client) throws Exception { + + // global + if (globalConfigPath != null) { + final byte[] globalConfig = readGlobalConfigFromFile(globalConfigPath); + if (globalConfig.length > 0) { + setupStellarStatically(client, Optional.of(new String(globalConfig))); + ConfigurationsUtils.writeGlobalConfigToZookeeper(readGlobalConfigFromFile(globalConfigPath), client); + } + } + } + */ + + public static void setupStellarStatically(CuratorFramework client) throws Exception { + byte[] ret = null; + try { + ret = readGlobalConfigBytesFromZookeeper(client); + } + catch(KeeperException.NoNodeException nne) { + //can't find the node + } + if(ret == null || ret.length == 0) { + setupStellarStatically(client, Optional.empty()); + } + else { + setupStellarStatically(client, Optional.of(new String(ret))); + } + } + + public static void setupStellarStatically(CuratorFramework client, Optional globalConfig) { + /* + In order to validate stellar functions, the function resolver must be initialized. Otherwise, + those utilities that require validation cannot validate the stellar expressions necessarily. + */ + Context.Builder builder = new Context.Builder().with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client) + ; + if(globalConfig.isPresent()) { + builder = builder.with(Context.Capabilities.GLOBAL_CONFIG, () -> GLOBAL.deserialize(globalConfig.get())) + .with(Context.Capabilities.STELLAR_CONFIG, () -> GLOBAL.deserialize(globalConfig.get())); + } + else { + builder = builder.with(Context.Capabilities.STELLAR_CONFIG, () -> new HashMap<>()); + } + Context stellarContext = builder.build(); + StellarFunctions.FUNCTION_RESOLVER().initialize(stellarContext); + } + + public static byte[] readGlobalConfigFromFile(String rootPath) throws IOException { + byte[] globalConfig = new byte[0]; + File configPath = new File(rootPath, GLOBAL.getName() + ".json"); + if (configPath.exists()) { + globalConfig = Files.readAllBytes(configPath.toPath()); + } + return globalConfig; + } + + public interface ConfigurationVisitor{ + void visit(ConfigurationType configurationType, String name, String data); + } + + public static void visitConfigs(CuratorFramework client, final ConfigurationVisitor callback) throws Exception { + visitConfigs(client, (type, name, data) -> { + setupStellarStatically(client, Optional.ofNullable(data)); + callback.visit(type, name, data); + }, GLOBAL); + } + + public static void visitConfigs(CuratorFramework client, ConfigurationVisitor callback, ConfigurationType configType) throws Exception { + + if (client.checkExists().forPath(configType.getZookeeperRoot()) != null) { + + if (configType.equals(GLOBAL)) { + byte[] globalConfigData = client.getData().forPath(configType.getZookeeperRoot()); + callback.visit(configType, "global", new String(globalConfigData)); + } + } + } + + public static void dumpConfigs(PrintStream out, CuratorFramework client) throws Exception { + ConfigurationsUtils.visitConfigs(client, (type, name, data) -> { + type.deserialize(data); + out.println(type + " Config: " + name + "\n" + data); + }); + } +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ArithmeticEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ArithmeticEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ArithmeticEvaluator.java new file mode 100644 index 0000000..8c1c768 --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ArithmeticEvaluator.java @@ -0,0 +1,103 @@ +/* + * 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.evaluators; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.metron.stellar.dsl.Token; +import org.apache.metron.stellar.common.FrameContext; + +import java.util.function.BiFunction; + +public enum ArithmeticEvaluator { + INSTANCE; + + public Token evaluate(BiFunction> function, + Pair, Token> p) { + if (p == null || p.getKey() == null || p.getValue() == null) { + throw new IllegalArgumentException(); + } + + final Number l = p.getKey().getValue(); + final Number r = p.getValue().getValue(); + + return function.apply(l == null ? 0 : l, r == null ? 0 : r); + } + + /** + * This is a helper class that defines how to handle arithmetic operations. The conversion between number + * types is taken for the Java spec: http://docs.oracle.com/javase/specs/jls/se8/html/jls-5.html#jls-5.6.2 + */ + public static final class ArithmeticEvaluatorFunctions { + public static BiFunction> addition(final FrameContext.Context context) { + return (Number l, Number r) -> { + if (l instanceof Double || r instanceof Double) { + return new Token<>(l.doubleValue() + r.doubleValue(), Double.class, context); + } else if (l instanceof Float || r instanceof Float) { + return new Token<>(l.floatValue() + r.floatValue(), Float.class, context); + } else if (l instanceof Long || r instanceof Long) { + return new Token<>(l.longValue() + r.longValue(), Long.class, context); + } else { + return new Token<>(l.intValue() + r.intValue(), Integer.class, context); + } + }; + } + + public static BiFunction> multiplication(final FrameContext.Context context) { + return (Number l, Number r) -> { + if (l instanceof Double || r instanceof Double) { + return new Token<>(l.doubleValue() * r.doubleValue(), Double.class, context); + } else if (l instanceof Float || r instanceof Float) { + return new Token<>(l.floatValue() * r.floatValue(), Float.class, context); + } else if (l instanceof Long || r instanceof Long) { + return new Token<>(l.longValue() * r.longValue(), Long.class, context); + } else { + return new Token<>(l.intValue() * r.intValue(), Integer.class, context); + } + }; + } + + public static BiFunction> subtraction(final FrameContext.Context context) { + return (Number l, Number r) -> { + if (l instanceof Double || r instanceof Double) { + return new Token<>(l.doubleValue() - r.doubleValue(), Double.class, context); + } else if (l instanceof Float || r instanceof Float) { + return new Token<>(l.floatValue() - r.floatValue(), Float.class, context); + } else if (l instanceof Long || r instanceof Long) { + return new Token<>(l.longValue() - r.longValue(), Long.class, context); + } else { + return new Token<>(l.intValue() - r.intValue(), Integer.class, context); + } + }; + } + + public static BiFunction> division(FrameContext.Context context) { + return (Number l, Number r) -> { + if (l instanceof Double || r instanceof Double) { + return new Token<>(l.doubleValue() / r.doubleValue(), Double.class, context); + } else if (l instanceof Float || r instanceof Float) { + return new Token<>(l.floatValue() / r.floatValue(), Float.class, context); + } else if (l instanceof Long || r instanceof Long) { + return new Token<>(l.longValue() / r.longValue(), Long.class, context); + } else { + return new Token<>(l.intValue() / r.intValue(), Integer.class, context); + } + }; + } + } +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonExpressionEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonExpressionEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonExpressionEvaluator.java new file mode 100644 index 0000000..f244981 --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonExpressionEvaluator.java @@ -0,0 +1,42 @@ +/* + * 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.evaluators; + +import org.apache.metron.stellar.dsl.Token; +import org.apache.metron.stellar.common.generated.StellarParser; + +/** + * This is used to determine what is needed to evaluate a Stellar comparison expression. A Stellar comparison + * expression is an expression that uses operators such as {@literal '<', '<=', '>', '>=', '==', '!=' } to compare + * values in Stellar. There are two main types of comparisons in Stellar, + * {@literal equality ('==', '!=') and comparison ('<', '<=', '>', '>='). } + */ +public interface ComparisonExpressionEvaluator { + + /** + * This will compare the values of {@code left} and {@code right} using the {@code op} input to determine a value + * to return. + * @param left The token representing the left side of a comparison expression. + * @param right The token representing the right side of a comparison expression. + * @param op This is a representation of a comparison operator {@literal (eg. <, <=, >, >=, ==, !=) } + * @return True if the expression is evaluated to be true, otherwise false. An example of expressions that + * should be true are {@code 1 == 1}, {@code 1f > 0}, etc. + */ + boolean evaluate(Token left, Token right, StellarParser.ComparisonOpContext op); +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonExpressionWithOperatorEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonExpressionWithOperatorEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonExpressionWithOperatorEvaluator.java new file mode 100644 index 0000000..70bc4ca --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonExpressionWithOperatorEvaluator.java @@ -0,0 +1,96 @@ +/* + * 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.evaluators; + +import org.apache.metron.stellar.dsl.ParseException; +import org.apache.metron.stellar.dsl.Token; +import org.apache.metron.stellar.common.FrameContext; +import org.apache.metron.stellar.common.generated.StellarParser; + +/** + * This is the evaluator used when evaluating Stellar comparison operators. + * + * @see EqualityOperatorsEvaluator + * @see ComparisonOperatorsEvaluator + */ +public enum ComparisonExpressionWithOperatorEvaluator { + /** + * The instance of {@link ComparisonExpressionWithOperatorEvaluator} used in + * order to evaluate Stellar comparison expressions. + */ + INSTANCE; + + /** + * The different strategies used to evaluate a Stellar comparison operator. They are broken into + * two categories: equality operator comparisons and comparison operator comparisons. + */ + enum Strategy { + /** + * The evaluator used to evaluate comparison operator expressions. + */ + COMPARISON_OPERATORS(new ComparisonOperatorsEvaluator()), + /** + * The evaluator used to evaluate equality operator expressions. + */ + EQUALITY_OPERATORS(new EqualityOperatorsEvaluator()), + ; + + /** + * The evaluator to be used when evaluating Stellar expressions. + */ + private ComparisonExpressionEvaluator evaluator; + + Strategy(final ComparisonExpressionEvaluator evaluator) { + this.evaluator = evaluator; + } + + /** + * + * @return The evaluator needed to evaluate Stellar comparison expressions. + */ + public ComparisonExpressionEvaluator evaluator() { + return evaluator; + } + } + + /** + * When evaluating comparison expressions with operators, they are broken into four cases: + * + * 1. Testing equality, see {@link EqualityOperatorsEvaluator} + * 2. Testing not equal, see {@link EqualityOperatorsEvaluator}. This will be the negation of {@link EqualityOperatorsEvaluator#evaluate(Token, Token, StellarParser.ComparisonOpContext)}. + * 3. Testing less than, less than or equal, greater than, and greater than or equal {@link ComparisonOperatorsEvaluator} + * 4. Otherwise thrown {@link ParseException}. + * + * @param left The value of the left side of the Stellar expression. + * @param right The value of the right side of the Stellar expression. + * @param op The operator in the Stellar expression. + * @return A token with type boolean. This is based on the comparison of the {@code right} and {@code left} values. + */ + public Token evaluate(final Token left, final Token right, final StellarParser.ComparisonOpContext op, FrameContext.Context context) { + if (op.EQ() != null) { + return new Token<>(Strategy.EQUALITY_OPERATORS.evaluator().evaluate(left, right, op), Boolean.class, context); + } else if (op.NEQ() != null) { + return new Token<>(!Strategy.EQUALITY_OPERATORS.evaluator().evaluate(left, right, op), Boolean.class, context); + } else if (op.LT() != null || op.GT() != null || op.LTE() != null || op.GTE() != null) { + return new Token<>(Strategy.COMPARISON_OPERATORS.evaluator().evaluate(left, right, op), Boolean.class, context); + } + + throw new ParseException("Unsupported operations. The following expression is invalid: " + left.getValue() + op.getText() + right.getValue()); + } +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonOperatorsEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonOperatorsEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonOperatorsEvaluator.java new file mode 100644 index 0000000..1dc8021 --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/ComparisonOperatorsEvaluator.java @@ -0,0 +1,185 @@ +/* + * 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.evaluators; + +import org.apache.metron.stellar.dsl.ParseException; +import org.apache.metron.stellar.dsl.Token; +import org.apache.metron.stellar.common.generated.StellarParser; + +/** + * {@link ComparisonOperatorsEvaluator} is used to evaluate comparison expressions using the following + * operator {@literal '<', '<=', '>', or '>='.} There are four major cases when evaluating a comparison expression. + */ +public class ComparisonOperatorsEvaluator implements ComparisonExpressionEvaluator { + + /** + * 1. If either the left or right's value is null then return false. + * 2. If both sides of the expression are instances of {@link Number} then: + * 1. If either side is a {@link Double} then get {@link Number#doubleValue()} from both sides and compare using given operator. + * 2. Else if either side is a {@link Float} then get {@link Number#floatValue()} from both sides and compare using given operator. + * 3. Else if either side is a {@link Long} then get {@link Number#longValue()} from both sides and compare using given operator. + * 4. Otherwise get {@link Number#intValue()} from both sides and compare using given operator. + * 3. If both sides are of the same type and implement the {@link Comparable} interface then use {@code compareTo} method. + * 4. If none of the above are met then a {@link ParseException} is thrown. + * + * @param left The token representing the left side of a comparison expression. + * @param right The token representing the right side of a comparison expression. + * @param op This is a representation of a comparison operator {@literal (eg. <, <=, >, >=, ==, !=) } + * @return A boolean value based on the comparison of {@code left} and {@code right}. + */ + @Override + public boolean evaluate(final Token left, final Token right, final StellarParser.ComparisonOpContext op) { + if (left.getValue() == null || right.getValue() == null) { + return false; + } else if (left.getValue() instanceof Number && right.getValue() instanceof Number) { + return compareNumbers((Number) left.getValue(), (Number) right.getValue(), op); + } else if (left.getValue().getClass() == right.getValue().getClass() + && left.getValue() instanceof Comparable && right.getValue() instanceof Comparable) { + return compare((Comparable) left.getValue(), (Comparable) right.getValue(), op); + } + + throw new ParseException("Unsupported operations. The following expression is invalid: " + left.getValue() + op + right.getValue()); + } + + /** + * This method uses the inputs' ability to compare with one another's values by using the {@code compareTo} method. It will use this and + * the operator to evaluate the output. + * + * @param l The value of the left side of the expression. + * @param r The value of the right side of the expression. + * @param op The operator to use when comparing. + * @param The type of values being compared. + * @return A boolean value representing the comparison of the two values with the given operator. For example, {@code 1 <= 1} would be true. + */ + @SuppressWarnings("unchecked") + private boolean compare(final T l, final T r, final StellarParser.ComparisonOpContext op) { + int compareTo = l.compareTo(r); + + if (op.LT() != null) { + return compareTo < 0; + } else if (op.LTE() != null) { + return compareTo <= 0; + } else if (op.GT() != null) { + return compareTo > 0; + } else if (op.GTE() != null) { + return compareTo >= 0; + } + + throw new ParseException("Unsupported operator: " + op); + } + + /** + * This method uses the inputs' ability to compare with one another's values by using the {@code compareTo} method. It will use this and + * the operator to evaluate the output. + * + * @param l The left side of the expression. + * @param r The right side of the expression + * @param op The operator used in the expression. + * @return A boolean value representing the comparison of the two values with the given operator. For example, {@code 1 <= 1} would be true. + */ + private boolean compareNumbers(final Number l, final Number r, final StellarParser.ComparisonOpContext op) { + if (op.LT() != null) { + return lessThan(l, r); + } else if (op.LTE() != null) { + return lessThanEqual(l, r); + } else if (op.GT() != null) { + return greaterThan(l, r); + } else if (op.GTE() != null) { + return greaterThanEqual(l, r); + } + + throw new ParseException("Unsupported operator: " + op); + } + + /** + * If the left side of the expression is less than the right then true otherwise false. + * + * @param l The value of the left side of the expression. + * @param r The value of the right side of the expression. + * @return If the left side of the expression is less than the right then true otherwise false. + */ + private boolean lessThan(final Number l, final Number r) { + if (l instanceof Double || r instanceof Double) { + return l.doubleValue() < r.doubleValue(); + } else if (l instanceof Float || r instanceof Float) { + return l.floatValue() < r.floatValue(); + } else if (l instanceof Long || r instanceof Long) { + return l.longValue() < r.longValue(); + } else { + return l.intValue() < r.intValue(); + } + } + + /** + * If the left side of the expression is less than or equal to the right then true otherwise false. + * + * @param l The value of the left side of the expression. + * @param r The value of the right side of the expression. + * @return If the left side of the expression is less than or equal to the right then true otherwise false. + */ + private boolean lessThanEqual(final Number l, final Number r) { + if (l instanceof Double || r instanceof Double) { + return l.doubleValue() <= r.doubleValue(); + } else if (l instanceof Float || r instanceof Float) { + return l.floatValue() <= r.floatValue(); + } else if (l instanceof Long || r instanceof Long) { + return l.longValue() <= r.longValue(); + } else { + return l.intValue() <= r.intValue(); + } + } + + /** + * If the left side of the expression is greater than the right then true otherwise false. + * + * @param l The value of the left side of the expression. + * @param r The value of the right side of the expression. + * @return If the left side of the expression is greater than the right then true otherwise false. + */ + private boolean greaterThan(final Number l, final Number r) { + if (l instanceof Double || r instanceof Double) { + return l.doubleValue() > r.doubleValue(); + } else if (l instanceof Float || r instanceof Float) { + return l.floatValue() > r.floatValue(); + } else if (l instanceof Long || r instanceof Long) { + return l.longValue() > r.longValue(); + } else { + return l.intValue() > r.intValue(); + } + } + + /** + * If the left side of the expression is greater than or equal to the right then true otherwise false. + * + * @param l The value of the left side of the expression. + * @param r The value of the right side of the expression. + * @return If the left side of the expression is greater than or equal to the right then true otherwise false. + */ + private boolean greaterThanEqual(final Number l, final Number r) { + if (l instanceof Double || r instanceof Double) { + return l.doubleValue() >= r.doubleValue(); + } else if (l instanceof Float || r instanceof Float) { + return l.floatValue() >= r.floatValue(); + } else if (l instanceof Long || r instanceof Long) { + return l.longValue() >= r.longValue(); + } else { + return l.intValue() >= r.intValue(); + } + } +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/DoubleLiteralEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/DoubleLiteralEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/DoubleLiteralEvaluator.java new file mode 100644 index 0000000..30167ae --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/DoubleLiteralEvaluator.java @@ -0,0 +1,34 @@ +/* + * 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.evaluators; + +import org.apache.metron.stellar.dsl.Token; +import org.apache.metron.stellar.common.FrameContext; +import org.apache.metron.stellar.common.generated.StellarParser; + +public class DoubleLiteralEvaluator implements NumberEvaluator { + @Override + public Token evaluate(StellarParser.DoubleLiteralContext context, FrameContext.Context contextVariety) { + if (context == null) { + throw new IllegalArgumentException("Cannot evaluate a context that is null."); + } + + return new Token<>(Double.parseDouble(context.getText()), Double.class, contextVariety); + } +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/EqualityOperatorsEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/EqualityOperatorsEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/EqualityOperatorsEvaluator.java new file mode 100644 index 0000000..c95eb34 --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/EqualityOperatorsEvaluator.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.metron.stellar.common.evaluators; + +import org.apache.metron.stellar.dsl.Token; +import org.apache.metron.stellar.common.generated.StellarParser; + +/** + * {@link EqualityOperatorsEvaluator} is used to evaluate equality expressions using the following operator '=='. There are + * four major cases when evaluating a equality expression. See {@link EqualityOperatorsEvaluator#evaluate(Token, Token, StellarParser.ComparisonOpContext)} + * for a description. + */ +public class EqualityOperatorsEvaluator implements ComparisonExpressionEvaluator { + + /** + * 1. If either side of the expression is null then check equality using Java's '==' expression. + * 2. Else if both sides of the expression are of type {@link Number} then: + * 1. If either side of the expression is a {@link Double} then use {@link Number#doubleValue()} to test equality. + * 2. Else if either side of the expression is a {@link Float} then use {@link Number#floatValue()} to test equality. + * 3. Else if either side of the expression is a {@link Long} then use {@link Number#longValue()} to test equality. + * 4. Otherwise use {@link Number#intValue()} to test equality + * 3. Otherwise use {@code equals} method compare the left side with the right side. + * @param left The token representing the left side of a comparison expression. + * @param right The token representing the right side of a comparison expression. + * @param op This is a representation of a comparison operator {@literal (eg. <, <=, >, >=, ==, !=) } + * @return A boolean value based on the comparison of {@code left} and {@code right}. + */ + @Override + public boolean evaluate(final Token left, final Token right, final StellarParser.ComparisonOpContext op) { + if (left.getValue() == null || right.getValue() == null) { + return left.getValue() == right.getValue(); + } else if (left.getValue() instanceof Number && right.getValue() instanceof Number) { + return eq((Number) left.getValue(), (Number) right.getValue()); + } else { + return left.getValue().equals(right.getValue()); + } + } + + /** + * This method follows Java's number promotions when comparing numbers. + * + * @param l The left side of the equality expression. + * @param r The right side of the equality expression. + * @return All comparisons use the '==' operator from Java. If either input is a double then compare double values. + * If either side is a float compare float values. If either side is a long compare long values. Otherwise compare + * int values. + */ + private boolean eq(final Number l, final Number r) { + if (l instanceof Double || r instanceof Double) { + return l.doubleValue() == r.doubleValue(); + } else if (l instanceof Float || r instanceof Float) { + return l.floatValue() == r.floatValue(); + } else if (l instanceof Long || r instanceof Long) { + return l.longValue() == r.longValue(); + } else { + return l.intValue() == r.intValue(); + } + } +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/FloatLiteralEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/FloatLiteralEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/FloatLiteralEvaluator.java new file mode 100644 index 0000000..bfa0bc9 --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/FloatLiteralEvaluator.java @@ -0,0 +1,34 @@ +/* + * 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.evaluators; + +import org.apache.metron.stellar.dsl.Token; +import org.apache.metron.stellar.common.FrameContext; +import org.apache.metron.stellar.common.generated.StellarParser; + +public class FloatLiteralEvaluator implements NumberEvaluator { + @Override + public Token evaluate(StellarParser.FloatLiteralContext context, FrameContext.Context contextVariety) { + if (context == null) { + throw new IllegalArgumentException("Cannot evaluate a context that is null."); + } + + return new Token<>(Float.parseFloat(context.getText()), Float.class, contextVariety); + } +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/IntLiteralEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/IntLiteralEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/IntLiteralEvaluator.java new file mode 100644 index 0000000..ad55700 --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/IntLiteralEvaluator.java @@ -0,0 +1,34 @@ +/* + * 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.evaluators; + +import org.apache.metron.stellar.dsl.Token; +import org.apache.metron.stellar.common.FrameContext; +import org.apache.metron.stellar.common.generated.StellarParser; + +public class IntLiteralEvaluator implements NumberEvaluator { + @Override + public Token evaluate(StellarParser.IntLiteralContext context, FrameContext.Context contextVariety) { + if (context == null) { + throw new IllegalArgumentException("Cannot evaluate a context that is null."); + } + + return new Token<>(Integer.parseInt(context.getText()), Integer.class, contextVariety); + } +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/LongLiteralEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/LongLiteralEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/LongLiteralEvaluator.java new file mode 100644 index 0000000..f5461ad --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/LongLiteralEvaluator.java @@ -0,0 +1,42 @@ +/* + * 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.evaluators; + +import org.apache.metron.stellar.dsl.ParseException; +import org.apache.metron.stellar.dsl.Token; +import org.apache.metron.stellar.common.FrameContext; +import org.apache.metron.stellar.common.generated.StellarParser; + +public class LongLiteralEvaluator implements NumberEvaluator { + @Override + public Token evaluate(StellarParser.LongLiteralContext context, FrameContext.Context contextVariety) { + if (context == null) { + throw new IllegalArgumentException("Cannot evaluate a context that is null."); + } + + String value = context.getText(); + if (value.endsWith("l") || value.endsWith("L")) { + value = value.substring(0, value.length() - 1); // Drop the 'L' or 'l'. Long.parseLong does not accept a string with either of these. + return new Token<>(Long.parseLong(value), Long.class, contextVariety); + } else { + // Technically this should never happen, but just being safe. + throw new ParseException("Invalid format for long. Failed trying to parse a long with the following value: " + value); + } + } +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/NumberEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/NumberEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/NumberEvaluator.java new file mode 100644 index 0000000..83aed62 --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/NumberEvaluator.java @@ -0,0 +1,27 @@ +/* + * 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.evaluators; + +import org.apache.metron.stellar.dsl.Token; +import org.apache.metron.stellar.common.FrameContext; +import org.apache.metron.stellar.common.generated.StellarParser; + +public interface NumberEvaluator { + Token evaluate(T context, FrameContext.Context contextVariety); +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/NumberLiteralEvaluator.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/NumberLiteralEvaluator.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/NumberLiteralEvaluator.java new file mode 100644 index 0000000..24dccba --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/evaluators/NumberLiteralEvaluator.java @@ -0,0 +1,71 @@ +/* + * 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.evaluators; + +import org.apache.metron.stellar.common.FrameContext; +import org.apache.metron.stellar.common.generated.StellarParser; +import org.apache.metron.stellar.dsl.ParseException; +import org.apache.metron.stellar.dsl.Token; + +import java.util.HashMap; +import java.util.Map; + +public enum NumberLiteralEvaluator { + INSTANCE; + public enum Strategy { + INTEGER(StellarParser.IntLiteralContext.class, new IntLiteralEvaluator()) + , DOUBLE(StellarParser.DoubleLiteralContext.class, new DoubleLiteralEvaluator()) + , FLOAT(StellarParser.FloatLiteralContext.class, new FloatLiteralEvaluator()) + , LONG(StellarParser.LongLiteralContext.class, new LongLiteralEvaluator()); + Class context; + NumberEvaluator evaluator; + private static Map, NumberEvaluator> strategyMap; + + static { + strategyMap = new HashMap<>(); + for (Strategy strat : Strategy.values()) { + strategyMap.put(strat.context, strat.evaluator); + } + } + + Strategy(Class context + , NumberEvaluator evaluator + ) { + this.context = context; + this.evaluator = evaluator; + } + } + + Token evaluate(StellarParser.Arithmetic_operandsContext context + , Map, NumberEvaluator> instanceMap + , FrameContext.Context contextVariety + ) + { + NumberEvaluator evaluator = instanceMap.get(context.getClass()); + if(evaluator == null) { + throw new ParseException("Does not support evaluation for type " + context.getClass()); + } + return evaluator.evaluate(context, contextVariety); + } + + public Token evaluate(StellarParser.Arithmetic_operandsContext context, FrameContext.Context contextVariety) { + return evaluate(context, Strategy.strategyMap, contextVariety); + } + +} http://git-wip-us.apache.org/repos/asf/metron/blob/a5b13777/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/generated/StellarBaseListener.java ---------------------------------------------------------------------- diff --git a/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/generated/StellarBaseListener.java b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/generated/StellarBaseListener.java new file mode 100644 index 0000000..2a1a406 --- /dev/null +++ b/metron-stellar/stellar-common/src/main/java/org/apache/metron/stellar/common/generated/StellarBaseListener.java @@ -0,0 +1,780 @@ +// Generated from org/apache/metron/stellar/common/generated/Stellar.g4 by ANTLR 4.5 +package org.apache.metron.stellar.common.generated; + +//CHECKSTYLE:OFF +/* + * 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. + */ + + +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.misc.NotNull; +import org.antlr.v4.runtime.tree.ErrorNode; +import org.antlr.v4.runtime.tree.TerminalNode; + +/** + * This class provides an empty implementation of {@link StellarListener}, + * which can be extended to create a listener which only needs to handle a subset + * of the available methods. + */ +public class StellarBaseListener implements StellarListener { + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterTransformation(StellarParser.TransformationContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitTransformation(StellarParser.TransformationContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterConditionalExpr(StellarParser.ConditionalExprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitConditionalExpr(StellarParser.ConditionalExprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterTransformationExpr(StellarParser.TransformationExprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitTransformationExpr(StellarParser.TransformationExprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterArithExpression(StellarParser.ArithExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitArithExpression(StellarParser.ArithExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterTransformationEntity(StellarParser.TransformationEntityContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitTransformationEntity(StellarParser.TransformationEntityContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterComparisonExpression(StellarParser.ComparisonExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitComparisonExpression(StellarParser.ComparisonExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLogicalExpression(StellarParser.LogicalExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLogicalExpression(StellarParser.LogicalExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterInExpression(StellarParser.InExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitInExpression(StellarParser.InExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterIf_expr(StellarParser.If_exprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitIf_expr(StellarParser.If_exprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterThen_expr(StellarParser.Then_exprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitThen_expr(StellarParser.Then_exprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterElse_expr(StellarParser.Else_exprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitElse_expr(StellarParser.Else_exprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterTernaryFuncWithoutIf(StellarParser.TernaryFuncWithoutIfContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitTernaryFuncWithoutIf(StellarParser.TernaryFuncWithoutIfContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterTernaryFuncWithIf(StellarParser.TernaryFuncWithIfContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitTernaryFuncWithIf(StellarParser.TernaryFuncWithIfContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLogicalExpressionAnd(StellarParser.LogicalExpressionAndContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLogicalExpressionAnd(StellarParser.LogicalExpressionAndContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLogicalExpressionOr(StellarParser.LogicalExpressionOrContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLogicalExpressionOr(StellarParser.LogicalExpressionOrContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterBoleanExpression(StellarParser.BoleanExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitBoleanExpression(StellarParser.BoleanExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterB_expr(StellarParser.B_exprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitB_expr(StellarParser.B_exprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterInExpressionStatement(StellarParser.InExpressionStatementContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitInExpressionStatement(StellarParser.InExpressionStatementContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterNInExpressionStatement(StellarParser.NInExpressionStatementContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitNInExpressionStatement(StellarParser.NInExpressionStatementContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterNotFunc(StellarParser.NotFuncContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitNotFunc(StellarParser.NotFuncContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterComparisonExpressionParens(StellarParser.ComparisonExpressionParensContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitComparisonExpressionParens(StellarParser.ComparisonExpressionParensContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterComparisonExpressionWithOperator(StellarParser.ComparisonExpressionWithOperatorContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitComparisonExpressionWithOperator(StellarParser.ComparisonExpressionWithOperatorContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterOperand(StellarParser.OperandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitOperand(StellarParser.OperandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterTransformation_entity(StellarParser.Transformation_entityContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitTransformation_entity(StellarParser.Transformation_entityContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterComparisonOp(StellarParser.ComparisonOpContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitComparisonOp(StellarParser.ComparisonOpContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterFunc_args(StellarParser.Func_argsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitFunc_args(StellarParser.Func_argsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterOp_list(StellarParser.Op_listContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitOp_list(StellarParser.Op_listContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterList_entity(StellarParser.List_entityContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitList_entity(StellarParser.List_entityContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterKv_list(StellarParser.Kv_listContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitKv_list(StellarParser.Kv_listContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterMap_entity(StellarParser.Map_entityContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitMap_entity(StellarParser.Map_entityContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterArithExpr_solo(StellarParser.ArithExpr_soloContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitArithExpr_solo(StellarParser.ArithExpr_soloContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterArithExpr_minus(StellarParser.ArithExpr_minusContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitArithExpr_minus(StellarParser.ArithExpr_minusContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterArithExpr_plus(StellarParser.ArithExpr_plusContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitArithExpr_plus(StellarParser.ArithExpr_plusContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterArithExpr_div(StellarParser.ArithExpr_divContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitArithExpr_div(StellarParser.ArithExpr_divContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterArithExpr_mul_solo(StellarParser.ArithExpr_mul_soloContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitArithExpr_mul_solo(StellarParser.ArithExpr_mul_soloContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterArithExpr_mul(StellarParser.ArithExpr_mulContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitArithExpr_mul(StellarParser.ArithExpr_mulContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterTransformationFunc(StellarParser.TransformationFuncContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitTransformationFunc(StellarParser.TransformationFuncContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterNumericFunctions(StellarParser.NumericFunctionsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitNumericFunctions(StellarParser.NumericFunctionsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterDoubleLiteral(StellarParser.DoubleLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitDoubleLiteral(StellarParser.DoubleLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterIntLiteral(StellarParser.IntLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitIntLiteral(StellarParser.IntLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLongLiteral(StellarParser.LongLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLongLiteral(StellarParser.LongLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterFloatLiteral(StellarParser.FloatLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitFloatLiteral(StellarParser.FloatLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterVariable(StellarParser.VariableContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitVariable(StellarParser.VariableContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterParenArith(StellarParser.ParenArithContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitParenArith(StellarParser.ParenArithContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterCondExpr(StellarParser.CondExprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitCondExpr(StellarParser.CondExprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLogicalConst(StellarParser.LogicalConstContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLogicalConst(StellarParser.LogicalConstContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLambdaWithArgsExpr(StellarParser.LambdaWithArgsExprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLambdaWithArgsExpr(StellarParser.LambdaWithArgsExprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLambdaWithoutArgsExpr(StellarParser.LambdaWithoutArgsExprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLambdaWithoutArgsExpr(StellarParser.LambdaWithoutArgsExprContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterArithmeticOperands(StellarParser.ArithmeticOperandsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitArithmeticOperands(StellarParser.ArithmeticOperandsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterStringLiteral(StellarParser.StringLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitStringLiteral(StellarParser.StringLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterList(StellarParser.ListContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitList(StellarParser.ListContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterMapConst(StellarParser.MapConstContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitMapConst(StellarParser.MapConstContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterNullConst(StellarParser.NullConstContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitNullConst(StellarParser.NullConstContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterExistsFunc(StellarParser.ExistsFuncContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitExistsFunc(StellarParser.ExistsFuncContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterCondExpr_paren(StellarParser.CondExpr_parenContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitCondExpr_paren(StellarParser.CondExpr_parenContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLambda_without_args(StellarParser.Lambda_without_argsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLambda_without_args(StellarParser.Lambda_without_argsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLambda_with_args(StellarParser.Lambda_with_argsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLambda_with_args(StellarParser.Lambda_with_argsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLambda_variables(StellarParser.Lambda_variablesContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLambda_variables(StellarParser.Lambda_variablesContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterSingle_lambda_variable(StellarParser.Single_lambda_variableContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitSingle_lambda_variable(StellarParser.Single_lambda_variableContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLambda_variable(StellarParser.Lambda_variableContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLambda_variable(StellarParser.Lambda_variableContext ctx) { } + + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterEveryRule(ParserRuleContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitEveryRule(ParserRuleContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void visitTerminal(TerminalNode node) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void visitErrorNode(ErrorNode node) { } +} \ No newline at end of file