You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2015/07/01 18:18:41 UTC

[2/4] hive git commit: HIVE-11055 HPL/SQL - Implementing Procedural SQL in Hive (PL/HQL Contribution) (Dmitry Tolpeko via gates)

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Select.java b/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
new file mode 100644
index 0000000..e0f4098
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
@@ -0,0 +1,411 @@
+/**
+ * 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.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.Stack;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.antlr.v4.runtime.misc.Interval;
+
+public class Select {
+
+  Exec exec = null;
+  Stack<Var> stack = null;
+  Conf conf;
+  
+  boolean trace = false; 
+  
+  Select(Exec e) {
+    exec = e;  
+    stack = exec.getStack();
+    conf = exec.getConf();
+    trace = exec.getTrace();
+  }
+   
+  /**
+   * Executing or building SELECT statement
+   */
+  public Integer select(HplsqlParser.Select_stmtContext ctx) { 
+    if (ctx.parent instanceof HplsqlParser.StmtContext) {
+      exec.stmtConnList.clear();
+      trace(ctx, "SELECT");
+    }
+    boolean oldBuildSql = exec.buildSql; 
+    exec.buildSql = true;
+    StringBuilder sql = new StringBuilder();
+    if (ctx.cte_select_stmt() != null) {
+      sql.append(evalPop(ctx.cte_select_stmt()).toString());
+      sql.append("\n");
+    }
+    sql.append(evalPop(ctx.fullselect_stmt()).toString());
+    exec.buildSql = oldBuildSql;
+    if (!(ctx.parent instanceof HplsqlParser.StmtContext)) {           // No need to execute at this stage
+      exec.stackPush(sql);
+      return 0;  
+    }    
+    if (trace && ctx.parent instanceof HplsqlParser.StmtContext) {
+      trace(ctx, sql.toString());
+    }
+    if (exec.getOffline()) {
+      trace(ctx, "Not executed - offline mode set");
+      return 0;
+    }
+    String conn = exec.getStatementConnection();
+    Query query = exec.executeQuery(ctx, sql.toString(), conn);
+    if (query.error()) { 
+      exec.signal(query);
+      return 1;
+    }
+    trace(ctx, "SELECT completed successfully");
+    exec.setSqlSuccess();
+    try {
+      ResultSet rs = query.getResultSet();
+      ResultSetMetaData rm = null;
+      if (rs != null) {
+        rm = rs.getMetaData();
+      }
+      HplsqlParser.Into_clauseContext into = getIntoClause(ctx);
+      if (into != null) {
+        trace(ctx, "SELECT INTO statement executed");
+        int cols = into.ident().size();          
+        if (rs.next()) {
+          for (int i = 1; i <= cols; i++) {
+            Var var = exec.findVariable(into.ident(i-1).getText());
+            if (var != null) {
+              var.setValue(rs, rm, i);
+              if (trace) {
+                trace(ctx, "COLUMN: " + rm.getColumnName(i) + ", " + rm.getColumnTypeName(i));
+                trace(ctx, "SET " + var.getName() + " = " + var.toString());
+              }            
+            } 
+            else if(trace) {
+              trace(ctx, "Variable not found: " + into.ident(i-1).getText());
+            }
+          }
+          exec.incRowCount();
+          exec.setSqlSuccess();
+        }
+        else {
+          exec.setSqlCode(100);
+          exec.signal(Signal.Type.NOTFOUND);
+        }
+      }
+      // Print all results for standalone SELECT statement
+      else if (ctx.parent instanceof HplsqlParser.StmtContext) {
+        int cols = rm.getColumnCount();
+        if (trace) {
+          trace(ctx, "Standalone SELECT executed: " + cols + " columns in the result set");
+        }
+        while (rs.next()) {
+          for (int i = 1; i <= cols; i++) {
+            if (i > 1) {
+              System.out.print("\t");
+            }
+            System.out.print(rs.getString(i));
+          }
+          System.out.println("");
+          exec.incRowCount();
+        }
+      }
+      // Scalar subquery
+      else {
+        trace(ctx, "Scalar subquery executed, first row and first column fetched only");
+        if(rs.next()) {
+          exec.stackPush(new Var().setValue(rs, rm, 1));
+          exec.setSqlSuccess();          
+        }
+        else {
+          evalNull();
+          exec.setSqlCode(100);
+        }
+      }
+    }
+    catch (SQLException e) {
+      exec.signal(query);
+      exec.closeQuery(query, exec.conf.defaultConnection);
+      return 1;
+    }
+    exec.closeQuery(query, exec.conf.defaultConnection);
+    return 0; 
+  }  
+
+  /**
+   * Common table expression (WITH clause)
+   */
+  public Integer cte(HplsqlParser.Cte_select_stmtContext ctx) {
+    int cnt = ctx.cte_select_stmt_item().size();
+    StringBuilder sql = new StringBuilder();
+    sql.append("WITH ");
+    for (int i = 0; i < cnt; i++) {
+      HplsqlParser.Cte_select_stmt_itemContext c = ctx.cte_select_stmt_item(i);      
+      sql.append(c.ident().getText());
+      if (c.cte_select_cols() != null) {
+        sql.append(" " + exec.getFormattedText(c.cte_select_cols()));
+      }
+      sql.append(" AS (");
+      sql.append(evalPop(ctx.cte_select_stmt_item(i).fullselect_stmt()).toString());
+      sql.append(")");
+      if (i + 1 != cnt) {
+        sql.append(",\n");
+      }
+    }
+    exec.stackPush(sql);
+    return 0;
+  }
+  
+  /**
+   * Part of SELECT
+   */
+  public Integer fullselect(HplsqlParser.Fullselect_stmtContext ctx) { 
+    int cnt = ctx.fullselect_stmt_item().size();
+    StringBuilder sql = new StringBuilder();
+    for (int i = 0; i < cnt; i++) {
+      String part = evalPop(ctx.fullselect_stmt_item(i)).toString(); 
+      sql.append(part);
+      if (i + 1 != cnt) {
+        sql.append("\n" + getText(ctx.fullselect_set_clause(i)) + "\n");
+      }
+    }
+    exec.stackPush(sql);
+    return 0; 
+  }
+  
+  public Integer subselect(HplsqlParser.Subselect_stmtContext ctx) {
+    StringBuilder sql = new StringBuilder();
+    if (ctx.T_SELECT() != null) {
+      sql.append(ctx.T_SELECT().getText());
+    }
+    sql.append(" " + evalPop(ctx.select_list()));
+    if (ctx.from_clause() != null) {
+      sql.append(" " + evalPop(ctx.from_clause()));
+    } else {
+      sql.append(" FROM " + conf.dualTable);
+    }
+    if (ctx.where_clause() != null) {
+      sql.append(" " + evalPop(ctx.where_clause()));
+    }
+    if (ctx.group_by_clause() != null) {
+      sql.append(" " + getText(ctx.group_by_clause()));
+    }
+    if (ctx.having_clause() != null) {
+      sql.append(" " + getText(ctx.having_clause()));
+    }
+    if (ctx.order_by_clause() != null) {
+      sql.append(" " + getText(ctx.order_by_clause()));
+    }
+    if (ctx.select_options() != null) {
+      sql.append(" " + evalPop(ctx.select_options()));
+    }
+    if (ctx.select_list().select_list_limit() != null) {
+      sql.append(" LIMIT " + evalPop(ctx.select_list().select_list_limit().expr()));
+    }
+    exec.stackPush(sql);
+    return 0; 
+  }
+  
+  /**
+   * SELECT list 
+   */
+  public Integer selectList(HplsqlParser.Select_listContext ctx) { 
+    StringBuilder sql = new StringBuilder();
+    if (ctx.select_list_set() != null) {
+      sql.append(exec.getText(ctx.select_list_set())).append(" ");
+    }
+    int cnt = ctx.select_list_item().size();
+    for (int i = 0; i < cnt; i++) {
+      if (ctx.select_list_item(i).select_list_asterisk() == null) {
+        sql.append(evalPop(ctx.select_list_item(i)));
+        if (ctx.select_list_item(i).select_list_alias() != null) {
+          sql.append(" " + exec.getText(ctx.select_list_item(i).select_list_alias()));
+        }
+      }
+      else {
+        sql.append(exec.getText(ctx.select_list_item(i).select_list_asterisk()));
+      }
+      if (i + 1 < cnt) {
+        sql.append(", ");
+      }
+    }
+    exec.stackPush(sql);
+    return 0;
+  }
+
+  /**
+   * FROM clause
+   */
+  public Integer from(HplsqlParser.From_clauseContext ctx) { 
+    StringBuilder sql = new StringBuilder();
+    sql.append(ctx.T_FROM().getText()).append(" ");
+    sql.append(evalPop(ctx.from_table_clause()));
+    int cnt = ctx.from_join_clause().size();
+    for (int i = 0; i < cnt; i++) {
+      sql.append(evalPop(ctx.from_join_clause(i)));
+    }
+    exec.stackPush(sql);
+    return 0;
+  }
+  
+  /**
+   * Single table name in FROM
+   */
+  public Integer fromTable(HplsqlParser.From_table_name_clauseContext ctx) {     
+    StringBuilder sql = new StringBuilder();
+    sql.append(evalPop(ctx.table_name()));
+    if (ctx.from_alias_clause() != null) {
+      sql.append(" ").append(exec.getText(ctx.from_alias_clause()));
+    }
+    exec.stackPush(sql);
+    return 0; 
+  }
+ 
+  /**
+   * JOIN clause in FROM
+   */
+  public Integer fromJoin(HplsqlParser.From_join_clauseContext ctx) {
+    StringBuilder sql = new StringBuilder();
+    if (ctx.T_COMMA() != null) {
+      sql.append(", ");
+      sql.append(evalPop(ctx.from_table_clause()));
+    }
+    else if (ctx.from_join_type_clause() != null) {
+      sql.append(" ");
+      sql.append(exec.getText(ctx.from_join_type_clause()));
+      sql.append(" ");
+      sql.append(evalPop(ctx.from_table_clause()));
+      sql.append(" ");
+      sql.append(exec.getText(ctx, ctx.T_ON().getSymbol(), ctx.bool_expr().getStop()));
+    }
+    exec.stackPush(sql);
+    return 0; 
+  }
+
+  /**
+   * FROM TABLE (VALUES ...) clause
+   */
+  public Integer fromTableValues(HplsqlParser.From_table_values_clauseContext ctx) {
+    StringBuilder sql = new StringBuilder();
+    int rows = ctx.from_table_values_row().size();
+    sql.append("(");
+    for (int i = 0; i < rows; i++) {
+      int cols = ctx.from_table_values_row(i).expr().size();
+      int cols_as = ctx.from_alias_clause().L_ID().size();
+      sql.append("SELECT ");
+      for (int j = 0; j < cols; j++) {
+        sql.append(evalPop(ctx.from_table_values_row(i).expr(j)));
+        if (j < cols_as) {
+          sql.append(" AS ");
+          sql.append(ctx.from_alias_clause().L_ID(j));  
+        }
+        if (j + 1 < cols) {
+          sql.append(", ");
+        }
+      }
+      sql.append(" FROM " + conf.dualTable);
+      if (i + 1 < rows) {
+        sql.append("\nUNION ALL\n");
+      }
+    }
+    sql.append(") ");
+    if (ctx.from_alias_clause() != null) {
+      sql.append(ctx.from_alias_clause().ident().getText());
+    }
+    exec.stackPush(sql);
+    return 0; 
+  }
+  
+  /**
+   * WHERE clause
+   */
+  public Integer where(HplsqlParser.Where_clauseContext ctx) { 
+    StringBuilder sql = new StringBuilder();
+    sql.append(ctx.T_WHERE().getText());
+    sql.append(" " + evalPop(ctx.bool_expr()));
+    exec.stackPush(sql);
+    return 0;
+  }
+  
+  /**
+   * Get INTO clause
+   */
+  HplsqlParser.Into_clauseContext getIntoClause(HplsqlParser.Select_stmtContext ctx) {
+    if (ctx.fullselect_stmt().fullselect_stmt_item(0).subselect_stmt() != null) {
+      return ctx.fullselect_stmt().fullselect_stmt_item(0).subselect_stmt().into_clause();
+    }
+    return null;
+  }
+  
+  /** 
+   * SELECT statement options - LIMIT n, WITH UR i.e
+   */
+  public Integer option(HplsqlParser.Select_options_itemContext ctx) { 
+    if (ctx.T_LIMIT() != null) {
+      exec.stackPush("LIMIT " + evalPop(ctx.expr()));
+    }
+    return 0; 
+  }
+  
+  /**
+   * Evaluate the expression to NULL
+   */
+  void evalNull() {
+    exec.stackPush(Var.Null); 
+  }
+
+  /**
+   * Evaluate the expression and pop value from the stack
+   */
+  Var evalPop(ParserRuleContext ctx) {
+    exec.visit(ctx);
+    if (!exec.stack.isEmpty()) { 
+      return exec.stackPop();
+    }
+    return Var.Empty;
+  }
+
+  /**
+   * Get node text including spaces
+   */
+  String getText(ParserRuleContext ctx) {
+    return ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
+  }
+  
+  /**
+   * Execute rules
+   */
+  Integer visit(ParserRuleContext ctx) {
+    return exec.visit(ctx);  
+  } 
+  
+  /**
+   * Execute children rules
+   */
+  Integer visitChildren(ParserRuleContext ctx) {
+    return exec.visitChildren(ctx);  
+  }  
+  
+  /**
+   * Trace information
+   */
+  void trace(ParserRuleContext ctx, String message) {
+    exec.trace(ctx, message);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java b/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java
new file mode 100644
index 0000000..6330ae3
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java
@@ -0,0 +1,48 @@
+/**
+ * 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;
+
+/**
+ * Signals and exceptions
+ */
+public class Signal {
+  public enum Type { LEAVE_LOOP, LEAVE_ROUTINE, SQLEXCEPTION, NOTFOUND, USERDEFINED };
+  Type type;
+  String value = "";
+  Exception exception = null;
+  
+  Signal(Type type, String value) {
+    this.type = type;
+    this.value = value;
+    this.exception = null;
+  }
+  
+  Signal(Type type, String value, Exception exception) {
+    this.type = type;
+    this.value = value;
+    this.exception = exception;
+  }
+  
+  /**
+   * Get the signal value (message text)
+   */
+  public String getValue() {
+    return value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
new file mode 100644
index 0000000..acc4907
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
@@ -0,0 +1,1021 @@
+/**
+ * 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.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.Stack;
+import java.util.UUID;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.hive.hplsql.Var.Type;
+import org.apache.hive.hplsql.HplsqlParser.Create_table_columns_itemContext;
+import org.apache.hive.hplsql.HplsqlParser.Create_table_columnsContext;
+
+/**
+ * HPL/SQL statements execution
+ */
+public class Stmt {
+
+  Exec exec = null;
+  Stack<Var> stack = null;
+  Conf conf;
+  
+  boolean trace = false; 
+  
+  Stmt(Exec e) {
+    exec = e;  
+    stack = exec.getStack();
+    conf = exec.getConf();
+    trace = exec.getTrace();
+  }
+  
+  /**
+   * DECLARE cursor statement
+   */
+  public Integer declareCursor(HplsqlParser.Declare_cursor_itemContext ctx) { 
+    String name = ctx.ident().getText();
+    if (trace) {
+      trace(ctx, "DECLARE CURSOR " + name);
+    }
+    Query query = new Query();
+    if (ctx.expr() != null) {
+      query.setExprCtx(ctx.expr());
+    }
+    else if (ctx.select_stmt() != null) {
+      query.setSelectCtx(ctx.select_stmt());
+    }
+    exec.addVariable(new Var(name, Type.CURSOR, query));
+    return 0; 
+  }
+  
+  /**
+   * CREATE TABLE statement
+   */
+  public Integer createTable(HplsqlParser.Create_table_stmtContext ctx) { 
+    trace(ctx, "CREATE TABLE");
+    StringBuilder sql = new StringBuilder();
+    sql.append(exec.getText(ctx, ctx.T_CREATE().getSymbol(), ctx.T_OPEN_P().getSymbol()));
+    int cnt = ctx.create_table_columns().create_table_columns_item().size();
+    int cols = 0;
+    for (int i = 0; i < cnt; i++) {
+      Create_table_columns_itemContext col = ctx.create_table_columns().create_table_columns_item(i);
+      if (col.create_table_column_cons() != null) {
+        continue;
+      }
+      if (cols > 0) {
+        sql.append(",\n");
+      }
+      sql.append(col.ident().getText());
+      sql.append(" ");
+      sql.append(exec.evalPop(col.dtype(), col.dtype_len()));
+      cols++;
+    }
+    sql.append("\n)");
+    if (ctx.create_table_options() != null) {
+      sql.append(" " + evalPop(ctx.create_table_options()).toString());
+    }
+    trace(ctx, sql.toString());
+    Query query = exec.executeSql(ctx, sql.toString(), exec.conf.defaultConnection);
+    if (query.error()) {
+      exec.signal(query);
+      return 1;
+    }
+    exec.setSqlSuccess();
+    exec.closeQuery(query, exec.conf.defaultConnection);
+    return 0; 
+  }  
+
+  /**
+   * CREATE TABLE options for Hive
+   */
+  public Integer createTableHiveOptions(HplsqlParser.Create_table_options_hive_itemContext ctx) {
+    if (ctx.create_table_hive_row_format() != null) {
+      createTableHiveRowFormat(ctx.create_table_hive_row_format());
+    }
+    return 0; 
+  }
+  
+  public Integer createTableHiveRowFormat(HplsqlParser.Create_table_hive_row_formatContext ctx) {
+    StringBuilder sql = new StringBuilder();
+    sql.append("ROW FORMAT DELIMITED");
+    int cnt = ctx.create_table_hive_row_format_fields().size();
+    for (int i = 0; i < cnt; i++) {
+      HplsqlParser.Create_table_hive_row_format_fieldsContext c = ctx.create_table_hive_row_format_fields(i);
+      if (c.T_FIELDS() != null) {
+        sql.append(" FIELDS TERMINATED BY " + evalPop(c.expr(0)).toSqlString());
+      }
+      else if (c.T_LINES() != null) {
+        sql.append(" LINES TERMINATED BY " + evalPop(c.expr(0)).toSqlString());
+      } 
+    }
+    evalString(sql);
+    return 0; 
+  }
+    
+  /**
+   * DECLARE TEMPORARY TABLE statement 
+   */
+  public Integer declareTemporaryTable(HplsqlParser.Declare_temporary_table_itemContext ctx) { 
+    String name = ctx.ident().getText();
+    if (trace) {
+      trace(ctx, "DECLARE TEMPORARY TABLE " + name);
+    }
+    return createTemporaryTable(ctx, ctx.create_table_columns(), name);
+  }
+  
+  /**
+   * CREATE LOCAL TEMPORARY | VOLATILE TABLE statement 
+   */
+  public Integer createLocalTemporaryTable(HplsqlParser.Create_local_temp_table_stmtContext ctx) { 
+    String name = ctx.ident().getText();
+    if (trace) {
+      trace(ctx, "CREATE LOCAL TEMPORARY TABLE " + name);
+    }
+    return createTemporaryTable(ctx, ctx.create_table_columns(), name);
+   }
+  
+  /**
+   * Create a temporary table statement 
+   */
+  public Integer createTemporaryTable(ParserRuleContext ctx, Create_table_columnsContext colCtx, String name) { 
+    String managedName = null;
+    String sql = null;
+    String columns = exec.getFormattedText(colCtx);
+    if (conf.tempTables == Conf.TempTables.NATIVE) {
+      sql = "CREATE TEMPORARY TABLE " + name + "\n(" + columns + "\n)";
+    } else if (conf.tempTables == Conf.TempTables.MANAGED) {
+      managedName = name + "_" + UUID.randomUUID().toString().replace("-","");
+      if (!conf.tempTablesSchema.isEmpty()) {
+        managedName = conf.tempTablesSchema + "." + managedName;
+      }      
+      sql = "CREATE TABLE " + managedName + "\n(" + columns + "\n)";
+      if (!conf.tempTablesLocation.isEmpty()) {
+        sql += "\nLOCATION '" + conf.tempTablesLocation + "/" + managedName + "'";
+      }
+      if (trace) {
+        trace(ctx, "Managed table name: " + managedName);
+      }
+    }  
+    if (sql != null) {
+      Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+      if (query.error()) {
+        exec.signal(query);
+        return 1;
+      }
+      if (managedName != null) {
+        exec.addManagedTable(name, managedName);
+      }
+      exec.setSqlSuccess();
+      exec.closeQuery(query, exec.conf.defaultConnection);
+    }    
+    return 0; 
+  }
+  
+  /**
+   * DROP statement
+   */
+  public Integer drop(HplsqlParser.Drop_stmtContext ctx) { 
+    trace(ctx, "DROP");
+    String sql = null;    
+    if (ctx.T_TABLE() != null) {
+      sql = "DROP TABLE ";
+      if (ctx.T_EXISTS() != null) {
+        sql += "IF NOT EXISTS ";
+      }
+      sql += evalPop(ctx.table_name()).toString();
+    }
+    if (sql != null) {
+      trace(ctx, sql);
+      Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+      if (query.error()) {
+        exec.signal(query);
+        return 1;
+      }
+      exec.setSqlSuccess();
+      exec.closeQuery(query, exec.conf.defaultConnection);
+    }
+    return 0; 
+  }
+  
+  /**
+   * OPEN cursor statement
+   */
+  public Integer open(HplsqlParser.Open_stmtContext ctx) { 
+    trace(ctx, "OPEN");
+    Query query = null;
+    Var var = null;
+    String cursor = ctx.L_ID().toString();   
+    String sql = null;
+    // Dynamic SQL
+    if (ctx.T_FOR() != null) {
+      sql = evalPop(ctx.expr()).toString();
+      if (trace) {
+        trace(ctx, cursor + ": " + sql);
+      }
+      query = new Query(sql);
+      var = new Var(cursor, Type.CURSOR, query);
+      exec.addVariable(var);
+    }
+    // Declared cursor
+    else {
+      var = exec.findVariable(cursor);      
+      if (var != null && var.type == Type.CURSOR) {
+        query = (Query)var.value;
+        if (query.sqlExpr != null) {
+          sql = evalPop(query.sqlExpr).toString();
+          query.setSql(sql);
+        }
+        else if (query.sqlSelect != null) {
+          sql = evalPop(query.sqlSelect).toString();
+          query.setSql(sql);
+        }
+        if (trace) {
+          trace(ctx, cursor + ": " + sql);
+        } 
+      }
+    }
+    // Open cursor now
+    if (query != null) {
+      exec.executeQuery(ctx, query, exec.conf.defaultConnection);
+      if (query.error()) {
+        exec.signal(query);
+        return 1;
+      }
+      else if (!exec.getOffline()) {
+        exec.setSqlCode(0);
+      }
+    }
+    else {
+      trace(ctx, "Cursor not found: " + cursor);
+      exec.setSqlCode(-1);
+      exec.signal(Signal.Type.SQLEXCEPTION);
+      return 1;
+    }
+    return 0; 
+  }
+  
+  /**
+   * FETCH cursor statement
+   */
+  public Integer fetch(HplsqlParser.Fetch_stmtContext ctx) { 
+    trace(ctx, "FETCH");
+    String name = ctx.L_ID(0).toString();
+    Var cursor = exec.findVariable(name);
+    if (cursor == null || cursor.type != Type.CURSOR) {
+      trace(ctx, "Cursor not found: " + name);
+      exec.setSqlCode(-1);
+      exec.signal(Signal.Type.SQLEXCEPTION);
+      return 1;
+    }    
+    else if (exec.getOffline()) {
+      exec.setSqlCode(100);
+      exec.signal(Signal.Type.NOTFOUND);
+      return 0;
+    }
+    // Assign values from the row to local variables
+    try {
+      Query query = (Query)cursor.value;
+      ResultSet rs = query.getResultSet();
+      ResultSetMetaData rsm = null;
+      if(rs != null) {
+        rsm = rs.getMetaData();
+      }
+      if(rs != null && rsm != null) {
+        int cols = ctx.L_ID().size() - 1;
+        if(rs.next()) {
+          for(int i=1; i <= cols; i++) {
+            Var var = exec.findVariable(ctx.L_ID(i).getText());
+            if(var != null) {
+              var.setValue(rs, rsm, i);
+              if(trace) {
+                trace(ctx, "COLUMN: " + rsm.getColumnName(i) + ", " + rsm.getColumnTypeName(i));
+                trace(ctx, "SET " + var.getName() + " = " + var.toString());
+              }            
+            } 
+            else if(trace) {
+              trace(ctx, "Variable not found: " + ctx.L_ID(i).getText());
+            }
+          }
+          exec.incRowCount();
+          exec.setSqlSuccess();
+        }
+        else {
+          exec.setSqlCode(100);
+          exec.signal(Signal.Type.NOTFOUND);
+        }
+      }
+    } 
+    catch (SQLException e) {
+      exec.setSqlCode(e);
+      exec.signal(Signal.Type.SQLEXCEPTION, e.getMessage(), e);
+    } 
+    return 0; 
+  }  
+  
+  /**
+   * CLOSE cursor statement
+   */
+  public Integer close(HplsqlParser.Close_stmtContext ctx) { 
+    trace(ctx, "CLOSE");
+    String name = ctx.L_ID().toString();
+    Var var = exec.findVariable(name);
+    if(var != null && var.type == Type.CURSOR) {
+      exec.closeQuery((Query)var.value, exec.conf.defaultConnection);
+      exec.setSqlCode(0);
+    }
+    else if(trace) {
+      trace(ctx, "Cursor not found: " + name);
+    }
+    return 0; 
+  }
+  
+  /**
+   * INCLUDE statement
+   */
+  public Integer include(HplsqlParser.Include_stmtContext ctx) {
+    String file = ctx.file_name().getText();
+    trace(ctx, "INCLUDE " + file);
+    exec.includeFile(file);
+    return 0; 
+  }
+  
+  /**
+   * IF statement (PL/SQL syntax)
+   */
+  public Integer ifPlsql(HplsqlParser.If_plsql_stmtContext ctx) {
+    boolean trueExecuted = false;
+    trace(ctx, "IF");
+    if (evalPop(ctx.bool_expr()).isTrue()) {
+      trace(ctx, "IF TRUE executed");
+      visit(ctx.block());
+      trueExecuted = true;
+    }
+    else if (ctx.elseif_block() != null) {
+      int cnt = ctx.elseif_block().size();
+      for (int i = 0; i < cnt; i++) {
+        if (evalPop(ctx.elseif_block(i).bool_expr()).isTrue()) {
+          trace(ctx, "ELSE IF executed");
+          visit(ctx.elseif_block(i).block());
+          trueExecuted = true;
+          break;
+        }
+      }
+    }
+    if (!trueExecuted && ctx.else_block() != null) {
+      trace(ctx, "ELSE executed");
+      visit(ctx.else_block());
+    }
+    return 0; 
+  }
+  
+  /**
+   * IF statement (Transact-SQL syntax)
+   */
+  public Integer ifTsql(HplsqlParser.If_tsql_stmtContext ctx) {
+    trace(ctx, "IF");
+    visit(ctx.bool_expr());
+    if(exec.stackPop().isTrue()) {
+      trace(ctx, "IF TRUE executed");
+      visit(ctx.single_block_stmt(0));
+    }
+    else if(ctx.T_ELSE() != null) {
+      trace(ctx, "ELSE executed");
+      visit(ctx.single_block_stmt(1));
+    }
+    return 0; 
+  }
+  
+  /**
+   * Assignment from SELECT statement 
+   */
+  public Integer assignFromSelect(HplsqlParser.Assignment_stmt_select_itemContext ctx) { 
+    String sql = evalPop(ctx.select_stmt()).toString();
+    if (trace) {
+      trace(ctx, sql.toString());
+    }
+    String conn = exec.getStatementConnection();
+    Query query = exec.executeQuery(ctx, sql.toString(), conn);
+    if (query.error()) { 
+      exec.signal(query);
+      return 1;
+    }
+    exec.setSqlSuccess();
+    try {
+      ResultSet rs = query.getResultSet();
+      ResultSetMetaData rm = null;
+      if (rs != null) {
+        rm = rs.getMetaData();
+        int cnt = ctx.ident().size();
+        if (rs.next()) {
+          for (int i = 1; i <= cnt; i++) {
+            Var var = exec.findVariable(ctx.ident(i-1).getText());
+            if (var != null) {
+              var.setValue(rs, rm, i);
+              if (trace) {
+                trace(ctx, "COLUMN: " + rm.getColumnName(i) + ", " + rm.getColumnTypeName(i));
+                trace(ctx, "SET " + var.getName() + " = " + var.toString());
+              }             
+            } 
+            else if(trace) {
+              trace(ctx, "Variable not found: " + ctx.ident(i-1).getText());
+            }
+          }
+          exec.incRowCount();
+          exec.setSqlSuccess();
+        }
+        else {
+          exec.setSqlCode(100);
+          exec.signal(Signal.Type.NOTFOUND);
+        }
+      }
+    }
+    catch (SQLException e) {
+      exec.signal(query);
+      return 1;
+    }
+    finally {
+      exec.closeQuery(query, conn);
+    }
+    return 0; 
+  }
+  
+  /**
+   * SQL INSERT statement
+   */
+  public Integer insert(HplsqlParser.Insert_stmtContext ctx) {
+    exec.stmtConnList.clear();
+    if (ctx.select_stmt() != null) {
+      return insertSelect(ctx);
+    }
+    return insertValues(ctx); 
+  }
+  
+  /**
+   * SQL INSERT SELECT statement
+   */
+  public Integer insertSelect(HplsqlParser.Insert_stmtContext ctx) { 
+    trace(ctx, "INSERT SELECT");
+    String table = evalPop(ctx.table_name()).toString();
+    String select = evalPop(ctx.select_stmt()).toString();
+    String sql = "INSERT INTO TABLE " + table + " " + select;    
+    trace(ctx, sql);
+    Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+    if (query.error()) {
+      exec.signal(query);
+      return 1;
+    }
+    exec.setSqlSuccess();
+    exec.closeQuery(query, exec.conf.defaultConnection);
+    return 0; 
+  }
+  
+  /**
+   * SQL INSERT VALUES statement
+   */
+  public Integer insertValues(HplsqlParser.Insert_stmtContext ctx) { 
+    trace(ctx, "INSERT VALUES");
+    String table = evalPop(ctx.table_name()).toString();
+    String conn = exec.getObjectConnection(ctx.table_name().getText());
+    Conn.Type type = exec.getConnectionType(conn); 
+    StringBuilder sql = new StringBuilder();
+    if (type == Conn.Type.HIVE) {
+      sql.append("INSERT INTO TABLE " + table + " ");
+      if (conf.insertValues == Conf.InsertValues.NATIVE) {
+        sql.append("VALUES\n("); 
+      }
+    }
+    else {
+      sql.append("INSERT INTO " + table);
+      if (ctx.insert_stmt_cols() != null) {
+        sql.append(" " + exec.getFormattedText(ctx.insert_stmt_cols()));
+      }
+      sql.append(" VALUES\n("); 
+    }
+    int rows = ctx.insert_stmt_rows().insert_stmt_row().size();
+    for (int i = 0; i < rows; i++) {
+      HplsqlParser.Insert_stmt_rowContext row =ctx.insert_stmt_rows().insert_stmt_row(i);
+      int cols = row.expr().size();
+      for (int j = 0; j < cols; j++) {         
+        String value = evalPop(row.expr(j)).toSqlString();
+        if (j == 0 && type == Conn.Type.HIVE && conf.insertValues == Conf.InsertValues.SELECT ) {
+          sql.append("SELECT ");
+        }
+        sql.append(value);
+        if (j + 1 != cols) {
+          sql.append(", ");
+        }       
+      }
+      if (type != Conn.Type.HIVE || conf.insertValues == Conf.InsertValues.NATIVE) {
+        if (i + 1 == rows) {
+          sql.append(")");
+        } else {
+          sql.append("),\n(");
+        } 
+      }
+      else if (type == Conn.Type.HIVE && conf.insertValues == Conf.InsertValues.SELECT) {
+        sql.append(" FROM " + conf.dualTable); 
+        if (i + 1 < rows) {
+          sql.append("\nUNION ALL\n");
+        }
+      }      
+    }    
+    if (trace) {
+      trace(ctx, sql.toString());
+    }
+    Query query = exec.executeSql(ctx, sql.toString(), conn);
+    if (query.error()) {
+      exec.signal(query);
+      return 1;
+    }
+    exec.setSqlSuccess();
+    exec.closeQuery(query, exec.conf.defaultConnection);
+    return 0; 
+  }
+  
+  /**
+   * GET DIAGNOSTICS EXCEPTION statement
+   */
+  public Integer getDiagnosticsException(HplsqlParser.Get_diag_stmt_exception_itemContext ctx) {
+    trace(ctx, "GET DIAGNOSTICS EXCEPTION");
+    Signal signal = exec.signalPeek();
+    if (signal == null || (signal != null && signal.type != Signal.Type.SQLEXCEPTION)) {
+      signal = exec.currentSignal;
+    }
+    if (signal != null) {
+      exec.setVariable(ctx.ident().getText(), signal.getValue());
+    }
+    return 0; 
+  }
+  
+  /**
+   * GET DIAGNOSTICS ROW_COUNT statement
+   */
+  public Integer getDiagnosticsRowCount(HplsqlParser.Get_diag_stmt_rowcount_itemContext ctx) {
+    trace(ctx, "GET DIAGNOSTICS ROW_COUNT");
+    exec.setVariable(ctx.ident().getText(), exec.getRowCount());
+    return 0;  
+  }
+  
+  /**
+   * USE statement
+   */
+  public Integer use(HplsqlParser.Use_stmtContext ctx) {
+    if(trace) {
+      trace(ctx, "USE");
+    }
+    String sql = ctx.T_USE().toString() + " " + evalPop(ctx.expr()).toString();
+    if(trace) {
+      trace(ctx, "Query: " + sql);
+    }    
+    Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+    if(query.error()) {
+      exec.signal(query);
+      return 1;
+    }
+    exec.setSqlCode(0);
+    exec.closeQuery(query, exec.conf.defaultConnection);
+    return 0; 
+  }
+  
+  /** 
+   * VALUES statement
+   */
+  public Integer values(HplsqlParser.Values_into_stmtContext ctx) { 
+    trace(ctx, "VALUES statement");    
+    int cnt = ctx.ident().size();        // Number of variables and assignment expressions
+    int ecnt = ctx.expr().size();    
+    for (int i = 0; i < cnt; i++) {
+      String name = ctx.ident(i).getText();      
+      if (i < ecnt) {
+        visit(ctx.expr(i));
+        Var var = exec.setVariable(name);        
+        if (trace) {
+          trace(ctx, "SET " + name + " = " + var.toString());      
+        } 
+      }      
+    }    
+    return 0; 
+  } 
+  
+  /**
+   * WHILE statement
+   */
+  public Integer while_(HplsqlParser.While_stmtContext ctx) {
+    trace(ctx, "WHILE - ENTERED");
+    String label = exec.labelPop();
+    while (true) {
+      if (evalPop(ctx.bool_expr()).isTrue()) {
+        exec.enterScope(Scope.Type.LOOP);
+        visit(ctx.block());
+        exec.leaveScope();        
+        if (canContinue(label)) {
+          continue;
+        }
+      }
+      break;
+    }    
+    trace(ctx, "WHILE - LEFT");
+    return 0; 
+  }
+  
+  /**
+   * FOR cursor statement
+   */
+  public Integer forCursor(HplsqlParser.For_cursor_stmtContext ctx) { 
+    trace(ctx, "FOR CURSOR - ENTERED");
+    exec.enterScope(Scope.Type.LOOP);
+    String cursor = ctx.L_ID().getText();
+    String sql = evalPop(ctx.select_stmt()).toString();   
+    trace(ctx, sql);
+    Query query = exec.executeQuery(ctx, sql, exec.conf.defaultConnection);
+    if (query.error()) { 
+      exec.signal(query);
+      return 1;
+    }
+    trace(ctx, "SELECT completed successfully");
+    exec.setSqlSuccess();
+    try {
+      ResultSet rs = query.getResultSet();
+      if (rs != null) {
+        ResultSetMetaData rm = rs.getMetaData();
+        int cols = rm.getColumnCount();
+        Var[] vars = new Var[cols];
+        for (int i = 0; i < cols; i++) {
+          vars[i] = new Var();
+          vars[i].setName(cursor + "." + rm.getColumnName(i + 1));
+          vars[i].setType(rm.getColumnType(i + 1));          
+          exec.addVariable(vars[i]);
+          if (trace) {
+            trace(ctx, "Column: " + vars[i].getName() + " " + rm.getColumnTypeName(i + 1));
+          }
+        }                
+        while (rs.next()) {
+          for (int i = 0; i < cols; i++) {
+            vars[i].setValue(rs, rm, i + 1);
+          }
+          visit(ctx.block());
+          exec.incRowCount();
+        }
+      }
+    }
+    catch (SQLException e) {
+      exec.signal(e);
+      exec.closeQuery(query, exec.conf.defaultConnection);
+      return 1;
+    }
+    exec.setSqlSuccess();
+    exec.closeQuery(query, exec.conf.defaultConnection);
+    exec.leaveScope();
+    trace(ctx, "FOR CURSOR - LEFT");
+    return 0; 
+  }
+  
+  /**
+   * FOR (integer range) statement
+   */
+  public Integer forRange(HplsqlParser.For_range_stmtContext ctx) { 
+    trace(ctx, "FOR RANGE - ENTERED");
+    int start = evalPop(ctx.expr(0)).intValue();
+    int end = evalPop(ctx.expr(1)).intValue();
+    int step = evalPop(ctx.expr(2), 1L).intValue();
+    exec.enterScope(Scope.Type.LOOP);
+    Var index = new Var(ctx.L_ID().getText(), new Long(start));       
+    exec.addVariable(index);     
+    if (ctx.T_REVERSE() == null) {
+      for (int i = start; i <= end; i += step) {
+        visit(ctx.block());
+        index.increment(new Long(step));
+      } 
+    } else {
+      for (int i = start; i >= end; i -= step) {
+        visit(ctx.block());
+        index.decrement(new Long(step));
+      }    
+    }
+    exec.leaveScope();
+    trace(ctx, "FOR RANGE - LEFT");
+    return 0; 
+  }  
+  
+  /**
+   * EXEC, EXECUTE and EXECUTE IMMEDIATE statement to execute dynamic SQL
+   */
+  public Integer exec(HplsqlParser.Exec_stmtContext ctx) { 
+    if(trace) {
+      trace(ctx, "EXECUTE");
+    }
+    Var vsql = evalPop(ctx.expr());
+    String sql = vsql.toString();
+    if(trace) {
+      trace(ctx, "Query: " + sql);
+    }
+    Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+    if(query.error()) {
+      exec.signal(query);
+      return 1;
+    }
+    ResultSet rs = query.getResultSet();
+    if(rs != null) {
+      try {
+        ResultSetMetaData rsm = rs.getMetaData();
+        // Assign to variables
+        if(ctx.T_INTO() != null) {
+          int cols = ctx.L_ID().size();
+          if(rs.next()) {
+            for(int i=0; i < cols; i++) {
+              Var var = exec.findVariable(ctx.L_ID(i).getText());
+              if(var != null) {
+                var.setValue(rs, rsm, i+1);
+                if(trace) {
+                  trace(ctx, "COLUMN: " + rsm.getColumnName(i+1) + ", " + rsm.getColumnTypeName(i+1));
+                  trace(ctx, "SET " + var.getName() + " = " + var.toString());
+                }
+              } 
+              else if(trace) {
+                trace(ctx, "Variable not found: " + ctx.L_ID(i).getText());
+              }
+            }
+            exec.setSqlCode(0);
+          }
+        }
+        // Print the results
+        else {
+          int cols = rsm.getColumnCount();
+          while(rs.next()) {
+            for(int i = 1; i <= cols; i++) {
+              if(i > 1) {
+                System.out.print("\t");
+              }
+              System.out.print(rs.getString(i));
+            }
+            System.out.println("");
+          }
+        }
+      } 
+      catch(SQLException e) {
+        exec.setSqlCode(e);
+      } 
+    }   
+    exec.closeQuery(query, exec.conf.defaultConnection);
+    return 0; 
+  }
+      
+  /**
+   * EXIT statement (leave the specified loop with a condition)
+   */
+  public Integer exit(HplsqlParser.Exit_stmtContext ctx) { 
+    trace(ctx, "EXIT");
+    String label = "";
+    if (ctx.L_ID() != null) {
+      label = ctx.L_ID().toString();
+    }
+    if (ctx.T_WHEN() != null) {
+      if (evalPop(ctx.bool_expr()).isTrue()) {
+        leaveLoop(label);
+      }
+    } else {
+      leaveLoop(label);
+    }
+    return 0;
+  }
+  
+  /**
+   * BREAK statement (leave the innermost loop unconditionally)
+   */
+  public Integer break_(HplsqlParser.Break_stmtContext ctx) { 
+    trace(ctx, "BREAK");
+    leaveLoop("");
+    return 0;
+  }
+  
+  /**
+   * LEAVE statement (leave the specified loop unconditionally)
+   */
+  public Integer leave(HplsqlParser.Leave_stmtContext ctx) { 
+    trace(ctx, "LEAVE");
+    String label = "";
+    if (ctx.L_ID() != null) {
+      label = ctx.L_ID().toString();
+    }
+    leaveLoop(label);    
+    return 0;
+  }
+  
+  /**
+   * Leave the specified or innermost loop unconditionally
+   */
+  public void leaveLoop(String value) { 
+    exec.signal(Signal.Type.LEAVE_LOOP, value);
+  }
+  
+  /**
+   * UPDATE statement
+   */
+  public Integer update(HplsqlParser.Update_stmtContext ctx) {
+    trace(ctx, "UPDATE");
+    String sql = exec.getFormattedText(ctx);
+    trace(ctx, sql);
+    Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+    if (query.error()) {
+      exec.signal(query);
+      return 1;
+    }
+    exec.setSqlSuccess();
+    exec.closeQuery(query, exec.conf.defaultConnection);
+    return 0;
+  }
+  
+  /**
+   * DELETE statement
+   */
+  public Integer delete(HplsqlParser.Delete_stmtContext ctx) {
+    trace(ctx, "DELETE");
+    String table = evalPop(ctx.table_name()).toString();
+    StringBuilder sql = new StringBuilder();
+    sql.append("DELETE FROM ");
+    sql.append(table);
+    if (ctx.where_clause() != null) {
+      boolean oldBuildSql = exec.buildSql; 
+      exec.buildSql = true;
+      sql.append(" " + evalPop(ctx.where_clause()).toString());
+      exec.buildSql = oldBuildSql;
+    }
+    trace(ctx, sql.toString());
+    Query query = exec.executeSql(ctx, sql.toString(), exec.conf.defaultConnection);
+    if (query.error()) {
+      exec.signal(query);
+      return 1;
+    }
+    exec.setSqlSuccess();
+    exec.closeQuery(query, exec.conf.defaultConnection);
+    return 0;
+  }
+  
+  /**
+   * MERGE statement
+   */
+  public Integer merge(HplsqlParser.Merge_stmtContext ctx) {
+    trace(ctx, "MERGE");
+    String sql = exec.getFormattedText(ctx);
+    trace(ctx, sql);
+    Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+    if (query.error()) {
+      exec.signal(query);
+      return 1;
+    }
+    exec.setSqlSuccess();
+    exec.closeQuery(query, exec.conf.defaultConnection);
+    return 0;
+  }
+  
+  /**
+   * PRINT Statement 
+   */
+  public Integer print(HplsqlParser.Print_stmtContext ctx) { 
+    trace(ctx, "PRINT");
+    if (ctx.expr() != null) {
+      visit(ctx.expr());
+      System.out.println(stack.pop().toString());
+    }
+	  return 0; 
+  }
+  
+  /**
+   * SIGNAL statement
+   */
+  public Integer signal(HplsqlParser.Signal_stmtContext ctx) {
+    trace(ctx, "SIGNAL");
+    Signal signal = new Signal(Signal.Type.USERDEFINED, ctx.ident().getText());
+    exec.signal(signal);
+    return 0; 
+  }  
+  
+  /**
+   * RESIGNAL statement
+   */
+  public Integer resignal(HplsqlParser.Resignal_stmtContext ctx) { 
+    trace(ctx, "RESIGNAL");
+    if (ctx.T_SQLSTATE() != null) {
+      String sqlstate = evalPop(ctx.expr(0)).toString();
+      String text = "";
+      if (ctx.T_MESSAGE_TEXT() != null) {
+        text = evalPop(ctx.expr(1)).toString();
+      }
+      SQLException exception = new SQLException(text, sqlstate, -1);
+      Signal signal = new Signal(Signal.Type.SQLEXCEPTION, text, exception);
+      exec.setSqlCode(exception);
+      exec.resignal(signal);
+    }
+    else {
+      exec.resignal();
+    }
+    return 0; 
+  }
+  
+  /**
+   * RETURN statement
+   */
+  public Integer return_(HplsqlParser.Return_stmtContext ctx) {
+    trace(ctx, "RETURN");
+    if (ctx.expr() != null) {
+      eval(ctx.expr());
+    }
+    exec.signal(Signal.Type.LEAVE_ROUTINE);    
+    return 0; 
+  }  
+  
+  /**
+   * Check if an exception is raised or EXIT executed, and we should leave the block
+   */
+  boolean canContinue(String label) {
+    Signal signal = exec.signalPeek();
+    if (signal != null && signal.type == Signal.Type.SQLEXCEPTION) {
+      return false;
+    }
+    signal = exec.signalPeek();
+    if (signal != null && signal.type == Signal.Type.LEAVE_LOOP) {
+      if (signal.value == null || signal.value.isEmpty() ||
+          (label != null && label.equalsIgnoreCase(signal.value))) {
+        exec.signalPop();
+      }
+      return false;
+    }
+    return true;     
+  }
+  
+  /**
+   * Evaluate the expression and push the value to the stack
+   */
+  void eval(ParserRuleContext ctx) {
+    exec.visit(ctx);
+  }
+  
+  /**
+   * Evaluate the expression to specified String value
+   */
+  void evalString(String string) {
+    exec.stackPush(new Var(string)); 
+  }
+  
+  void evalString(StringBuilder string) {
+    evalString(string.toString()); 
+  }
+  
+  /**
+   * Evaluate the expression and pop value from the stack
+   */
+  Var evalPop(ParserRuleContext ctx) {
+    visit(ctx);
+    if (!exec.stack.isEmpty()) { 
+      return exec.stackPop();
+    }
+    return Var.Empty;
+  }
+  
+  Var evalPop(ParserRuleContext ctx, long def) {
+    if (ctx != null) {
+      exec.visit(ctx);
+      return exec.stackPop();
+    }
+    return new Var(def);
+  }
+  
+  /**
+   * Execute rules
+   */
+  Integer visit(ParserRuleContext ctx) {
+    return exec.visit(ctx);  
+  } 
+  
+  /**
+   * Execute children rules
+   */
+  Integer visitChildren(ParserRuleContext ctx) {
+	  return exec.visitChildren(ctx);  
+  }  
+  
+  /**
+   * Trace information
+   */
+  void trace(ParserRuleContext ctx, String message) {
+	  exec.trace(ctx, message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/StreamGobbler.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/StreamGobbler.java b/hplsql/src/main/java/org/apache/hive/hplsql/StreamGobbler.java
new file mode 100644
index 0000000..d5a7cc4
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/StreamGobbler.java
@@ -0,0 +1,51 @@
+/**
+ * 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.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.BufferedReader;
+import java.io.IOException;
+
+/**
+ * Read a stream from an external process 
+ */
+public class StreamGobbler extends Thread {
+  InputStream is;
+  
+  StreamGobbler(InputStream is) {
+    this.is = is;
+  }
+  
+  public void run() {
+    try {
+      InputStreamReader isr = new InputStreamReader(is);
+      BufferedReader br = new BufferedReader(isr);
+      while(true) {
+        String line = br.readLine();
+        if(line == null) {
+          break;
+        }        
+        System.out.println(line);
+      }
+    } catch (IOException ioe) {
+      ioe.printStackTrace();  
+    }
+ }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Timer.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Timer.java b/hplsql/src/main/java/org/apache/hive/hplsql/Timer.java
new file mode 100644
index 0000000..9330eb4
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Timer.java
@@ -0,0 +1,59 @@
+/**
+ * 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;
+
+public class Timer {
+  long start = 0;
+  long stop = 0;
+  long elapsed = 0;
+  
+  /**
+   * Start the timer
+   */
+  public long start() {
+    start = System.currentTimeMillis();
+    return start;
+  }
+  
+  /**
+   * Get intermediate timer value
+   */
+  public long current() {
+    return System.currentTimeMillis();
+  }
+  
+  /**
+   * Stop the timer and return elapsed time
+   */
+  public long stop() {
+    stop = System.currentTimeMillis();
+    elapsed = stop - start;
+    return elapsed;
+  }
+  
+  /**
+   * Format the elapsed time
+   */
+  public String format() {
+    if (elapsed < 1000) {
+      return String.valueOf(elapsed) + " ms";
+    }
+    return String.format("%.2f", ((float)elapsed)/1000) + " sec";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Udf.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Udf.java b/hplsql/src/main/java/org/apache/hive/hplsql/Udf.java
new file mode 100644
index 0000000..9c29eeb
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Udf.java
@@ -0,0 +1,117 @@
+/**
+ * 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.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.UDFType;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+
+@Description(name = "hplsql", value = "_FUNC_('query' [, :1, :2, ...n]) - Execute HPL/SQL query", extended = "Example:\n" + " > SELECT _FUNC_('CURRENT_DATE') FROM src LIMIT 1;\n")
+@UDFType(deterministic = false)
+public class Udf extends GenericUDF {
+  
+  Exec exec;
+  StringObjectInspector queryOI;
+  ObjectInspector[] argumentsOI;
+  
+  /**
+   * Initialize UDF
+   */
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+    if (arguments.length == 0) {
+      throw new UDFArgumentLengthException("At least one argument must be specified");
+    }    
+    if (!(arguments[0] instanceof StringObjectInspector)) {
+      throw new UDFArgumentException("First argument must be a string");
+    }
+    queryOI = (StringObjectInspector)arguments[0];
+    argumentsOI = arguments;
+    return PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+  }
+  
+  /**
+   * Execute UDF
+   */
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    if (exec == null) {
+      exec = new Exec(); 
+      String query = queryOI.getPrimitiveJavaObject(arguments[0].get());
+      String[] args = { "-e", query, "-trace" };
+      try {
+        exec.setUdfRun(true);
+        exec.init(args);
+      } catch (Exception e) {
+        throw new HiveException(e.getMessage());
+      }
+    }
+    if (arguments.length > 1) {
+      setParameters(arguments);
+    }
+    Var result = exec.run();
+    if (result != null) {
+      return result.toString();
+    }
+    return null;
+  }
+  
+  /**
+   * Set parameters for the current call
+   */
+  void setParameters(DeferredObject[] arguments) throws HiveException {
+    for (int i = 1; i < arguments.length; i++) {
+      String name = ":" + i;      
+      if (argumentsOI[i] instanceof StringObjectInspector) {
+        String value = ((StringObjectInspector)argumentsOI[i]).getPrimitiveJavaObject(arguments[i].get());
+        if (value != null) {
+          exec.setVariable(name, value);
+        }        
+      }
+      else if (argumentsOI[i] instanceof IntObjectInspector) {
+        Integer value = (Integer)((IntObjectInspector)argumentsOI[i]).getPrimitiveJavaObject(arguments[i].get());
+        if (value != null) {
+          exec.setVariable(name, new Var(new Long(value)));
+        }        
+      }
+      else if (argumentsOI[i] instanceof LongObjectInspector) {
+        Long value = (Long)((LongObjectInspector)argumentsOI[i]).getPrimitiveJavaObject(arguments[i].get());
+        if (value != null) {
+          exec.setVariable(name, new Var(value));
+        }        
+      }
+      else {
+        exec.setVariableToNull(name);
+      }
+    }
+  }
+  
+  @Override
+  public String getDisplayString(String[] children) {
+    return "hplsql";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java b/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java
new file mode 100644
index 0000000..da0d878
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java
@@ -0,0 +1,289 @@
+/**
+ * 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.sql.Date;
+import java.sql.Timestamp;
+
+public class Utils {
+
+  /**
+   * Unquote string and remove escape characters inside the script 
+   */
+  public static String unquoteString(String s) {
+	  if(s == null) {
+		  return null;
+	  }
+	  
+	  int len = s.length();
+	  StringBuffer s2 = new StringBuffer(len);	  
+	  
+	  for (int i = 0; i < len; i++) {
+		char ch = s.charAt(i);
+		char ch2 = (i < len - 1) ? s.charAt(i+1) : 0;
+		  
+	    if((i == 0 || i == len -1) && (ch == '\'' || ch == '"'))
+	      continue;
+	    else
+	    // \' and '' escape sequences
+	    if((ch == '\\' && ch2 == '\'') || (ch == '\'' && ch2 == '\''))
+	      continue;
+	    
+	    s2.append(ch);	
+	  }
+	  
+	  return s2.toString();
+  }
+
+  /**
+   * Quote string and escape characters - ab'c -> 'ab''c'
+   */
+  public static String quoteString(String s) {
+    if(s == null) {
+      return null;
+    }    
+    int len = s.length();
+    StringBuilder s2 = new StringBuilder(len + 2).append('\'');
+    
+    for (int i = 0; i < len; i++) {
+      char ch = s.charAt(i);
+      s2.append(ch);
+      if(ch == '\'') {
+        s2.append(ch);
+      }      
+    }
+    s2.append('\'');
+    return s2.toString();
+  }
+  
+  /**
+   * Merge quoted strings: 'a' 'b' -> 'ab'; 'a''b' 'c' -> 'a''bc'
+   */
+  public static String mergeQuotedStrings(String s1, String s2) {
+	  if(s1 == null || s2 == null) {
+		  return null;
+	  }
+	  
+	  int len1 = s1.length();
+	  int len2 = s2.length();
+	  
+	  if(len1 == 0 || len2 == 0) {
+		  return s1;
+	  }
+	  
+	  return s1.substring(0, len1 - 1) + s2.substring(1);
+  }
+  
+  /**
+   * Convert String to Date
+   */
+  public static Date toDate(String s) {
+    int len = s.length();
+    if(len >= 10) {
+      int c4 = s.charAt(4);
+      int c7 = s.charAt(7);
+      // YYYY-MM-DD
+      if(c4 == '-' && c7 == '-') {
+        return Date.valueOf(s.substring(0, 10));
+      }
+    }
+    return null;    
+  }
+  
+  /**
+   * Convert String to Timestamp
+   */
+  public static Timestamp toTimestamp(String s) {
+    int len = s.length();
+    if(len >= 10) {
+      int c4 = s.charAt(4);
+      int c7 = s.charAt(7);
+      // YYYY-MM-DD 
+      if(c4 == '-' && c7 == '-') {
+        // Convert DB2 syntax: YYYY-MM-DD-HH.MI.SS.FFF
+        if(len > 19) {
+          if(s.charAt(10) == '-') {
+            String s2 = s.substring(0, 10) + ' ' + s.substring(11, 13) + ':' + s.substring(14, 16) + ':' + 
+                s.substring(17);
+            return Timestamp.valueOf(s2);
+          }          
+        }
+        else if(len == 10) {
+          s += " 00:00:00.000";
+        }
+        return Timestamp.valueOf(s);
+      }
+    }
+    return null;    
+  }
+  
+  /**
+   * Compare two String values and return min or max 
+   */
+  public static String minMaxString(String s1, String s2, boolean max) {
+    if(s1 == null) {
+      return s2;
+    } 
+    else if(s2 == null) {
+      return s1;
+    }    
+    int cmp = s1.compareTo(s2);
+    if((max && cmp < 0) || (!max && cmp > 0)) {
+      return s2;
+    }
+    return s1;
+  }
+
+  /**
+   * Compare two Int values and return min or max 
+   */
+  public static Long minMaxInt(Long i1, String s, boolean max) {
+    Long i2 = null;
+    try {
+      i2 = Long.parseLong(s);
+    }
+    catch(NumberFormatException e) {}
+    if(i1 == null) {
+      return i2;
+    } 
+    else if(i2 == null) {
+      return i1;
+    }    
+    if((max && i1.longValue() < i2.longValue()) || (!max && i1.longValue() > i2.longValue())) {
+      return i2;
+    }
+    return i1;
+  }
+  
+  /**
+   * Compare two Date values and return min or max 
+   */
+  public static Date minMaxDate(Date d1, String s, boolean max) {
+    Date d2 = Utils.toDate(s);
+    if(d1 == null) {
+      return d2;
+    } else if(d2 == null) {
+      return d1;
+    }    
+    if((max && d1.before(d2)) || (!max && d1.after(d2))) {
+      return d2;
+    }
+    return d1;
+  }
+  
+  /**
+   * Convert String array to a string with the specified delimiter
+   */
+  public static String toString(String[] a, char del) {
+    StringBuilder s = new StringBuilder();
+    for(int i=0; i < a.length; i++) {
+      if(i > 0) {
+        s.append(del);
+      }
+      s.append(a[i]);
+    }
+    return s.toString();
+  }
+  
+  /**
+   * Convert SQL datetime format string to Java SimpleDateFormat
+   */
+  public static String convertSqlDatetimeFormat(String in) {
+    StringBuilder out = new StringBuilder();
+    int len = in.length();
+    int i = 0;
+    while (i < len) {
+      if (i + 4 <= len && in.substring(i, i + 4).compareTo("YYYY") == 0) {
+        out.append("yyyy");
+        i += 4;
+      }
+      else if (i + 2 <= len && in.substring(i, i + 2).compareTo("mm") == 0) {
+        out.append("MM");
+        i += 2;
+      }
+      else if (i + 2 <= len && in.substring(i, i + 2).compareTo("DD") == 0) {
+        out.append("dd");
+        i += 2;
+      }
+      else if (i + 4 <= len && in.substring(i, i + 4).compareToIgnoreCase("HH24") == 0) {
+        out.append("HH");
+        i += 4;
+      }
+      else if (i + 2 <= len && in.substring(i, i + 2).compareToIgnoreCase("MI") == 0) {
+        out.append("mm");
+        i += 2;
+      }
+      else if (i + 2 <= len && in.substring(i, i + 2).compareTo("SS") == 0) {
+        out.append("ss");
+        i += 2;
+      }
+      else {
+        out.append(in.charAt(i));
+        i++;
+      }
+    }
+    return out.toString();
+  }
+  
+  /**
+   * Get the executable directory
+   */
+  public static String getExecDir() {
+    String dir = Hplsql.class.getProtectionDomain().getCodeSource().getLocation().getPath();
+    if (dir.endsWith(".jar")) {   
+      dir = dir.substring(0, dir.lastIndexOf("/") + 1);
+    }
+    return dir;
+  }
+  
+  /**
+   * Format size value specified in bytes
+   */
+  public static String formatSizeInBytes(long bytes, String postfix) {
+    String out; 
+    if (bytes < 1024) {
+      out = bytes + " bytes";
+    }
+    else if (bytes < 1024 * 1024) {
+      out = String.format("%.1f", ((float)bytes)/1024) + " KB";
+    }
+    else if (bytes < 1024 * 1024 * 1024) {
+      out = String.format("%.1f", ((float)bytes)/(1024 * 1024)) + " MB";
+    }
+    else {
+      out = String.format("%.1f", ((float)bytes)/(1024 * 1024 * 1024)) + " GB";
+    }
+    if (postfix != null && !postfix.isEmpty()) {
+      out += postfix;
+    }
+    return out;
+  }
+  
+  public static String formatSizeInBytes(long bytes) {
+    return Utils.formatSizeInBytes(bytes, null);
+  }
+  
+  /**
+   * Format bytes per second rate
+   */
+  public static String formatBytesPerSec(long bytes, long msElapsed) {
+    float bytesPerSec = ((float)bytes)/msElapsed*1000;
+    return Utils.formatSizeInBytes((long)bytesPerSec, "/sec");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
new file mode 100644
index 0000000..0a4ead2
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
@@ -0,0 +1,430 @@
+/**
+ * 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.math.BigDecimal;
+import java.util.ArrayList;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Date;
+import java.sql.Timestamp;
+
+/**
+ * Variable or the result of expression 
+ */
+public class Var {
+
+	// Data types
+	public enum Type {BOOL, CURSOR, DATE, DEC, FILE, IDENT, BIGINT, INTERVAL, STRING, STRINGLIST, TIMESTAMP, NULL};
+	public static Var Empty = new Var();
+	public static Var Null = new Var(Type.NULL);
+	
+	public String name;
+	public Type type; 
+	public Object value;
+	
+	int len;
+	int scale;
+	
+	public Var() {
+	  type = Type.NULL;  
+	}
+	
+	public Var(Var var) {
+	  name = var.name;
+    type = var.type;
+    value = var.value;
+    len = var.len;
+    scale = var.scale;
+  }
+	
+	public Var(Long value) {
+    this.type = Type.BIGINT;
+    this.value = value;
+	}
+	
+	public Var(BigDecimal value) {
+    this.type = Type.DEC;
+    this.value = value;
+  }
+  
+	public Var(String name, Long value) {
+    this.type = Type.BIGINT;
+    this.name = name;    
+    this.value = value;
+  }
+  
+	public Var(String value) {
+    this.type = Type.STRING;
+    this.value = value;
+  }
+  
+	public Var(Date value) {
+    this.type = Type.DATE;
+    this.value = value;
+  }
+
+	public Var(Timestamp value, int scale) {
+    this.type = Type.TIMESTAMP;
+    this.value = value;
+    this.scale = scale;
+  }
+	
+	public Var(Interval value) {
+    this.type = Type.INTERVAL;
+    this.value = value;
+  }
+
+	public Var(ArrayList<String> value) {
+    this.type = Type.STRINGLIST;
+    this.value = value;
+  }
+  
+	public Var(Boolean b) {
+    type = Type.BOOL;
+    value = b;
+  }
+	
+	public Var(Type type, String name) {
+    this.type = type;
+    this.name = name;
+  }
+  
+	public Var(Type type, Object value) {
+    this.type = type;
+    this.value = value;
+  }
+  
+	public Var(String name, Type type, Object value) {
+    this.name = name;
+    this.type = type;
+    this.value = value;
+  }
+	
+	public Var(Type type) {
+    this.type = type;
+  }
+
+	public Var(String name, String type, String len, String scale, Var def) {
+	  this.name = name;
+	  setType(type);	  
+	  if (len != null) {
+	    this.len = Integer.parseInt(len);
+	  }
+    if (scale != null) {
+	    this.scale = Integer.parseInt(scale);
+    }
+    if (def != null) {
+      cast(def);
+    }
+	}
+	
+	/**
+	 * Cast a new value to the variable 
+	 */
+	public Var cast(Var val) {
+	  if (val == null || val.value == null) {
+	    value = null;
+	  }
+	  else if (type == val.type && type == Type.STRING) {
+	    cast((String)val.value);
+	  }
+	  else if (type == val.type) {
+	    value = val.value;
+	  }
+	  else if (type == Type.STRING) {
+	    cast(val.toString());
+	  }
+	  else if (type == Type.DATE) {
+	    value = Utils.toDate(val.toString());
+    }
+    else if (type == Type.TIMESTAMP) {
+      value = Utils.toTimestamp(val.toString());
+    }
+	  return this;
+	}
+
+  /**
+   * Cast a new string value to the variable 
+   */
+  public Var cast(String val) {
+    if (type == Type.STRING) {
+      if (len != 0 ) {
+        int l = val.length();
+        if (l > len) {
+          value = val.substring(0, len);
+          return this;
+        }
+      }
+      value = val;
+    }
+    return this;
+  }
+	
+	/**
+	 * Set the new value 
+	 */
+	public void setValue(String str) {
+	  if(type == Type.STRING) {
+	    value = str;
+	  }
+	}
+	
+	public Var setValue(Long val) {
+    if (type == Type.BIGINT) {
+      value = val;
+    }
+    return this;
+  }
+	
+	/**
+   * Set the new value from a result set
+   */
+  public Var setValue(ResultSet rs, ResultSetMetaData rsm, int idx) throws SQLException {
+    int type = rsm.getColumnType(idx);
+    if (type == java.sql.Types.CHAR || type == java.sql.Types.VARCHAR) {
+      cast(new Var(rs.getString(idx)));
+    }
+    else if (type == java.sql.Types.INTEGER || type == java.sql.Types.BIGINT) {
+      cast(new Var(new Long(rs.getLong(idx))));
+    }
+    return this;
+  }
+	
+	/**
+	 * Set the data type from string representation
+	 */
+	void setType(String type) {
+	  this.type = defineType(type);
+	}
+	
+	/**
+   * Set the data type from JDBC type code
+   */
+  void setType(int type) {
+    this.type = defineType(type);
+  }	
+	
+	/**
+   * Define the data type from string representation
+   */
+  public static Type defineType(String type) {
+    if (type == null) {
+      return Type.NULL;
+    }    
+    else if (type.equalsIgnoreCase("INT") || type.equalsIgnoreCase("INTEGER")) {
+      return Type.BIGINT;
+    }
+    else if (type.equalsIgnoreCase("CHAR") || type.equalsIgnoreCase("VARCHAR") || type.equalsIgnoreCase("STRING")) {
+      return Type.STRING;
+    }
+    else if (type.equalsIgnoreCase("DATE")) {
+      return Type.DATE;
+    }
+    else if (type.equalsIgnoreCase("TIMESTAMP")) {
+      return Type.TIMESTAMP;
+    }
+    else if (type.equalsIgnoreCase("UTL_FILE.FILE_TYPE")) {
+      return Type.FILE;
+    }
+    return Type.NULL;
+  }
+  
+  /**
+   * Define the data type from JDBC type code
+   */
+  public static Type defineType(int type) {
+    if (type == java.sql.Types.CHAR || type == java.sql.Types.VARCHAR) {
+      return Type.STRING;
+    }
+    else if (type == java.sql.Types.INTEGER || type == java.sql.Types.BIGINT) {
+      return Type.BIGINT;
+    }
+    return Type.NULL;
+  }
+	
+	/**
+	 * Remove value
+	 */
+	public void removeValue() {
+	  type = Type.NULL;  
+    name = null;
+    value = null;
+    len = 0;
+    scale = 0;
+	}
+	
+	/*
+	 * Compare values
+	 */
+	@Override
+  public boolean equals(Object obj) {
+	  if (this == obj) {
+      return true;
+	  }
+	  else if (obj == null || this.value == null) {
+      return false;
+    }
+	  else if (getClass() != obj.getClass()) {
+      return false;
+	  }
+	  
+    Var var = (Var)obj;    
+    if (type == Type.BIGINT && var.type == Type.BIGINT &&
+       ((Long)value).longValue() == ((Long)var.value).longValue()) {
+      return true;
+    }
+    else if (type == Type.STRING && var.type == Type.STRING &&
+            ((String)value).equals((String)var.value)) {
+      return true;
+    }
+    return false;
+	}
+	
+	/*
+   * Compare values
+   */
+  public int compareTo(Var v) {
+    if (this == v) {
+      return 0;
+    }
+    else if (v == null) {
+      return -1;
+    }
+    else if (type == Type.BIGINT && v.type == Type.BIGINT) {
+      return ((Long)value).compareTo((Long)v.value);
+    }
+    else if (type == Type.STRING && v.type == Type.STRING) {
+      return ((String)value).compareTo((String)v.value);
+    }
+    return -1;
+  }
+	
+	 /**
+   * Increment an integer value
+   */
+  public Var increment(Long i) {
+    if (type == Type.BIGINT) {
+      value = new Long(((Long)value).longValue() + i);
+    }
+    return this;
+  }
+
+  /**
+  * Decrement an integer value
+  */
+ public Var decrement(Long i) {
+   if (type == Type.BIGINT) {
+     value = new Long(((Long)value).longValue() - i);
+   }
+   return this;
+ }
+  
+	/**
+	 * Return an integer value
+	 */
+	public int intValue() {
+	  if (type == Type.BIGINT) {
+	    return ((Long)value).intValue();
+	  }
+	  return -1;
+	}
+	
+	/**
+	 * Return true/false for BOOL type
+	 */
+	public boolean isTrue() {
+	  if(type == Type.BOOL && value != null) {
+	    return ((Boolean)value).booleanValue();
+	  }
+	  return false;
+	}
+	
+	/**
+	 * Check if the variable contains NULL
+	 */
+	public boolean isNull() {
+    if (type == Type.NULL || value == null) {
+      return true;
+    }
+    return false;
+  }
+	
+	/**
+	 * Convert value to String
+	 */
+	@Override
+  public String toString() {
+	  if (type == Type.IDENT) {
+      return name;
+    }   
+	  else if (value == null) {
+	    return null;
+	  }
+	  else if (type == Type.BIGINT) {
+	    return ((Long)value).toString();
+	  }
+	  else if (type == Type.STRING) {
+      return (String)value;
+    }
+    else if (type == Type.DATE) {
+      return ((Date)value).toString();
+    }
+    else if (type == Type.TIMESTAMP) {
+      int len = 19;
+      String t = ((Timestamp)value).toString();   // .0 returned if the fractional part not set
+      if (scale > 0) {
+        len += scale + 1;
+      }
+      if (t.length() > len) {
+        t = t.substring(0, len);
+      }
+      return t;
+    }
+	  return value.toString();
+	}
+
+  /**
+   * Convert value to SQL string - string literals are quoted and escaped, ab'c -> 'ab''c'
+   */
+  public String toSqlString() {
+    if (value == null) {
+      return "NULL";
+    }
+    else if (type == Type.STRING) {
+      return Utils.quoteString((String)value);
+    }
+    return toString();
+  }
+	
+  /**
+   * Set variable name
+   */
+  public void setName(String name) {
+    this.name = name;
+  }
+  
+	/**
+	 * Get variable name
+	 */
+	public String getName() {
+	  return name;
+	}
+}