[ 
https://issues.apache.org/jira/browse/DRILL-6870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16699197#comment-16699197
 ] 

ASF GitHub Bot commented on DRILL-6870:
---------------------------------------

asfgit closed pull request #1554: DRILL-6870: Upgrade to ANTLR4
URL: https://github.com/apache/drill/pull/1554
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/common/pom.xml b/common/pom.xml
index c931c017ff8..fba9efc0d54 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -101,12 +101,6 @@
       <artifactId>metrics-jmx</artifactId>
     </dependency>
 
-    <dependency>
-      <groupId>org.antlr</groupId>
-      <artifactId>antlr-runtime</artifactId>
-      <version>3.4</version>
-    </dependency>
-
     <dependency>
         <groupId>joda-time</groupId>
         <artifactId>joda-time</artifactId>
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 d949634c815..d26487cf7a8 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,6 +18,7 @@
 
 package org.apache.drill.exec.planner.index;
 
+import org.apache.drill.common.parser.LogicalExpressionParser;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 import com.mapr.db.Admin;
 import com.mapr.db.MapRDB;
@@ -25,16 +26,12 @@
 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 @@
 
 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 IndexCollection getTableIndex(String tableName) {
   /**
    * 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 DrillTable getNativeDrillTable(IndexDescriptor 
idxDescriptor) {
         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 @@ private LogicalExpression castFunctionSQLSyntax(String 
field, String type) throw
     }
     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 @@ private CollationContext 
buildCollationContext(List<LogicalExpression> indexFiel
     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 @@ private Admin admin() {
 
     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 15a6e4f8cf8..6725bb9558d 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.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 static FileSystem getLocalFileSystem() throws 
IOException {
     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 @@ protected DrillbitContext mockDrillbitContext() throws 
Exception {
     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 91ce6530b0f..631f1f52a07 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 2e214088e92..7d150a26f5e 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.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 void testUnnestNonArrayColumn() {
 
     // 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 @@ private boolean isTerminal(RecordBatch.IterOutcome 
outcome) {
 
     // 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 a6b59066e57..d65cb3e6ebd 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 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 300e88bf243..84a7c785bdc 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.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.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 void teardown() {
     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 d45bd6f2e87..37e05cd7c9b 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.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 TestBuilder 
preparedStatement(PreparedStatementHandle preparedStatementHa
   }
 
   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 TestBuilder approximateEquality() {
 
   // 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 TestBuilder baselineTypes(Map<SchemaPath, 
TypeProtos.MajorType> baselineT
   }
 
   boolean typeInfoSet() {
-    if (baselineTypeMap != null) {
-      return true;
-    } else {
-      return false;
-    }
+    return baselineTypeMap != null;
   }
 
   /**
@@ -513,11 +495,7 @@ protected TestBuilder reset() {
 
     @Override
     boolean typeInfoSet() {
-      if (super.typeInfoSet() || baselineTypes != null) {
-        return true;
-      } else {
-        return false;
-      }
+      return super.typeInfoSet() || baselineTypes != null;
     }
 
     @Override
@@ -683,7 +661,7 @@ boolean typeInfoSet() {
   }
 
   /**
-   * 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 ed9abb768e6..1d7325a4c24 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>
@@ -112,15 +111,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 93dba9478f8..53b39bd98a7 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 78a7cc3297f..d868cff3e7c 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 cda77c3e770..3cfbade080e 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 @@
 
 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 LogicalExpression deserialize(JsonParser jp, 
DeserializationContext ctxt)
       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 f1986209971..6c6a094513c 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 ArraySegment(String numberAsText, PathSegment child) {
       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 boolean segmentEquals(PathSegment obj) {
 
     @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 PathSegment clone() {
 
     @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 ba71712b7f3..d514d2e280d 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.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 static SchemaPath parseFromString(String expr) {
     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 00000000000..fe943683d84
--- /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 00000000000..ad30a202863
--- /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 6dd489eb217..0d3471663c2 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 d69c67993b9..d98af691257 100644
--- a/pom.xml
+++ b/pom.xml
@@ -88,6 +88,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>
@@ -1644,6 +1645,11 @@
           </exclusion>
         </exclusions>
       </dependency>
+      <dependency>
+        <groupId>org.antlr</groupId>
+        <artifactId>antlr4-runtime</artifactId>
+        <version>${antlr.version}</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 
@@ -2703,11 +2709,6 @@
               </exclusion>
             </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>


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Upgrade to ANTLR4
> -----------------
>
>                 Key: DRILL-6870
>                 URL: https://issues.apache.org/jira/browse/DRILL-6870
>             Project: Apache Drill
>          Issue Type: Task
>            Reporter: Arina Ielchiieva
>            Assignee: Arina Ielchiieva
>            Priority: Major
>              Labels: ready-to-commit
>             Fix For: 1.15.0
>
>
> Upgrade from ANTLR3 to ANTLR4.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to