You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2021/03/18 23:17:41 UTC

[GitHub] [hive] mustafaiman commented on a change in pull request #2059: HIVE-24315. Improve validation and error handling in HPL/SQL (amagyar)

mustafaiman commented on a change in pull request #2059:
URL: https://github.com/apache/hive/pull/2059#discussion_r597255333



##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -845,6 +854,13 @@ public Integer init(String[] args) throws Exception {
     return 0;
   }
 
+  private HplsqlParser newParser(CommonTokenStream tokens) {
+    HplsqlParser parser = new HplsqlParser(tokens);
+    parser.removeErrorListeners();

Review comment:
       This looks unconventional. Can you add a comment explaining why this is necessary?

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -2256,24 +2276,26 @@ public Integer visitIdent(HplsqlParser.IdentContext ctx) {
           Var var1 = new Var(var);
           var1.negate();
           exec.stackPush(var1);
-        }
-        else {
+        } else {
           exec.stackPush(var);
         }
-      }
-      else {
+      } else {
         exec.stackPush(new Var(ident, Var.Type.STRING, var.toSqlString()));
       }
-    }
-    else {
-      if (!exec.buildSql && !exec.inCallStmt && exec.functions.exec(ident.toUpperCase(), null)) {
-        return 0;
+    } else {
+      if (exec.buildSql || exec.inCallStmt) {

Review comment:
       I could not understand what is happening here. Can you give some info on what this if-else statement is for?

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -974,14 +990,16 @@ void cleanup() {
   void printExceptions() {
     while (!signals.empty()) {
       Signal sig = signals.pop();
-      if (sig.type == Signal.Type.SQLEXCEPTION) {
+      if (sig.type == Signal.Type.VALIDATION) {

Review comment:
       What about "NOTFOUND" and "UNSUPPORTED_OPERATION"? Do we suppress them or are they handled somewhere else?

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -501,6 +501,10 @@ public void signal(QueryResult query) {
     signal(Signal.Type.SQLEXCEPTION, query.errorText(), query.exception());
   }
 
+  public void signalHplsql(HplValidationException exception) {

Review comment:
       This looks unused

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/SyntaxErrorReporter.java
##########
@@ -0,0 +1,50 @@
+/*
+ *  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.hive.hplsql;
+
+import java.util.BitSet;
+
+import org.antlr.v4.runtime.BaseErrorListener;
+import org.antlr.v4.runtime.Parser;
+import org.antlr.v4.runtime.RecognitionException;
+import org.antlr.v4.runtime.Recognizer;
+import org.antlr.v4.runtime.atn.ATNConfigSet;
+import org.antlr.v4.runtime.dfa.DFA;
+
+public class SyntaxErrorReporter extends BaseErrorListener {
+  private final Console console;
+
+  public SyntaxErrorReporter(Console console) {
+    this.console = console;
+  }
+
+  @Override
+  public void syntaxError(Recognizer<?, ?> recognizer, Object offendingSymbol, int line, int charPositionInLine, String msg, RecognitionException e) {
+    console.printError("Syntax error at line " + line + ":" + charPositionInLine + " " + msg);
+  }
+
+  public void reportAmbiguity(Parser recognizer, DFA dfa, int startIndex, int stopIndex, boolean exact, BitSet ambigAlts, ATNConfigSet configs) {

Review comment:
       We can remove these three empty overrides.

##########
File path: hplsql/src/test/results/local/func_no_return.out.txt
##########
@@ -0,0 +1 @@
+Ln:1 identifier 'CREATE' must be declared.

Review comment:
       This is not the correct error.  We most certainly do not need to declare 'CREATE'.

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
##########
@@ -369,21 +380,32 @@ public void operatorSub(HplsqlParser.ExprContext ctx) {
     Var v2 = evalPop(ctx.expr(1));
     if (v1.value == null || v2.value == null) {
       evalNull();
-    }
-    else if (v1.type == Type.BIGINT && v2.type == Type.BIGINT) {
-      exec.stackPush(new Var((Long)v1.value - (Long)v2.value)); 
-    }
-    else if (v1.type == Type.DATE && v2.type == Type.BIGINT) {
-      exec.stackPush(changeDateByInt((Date)v1.value, (Long)v2.value, false /*subtract*/));
-    }
-    else if (v1.type == Type.DATE && v2.type == Type.INTERVAL) {
+    } else if (v1.type == Type.BIGINT && v2.type == Type.BIGINT) {
+      exec.stackPush(new Var((long)v1.value - (long)v2.value));
+    } else if (v1.type == Type.BIGINT && v2.type == Type.DECIMAL) {
+      exec.stackPush(new Var((new BigDecimal((long)v1.value)).subtract((BigDecimal)v2.value)));
+    } else if (v1.type == Type.BIGINT && v2.type == Type.DOUBLE) {
+      exec.stackPush(new Var((long)v1.value - (double)v2.value));
+    } else if (v1.type == Type.DECIMAL && v2.type == Type.DECIMAL) {
+      exec.stackPush(new Var(((BigDecimal) v1.value).subtract((BigDecimal) v2.value)));
+    } else if (v1.type == Type.DECIMAL && v2.type == Type.BIGINT) {
+      exec.stackPush(new Var(((BigDecimal)v1.value).subtract(new BigDecimal((long)v2.value))));
+    } else if (v1.type == Type.DECIMAL && v2.type == Type.DOUBLE) {
+      exec.stackPush(new Var(((BigDecimal) v1.value).subtract(new BigDecimal((double) v2.value))));
+    } else if (v1.type == Type.DOUBLE && v2.type == Type.DOUBLE) {
+      exec.stackPush(new Var((double) v1.value - (double) v2.value));
+    } else if (v1.type == Type.DOUBLE && v2.type == Type.DECIMAL) {
+      exec.stackPush(new Var((new BigDecimal((double) v1.value)).subtract((BigDecimal) v2.value)));
+    } else if (v1.type == Type.DOUBLE && v2.type == Type.BIGINT) {
+      exec.stackPush(new Var(((double) v1.value) - (long)v2.value));
+    } else if (v1.type == Type.DATE && v2.type == Type.BIGINT) {
+      exec.stackPush(changeDateByInt((Date)v1.value, (long)v2.value, false /*subtract*/));
+    } else if (v1.type == Type.DATE && v2.type == Type.INTERVAL) {
       exec.stackPush(new Var(((Interval)v2.value).dateChange((Date)v1.value, false /*subtract*/)));
-    }
-    else if (v1.type == Type.TIMESTAMP && v2.type == Type.INTERVAL) {
+    } else if (v1.type == Type.TIMESTAMP && v2.type == Type.INTERVAL) {
       exec.stackPush(new Var(((Interval)v2.value).timestampChange((Timestamp)v1.value, false /*subtract*/), v1.scale));
-    }
-    else {
-      evalNull();
+    } else {
+      exec.signal(Signal.Type.UNSUPPORTED_OPERATION, "Unsupported data types in subtraction operator");

Review comment:
       I think we should report the data types of both operands in the error message. The same goes for other add, division and multiply too of course.

##########
File path: hplsql/src/test/queries/local/conversion.sql
##########
@@ -0,0 +1,55 @@
+declare a BIGINT = 3;
+declare b DECIMAL = 4;
+declare c DOUBLE = 5;
+
+-- add
+a + a;
+a + b;
+a + c;
+
+b + b;
+b + a;
+b + c;
+
+c + c;
+c + a;
+c + b;
+
+-- sub
+a - a;
+a - b;
+a - c;
+
+b - b;
+b - a;
+b - c;
+
+c - c;
+c - a;
+c - b;
+
+-- mul
+a * a;
+a * b;
+a * c;
+
+b * b;
+b * a;
+b * c;
+
+c * c;
+c * a;
+c * b;
+
+-- div
+a / a;
+a / b;
+a / c;
+
+b / b;
+b / (a - 2);
+b / c;
+
+c / c;
+c / a;
+c / b;

Review comment:
       I know this is a boring and annoying test but we should cover all combinations of possible types and operators. There are STRING, DATE, TIMESTAMP, INTERVAL types and their operations which were not covered here.

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/UndefinedException.java
##########
@@ -0,0 +1,27 @@
+/*
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hive.hplsql;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+
+public class UndefinedException extends HplValidationException {

Review comment:
       minor: UndefinedException sounds like the exception itself is undefined. Maybe we can name this UndefinedIdentifierException.

##########
File path: hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
##########
@@ -407,19 +408,68 @@ public void testWhile() throws Exception {
     run("while");
   }
 
+  @Test
+  public void testArity() throws Exception {
+    run("arity");
+  }
+
+  @Test
+  public void testTypeCheck() throws Exception {
+    run("type_check");
+  }
+
+  @Test
+  public void testUndefFunc() throws Exception {
+    run("undef_func");
+  }
+
+  @Test
+  public void testUndefVar() throws Exception {
+    run("undef_var");
+  }
+
+  @Test
+  public void testNull() throws Exception {
+    run("null");
+  }
+
+  @Test
+  public void testFuncNoReturn() throws Exception {
+    run("func_no_return");
+  }
+
+  @Test
+  public void testInvalidSyntax() throws Exception {
+    run("invalid_syntax");
+  }
+
+  @Test
+  public void testPrecedence() throws Exception {
+    run("preced");
+  }
+
+  @Test
+  public void testConversion() throws Exception {
+    run("conversion");
+  }
+
+
   /**
    * Run a test file
    */
   void run(String testFile) throws Exception {
     System.setOut(new PrintStream(out));
+    System.setErr(new PrintStream(err));
     Exec exec = new Exec();
     String[] args = { "-f", "src/test/queries/local/" + testFile + ".sql", "-trace" };
     exec.run(args);
-    String s = getTestOutput(out.toString()).trim();
-    FileUtils.writeStringToFile(new java.io.File("target/tmp/log/" + testFile + ".out.txt"), s);
+    String sout = getTestOutput(out.toString()).trim();

Review comment:
       I think we should not trim output files. Sure, it is not a major failure if the tests output a few extra new lines. However, it would be still weird behavior that will need to be investigated.

##########
File path: hplsql/src/test/queries/local/arity.sql
##########
@@ -0,0 +1,8 @@
+create procedure p(a int, b int)
+begin
+  print 'a=' || a;
+  print 'b=' || b;
+end;
+
+call p(1,2);
+call p(1);

Review comment:
       what about the case where we call with extra arguments?

##########
File path: hplsql/src/test/queries/local/preced.sql
##########
@@ -0,0 +1,9 @@
+declare a int = 0;
+declare b int = 0;
+declare c int = -1;
+
+a - b + c;
+
+2.0 * 0 * 0 + -1.0;
+
+(10 - 2) * 3;

Review comment:
       None of these actually test operator precedence. All three are calculated from left to right correctly. You would not know if parser did not care about precedence and did the operations from left to right.

##########
File path: hplsql/src/test/queries/local/func_no_return.sql
##########
@@ -0,0 +1,6 @@
+create function f()
+begin
+  print 'no return';
+end;
+
+print f();

Review comment:
       The error happens in create function statement. We do not reach here. print f() should be removed from the test.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org