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/24 19:45:02 UTC

hive git commit: HIVE-11290 Cursor attributes %ISOPEN, %FOUND, %NOTFOUND and SYS_REFCURSOR variable (Dmitry Tolpeko via gates)

Repository: hive
Updated Branches:
  refs/heads/master 2773b9d9f -> 92e98858e


HIVE-11290 Cursor attributes %ISOPEN, %FOUND, %NOTFOUND and SYS_REFCURSOR variable (Dmitry Tolpeko via gates)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/92e98858
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/92e98858
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/92e98858

Branch: refs/heads/master
Commit: 92e98858e742bbb669ccbf790a71a618c581df21
Parents: 2773b9d
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Jul 24 10:43:45 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Fri Jul 24 10:43:45 2015 -0700

----------------------------------------------------------------------
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     | 38 ++++++++++--
 .../main/java/org/apache/hive/hplsql/Conn.java  |  6 ++
 .../main/java/org/apache/hive/hplsql/Exec.java  | 21 ++++++-
 .../java/org/apache/hive/hplsql/Expression.java | 28 +++++++++
 .../main/java/org/apache/hive/hplsql/Query.java | 55 +++++++++++++++++
 .../main/java/org/apache/hive/hplsql/Stmt.java  | 60 ++++++++++++------
 .../main/java/org/apache/hive/hplsql/Var.java   | 20 +++++-
 .../src/test/queries/db/cursor_attributes.sql   | 60 ++++++++++++++++++
 hplsql/src/test/queries/db/select_into.sql      | 17 +++++
 .../src/test/queries/db/set_current_schema.sql  |  6 ++
 hplsql/src/test/queries/db/sys_refcursor.sql    | 65 ++++++++++++++++++++
 hplsql/src/test/queries/db/use.sql              |  2 +
 .../test/results/db/cursor_attributes.out.txt   | 33 ++++++++++
 hplsql/src/test/results/db/select_into.out.txt  | 19 ++++++
 .../test/results/db/set_current_schema.out.txt  | 12 ++++
 .../src/test/results/db/sys_refcursor.out.txt   | 36 +++++++++++
 hplsql/src/test/results/db/use.out.txt          |  4 ++
 17 files changed, 453 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
----------------------------------------------------------------------
diff --git a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4 b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
index 70312b2..4104d13 100644
--- a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
+++ b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
@@ -98,7 +98,8 @@ expr_stmt :             // Standalone expression
      ;
 
 assignment_stmt :       // Assignment statement
-       T_SET? assignment_stmt_item (T_COMMA assignment_stmt_item)*  
+       T_SET set_session_option
+     | T_SET? assignment_stmt_item (T_COMMA assignment_stmt_item)*  
      ;
 
 assignment_stmt_item : 
@@ -145,9 +146,9 @@ declare_block :         // Declaration block
    
      
 declare_stmt_item :
-       declare_var_item 
+       declare_cursor_item
+     | declare_var_item 
      | declare_condition_item  
-     | declare_cursor_item
      | declare_handler_item
      | declare_temporary_table_item
      ;
@@ -254,6 +255,7 @@ dtype :                  // Data types
      | T_RESULT_SET_LOCATOR T_VARYING
      | T_SMALLINT
      | T_STRING
+     | T_SYS_REFCURSOR
      | T_TIMESTAMP
      | T_VARCHAR
      | T_VARCHAR2
@@ -391,7 +393,7 @@ map_object_stmt :
      ;
      
 open_stmt :             // OPEN cursor statement
-       T_OPEN L_ID (T_FOR (expr | select_stmt))?
+       T_OPEN L_ID (T_FOR (select_stmt | expr))?
      ;
 
 fetch_stmt :            // FETCH cursor statement
@@ -460,6 +462,14 @@ rollback_stmt :         // ROLLBACK statement
        T_ROLLBACK T_WORK?
      ;
      
+set_session_option :          
+       set_current_schema_option
+     ;
+
+set_current_schema_option :          
+       ((T_CURRENT? T_SCHEMA) | T_CURRENT_SCHEMA) T_EQUAL? expr
+     ;
+     
 signal_stmt :          // SIGNAL statement
        T_SIGNAL ident
      ;
@@ -675,6 +685,7 @@ bool_expr :                               // Boolean condition
 bool_expr_atom :
       bool_expr_unary
     | bool_expr_binary
+    | expr
     ;
     
 bool_expr_unary :
@@ -722,6 +733,7 @@ expr :
      | T_OPEN_P expr T_CLOSE_P 
      | expr_concat
      | expr_case
+     | expr_cursor_attribute
      | expr_agg_window_func
      | expr_spec_func
      | expr_func                          
@@ -777,6 +789,10 @@ expr_case_searched :
        T_CASE (T_WHEN bool_expr T_THEN expr)+ (T_ELSE expr)? T_END
      ;
      
+expr_cursor_attribute :
+      ident '%' (T_ISOPEN | T_FOUND | T_NOTFOUND)
+    ;
+     
 expr_agg_window_func :
        T_AVG T_OPEN_P expr_func_all_distinct? expr T_CLOSE_P expr_func_over_clause?
      | T_COUNT T_OPEN_P ((expr_func_all_distinct? expr) | '*') T_CLOSE_P expr_func_over_clause?
@@ -937,6 +953,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_CS
      | T_CURRENT 
      | T_CURRENT_DATE
+     | T_CURRENT_SCHEMA
      | T_CURRENT_TIMESTAMP
      | T_CURRENT_USER
      | T_CURSOR  
@@ -1007,6 +1024,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_INVOKER     
      | T_ITEMS     
      | T_IS    
+     | T_ISOPEN
      | T_JOIN     
      | T_KEY
      | T_KEYS
@@ -1033,7 +1051,8 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_MICROSECONDS
      | T_MIN
      | T_MULTISET
-     | T_NOT          
+     | T_NOT         
+     | T_NOTFOUND     
      // T_NULL reserved word       
      | T_NUMBER   
      | T_OBJECT     
@@ -1072,6 +1091,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_ROWS
      | T_ROW_COUNT
      | T_ROW_NUMBER
+     | T_SCHEMA
      | T_SECURITY
      | T_SEL          
      | T_SELECT       
@@ -1089,7 +1109,8 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_STRING    
      | T_SUBSTRING
      | T_SUM
-     | T_SYSDATE     
+     | T_SYSDATE 
+     | T_SYS_REFCURSOR     
      | T_TABLE
      | T_TEMPORARY
      | T_TERMINATED
@@ -1160,6 +1181,7 @@ T_CREATE          : C R E A T E ;
 T_CREATOR         : C R E A T O R ;
 T_CS              : C S;
 T_CURRENT         : C U R R E N T ;
+T_CURRENT_SCHEMA  : C U R R E N T '_' S C H E M A ;
 T_CURSOR          : C U R S O R ;
 T_DATE            : D A T E ;
 T_DAY             : D A Y ;
@@ -1225,6 +1247,7 @@ T_INTERSECT       : I N T E R S E C T ;
 T_INTO            : I N T O ;
 T_INVOKER         : I N V O K E R ;
 T_IS              : I S ;
+T_ISOPEN          : I S O P E N ;
 T_ITEMS           : I T E M S ; 
 T_JOIN            : J O I N ;
 T_KEY             : K E Y ;
@@ -1250,6 +1273,7 @@ T_MICROSECONDS    : M I C R O S E C O N D S;
 T_MIN             : M I N ;
 T_MULTISET        : M U L T I S E T ; 
 T_NOT             : N O T ;
+T_NOTFOUND        : N O T F O U N D ; 
 T_NULL            : N U L L ;
 T_NUMBER          : N U M B E R ;
 T_OBJECT          : O B J E C T ; 
@@ -1286,6 +1310,7 @@ T_ROW_COUNT       : R O W '_' C O U N T ;
 T_RR              : R R;
 T_RS              : R S ;
 T_TRIM            : T R I M ;
+T_SCHEMA          : S C H E M A ;
 T_SECURITY        : S E C U R I T Y ; 
 T_SEL             : S E L ;
 T_SELECT          : S E L E C T ; 
@@ -1302,6 +1327,7 @@ T_STEP            : S T E P ;
 T_STRING          : S T R I N G ;
 T_SUBSTRING       : S U B S T R I N G ; 
 T_SUM             : S U M ;
+T_SYS_REFCURSOR   : S Y S '_' R E F C U R S O R ; 
 T_TABLE           : T A B L E ;
 T_TEMPORARY       : T E M P O R A R Y ;
 T_TERMINATED      : T E R M I N A T E D ; 

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java b/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
index ac4b521..c8cc910 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
@@ -57,6 +57,7 @@ public class Conn {
       Connection conn = getConnection(connName);
       runPreSql(connName, conn);
       Statement stmt = conn.createStatement();
+      exec.info(null, "Starting query");
       timer.start();
       ResultSet rs = stmt.executeQuery(query.sql);
       timer.stop();
@@ -84,10 +85,15 @@ public class Conn {
       runPreSql(connName, conn);
       Statement stmt = conn.createStatement();
       ResultSet rs = null;
+      exec.info(null, "Starting SQL statement");
+      timer.start();
       if (stmt.execute(sql)) {
         rs = stmt.getResultSet();        
       } 
       query.set(conn, stmt, rs);
+      if (info) {
+        exec.info(null, "SQL statement executed successfully (" + timer.format() + ")");
+      } 
     } catch (Exception e) {
       query.setError(e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
index b35344f..e143136 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
@@ -232,7 +232,7 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   /**
    * Push a boolean value to the stack
    */
-  public void stackPush(boolean val) {
+  public void stackPush(Boolean val) {
     exec.stack.push(new Var(val));  
   }
 
@@ -832,7 +832,7 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
     while (!signals.empty()) {
       Signal sig = signals.pop();
       if (sig.type == Signal.Type.SQLEXCEPTION) {
-        System.err.println("Unhandled exception in PL/HQL");
+        System.err.println("Unhandled exception in HPL/SQL");
       }
       if (sig.exception != null) {
         sig.exception.printStackTrace(); 
@@ -1320,6 +1320,15 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
     }
     return 0; 
   }
+  
+  /**
+   * Cursor attribute %ISOPEN, %FOUND and %NOTFOUND
+   */
+  @Override 
+  public Integer visitExpr_cursor_attribute(HplsqlParser.Expr_cursor_attributeContext ctx) {
+    exec.expr.execCursorAttribute(ctx);
+    return 0; 
+  }
     
   /**
    * Function call
@@ -1498,6 +1507,14 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
     return exec.stmt.return_(ctx); 
   }  
   
+  /** 
+   * SET session options
+   */
+  @Override 
+  public Integer visitSet_current_schema_option(HplsqlParser.Set_current_schema_optionContext ctx) { 
+    return exec.stmt.setCurrentSchema(ctx); 
+  }
+  
   /**
    * MAP OBJECT statement
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java b/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
index f8b01e1..772c98a 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
@@ -268,6 +268,34 @@ public class Expression {
   }
   
   /**
+   * Cursor attribute %ISOPEN, %FOUND and %NOTFOUND
+   */
+  public void execCursorAttribute(HplsqlParser.Expr_cursor_attributeContext ctx) {
+    String name = ctx.ident().getText();
+    Var val = new Var(Var.Type.BOOL);
+    Var cursor = exec.findCursor(name);
+    if (cursor != null) {
+      Query query = (Query)cursor.value;
+      if (query != null) {
+        if (ctx.T_ISOPEN() != null) {
+          val.setValue(query.isOpen());
+        }
+        else if (ctx.T_FOUND() != null) {
+          val.setValue(query.isFound());
+        }
+        else if (ctx.T_NOTFOUND() != null) {
+          val.setValue(query.isNotFound());
+        }
+      }
+      exec.stackPush(val);
+    }
+    else {
+      trace(ctx, "Cursor not found: " + name);
+      exec.signal(Signal.Type.SQLEXCEPTION);
+    }
+  }
+  
+  /**
    * Addition operator
    */
   public void operatorAdd(HplsqlParser.ExprContext ctx) {

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Query.java b/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
index eaaaa67..08cd6a7 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
@@ -34,6 +34,9 @@ public class Query {
   Statement stmt;
   ResultSet rs;
   Exception exception;
+
+  public enum State { OPEN, FETCHED_OK, FETCHED_NODATA, CLOSE };
+  State state = State.CLOSE;
   
   boolean withReturn = false;
   
@@ -51,6 +54,21 @@ public class Query {
     this.conn = conn;
     this.stmt = stmt;
     this.rs = rs;
+    if (rs != null) {
+      state = State.OPEN;
+    }
+  }
+  
+  /**
+   * Set the fetch status
+   */
+  public void setFetch(boolean ok) {
+    if (ok == true) {
+      state = State.FETCHED_OK;
+    }
+    else {
+      state = State.FETCHED_NODATA;
+    }
   }
   
   /**
@@ -66,6 +84,42 @@ public class Query {
   }
   
   /**
+   * Check if the cursor is open
+   */
+  public boolean isOpen() {
+    if (rs != null) {
+      return true;
+    }
+    return false;
+  }
+  
+  /**
+   * Check if the cursor was fetched and a row was returned
+   */
+  public Boolean isFound() {
+    if (state == State.OPEN || state == State.CLOSE) {
+      return null;
+    }
+    if (state == State.FETCHED_OK) {
+      return Boolean.valueOf(true);
+    } 
+    return Boolean.valueOf(false);    
+  }
+  
+  /**
+   * Check if the cursor was fetched and no row was returned
+   */
+  public Boolean isNotFound() {
+    if (state == State.OPEN || state == State.CLOSE) {
+      return null;
+    }
+    if (state == State.FETCHED_NODATA) {
+      return Boolean.valueOf(true);
+    }
+    return Boolean.valueOf(false);
+  }
+  
+  /**
    * Close statement results
    */
   public void closeStatement() {
@@ -78,6 +132,7 @@ public class Query {
         stmt.close();
         stmt = null;
       }
+      state = State.CLOSE;
     } catch (SQLException e) {
       e.printStackTrace();
     }   

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/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
index bfb76cd..9a5e970 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
@@ -280,18 +280,24 @@ public class Stmt {
     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);
+    if (ctx.T_FOR() != null) {                             // SELECT statement or dynamic SQL
+      if (ctx.expr() != null) {
+        sql = evalPop(ctx.expr()).toString();
+      }
+      else {
+        sql = evalPop(ctx.select_stmt()).toString();
       }
       query = new Query(sql);
-      var = new Var(cursor, Type.CURSOR, query);
-      exec.addVariable(var);
+      var = exec.findCursor(cursor);                      // Can be a ref cursor variable
+      if (var == null) {
+        var = new Var(cursor, Type.CURSOR, query);
+        exec.addVariable(var);
+      }
+      else {
+        var.setValue(query);
+      }
     }
-    // Declared cursor
-    else {
+    else {                                                 // Declared cursor
       var = exec.findVariable(cursor);      
       if (var != null && var.type == Type.CURSOR) {
         query = (Query)var.value;
@@ -303,13 +309,12 @@ public class Stmt {
           sql = evalPop(query.sqlSelect).toString();
           query.setSql(sql);
         }
-        if (trace) {
-          trace(ctx, cursor + ": " + sql);
-        } 
       }
     }
-    // Open cursor now
     if (query != null) {
+      if (trace) {
+        trace(ctx, cursor + ": " + sql);
+      } 
       exec.executeQuery(ctx, query, exec.conf.defaultConnection);
       if (query.error()) {
         exec.signal(query);
@@ -343,7 +348,13 @@ public class Stmt {
       exec.setSqlCode(-1);
       exec.signal(Signal.Type.SQLEXCEPTION);
       return 1;
-    }    
+    }  
+    else if (cursor.value == null) {
+      trace(ctx, "Cursor not open: " + name);
+      exec.setSqlCode(-1);
+      exec.signal(Signal.Type.SQLEXCEPTION);
+      return 1;
+    }  
     else if (exec.getOffline()) {
       exec.setSqlCode(100);
       exec.signal(Signal.Type.NOTFOUND);
@@ -360,6 +371,7 @@ public class Stmt {
       if(rs != null && rsm != null) {
         int cols = ctx.L_ID().size() - 1;
         if(rs.next()) {
+          query.setFetch(true);
           for(int i=1; i <= cols; i++) {
             Var var = exec.findVariable(ctx.L_ID(i).getText());
             if(var != null) {
@@ -377,6 +389,7 @@ public class Stmt {
           exec.setSqlSuccess();
         }
         else {
+          query.setFetch(false);
           exec.setSqlCode(100);
         }
       }
@@ -639,12 +652,13 @@ public class Stmt {
    * USE statement
    */
   public Integer use(HplsqlParser.Use_stmtContext ctx) {
+    trace(ctx, "USE");
+    return use(ctx, ctx.T_USE().toString() + " " + evalPop(ctx.expr()).toString());
+  }
+  
+  public Integer use(ParserRuleContext ctx, String sql) {
     if(trace) {
-      trace(ctx, "USE");
-    }
-    String sql = ctx.T_USE().toString() + " " + evalPop(ctx.expr()).toString();
-    if(trace) {
-      trace(ctx, "Query: " + sql);
+      trace(ctx, "SQL statement: " + sql);
     }    
     Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
     if(query.error()) {
@@ -959,6 +973,14 @@ public class Stmt {
 	  return 0; 
   }
   
+  /** 
+   * SET current schema 
+   */
+  public Integer setCurrentSchema(HplsqlParser.Set_current_schema_optionContext ctx) { 
+    trace(ctx, "SET CURRENT SCHEMA");
+    return use(ctx, "USE " + evalPop(ctx.expr()).toString());
+  }
+  
   /**
    * SIGNAL statement
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/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
index 87b42f9..7e4ae52 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
@@ -32,7 +32,7 @@ import java.sql.Timestamp;
 public class Var {
 
 	// Data types
-	public enum Type {BOOL, CURSOR, DATE, DEC, FILE, IDENT, BIGINT, INTERVAL, RS_LOCATOR, STRING, STRINGLIST, TIMESTAMP, NULL};
+	public enum Type {BOOL, CURSOR, DATE, DECIMAL, FILE, IDENT, BIGINT, INTERVAL, RS_LOCATOR, STRING, STRINGLIST, TIMESTAMP, NULL};
 	public static Var Empty = new Var();
 	public static Var Null = new Var(Type.NULL);
 	
@@ -61,7 +61,7 @@ public class Var {
 	}
 	
 	public Var(BigDecimal value) {
-    this.type = Type.DEC;
+    this.type = Type.DECIMAL;
     this.value = value;
   }
   
@@ -194,6 +194,13 @@ public class Var {
     return this;
   }
 	
+	public Var setValue(Boolean val) {
+    if (type == Type.BOOL) {
+      value = val;
+    }
+    return this;
+  }
+	
 	public void setValue(Object value) {
     this.value = value;
   }
@@ -209,6 +216,9 @@ public class Var {
     else if (type == java.sql.Types.INTEGER || type == java.sql.Types.BIGINT) {
       cast(new Var(new Long(rs.getLong(idx))));
     }
+    else if (type == java.sql.Types.DECIMAL || type == java.sql.Types.NUMERIC) {
+      cast(new Var(rs.getBigDecimal(idx)));
+    }
     return this;
   }
 	
@@ -239,12 +249,18 @@ public class Var {
     else if (type.equalsIgnoreCase("CHAR") || type.equalsIgnoreCase("VARCHAR") || type.equalsIgnoreCase("STRING")) {
       return Type.STRING;
     }
+    else if (type.equalsIgnoreCase("DEC") || type.equalsIgnoreCase("DECIMAL") || type.equalsIgnoreCase("NUMERIC")) {
+      return Type.DECIMAL;
+    }
     else if (type.equalsIgnoreCase("DATE")) {
       return Type.DATE;
     }
     else if (type.equalsIgnoreCase("TIMESTAMP")) {
       return Type.TIMESTAMP;
     }
+    else if (type.equalsIgnoreCase("SYS_REFCURSOR")) {
+      return Type.CURSOR;
+    }
     else if (type.equalsIgnoreCase("UTL_FILE.FILE_TYPE")) {
       return Type.FILE;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/queries/db/cursor_attributes.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/cursor_attributes.sql b/hplsql/src/test/queries/db/cursor_attributes.sql
new file mode 100644
index 0000000..9b1e9db
--- /dev/null
+++ b/hplsql/src/test/queries/db/cursor_attributes.sql
@@ -0,0 +1,60 @@
+DECLARE 
+  CURSOR c1 IS SELECT 'A' FROM src LIMIT 1;
+  v1 VARCHAR(30);
+BEGIN
+  IF c1%ISOPEN THEN
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN failed');
+  ELSE 
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN correct');
+  END IF; 
+  
+  OPEN c1;
+
+  IF c1%ISOPEN THEN
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN correct');
+  ELSE 
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN failed');
+  END IF; 
+  
+  IF c1%FOUND IS NULL THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  END IF;
+
+  IF c1%NOTFOUND IS NULL THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  END IF;
+  
+  FETCH c1 INTO v1;
+  
+  IF c1%FOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  END IF;
+
+  IF c1%NOTFOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  END IF;  
+  
+  FETCH c1 INTO v1;
+  
+  IF c1%FOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  END IF;
+
+  IF c1%NOTFOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  END IF;  
+
+  CLOSE c1; 
+END;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/queries/db/select_into.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/select_into.sql b/hplsql/src/test/queries/db/select_into.sql
new file mode 100644
index 0000000..3995ba2
--- /dev/null
+++ b/hplsql/src/test/queries/db/select_into.sql
@@ -0,0 +1,17 @@
+DECLARE v_int INT;
+DECLARE v_dec DECIMAL(18,2);
+DECLARE v_dec0 DECIMAL(18,0);
+
+SELECT TOP 1 
+  CAST(1 AS INT), 
+  CAST(1.1 AS DECIMAL(18,2)),
+  CAST(1.1 AS DECIMAL(18,0))   
+INTO 
+  v_int,
+  v_dec,
+  v_dec0  
+FROM src ;
+        
+PRINT 'INT: ' || v_int;
+PRINT 'DECIMAL: ' || v_dec;
+PRINT 'DECIMAL0: ' || v_dec0;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/queries/db/set_current_schema.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/set_current_schema.sql b/hplsql/src/test/queries/db/set_current_schema.sql
new file mode 100644
index 0000000..059ef22
--- /dev/null
+++ b/hplsql/src/test/queries/db/set_current_schema.sql
@@ -0,0 +1,6 @@
+SET CURRENT SCHEMA = default;
+SET CURRENT_SCHEMA = default;
+SET SCHEMA = default;
+SET SCHEMA default;
+SET SCHEMA = 'default';
+SET SCHEMA 'def' || 'ault';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/queries/db/sys_refcursor.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/sys_refcursor.sql b/hplsql/src/test/queries/db/sys_refcursor.sql
new file mode 100644
index 0000000..eeb98ed
--- /dev/null
+++ b/hplsql/src/test/queries/db/sys_refcursor.sql
@@ -0,0 +1,65 @@
+CREATE PROCEDURE spResultSet1 (cur OUT SYS_REFCURSOR)
+BEGIN
+  OPEN cur FOR SELECT 'A' FROM src LIMIT 1;
+END; 
+
+DECLARE 
+  c1 SYS_REFCURSOR;
+  v1 VARCHAR(30);
+BEGIN
+  IF c1%ISOPEN THEN
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN failed');
+  ELSE 
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN correct');
+  END IF; 
+  
+  CALL spResultSet1(c1);
+  
+  IF c1%ISOPEN THEN
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN correct');
+  ELSE 
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN failed');
+  END IF; 
+  
+  IF c1%FOUND IS NULL THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  END IF;
+
+  IF c1%NOTFOUND IS NULL THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  END IF;
+  
+  FETCH c1 INTO v1;
+  
+  IF c1%FOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  END IF;
+
+  IF c1%NOTFOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  END IF;  
+  
+  FETCH c1 INTO v1;
+  
+  IF c1%FOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  END IF;
+
+  IF c1%NOTFOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  END IF;  
+
+  CLOSE c1; 
+END;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/queries/db/use.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/use.sql b/hplsql/src/test/queries/db/use.sql
new file mode 100644
index 0000000..dab0615
--- /dev/null
+++ b/hplsql/src/test/queries/db/use.sql
@@ -0,0 +1,2 @@
+USE default;
+USE SUBSTR('default', 1, 7);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/results/db/cursor_attributes.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/cursor_attributes.out.txt b/hplsql/src/test/results/db/cursor_attributes.out.txt
new file mode 100644
index 0000000..fbc8eef
--- /dev/null
+++ b/hplsql/src/test/results/db/cursor_attributes.out.txt
@@ -0,0 +1,33 @@
+Ln:2 DECLARE CURSOR c1
+Ln:3 DECLARE v1 VARCHAR
+Ln:5 IF
+Ln:5 ELSE executed
+%ISOPEN correct
+Ln:11 OPEN
+Ln:11 c1: SELECT 'A' FROM src LIMIT 1
+Ln:13 IF
+Ln:13 IF TRUE executed
+%ISOPEN correct
+Ln:19 IF
+Ln:19 IF TRUE executed
+%FOUND correct
+Ln:25 IF
+Ln:25 IF TRUE executed
+%NOTFOUND correct
+Ln:31 FETCH
+Ln:31 COLUMN: _c0, string
+Ln:31 SET v1 = A
+Ln:33 IF
+Ln:33 IF TRUE executed
+%FOUND correct
+Ln:39 IF
+Ln:39 ELSE executed
+%NOTFOUND correct
+Ln:45 FETCH
+Ln:47 IF
+Ln:47 ELSE executed
+%FOUND correct
+Ln:53 IF
+Ln:53 IF TRUE executed
+%NOTFOUND correct
+Ln:59 CLOSE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/results/db/select_into.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/select_into.out.txt b/hplsql/src/test/results/db/select_into.out.txt
new file mode 100644
index 0000000..80d067e
--- /dev/null
+++ b/hplsql/src/test/results/db/select_into.out.txt
@@ -0,0 +1,19 @@
+Ln:1 DECLARE v_int INT
+Ln:2 DECLARE v_dec DECIMAL
+Ln:3 DECLARE v_dec0 DECIMAL
+Ln:5 SELECT
+Ln:5 SELECT CAST(1 AS INT), CAST(1.1 AS DECIMAL(18,2)), CAST(1.1 AS DECIMAL(18,0)) FROM src LIMIT 1
+Ln:5 SELECT completed successfully
+Ln:5 SELECT INTO statement executed
+Ln:5 COLUMN: _c0, int
+Ln:5 SET v_int = 1
+Ln:5 COLUMN: _c1, decimal
+Ln:5 SET v_dec = 1.1
+Ln:5 COLUMN: _c2, decimal
+Ln:5 SET v_dec0 = 1
+Ln:15 PRINT
+INT: 1
+Ln:16 PRINT
+DECIMAL: 1.1
+Ln:17 PRINT
+DECIMAL0: 1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/results/db/set_current_schema.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/set_current_schema.out.txt b/hplsql/src/test/results/db/set_current_schema.out.txt
new file mode 100644
index 0000000..6ef8572
--- /dev/null
+++ b/hplsql/src/test/results/db/set_current_schema.out.txt
@@ -0,0 +1,12 @@
+Ln:1 SET CURRENT SCHEMA
+Ln:1 SQL statement: USE default
+Ln:2 SET CURRENT SCHEMA
+Ln:2 SQL statement: USE default
+Ln:3 SET CURRENT SCHEMA
+Ln:3 SQL statement: USE default
+Ln:4 SET CURRENT SCHEMA
+Ln:4 SQL statement: USE default
+Ln:5 SET CURRENT SCHEMA
+Ln:5 SQL statement: USE default
+Ln:6 SET CURRENT SCHEMA
+Ln:6 SQL statement: USE default
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/results/db/sys_refcursor.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/sys_refcursor.out.txt b/hplsql/src/test/results/db/sys_refcursor.out.txt
new file mode 100644
index 0000000..08239cf
--- /dev/null
+++ b/hplsql/src/test/results/db/sys_refcursor.out.txt
@@ -0,0 +1,36 @@
+Ln:1 CREATE PROCEDURE spResultSet1
+Ln:7 DECLARE c1 SYS_REFCURSOR
+Ln:8 DECLARE v1 VARCHAR
+Ln:10 IF
+Ln:10 ELSE executed
+%ISOPEN correct
+Ln:16 EXEC PROCEDURE spResultSet1
+Ln:16 SET PARAM cur = null
+Ln:3 OPEN
+Ln:3 cur: SELECT 'A' FROM src LIMIT 1
+Ln:18 IF
+Ln:18 IF TRUE executed
+%ISOPEN correct
+Ln:24 IF
+Ln:24 IF TRUE executed
+%FOUND correct
+Ln:30 IF
+Ln:30 IF TRUE executed
+%NOTFOUND correct
+Ln:36 FETCH
+Ln:36 COLUMN: _c0, string
+Ln:36 SET v1 = A
+Ln:38 IF
+Ln:38 IF TRUE executed
+%FOUND correct
+Ln:44 IF
+Ln:44 ELSE executed
+%NOTFOUND correct
+Ln:50 FETCH
+Ln:52 IF
+Ln:52 ELSE executed
+%FOUND correct
+Ln:58 IF
+Ln:58 IF TRUE executed
+%NOTFOUND correct
+Ln:64 CLOSE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/results/db/use.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/use.out.txt b/hplsql/src/test/results/db/use.out.txt
new file mode 100644
index 0000000..4a478f7
--- /dev/null
+++ b/hplsql/src/test/results/db/use.out.txt
@@ -0,0 +1,4 @@
+Ln:1 USE
+Ln:1 SQL statement: USE default
+Ln:2 USE
+Ln:2 SQL statement: USE default
\ No newline at end of file