You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by jh...@apache.org on 2013/08/13 12:40:04 UTC

git commit: TAJO-100: Port the parse error handling to the new parser.

Updated Branches:
  refs/heads/master 301da59dd -> a2693722b


TAJO-100: Port the parse error handling to the new parser.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tajo/commit/a2693722
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tajo/tree/a2693722
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tajo/diff/a2693722

Branch: refs/heads/master
Commit: a2693722be849bc7174d051a6df6e56c70f69c2e
Parents: 301da59
Author: jinossy <ji...@gmail.com>
Authored: Tue Aug 13 19:35:54 2013 +0900
Committer: jinossy <ji...@gmail.com>
Committed: Tue Aug 13 19:35:54 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../org/apache/tajo/engine/parser/SQLParser.g4  |  24 ++---
 .../apache/tajo/engine/parser/SQLAnalyzer.java  |  14 ++-
 .../tajo/engine/parser/SQLErrorListener.java    |  39 +++++++
 .../tajo/engine/parser/SQLErrorStrategy.java    |  66 ++++++++++++
 .../engine/query/exception/SQLParseError.java   | 105 +++++++++++++++++++
 .../engine/query/exception/SQLSyntaxError.java  |  49 +++++++++
 .../engine/query/exception/TQLParseError.java   |  44 --------
 .../engine/query/exception/TQLSyntaxError.java  |  92 ----------------
 .../org/apache/tajo/master/ClientService.java   |  25 +++--
 tajo-project/pom.xml                            |   6 +-
 11 files changed, 306 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f41a886..808d9cb 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -15,6 +15,8 @@ Release 0.2.0 - unreleased
 
   IMPROVEMENTS
 
+    TAJO-100: Port the parse error handling to the new parser. (jinho)
+
     TAJO-121: Add LogicalPlanVisitor and Refactor LogicalOptimizer to use the
     visitor. (hyunsik)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
index 2354a32..765f3d6 100644
--- a/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
+++ b/tajo-core/tajo-core-backend/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4
@@ -60,7 +60,7 @@ character_string_type
   ;
 
 type_length
-  : LEFT_PAREN! NUMBER RIGHT_PAREN!
+  : LEFT_PAREN NUMBER RIGHT_PAREN
   ;
 
 national_character_string_type
@@ -107,8 +107,8 @@ approximate_numeric_type
   ;
 
 precision_param
-  : LEFT_PAREN! precision=NUMBER RIGHT_PAREN!
-  | LEFT_PAREN! precision=NUMBER COMMA! scale=NUMBER RIGHT_PAREN!
+  : LEFT_PAREN precision=NUMBER RIGHT_PAREN
+  | LEFT_PAREN precision=NUMBER COMMA scale=NUMBER RIGHT_PAREN
   ;
 
 boolean_type
@@ -389,7 +389,7 @@ join_specification
   ;
 
 join_condition
-  : ON^ search_condition
+  : ON search_condition
   ;
 
 named_columns_join
@@ -410,7 +410,7 @@ derived_table
   ;
 
 where_clause
-  : WHERE^ search_condition
+  : WHERE search_condition
   ;
 
 search_condition
@@ -438,7 +438,7 @@ table_subquery
   ;
 
 subquery
-	:  LEFT_PAREN! query_expression RIGHT_PAREN!
+	:  LEFT_PAREN query_expression RIGHT_PAREN
 	;
 
 /*
@@ -497,7 +497,7 @@ empty_grouping_set
   ;
 
 having_clause
-  : HAVING^ boolean_value_expression
+  : HAVING boolean_value_expression
   ;
 
 /*
@@ -541,11 +541,11 @@ null_ordering
 */
 
 boolean_value_expression
-  : and_predicate (OR^ and_predicate)*
+  : and_predicate (OR and_predicate)*
   ;
 
 and_predicate
-  : boolean_factor (AND^ boolean_factor)*
+  : boolean_factor (AND boolean_factor)*
   ;
 
 boolean_factor
@@ -569,7 +569,7 @@ boolean_primary
   : predicate
   | numeric_value_expression  
   | case_expression
-  | LEFT_PAREN! boolean_value_expression RIGHT_PAREN!
+  | LEFT_PAREN boolean_value_expression RIGHT_PAREN
   ;
 
 /*
@@ -620,7 +620,7 @@ in_predicate
 
 in_predicate_value
   : table_subquery
-  | LEFT_PAREN! in_value_list  RIGHT_PAREN!
+  | LEFT_PAREN in_value_list  RIGHT_PAREN
   ;
 
 in_value_list
@@ -719,7 +719,7 @@ numeric_primary
   | column_reference  
   | routine_invocation
   | scalar_subquery
-  | LEFT_PAREN! numeric_value_expression RIGHT_PAREN!
+  | LEFT_PAREN numeric_value_expression RIGHT_PAREN
   ;
 
 literal

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
index 92d0336..aad275d 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
@@ -25,6 +25,8 @@ import org.apache.tajo.algebra.*;
 import org.apache.tajo.algebra.Aggregation.GroupType;
 import org.apache.tajo.algebra.LiteralValue.LiteralType;
 import org.apache.tajo.engine.parser.SQLParser.*;
+import org.apache.tajo.engine.query.exception.SQLParseError;
+import org.apache.tajo.engine.query.exception.SQLSyntaxError;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -45,7 +47,17 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
     CommonTokenStream tokens = new CommonTokenStream(lexer);
     this.parser = new SQLParser(tokens);
     parser.setBuildParseTree(true);
-    SqlContext context = parser.sql();
+    parser.removeErrorListeners();
+
+    parser.setErrorHandler(new SQLErrorStrategy());
+    parser.addErrorListener(new SQLErrorListener());
+
+    SqlContext context;
+    try {
+      context = parser.sql();
+    } catch (SQLParseError e) {
+      throw new SQLSyntaxError(e);
+    }
     return visitSql(context);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLErrorListener.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLErrorListener.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLErrorListener.java
new file mode 100644
index 0000000..24ef3cb
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLErrorListener.java
@@ -0,0 +1,39 @@
+/**
+ * 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.tajo.engine.parser;
+
+import org.antlr.v4.runtime.*;
+import org.apache.commons.lang.StringUtils;
+import org.apache.tajo.engine.query.exception.SQLParseError;
+
+public class SQLErrorListener extends BaseErrorListener {
+  public void syntaxError(Recognizer<?, ?> recognizer,
+                          Object offendingSymbol,
+                          int line, int charPositionInLine,
+                          String msg,
+                          RecognitionException e) {
+    CommonTokenStream tokens = (CommonTokenStream) recognizer.getInputStream();
+    String input = tokens.getTokenSource().getInputStream().toString();
+    Token token = (Token) offendingSymbol;
+    String[] lines = StringUtils.splitPreserveAllTokens(input, '\n');
+    String errorLine = lines[line - 1];
+
+    throw new SQLParseError(token, line, charPositionInLine, msg, errorLine);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLErrorStrategy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLErrorStrategy.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLErrorStrategy.java
new file mode 100644
index 0000000..7e1d43c
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLErrorStrategy.java
@@ -0,0 +1,66 @@
+/**
+ * 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.tajo.engine.parser;
+
+import org.antlr.v4.runtime.*;
+import org.antlr.v4.runtime.misc.NotNull;
+
+public class SQLErrorStrategy extends DefaultErrorStrategy {
+
+  @Override
+  public void reportError(Parser recognizer, RecognitionException e) {
+    // if we've already reported an error and have not matched a token
+    // yet successfully, don't report any errors.
+    if (inErrorRecoveryMode(recognizer)) {
+      return; // don't report spurious errors
+    }
+    beginErrorCondition(recognizer);
+    if (e instanceof NoViableAltException) {
+      reportNoViableAltException(recognizer, (NoViableAltException) e);
+    } else if (e instanceof InputMismatchException) {
+      reportInputMismatchException(recognizer, (InputMismatchException) e);
+    } else if (e instanceof FailedPredicateException) {
+      reportFailedPredicate(recognizer, (FailedPredicateException) e);
+    } else {
+      recognizer.notifyErrorListeners(e.getOffendingToken(), e.getMessage(), e);
+    }
+  }
+
+  protected void reportNoViableAltException(@NotNull Parser recognizer, @NotNull NoViableAltException e) {
+    TokenStream tokens = recognizer.getInputStream();
+    String msg;
+    Token token = e.getStartToken();
+    if (tokens != null) {
+      if (tokens.LT(-1) != null && token.getType() == Token.EOF) {
+        token = tokens.LT(-1);
+      }
+      msg = "syntax error at or near " + getTokenErrorDisplay(token);
+    } else {
+      msg = "no viable alternative at input " + escapeWSAndQuote("<unknown input>");
+    }
+    recognizer.notifyErrorListeners(token, msg, e);
+  }
+
+  protected void reportInputMismatchException(@NotNull Parser recognizer,
+                                              @NotNull InputMismatchException e) {
+    String msg = "mismatched input " + getTokenErrorDisplay(e.getOffendingToken()) +
+        " expecting " + e.getExpectedTokens().toString(recognizer.getTokenNames());
+    recognizer.notifyErrorListeners(e.getOffendingToken(), msg, e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/SQLParseError.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/SQLParseError.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/SQLParseError.java
new file mode 100644
index 0000000..34b639b
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/SQLParseError.java
@@ -0,0 +1,105 @@
+/**
+ * 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.tajo.engine.query.exception;
+
+
+import org.antlr.v4.runtime.Token;
+import org.apache.commons.lang.StringUtils;
+
+public class SQLParseError extends RuntimeException {
+  private String header;
+  private String errorLine;
+  private int charPositionInLine;
+  private int line;
+  private Token offendingToken;
+  private String detailedMessage;
+
+  public SQLParseError(Token offendingToken,
+                       int line, int charPositionInLine,
+                       String msg,
+                       String errorLine) {
+    super(msg);
+    this.offendingToken = offendingToken;
+    this.charPositionInLine = charPositionInLine;
+    this.line = line;
+    this.errorLine = errorLine;
+    this.header = msg;
+  }
+
+  @Override
+  public String getMessage() {
+    if (detailedMessage == null) {
+      if (offendingToken != null) {
+        detailedMessage = getDetailedMessageWithLocation();
+      } else {
+        StringBuilder sb = new StringBuilder();
+        sb.append("ERROR: ").append(header).append("\n");
+        sb.append("LINE: ").append(errorLine);
+        detailedMessage = sb.toString();
+      }
+    }
+
+    return detailedMessage;
+  }
+
+  public String getMessageHeader(){
+    return this.header;
+  }
+
+  private String getDetailedMessageWithLocation() {
+    StringBuilder sb = new StringBuilder();
+    int displayLimit = 80;
+    String queryPrefix = "LINE " + line + ":" + charPositionInLine + " ";
+    String prefixPadding = StringUtils.repeat(" ", queryPrefix.length());
+    String locationString;
+
+    int tokenLength = offendingToken.getStopIndex() - offendingToken.getStartIndex() + 1;
+    if(tokenLength > 0){
+      locationString = StringUtils.repeat(" ", charPositionInLine) + StringUtils.repeat("^", tokenLength);
+    } else {
+      locationString = StringUtils.repeat(" ", charPositionInLine) + "^";
+    }
+
+    sb.append("ERROR: ").append(header).append("\n");
+    sb.append(queryPrefix);
+
+    if (errorLine.length() > displayLimit) {
+      int padding = (displayLimit / 2);
+
+      String ellipsis = " ... ";
+      int startPos = locationString.length() - padding - 1;
+      if (startPos <= 0) {
+        startPos = 0;
+        sb.append(errorLine.substring(startPos, displayLimit)).append(ellipsis).append("\n");
+        sb.append(prefixPadding).append(locationString);
+      } else if (errorLine.length() - (locationString.length() + padding) <= 0) {
+        startPos = errorLine.length() - displayLimit - 1;
+        sb.append(ellipsis).append(errorLine.substring(startPos)).append("\n");
+        sb.append(prefixPadding).append(locationString.substring(startPos - ellipsis.length()));
+      } else {
+        sb.append(ellipsis).append(errorLine.substring(startPos, startPos + displayLimit)).append(ellipsis).append("\n");
+        sb.append(prefixPadding).append(locationString.substring(startPos - ellipsis.length()));
+      }
+    } else {
+      sb.append(errorLine).append("\n");
+      sb.append(prefixPadding).append(locationString);
+    }
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/SQLSyntaxError.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/SQLSyntaxError.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/SQLSyntaxError.java
new file mode 100644
index 0000000..619b11d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/SQLSyntaxError.java
@@ -0,0 +1,49 @@
+/**
+ * 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.tajo.engine.query.exception;
+
+
+public class SQLSyntaxError extends InvalidQueryException {
+  private static final long serialVersionUID = 5388279335175632066L;
+
+  private String errorMessage;
+  private String detailedMessage;
+  private SQLParseError parseError;
+
+  public SQLSyntaxError(String errorMessage) {
+    this.errorMessage = errorMessage;
+  }
+
+  public SQLSyntaxError(SQLParseError e) {
+    this.errorMessage = e.getMessageHeader();
+    this.parseError = e;
+  }
+
+  @Override
+  public String getMessage() {
+    if (detailedMessage == null) {
+      if (parseError != null) {
+        detailedMessage = parseError.getMessage();
+      } else {
+        detailedMessage = String.format("ERROR: %s\n", errorMessage);
+      }
+    }
+    return detailedMessage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/TQLParseError.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/TQLParseError.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/TQLParseError.java
deleted file mode 100644
index 5f9258e..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/TQLParseError.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * 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.tajo.engine.query.exception;
-
-
-@SuppressWarnings("UnusedDeclaration")
-public class TQLParseError extends RuntimeException {
-  private int errorLine;
-  private int errorPosition;
-
-  public TQLParseError(String parseErrorMessage) {
-    super(parseErrorMessage);
-  }
-
-  public TQLParseError(String parseErrorMessage, int line, int position) {
-    super(parseErrorMessage);
-    this.errorLine = line;
-    this.errorPosition = position;
-  }
-
-  public int getErrorPosition(){
-    return this.errorPosition;
-  }
-
-  public int getErrorLine(){
-    return this.errorLine;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/TQLSyntaxError.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/TQLSyntaxError.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/TQLSyntaxError.java
deleted file mode 100644
index 80a767f..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/exception/TQLSyntaxError.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
- * 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.tajo.engine.query.exception;
-
-
-import org.apache.commons.lang.StringUtils;
-
-public class TQLSyntaxError extends InvalidQueryException {
-  private static final long serialVersionUID = 5388279335175632066L;
-
-  private String query;
-  private String errorMessage;
-  private String detailedMessage;
-  private TQLParseError parseError;
-
-  public TQLSyntaxError(String query, String errorMessage) {
-    this.query = query;
-    this.errorMessage = errorMessage;
-  }
-
-  public TQLSyntaxError(String query, TQLParseError e) {
-    this.query = query;
-    this.errorMessage = e.getMessage();
-    this.parseError = e;
-  }
-
-  @Override
-  public String getMessage() {
-    if (detailedMessage == null) {
-      if (parseError != null) {
-        detailedMessage = getDetailedMessageWithLocation();
-      } else {
-        StringBuilder sb = new StringBuilder();
-        sb.append("ERROR: " + errorMessage).append("\n");
-        sb.append("LINE: " + query);
-        detailedMessage = sb.toString();
-      }
-    }
-
-    return detailedMessage;
-  }
-
-  private String getDetailedMessageWithLocation() {
-    StringBuilder sb = new StringBuilder();
-    int displayLimit = 80;
-    String queryPrefix = "LINE " + parseError.getErrorLine() + ":" +parseError.getErrorPosition() + " ";
-    String prefixPadding = StringUtils.repeat(" ", queryPrefix.length());
-    String locationString = StringUtils.repeat(" ", parseError.getErrorPosition()) + "^";
-
-    sb.append("ERROR: " + this.errorMessage).append("\n");
-    sb.append(queryPrefix);
-
-    if (query.length() > displayLimit) {
-      int padding = (displayLimit / 2);
-
-      String ellipsis = " ... ";
-      int startPos = locationString.length() - padding - 1;
-      if (startPos <= 0) {
-        startPos = 0;
-        sb.append(query.substring(startPos, displayLimit)).append(ellipsis).append("\n");
-        sb.append(prefixPadding).append(locationString);
-      } else if (query.length() - (locationString.length() + padding) <= 0) {
-        startPos = query.length() - displayLimit - 1;
-        sb.append(ellipsis).append(query.substring(startPos)).append("\n");
-        sb.append(prefixPadding).append(locationString.substring(startPos - ellipsis.length()));
-      } else {
-        sb.append(ellipsis).append(query.substring(startPos, startPos + displayLimit)).append(ellipsis).append("\n");
-        sb.append(prefixPadding).append(locationString.substring(startPos - ellipsis.length()));
-      }
-    } else {
-      sb.append(query).append("\n");
-      sb.append(prefixPadding).append(locationString);
-    }
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ClientService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ClientService.java
index a6a5f68..c980dcf 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ClientService.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ClientService.java
@@ -40,6 +40,7 @@ import org.apache.tajo.client.ClientProtocol;
 import org.apache.tajo.client.ClientProtocol.*;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.engine.query.exception.SQLSyntaxError;
 import org.apache.tajo.master.TajoMaster.MasterContext;
 import org.apache.tajo.master.event.QueryEvent;
 import org.apache.tajo.master.event.QueryEventType;
@@ -122,27 +123,35 @@ public class ClientService extends AbstractService {
 
     @Override
     public SubmitQueryRespose submitQuery(RpcController controller,
-                                           QueryRequest request)
+                                          QueryRequest request)
         throws ServiceException {
 
       QueryId queryId;
-
+      SubmitQueryRespose.Builder build = SubmitQueryRespose.newBuilder();
       try {
         queryId = context.getGlobalEngine().executeQuery(request.getQuery());
+      } catch (SQLSyntaxError e) {
+        build.setResultCode(ResultCode.ERROR);
+        build.setErrorMessage(e.getMessage());
+        return build.build();
+
       } catch (Exception e) {
-        SubmitQueryRespose.Builder build = SubmitQueryRespose.newBuilder();
         build.setResultCode(ResultCode.ERROR);
-        if (e.getMessage() != null) {
-          build.setErrorMessage(ExceptionUtils.getStackTrace(e));
+        String msg = e.getMessage();
+        if (msg == null) {
+          msg = "Internal Error";
+        }
+
+        if (LOG.isDebugEnabled()) {
+          LOG.error(msg, e);
         } else {
-          LOG.error("Internal Error", e);
-          build.setErrorMessage("Internal Error");
+          LOG.error(msg);
         }
+        build.setErrorMessage(msg);
         return build.build();
       }
 
       LOG.info("Query " + queryId + " is submitted");
-      SubmitQueryRespose.Builder build = SubmitQueryRespose.newBuilder();
       build.setResultCode(ResultCode.OK);
       build.setQueryId(queryId.getProto());
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/a2693722/tajo-project/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml
index 108a7ba..b03880e 100644
--- a/tajo-project/pom.xml
+++ b/tajo-project/pom.xml
@@ -326,7 +326,7 @@
         <plugin>
           <groupId>org.antlr</groupId>
           <artifactId>antlr4-maven-plugin</artifactId>
-          <version>4.0</version>
+          <version>4.1</version>
         </plugin>
         <!--This plugin's configuration is used to store Eclipse m2e settings only. 
           It has no influence on the Maven build itself. -->
@@ -662,14 +662,14 @@
       <dependency>
         <groupId>org.antlr</groupId>
         <artifactId>antlr4</artifactId>
-        <version>4.0</version>
+        <version>4.1</version>
         <type>jar</type>
         <scope>compile</scope>
       </dependency>
       <dependency>
         <groupId>org.antlr</groupId>
         <artifactId>antlr4-runtime</artifactId>
-        <version>4.0</version>
+        <version>4.1</version>
         <type>jar</type>
         <scope>compile</scope>
       </dependency>