This is an automated email from the ASF dual-hosted git repository.
volodymyr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git
commit 44b990be5c15e1c480725cfb78fcabb40216ebf0
Author: Arina Ielchiieva <arina.yelchiyeva@gmail.com>
AuthorDate: Tue Nov 20 14:27:55 2018 +0200
DRILL-6870: Upgrade to ANTLR4
closes #1554
---
common/pom.xml | 6 --
.../exec/planner/index/MapRDBIndexDiscover.java | 32 ++-----
.../test/java/org/apache/drill/exec/ExecTest.java | 27 +-----
.../drill/exec/compile/TestEvaluationVisitor.java | 50 +++-------
.../unnest/TestUnnestWithLateralCorrectness.java | 8 +-
...alTestutils.java => DrillLogicalTestUtils.java} | 28 ++----
.../apache/drill/test/PhysicalOpUnitTestBase.java | 14 +--
.../java/org/apache/drill/test/TestBuilder.java | 46 +++------
logical/pom.xml | 12 +--
.../drill/common/expression/parser/ExprLexer.g4} | 46 +++------
.../drill/common/expression/parser/ExprParser.g4} | 81 +++++++++-------
.../drill/common/expression/LogicalExpression.java | 23 +----
.../drill/common/expression/PathSegment.java | 8 +-
.../apache/drill/common/expression/SchemaPath.java | 30 ++----
.../apache/drill/common/parser/ErrorListener.java | 38 ++++++++
.../common/parser/LogicalExpressionParser.java | 55 +++++++++++
.../drill/common/expression/parser/TreeTest.java | 106 +++++++++++----------
pom.xml | 11 ++-
18 files changed, 291 insertions(+), 330 deletions(-)
diff --git a/common/pom.xml b/common/pom.xml
index c931c01..fba9efc 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -102,12 +102,6 @@
</dependency>
<dependency>
- <groupId>org.antlr</groupId>
- <artifactId>antlr-runtime</artifactId>
- <version>3.4</version>
- </dependency>
-
- <dependency>
<groupId>joda-time</groupId>
<artifactId>joda-time</artifactId>
<version>2.9</version>
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
index 95b9813..040b5e9 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
@@ -18,22 +18,19 @@
package org.apache.drill.exec.planner.index;
+import org.apache.drill.common.parser.LogicalExpressionParser;
import com.mapr.db.Admin;
import com.mapr.db.MapRDB;
import com.mapr.db.exceptions.DBException;
import com.mapr.db.index.IndexDesc;
import com.mapr.db.index.IndexDesc.MissingAndNullOrdering;
import com.mapr.db.index.IndexFieldDesc;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
import org.apache.calcite.rel.RelFieldCollation;
import org.apache.calcite.rel.RelFieldCollation.NullDirection;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.drill.common.exceptions.DrillRuntimeException;
import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
import org.apache.drill.common.types.TypeProtos;
import org.apache.drill.common.types.Types;
import org.apache.drill.common.util.DrillFileUtils;
@@ -67,8 +64,7 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDiscover {
- static final String DEFAULT_STRING_CAST_LEN_STR = "256";
- static final String FIELD_DELIMITER = ":";
+ public static final String DEFAULT_STRING_CAST_LEN_STR = "256";
public MapRDBIndexDiscover(GroupScan inScan, DrillScanRelBase scanRel) {
super((AbstractDbGroupScan) inScan, scanRel);
@@ -86,7 +82,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
/**
* For a given table name get the list of indexes defined on the table according to the visibility of
* the indexes based on permissions.
- * @param tableName
+ * @param tableName table name
* @return an IndexCollection representing the list of indexes for that table
*/
private IndexCollection getTableIndexFromMFS(String tableName) {
@@ -137,7 +133,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
return null;
}
MapRDBFormatPlugin maprFormatPlugin = ((MapRDBGroupScan) origScan).getFormatPlugin();
- FileSystemPlugin fsPlugin = (FileSystemPlugin) (((MapRDBGroupScan) origScan).getStoragePlugin());
+ FileSystemPlugin fsPlugin = (FileSystemPlugin) (origScan.getStoragePlugin());
DrillFileSystem fs = ImpersonationUtil.createFileSystem(origScan.getUserName(), fsPlugin.getFsConf());
MapRDBFormatMatcher matcher = (MapRDBFormatMatcher) (maprFormatPlugin.getMatcher());
@@ -240,14 +236,9 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
}
try {
String castFunc = String.format("cast( %s as %s)", field, castTypeStr);
- final ExprLexer lexer = new ExprLexer(new ANTLRStringStream(castFunc));
- final CommonTokenStream tokens = new CommonTokenStream(lexer);
- final ExprParser parser = new ExprParser(tokens);
- final ExprParser.parse_return ret = parser.parse();
- logger.trace("{}, {}", tokens, ret);
- return ret.e;
- }catch(Exception ex) {
- logger.error("parse failed{}", ex);
+ return LogicalExpressionParser.parse(castFunc);
+ } catch (Exception ex) {
+ logger.error("parse failed: {}", ex);
}
return null;
}
@@ -313,8 +304,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
for (int i = 0; i < indexFieldCollations.size(); i++) {
collationMap.put(indexFields.get(i), indexFieldCollations.get(i));
}
- CollationContext collationContext = new CollationContext(collationMap, indexFieldCollations);
- return collationContext;
+ return new CollationContext(collationMap, indexFieldCollations);
}
private DrillIndexDescriptor buildIndexDescriptor(String tableName, IndexDesc desc)
@@ -363,11 +353,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
final Admin admin;
try {
- admin = currentUser.doAs(new PrivilegedExceptionAction<Admin>() {
- public Admin run() throws Exception {
- return MapRDB.getAdmin(conf);
- }
- });
+ admin = currentUser.doAs((PrivilegedExceptionAction<Admin>) () -> MapRDB.getAdmin(conf));
} catch (Exception e) {
throw new DrillRuntimeException("Failed to get Admin instance for user: " + currentUser.getUserName(), e);
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
index 15a6e4f..6725bb9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
@@ -18,16 +18,11 @@
package org.apache.drill.exec;
import com.codahale.metrics.MetricRegistry;
-import org.apache.drill.shaded.guava.com.google.common.io.Files;
+import org.apache.drill.common.parser.LogicalExpressionParser;
import mockit.Mock;
import mockit.MockUp;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
import org.apache.drill.common.scanner.ClassPathScanner;
import org.apache.drill.exec.compile.CodeCompilerTestFactory;
import org.apache.drill.exec.memory.RootAllocatorFactory;
@@ -48,7 +43,6 @@ import org.junit.After;
import org.junit.BeforeClass;
import org.junit.ClassRule;
-import java.io.File;
import java.io.IOException;
import java.text.DateFormatSymbols;
import java.util.Locale;
@@ -94,17 +88,6 @@ public class ExecTest extends DrillTest {
return FileSystem.get(configuration);
}
- /**
- * Create a temp directory to store the given <i>dirName</i>.
- * Directory will be deleted on exit.
- * @param dirName directory name
- * @return Full path including temp parent directory and given directory name.
- */
- public static String getTempDir(final String dirName) {
- final File dir = Files.createTempDir();
- return dir.getAbsolutePath() + File.separator + dirName;
- }
-
protected DrillbitContext mockDrillbitContext() throws Exception {
final DrillbitContext context = mock(DrillbitContext.class);
@@ -118,12 +101,8 @@ public class ExecTest extends DrillTest {
return context;
}
- protected LogicalExpression parseExpr(String expr) throws RecognitionException {
- final ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
- final CommonTokenStream tokens = new CommonTokenStream(lexer);
- final ExprParser parser = new ExprParser(tokens);
- final ExprParser.parse_return ret = parser.parse();
- return ret.e;
+ public LogicalExpression parseExpr(String expr) {
+ return LogicalExpressionParser.parse(expr);
}
/**
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java
index 91ce653..631f1f5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java
@@ -17,68 +17,46 @@
*/
package org.apache.drill.exec.compile;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
-import org.apache.drill.common.expression.parser.ExprParser.parse_return;
+import org.apache.drill.common.parser.LogicalExpressionParser;
import org.apache.drill.common.types.TypeProtos.MinorType;
import org.apache.drill.common.types.Types;
import org.apache.drill.exec.expr.CodeGenerator;
import org.apache.drill.exec.expr.EvaluationVisitor;
import org.apache.drill.exec.expr.ValueVectorReadExpression;
import org.apache.drill.exec.expr.ValueVectorWriteExpression;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
import org.apache.drill.exec.physical.impl.project.Projector;
import org.apache.drill.exec.record.TypedFieldId;
import org.junit.Test;
public class TestEvaluationVisitor {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestEvaluationVisitor.class);
-
@Test
- public void x() throws Exception{
- DrillConfig c = DrillConfig.create();
-
- FunctionImplementationRegistry reg = new FunctionImplementationRegistry(c);
+ public void testEvaluation() {
EvaluationVisitor v = new EvaluationVisitor();
CodeGenerator<?> g = CodeGenerator.get(Projector.TEMPLATE_DEFINITION, null);
- SchemaPath path = (SchemaPath) getExpr("a.b[4][2].c[6]");
+ SchemaPath path = (SchemaPath) LogicalExpressionParser.parse("a.b[4][2].c[6]");
- TypedFieldId id = TypedFieldId.newBuilder() //
- .addId(1) //
- .addId(3) //
- .remainder(path.getRootSegment()) //
+ TypedFieldId id = TypedFieldId.newBuilder()
+ .addId(1)
+ .addId(3)
+ .remainder(path.getRootSegment())
.intermediateType(Types.optional(MinorType.MAP))
- .finalType(Types.repeated(MinorType.MAP)) //
- .hyper() //
- .withIndex() //
+ .finalType(Types.repeated(MinorType.MAP))
+ .hyper()
+ .withIndex()
.build();
ValueVectorReadExpression e = new ValueVectorReadExpression(id);
- TypedFieldId outId = TypedFieldId.newBuilder() //
- .addId(1) //
- .finalType(Types.repeated(MinorType.MAP)) //
- .intermediateType(Types.repeated(MinorType.MAP)) //
+ TypedFieldId outId = TypedFieldId.newBuilder()
+ .addId(1)
+ .finalType(Types.repeated(MinorType.MAP))
+ .intermediateType(Types.repeated(MinorType.MAP))
.build();
ValueVectorWriteExpression e2 = new ValueVectorWriteExpression(outId, e, true);
v.addExpr(e2, g.getRoot());
- logger.debug(g.generateAndGet());
}
- private LogicalExpression getExpr(String expr) throws Exception{
- ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
- CommonTokenStream tokens = new CommonTokenStream(lexer);
- ExprParser parser = new ExprParser(tokens);
- parse_return ret = parser.parse();
-
- return ret.e;
-
- }
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java
index 2e21408..7d150a2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java
@@ -37,7 +37,7 @@ import org.apache.drill.exec.physical.impl.project.ProjectRecordBatch;
import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
import org.apache.drill.exec.planner.common.DrillLateralJoinRelBase;
import org.apache.drill.exec.planner.common.DrillUnnestRelBase;
-import org.apache.drill.exec.planner.logical.DrillLogicalTestutils;
+import org.apache.drill.exec.planner.logical.DrillLogicalTestUtils;
import org.apache.drill.exec.record.RecordBatch;
import org.apache.drill.exec.record.VectorContainer;
import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -590,7 +590,7 @@ public class TestUnnestWithLateralCorrectness extends SubOperatorTest {
// project is required to rename the columns so as to disambiguate the same column name from
// unnest operator and the regular scan.
- final Project projectPopConfig = new Project(DrillLogicalTestutils.parseExprs("unnestColumn", "unnestColumn1",
+ final Project projectPopConfig = new Project(DrillLogicalTestUtils.parseExprs("unnestColumn", "unnestColumn1",
unnestPopConfig.getImplicitColumn(), unnestPopConfig.getImplicitColumn()), null);
final ProjectRecordBatch projectBatch =
@@ -899,12 +899,12 @@ public class TestUnnestWithLateralCorrectness extends SubOperatorTest {
// Create intermediate Project
final Project projectPopConfig1 =
- new Project(DrillLogicalTestutils.parseExprs("unnestColumn.colB", "colB",
+ new Project(DrillLogicalTestUtils.parseExprs("unnestColumn.colB", "colB",
unnestPopConfig1.getImplicitColumn(), unnestPopConfig1.getImplicitColumn()), unnestPopConfig1);
final ProjectRecordBatch projectBatch1 =
new ProjectRecordBatch(projectPopConfig1, unnestBatch1, fixture.getFragmentContext());
final Project projectPopConfig2 =
- new Project(DrillLogicalTestutils.parseExprs("colB", "unnestColumn2",
+ new Project(DrillLogicalTestUtils.parseExprs("colB", "unnestColumn2",
unnestPopConfig2.getImplicitColumn(), unnestPopConfig2.getImplicitColumn()), unnestPopConfig2);
final ProjectRecordBatch projectBatch2 =
new ProjectRecordBatch(projectPopConfig2, unnestBatch2, fixture.getFragmentContext());
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestutils.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestUtils.java
similarity index 65%
rename from exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestutils.java
rename to exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestUtils.java
index a6b5906..d65cb3e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestutils.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestUtils.java
@@ -17,17 +17,12 @@
*/
package org.apache.drill.exec.planner.logical;
+import org.apache.drill.common.parser.LogicalExpressionParser;
import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
import org.apache.calcite.rel.RelFieldCollation;
import org.apache.drill.common.expression.FieldReference;
-import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.common.expression.PathSegment;
import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
import org.apache.drill.common.logical.data.JoinCondition;
import org.apache.drill.common.logical.data.NamedExpression;
import org.apache.drill.common.logical.data.Order;
@@ -35,36 +30,27 @@ import org.apache.drill.common.logical.data.Order;
import java.util.ArrayList;
import java.util.List;
-public class DrillLogicalTestutils {
+public class DrillLogicalTestUtils {
public static Order.Ordering ordering(String expression,
RelFieldCollation.Direction direction,
RelFieldCollation.NullDirection nullDirection) {
- return new Order.Ordering(direction, parseExpr(expression), nullDirection);
+ return new Order.Ordering(direction, LogicalExpressionParser.parse(expression), nullDirection);
}
public static JoinCondition joinCond(String leftExpr, String relationship, String rightExpr) {
- return new JoinCondition(relationship, parseExpr(leftExpr), parseExpr(rightExpr));
+ return new JoinCondition(relationship, LogicalExpressionParser.parse(leftExpr), LogicalExpressionParser.parse(rightExpr));
}
public static List<NamedExpression> parseExprs(String... expressionsAndOutputNames) {
Preconditions.checkArgument(expressionsAndOutputNames.length % 2 == 0,
- "List of expressions and output field names" + " is not complete, each expression must explicitly give and output name,");
+ "List of expressions and output field names"
+ + " is not complete, each expression must explicitly give and output name,");
List<NamedExpression> ret = new ArrayList<>();
for (int i = 0; i < expressionsAndOutputNames.length; i += 2) {
- ret.add(new NamedExpression(parseExpr(expressionsAndOutputNames[i]),
+ ret.add(new NamedExpression(LogicalExpressionParser.parse(expressionsAndOutputNames[i]),
new FieldReference(new SchemaPath(new PathSegment.NameSegment(expressionsAndOutputNames[i + 1])))));
}
return ret;
}
- public static LogicalExpression parseExpr(String expr) {
- ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
- CommonTokenStream tokens = new CommonTokenStream(lexer);
- ExprParser parser = new ExprParser(tokens);
- try {
- return parser.parse().e;
- } catch (RecognitionException e) {
- throw new RuntimeException("Error parsing expression: " + expr);
- }
- }
}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
index 300e88b..84a7c78 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.ops.FragmentContext;
import org.apache.drill.exec.ops.FragmentStats;
import org.apache.drill.exec.physical.base.PhysicalVisitor;
import org.apache.drill.exec.planner.PhysicalPlanReader;
-import org.apache.drill.exec.planner.logical.DrillLogicalTestutils;
+import org.apache.drill.exec.planner.logical.DrillLogicalTestUtils;
import org.apache.drill.exec.proto.CoordinationProtos;
import org.apache.drill.exec.rpc.control.Controller;
import org.apache.drill.exec.rpc.control.WorkEventBus;
@@ -42,7 +42,6 @@ import org.apache.drill.exec.work.filter.RuntimeFilterSink;
import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.logical.data.JoinCondition;
import org.apache.drill.common.logical.data.NamedExpression;
@@ -115,21 +114,16 @@ public class PhysicalOpUnitTestBase extends ExecTest {
scanDecodeExecutor.shutdownNow();
}
- @Override
- protected LogicalExpression parseExpr(String expr) {
- return DrillLogicalTestutils.parseExpr(expr);
- }
-
protected Order.Ordering ordering(String expression, RelFieldCollation.Direction direction, RelFieldCollation.NullDirection nullDirection) {
- return DrillLogicalTestutils.ordering(expression, direction, nullDirection);
+ return DrillLogicalTestUtils.ordering(expression, direction, nullDirection);
}
protected JoinCondition joinCond(String leftExpr, String relationship, String rightExpr) {
- return DrillLogicalTestutils.joinCond(leftExpr, relationship, rightExpr);
+ return DrillLogicalTestUtils.joinCond(leftExpr, relationship, rightExpr);
}
protected List<NamedExpression> parseExprs(String... expressionsAndOutputNames) {
- return DrillLogicalTestutils.parseExprs(expressionsAndOutputNames);
+ return DrillLogicalTestUtils.parseExprs(expressionsAndOutputNames);
}
protected static class BatchIterator implements Iterable<VectorAccessible> {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
index d45bd6f..37e05cd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
@@ -30,13 +30,10 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
+import org.apache.drill.common.parser.LogicalExpressionParser;
import org.apache.drill.common.types.TypeProtos;
import org.apache.drill.common.types.Types;
import org.apache.drill.exec.expr.fn.impl.DateUtility;
@@ -159,15 +156,13 @@ public class TestBuilder {
}
public TestBuilder sqlQueryFromFile(String queryFile) throws IOException {
- String query = BaseTestQuery.getFile(queryFile);
- this.query = query;
+ this.query = BaseTestQuery.getFile(queryFile);
queryType = UserBitShared.QueryType.SQL;
return this;
}
public TestBuilder physicalPlanFromFile(String queryFile) throws IOException {
- String query = BaseTestQuery.getFile(queryFile);
- this.query = query;
+ this.query = BaseTestQuery.getFile(queryFile);
queryType = UserBitShared.QueryType.PHYSICAL;
return this;
}
@@ -226,20 +221,11 @@ public class TestBuilder {
// modified code from SchemaPath.De class. This should be used sparingly and only in tests if absolutely needed.
public static SchemaPath parsePath(String path) {
- try {
- ExprLexer lexer = new ExprLexer(new ANTLRStringStream(path));
- CommonTokenStream tokens = new CommonTokenStream(lexer);
- ExprParser parser = new ExprParser(tokens);
-
- ExprParser.parse_return ret = parser.parse();
-
- if (ret.e instanceof SchemaPath) {
- return (SchemaPath) ret.e;
- } else {
- throw new IllegalStateException("Schema path is not a valid format.");
- }
- } catch (RecognitionException e) {
- throw new RuntimeException(e);
+ LogicalExpression expr = LogicalExpressionParser.parse(path);
+ if (expr instanceof SchemaPath) {
+ return (SchemaPath) expr;
+ } else {
+ throw new IllegalStateException(String.format("Schema path is not a valid format: %s.", expr));
}
}
@@ -290,11 +276,7 @@ public class TestBuilder {
}
boolean typeInfoSet() {
- if (baselineTypeMap != null) {
- return true;
- } else {
- return false;
- }
+ return baselineTypeMap != null;
}
/**
@@ -513,11 +495,7 @@ public class TestBuilder {
@Override
boolean typeInfoSet() {
- if (super.typeInfoSet() || baselineTypes != null) {
- return true;
- } else {
- return false;
- }
+ return super.typeInfoSet() || baselineTypes != null;
}
@Override
@@ -683,7 +661,7 @@ public class TestBuilder {
}
/**
- * Convenience method to create a {@link JsonStringHashMap<String, Object> map} instance with the given key value sequence.
+ * Convenience method to create a {@link JsonStringHashMap<String, Object>} map instance with the given key value sequence.
*
* Key value sequence consists of key - value pairs such that a key precedes its value. For instance:
*
diff --git a/logical/pom.xml b/logical/pom.xml
index aa8fb06..94bef6c 100644
--- a/logical/pom.xml
+++ b/logical/pom.xml
@@ -84,8 +84,7 @@
<dependency>
<groupId>org.antlr</groupId>
- <artifactId>antlr-runtime</artifactId>
- <version>3.4</version>
+ <artifactId>antlr4-runtime</artifactId>
</dependency>
<dependency>
@@ -111,15 +110,12 @@
</plugin>
<plugin>
<groupId>org.antlr</groupId>
- <artifactId>antlr3-maven-plugin</artifactId>
- <version>3.4</version>
- <configuration>
-
- </configuration>
+ <artifactId>antlr4-maven-plugin</artifactId>
+ <version>${antlr.version}</version>
<executions>
<execution>
<goals>
- <goal>antlr</goal>
+ <goal>antlr4</goal>
</goals>
</execution>
</executions>
diff --git a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g b/logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprLexer.g4
similarity index 79%
rename from logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
rename to logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprLexer.g4
index 93dba94..53b39bd 100644
--- a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
+++ b/logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprLexer.g4
@@ -22,8 +22,6 @@ options {
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
-package org.apache.drill.common.expression.parser;
}
If : 'if';
@@ -99,40 +97,35 @@ Bool
Number
: Int ('.' Digit*)? (('e' | 'E') ('+' | '-')? Digit*)?
;
-
-//Float
-// : Int ('.' Digit*)? ('e' ('+' | '-')? Digit*)?
-// ;
-//
-//Integer
-// : Digit Digit*
-// ;
Identifier
: ('a'..'z' | 'A'..'Z' | '_' | '$') ('a'..'z' | 'A'..'Z' | '_' | '$' | Digit)*
;
QuotedIdentifier
-@after {
- setText(getText().substring(1, getText().length()-1).replaceAll("\\\\(.)", "$1"));
-}
- : '`' (~('`' | '\\') | '\\' ('\\' | '`'))* '`'
+ : '`' (~('`' | '\\') | '\\' ('\\' | '`'))* '`'
+ {
+ setText(getText().substring(1, getText().length()-1).replaceAll("\\\\(.)", "$1"));
+ }
;
String
-@after {
- setText(getText().substring(1, getText().length()-1).replaceAll("\\\\(.)", "$1"));
-}
: '\'' (~('\'' | '\\') | '\\' ('\\' | '\''))* '\''
+ {
+ setText(getText().substring(1, getText().length()-1).replaceAll("\\\\(.)", "$1"));
+ }
;
-Comment
- : '//' ~('\r' | '\n')* {skip();}
- | '/*' .* '*/' {skip();}
+LineComment
+ : '//' ~[\r\n]* -> skip
+ ;
+
+BlockComment
+ : '/*' .*? '*/' -> skip
;
Space
- : (' ' | '\t' | '\r' | '\n' | '\u000C') {skip();}
+ : [ \n\t\r\u000C]+ -> skip
;
fragment Int
@@ -143,14 +136,3 @@ fragment Int
fragment Digit
: '0'..'9'
;
-
-FallThrough
- @after{
- throw new RuntimeException(java.lang.String.format(
- "Encountered an illegal char on line \%d, column \%d: '\%s'",
- getLine(), getCharPositionInLine(), getText()
- )
- );
- }
- :
- ;
diff --git a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g b/logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprParser.g4
similarity index 85%
rename from logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
rename to logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprParser.g4
index 78a7cc3..d868cff 100644
--- a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
+++ b/logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprParser.g4
@@ -1,15 +1,10 @@
parser grammar ExprParser;
options{
- output=AST;
language=Java;
tokenVocab=ExprLexer;
- backtrack=true;
- memoize=true;
}
-
-
@header {
/*
* Licensed to the Apache Software Foundation (ASF) under one
@@ -28,11 +23,8 @@ options{
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
-package org.apache.drill.common.expression.parser;
//Explicit import...
-import org.antlr.runtime.BitSet;
import java.util.*;
import org.apache.drill.common.expression.*;
import org.apache.drill.common.expression.PathSegment.NameSegment;
@@ -55,25 +47,20 @@ import org.apache.drill.common.exceptions.ExpressionParsingException;
public ExpressionPosition pos(Token token){
return new ExpressionPosition(fullExpression, token.getTokenIndex());
}
-
- @Override
- public void displayRecognitionError(String[] tokenNames, RecognitionException e) {
- String hdr = getErrorHeader(e);
- String msg = getErrorMessage(e, tokenNames);
- throw new ExpressionParsingException("Expression has syntax error! " + hdr + ":" + msg);
- }
}
parse returns [LogicalExpression e]
: expression EOF {
- $e = $expression.e;
- if(fullExpression == null) fullExpression = $expression.text;
+ $e = $expression.e;
+ if (fullExpression == null) fullExpression = $expression.text;
tokenPos = $expression.start.getTokenIndex();
}
;
functionCall returns [LogicalExpression e]
- : Identifier OParen exprList? CParen {$e = FunctionCallFactory.createExpression($Identifier.text, pos($Identifier), $exprList.listE); }
+ : Identifier OParen exprList? CParen {$e =
+ FunctionCallFactory.createExpression($Identifier.text, pos($Identifier),
+ ($exprList.ctx == null ? new ArrayList<>() : $exprList.listE)); }
;
convertCall returns [LogicalExpression e]
@@ -82,16 +69,18 @@ convertCall returns [LogicalExpression e]
;
anyValueCall returns [LogicalExpression e]
- : AnyValue OParen exprList? CParen {$e = FunctionCallFactory.createExpression($AnyValue.text, pos($AnyValue), $exprList.listE); }
+ : AnyValue OParen exprList? CParen {$e =
+ FunctionCallFactory.createExpression($AnyValue.text, pos($AnyValue),
+ ($exprList.ctx == null ? new ArrayList<>() : $exprList.listE)); }
;
castCall returns [LogicalExpression e]
@init{
- List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
+ List<LogicalExpression> exprs = new ArrayList<>();
ExpressionPosition p = null;
}
: Cast OParen expression As dataType repeat? CParen
- { if ($repeat.isRep!=null && $repeat.isRep.compareTo(Boolean.TRUE)==0)
+ { if ($repeat.ctx != null && $repeat.isRep.compareTo(Boolean.TRUE)==0)
$e = FunctionCallFactory.createCast(TypeProtos.MajorType.newBuilder().mergeFrom($dataType.type).setMode(DataMode.REPEATED).build(), pos($Cast), $expression.e);
else
$e = FunctionCallFactory.createCast($dataType.type, pos($Cast), $expression.e);}
@@ -190,7 +179,7 @@ caseElseStat returns [LogicalExpression e]
exprList returns [List<LogicalExpression> listE]
@init{
- $listE = new ArrayList<LogicalExpression>();
+ $listE = new ArrayList<>();
}
: e1=expression {$listE.add($e1.e); } (Comma e2=expression {$listE.add($e2.e); } )*
;
@@ -207,7 +196,7 @@ condExpr returns [LogicalExpression e]
orExpr returns [LogicalExpression e]
@init{
- List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
+ List<LogicalExpression> exprs = new ArrayList<>();
ExpressionPosition p = null;
}
@after{
@@ -222,7 +211,7 @@ orExpr returns [LogicalExpression e]
andExpr returns [LogicalExpression e]
@init{
- List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
+ List<LogicalExpression> exprs = new ArrayList<>();
ExpressionPosition p = null;
}
@after{
@@ -237,8 +226,8 @@ andExpr returns [LogicalExpression e]
equExpr returns [LogicalExpression e]
@init{
- List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
- List<String> cmps = new ArrayList();
+ List<LogicalExpression> exprs = new ArrayList<>();
+ List<String> cmps = new ArrayList<>();
ExpressionPosition p = null;
}
@after{
@@ -254,8 +243,8 @@ relExpr returns [LogicalExpression e]
addExpr returns [LogicalExpression e]
@init{
- List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
- List<String> ops = new ArrayList();
+ List<LogicalExpression> exprs = new ArrayList<>();
+ List<String> ops = new ArrayList<>();
ExpressionPosition p = null;
}
@after{
@@ -266,8 +255,8 @@ addExpr returns [LogicalExpression e]
mulExpr returns [LogicalExpression e]
@init{
- List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
- List<String> ops = new ArrayList();
+ List<LogicalExpression> exprs = new ArrayList<>();
+ List<String> ops = new ArrayList<>();
ExpressionPosition p = null;
}
@after{
@@ -278,8 +267,8 @@ mulExpr returns [LogicalExpression e]
xorExpr returns [LogicalExpression e]
@init{
- List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
- List<String> ops = new ArrayList();
+ List<LogicalExpression> exprs = new ArrayList<>();
+ List<String> ops = new ArrayList<>();
ExpressionPosition p = null;
}
@after{
@@ -305,15 +294,35 @@ pathSegment returns [NameSegment seg]
;
nameSegment returns [NameSegment seg]
- : QuotedIdentifier ( (Period s1=pathSegment) | s2=arraySegment)? {$seg = new NameSegment($QuotedIdentifier.text, ($s1.seg == null ? $s2.seg : $s1.seg) ); }
- | Identifier ( (Period s1=pathSegment) | s2=arraySegment)? {$seg = new NameSegment($Identifier.text, ($s1.seg == null ? $s2.seg : $s1.seg) ); }
+ : QuotedIdentifier ( (Period s1=pathSegment) | s2=arraySegment)?
+ {
+ if ($s1.ctx == null && $s2.ctx == null) {
+ $seg = new NameSegment($QuotedIdentifier.text);
+ } else {
+ $seg = new NameSegment($QuotedIdentifier.text, ($s1.ctx == null ? $s2.seg : $s1.seg));
+ }
+ }
+ | Identifier ( (Period s1=pathSegment) | s2=arraySegment)?
+ {
+ if ($s1.ctx == null && $s2.ctx == null) {
+ $seg = new NameSegment($Identifier.text);
+ } else {
+ $seg = new NameSegment($Identifier.text, ($s1.ctx == null ? $s2.seg : $s1.seg));
+ }
+ }
;
arraySegment returns [PathSegment seg]
- : OBracket Number CBracket ( (Period s1=pathSegment) | s2=arraySegment)? {$seg = new ArraySegment($Number.text, ($s1.seg == null ? $s2.seg : $s1.seg) ); }
+ : OBracket Number CBracket ( (Period s1=pathSegment) | s2=arraySegment)?
+ {
+ if ($s1.ctx == null && $s2.ctx == null) {
+ $seg = new ArraySegment($Number.text);
+ } else {
+ $seg = new ArraySegment($Number.text, ($s1.ctx == null ? $s2.seg : $s1.seg));
+ }
+ }
;
-
lookup returns [LogicalExpression e]
: functionCall {$e = $functionCall.e ;}
| convertCall {$e = $convertCall.e; }
diff --git a/logical/src/main/java/org/apache/drill/common/expression/LogicalExpression.java b/logical/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
index cda77c3..3cfbade 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
@@ -19,14 +19,9 @@ package org.apache.drill.common.expression;
import java.io.IOException;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
-import org.apache.drill.common.expression.parser.ExprParser.parse_return;
import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.parser.LogicalExpressionParser;
import org.apache.drill.common.types.TypeProtos.MajorType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -68,20 +63,8 @@ public interface LogicalExpression extends Iterable<LogicalExpression>{
if (expr == null || expr.isEmpty()) {
return null;
}
- try {
- ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
- CommonTokenStream tokens = new CommonTokenStream(lexer);
- ExprParser parser = new ExprParser(tokens);
-
- //TODO: move functionregistry and error collector to injectables.
- //ctxt.findInjectableValue(valueId, forProperty, beanInstance)
- parse_return ret = parser.parse();
-
- // ret.e.resolveAndValidate(expr, errorCollector);
- return ret.e;
- } catch (RecognitionException e) {
- throw new RuntimeException(e);
- }
+
+ return LogicalExpressionParser.parse(expr);
}
}
diff --git a/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java b/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java
index f198620..6c6a094 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java
@@ -39,6 +39,10 @@ public abstract class PathSegment {
this(Integer.parseInt(numberAsText), child);
}
+ public ArraySegment(String numberAsText) {
+ this(Integer.parseInt(numberAsText), null);
+ }
+
public ArraySegment(int index, PathSegment child) {
super(child);
this.index = index;
@@ -105,7 +109,7 @@ public abstract class PathSegment {
@Override
public PathSegment clone() {
- PathSegment seg = index < 0 ? new ArraySegment(null) : new ArraySegment(index);
+ PathSegment seg = index < 0 ? new ArraySegment((PathSegment) null) : new ArraySegment(index);
if (getChild() != null) {
seg.setChild(getChild().clone());
}
@@ -114,7 +118,7 @@ public abstract class PathSegment {
@Override
public ArraySegment cloneWithNewChild(PathSegment newChild) {
- ArraySegment seg = index < 0 ? new ArraySegment(null) : new ArraySegment(index);
+ ArraySegment seg = index < 0 ? new ArraySegment((PathSegment) null) : new ArraySegment(index);
if (getChild() != null) {
seg.setChild(getChild().cloneWithNewChild(newChild));
} else {
diff --git a/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java b/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
index ba71712..d514d2e 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
@@ -21,15 +21,10 @@ import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
import org.apache.drill.common.expression.PathSegment.ArraySegment;
import org.apache.drill.common.expression.PathSegment.NameSegment;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
-import org.apache.drill.common.expression.parser.ExprParser.parse_return;
import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.parser.LogicalExpressionParser;
import org.apache.drill.common.types.TypeProtos.MajorType;
import org.apache.drill.common.types.Types;
import org.apache.drill.exec.proto.UserBitShared.NamePart;
@@ -179,23 +174,16 @@ public class SchemaPath extends LogicalExpressionBase {
if (expr == null || expr.isEmpty()) {
return null;
}
- try {
- if (SchemaPath.DYNAMIC_STAR.equals(expr)) {
- return SchemaPath.getSimplePath(expr);
- }
- ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
- CommonTokenStream tokens = new CommonTokenStream(lexer);
- ExprParser parser = new ExprParser(tokens);
- parse_return ret = parser.parse();
+ if (SchemaPath.DYNAMIC_STAR.equals(expr)) {
+ return SchemaPath.getSimplePath(expr);
+ }
- if (ret.e instanceof SchemaPath) {
- return (SchemaPath) ret.e;
- } else {
- throw new IllegalStateException("Schema path is not a valid format.");
- }
- } catch (RecognitionException e) {
- throw new RuntimeException(e);
+ LogicalExpression logicalExpression = LogicalExpressionParser.parse(expr);
+ if (logicalExpression instanceof SchemaPath) {
+ return (SchemaPath) logicalExpression;
+ } else {
+ throw new IllegalStateException(String.format("Schema path is not a valid format: %s.", logicalExpression));
}
}
diff --git a/logical/src/main/java/org/apache/drill/common/parser/ErrorListener.java b/logical/src/main/java/org/apache/drill/common/parser/ErrorListener.java
new file mode 100644
index 0000000..fe94368
--- /dev/null
+++ b/logical/src/main/java/org/apache/drill/common/parser/ErrorListener.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.common.parser;
+
+import org.antlr.v4.runtime.BaseErrorListener;
+import org.antlr.v4.runtime.RecognitionException;
+import org.antlr.v4.runtime.Recognizer;
+import org.apache.drill.common.exceptions.ExpressionParsingException;
+
+/**
+ * Custom error listener that converts all syntax errors into {@link ExpressionParsingException}.
+ */
+public class ErrorListener extends BaseErrorListener {
+
+ public static final ErrorListener INSTANCE = new ErrorListener();
+
+ @Override
+ public void syntaxError(Recognizer<?, ?> recognizer, Object offendingSymbol, int line,
+ int charPositionInLine, String msg, RecognitionException e) {
+ throw new ExpressionParsingException(msg);
+ }
+
+}
diff --git a/logical/src/main/java/org/apache/drill/common/parser/LogicalExpressionParser.java b/logical/src/main/java/org/apache/drill/common/parser/LogicalExpressionParser.java
new file mode 100644
index 0000000..ad30a20
--- /dev/null
+++ b/logical/src/main/java/org/apache/drill/common/parser/LogicalExpressionParser.java
@@ -0,0 +1,55 @@
+/*
+ * 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.drill.common.parser;
+
+import org.antlr.v4.runtime.CharStreams;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.parser.ExprLexer;
+import org.apache.drill.common.expression.parser.ExprParser;
+
+/**
+ * Helper class for parsing logical expression.
+ */
+public class LogicalExpressionParser {
+
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LogicalExpressionParser.class);
+
+ /**
+ * Initializes logical expression lexer and parser, add error listener that converts all
+ * syntax error into {@link org.apache.drill.common.exceptions.ExpressionParsingException}.
+ * Parses given expression into logical expression instance.
+ *
+ * @param expr expression to be parsed
+ * @return logical expression instance
+ */
+ public static LogicalExpression parse(String expr) {
+ ExprLexer lexer = new ExprLexer(CharStreams.fromString(expr));
+ lexer.removeErrorListeners(); // need to remove since default listener will output warning
+ lexer.addErrorListener(ErrorListener.INSTANCE);
+ CommonTokenStream tokens = new CommonTokenStream(lexer);
+
+ ExprParser parser = new ExprParser(tokens);
+ parser.removeErrorListeners(); // need to remove since default listener will output warning
+ parser.addErrorListener(ErrorListener.INSTANCE);
+ ExprParser.ParseContext parseContext = parser.parse();
+ logger.trace("Tokens: [{}]. Parsing details: [{}].", tokens.getText(), parseContext.toInfoString(parser));
+ return parseContext.e;
+ }
+
+}
diff --git a/logical/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java b/logical/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java
index 6dd489e..0d34716 100644
--- a/logical/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java
+++ b/logical/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java
@@ -17,103 +17,113 @@
*/
package org.apache.drill.common.expression.parser;
-import java.io.IOException;
-
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
+import org.apache.drill.common.exceptions.ExpressionParsingException;
import org.apache.drill.common.expression.ExpressionStringBuilder;
import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.parser.ExprParser.parse_return;
+import org.apache.drill.common.parser.LogicalExpressionParser;
import org.apache.drill.test.DrillTest;
+import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertEquals;
public class TreeTest extends DrillTest {
- private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TreeTest.class);
+
+ @Rule
+ public ExpectedException thrown = ExpectedException.none();
@Test
- public void escapeStringLiteral() throws Exception {
+ public void escapeStringLiteral() {
String expr = "func(`identifier`, '\\\\d+', 0, 'fjds')";
- testExpressionParsing(expr);
+ testExpressionParsing(expr, expr);
}
@Test
- public void escapeQuotedIdentifier() throws Exception {
+ public void escapeQuotedIdentifier() {
String expr = "`a\\\\b` + `c'd`";
- testExpressionParsing(expr);
+ testExpressionParsing(expr, "add(`a\\\\b`, `c'd`)");
}
@Test
- public void testIfWithCase() throws Exception{
- testExpressionParsing("if ($F1) then case when (_MAP.R_NAME = 'AFRICA') then 2 else 4 end else if(4==3) then 1 else if(x==3) then 7 else (if(2==1) then 6 else 4 end) end");
+ public void testIfWithCase() {
+ testExpressionParsing("if ($F1) then case when (_MAP.R_NAME = 'AFRICA') then 2 else 4 end else if(4==3) then 1 else if(x==3) then 7 else (if(2==1) then 6 else 4 end) end",
+ "( if (equal(`x`, 3) ) then (7 ) else ( ( if (equal(2, 1) ) then (6 ) else (4 ) end ) ) end )");
}
@Test
- public void testAdd() throws Exception{
- testExpressionParsing("2+2");
+ public void testAdd() {
+ testExpressionParsing("2+2", "add(2, 2)");
}
@Test
- public void testIf() throws Exception{
- testExpressionParsing("if ('blue.red') then 'orange' else if (false) then 1 else 0 end");
+ public void testIf() {
+ testExpressionParsing("if ('blue.red') then 'orange' else if (false) then 1 else 0 end",
+ "( if (false ) then (1 ) else (0 ) end )");
}
@Test
- public void testQuotedIdentifier() throws Exception{
- testExpressionParsing("`hello friend`.`goodbye`");
+ public void testQuotedIdentifier() {
+ String expr = "`hello friend`.`goodbye`";
+ testExpressionParsing(expr, expr);
}
@Test
- public void testSpecialQuoted() throws Exception{
- testExpressionParsing("`*0` + `*` ");
+ public void testSpecialQuoted() {
+ testExpressionParsing("`*0` + `*` ", "add(`*0`, `*`)");
}
@Test
- public void testQuotedIdentifier2() throws Exception{
- testExpressionParsing("`hello friend`.goodbye");
+ public void testQuotedIdentifier2() {
+ testExpressionParsing("`hello friend`.goodbye", "`hello friend`.`goodbye`");
}
@Test
- public void testComplexIdentifier() throws Exception{
- testExpressionParsing("goodbye[4].`hello`");
+ public void testComplexIdentifier() {
+ testExpressionParsing("goodbye[4].`hello`", "`goodbye`[4].`hello`");
}
@Test // DRILL-2606
- public void testCastToBooleanExpr() throws Exception{
- testExpressionParsing("cast( (cast( (`bool_col` ) as VARCHAR(100) ) ) as BIT )");
+ public void testCastToBooleanExpr() {
+ String expr = "cast( (cast( (`bool_col` ) as VARCHAR(100) ) ) as BIT )";
+ testExpressionParsing(expr, expr);
}
- private LogicalExpression parseExpression(String expr) throws RecognitionException, IOException{
+ @Test
+ public void testComments() {
+ testExpressionParsing("cast /* block comment */ ( // single comment\n" +
+ "1 as int)", "cast( (1 ) as INT )");
+ }
- ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
- CommonTokenStream tokens = new CommonTokenStream(lexer);
+ @Test
+ public void testParsingException() {
+ thrown.expect(ExpressionParsingException.class);
+ thrown.expectMessage(containsString("mismatched input 'i' expecting"));
+ testExpressionParsing("cast(1 as i)", "");
+ }
- ExprParser parser = new ExprParser(tokens);
- parse_return ret = parser.parse();
+ @Test
+ public void testFunctionCallWithoutParams() {
+ String expr = "now()";
+ testExpressionParsing(expr, expr);
+ }
- return ret.e;
+ /**
+ * Attempt to parse an expression. Once parsed, convert it to a string and then parse it again to make sure serialization works.
+ */
+ private void testExpressionParsing(String expr, String expected) {
+ LogicalExpression e1 = LogicalExpressionParser.parse(expr);
+ String newStringExpr = serializeExpression(e1);
+ assertEquals(expected, newStringExpr.trim());
+ LogicalExpressionParser.parse(newStringExpr);
}
private String serializeExpression(LogicalExpression expr){
-
ExpressionStringBuilder b = new ExpressionStringBuilder();
StringBuilder sb = new StringBuilder();
expr.accept(b, sb);
return sb.toString();
}
- /**
- * Attempt to parse an expression. Once parsed, convert it to a string and then parse it again to make sure serialization works.
- * @param expr
- * @throws RecognitionException
- * @throws IOException
- */
- private void testExpressionParsing(String expr) throws RecognitionException, IOException{
- logger.debug("-----" + expr + "-----");
- LogicalExpression e = parseExpression(expr);
-
- String newStringExpr = serializeExpression(e);
- logger.debug(newStringExpr);
- LogicalExpression e2 = parseExpression(newStringExpr);
- }
}
diff --git a/pom.xml b/pom.xml
index ca847d2..bd512c4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -89,6 +89,7 @@
<rat.skip>true</rat.skip>
<license.skip>true</license.skip>
<docker.repository>drill/apache-drill</docker.repository>
+ <antlr.version>4.7.1</antlr.version>
</properties>
<scm>
@@ -1648,6 +1649,11 @@
</exclusion>
</exclusions>
</dependency>
+ <dependency>
+ <groupId>org.antlr</groupId>
+ <artifactId>antlr4-runtime</artifactId>
+ <version>${antlr.version}</version>
+ </dependency>
</dependencies>
</dependencyManagement>
@@ -2708,11 +2714,6 @@
</exclusions>
</dependency>
<dependency>
- <groupId>org.antlr</groupId>
- <artifactId>antlr4-runtime</artifactId>
- <version>4.5</version>
- </dependency>
- <dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-maprdb-json-handler</artifactId>
<version>${hive.version}</version>
|