You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/07/02 03:49:15 UTC

[10/25] 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/functions/Function.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
new file mode 100644
index 0000000..9895b5e
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
@@ -0,0 +1,709 @@
+/**
+ * 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.functions;
+
+import java.sql.ResultSet;
+import java.sql.Date;
+import java.sql.SQLException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang.StringUtils;
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.hive.hplsql.*;
+
+interface FuncCommand {
+  void run(HplsqlParser.Expr_func_paramsContext ctx);
+}
+
+interface FuncSpecCommand {
+  void run(HplsqlParser.Expr_spec_funcContext ctx);
+}
+
+/**
+ * HPL/SQL functions
+ */
+public class Function {
+  Exec exec;
+  HashMap<String, FuncCommand> map = new HashMap<String, FuncCommand>();  
+  HashMap<String, FuncSpecCommand> specMap = new HashMap<String, FuncSpecCommand>();
+  HashMap<String, FuncSpecCommand> specSqlMap = new HashMap<String, FuncSpecCommand>();
+  HashMap<String, HplsqlParser.Create_function_stmtContext> userMap = new HashMap<String, HplsqlParser.Create_function_stmtContext>();
+  HashMap<String, HplsqlParser.Create_procedure_stmtContext> procMap = new HashMap<String, HplsqlParser.Create_procedure_stmtContext>();
+  boolean trace = false; 
+  
+  public Function(Exec e) {
+    exec = e;  
+    trace = exec.getTrace();
+  }
+  
+  /** 
+   * Register functions
+   */
+  public void register(Function f) {    
+  }
+  
+  /**
+   * Execute a function
+   */
+  public void exec(String name, HplsqlParser.Expr_func_paramsContext ctx) {
+    if (execUser(ctx, name)) {
+      return;
+    }
+    if (name.indexOf(".") != -1) {               // Name can be qualified and spaces are allowed between parts
+      String[] parts = name.split("\\.");
+      StringBuilder str = new StringBuilder();
+      for (int i = 0; i < parts.length; i++) {
+        if (i > 0) {
+          str.append(".");
+        }
+        str.append(parts[i].trim());        
+      }
+      name = str.toString();      
+    } 
+    if (trace && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
+      trace(ctx, "FUNC " + name);      
+    }
+    FuncCommand func = map.get(name);    
+    if (func != null) {
+      func.run(ctx);
+    }    
+    else {
+      evalNull();
+    }
+  }
+  
+  /**
+   * User-defined function in a SQL query
+   */
+  public void execSql(String name, HplsqlParser.Expr_func_paramsContext ctx) {
+    if (execUserSql(ctx, name)) {
+      return;
+    }
+    StringBuilder sql = new StringBuilder();
+    sql.append(name);
+    sql.append("(");
+    int cnt = ctx.expr().size();
+    for (int i = 0; i < cnt; i++) {
+      sql.append(evalPop(ctx.expr(i)));
+      if (i + 1 < cnt) {
+        sql.append(", ");
+      }
+    }
+    sql.append(")");
+    exec.stackPush(sql);
+  }
+  
+  /**
+   * Aggregate or window function in a SQL query
+   */
+  public void execAggWindowSql(HplsqlParser.Expr_agg_window_funcContext ctx) {
+    exec.stackPush(exec.getFormattedText(ctx));
+  }
+  
+  /**
+   * Execute a user-defined function
+   */
+  public boolean execUser(HplsqlParser.Expr_func_paramsContext ctx, String name) {
+    HplsqlParser.Create_function_stmtContext userCtx = userMap.get(name.toUpperCase());
+    if (userCtx == null) {
+      return false;
+    }
+    if (trace) {
+      trace(ctx, "EXEC FUNCTION " + name);
+    }
+    exec.enterScope(Scope.Type.ROUTINE);
+    setCallParameters(ctx, userCtx.create_routine_params(), null);
+    visit(userCtx.single_block_stmt());
+    exec.leaveScope(); 
+    return true;
+  }
+  
+  /**
+   * Execute a HPL/SQL user-defined function in a query 
+   */
+  public boolean execUserSql(HplsqlParser.Expr_func_paramsContext ctx, String name) {
+    HplsqlParser.Create_function_stmtContext userCtx = userMap.get(name.toUpperCase());
+    if (userCtx == null) {
+      return false;
+    }
+    StringBuilder sql = new StringBuilder();
+    sql.append("hplsql('");
+    sql.append(name);
+    sql.append("(");
+    int cnt = ctx.expr().size();
+    for (int i = 0; i < cnt; i++) {
+      sql.append(":" + (i + 1));
+      if (i + 1 < cnt) {
+        sql.append(", ");
+      }
+    }
+    sql.append(")'");
+    if (cnt > 0) {
+      sql.append(", ");
+    }
+    for (int i = 0; i < cnt; i++) {
+      sql.append(evalPop(ctx.expr(i)));
+      if (i + 1 < cnt) {
+        sql.append(", ");
+      }
+    }
+    sql.append(")");
+    exec.stackPush(sql);
+    exec.registerUdf();
+    return true;
+  }
+  
+  /**
+   * Execute a stored procedure as the entry point of the script (defined by -main option)
+   */
+  public boolean execProc(String name) {
+    if (trace) {
+      trace("EXEC PROCEDURE " + name);
+    }
+    HplsqlParser.Create_procedure_stmtContext procCtx = procMap.get(name.toUpperCase());    
+    if (procCtx == null) {
+      trace("Procedure not found");
+      return false;
+    }    
+    exec.enterScope(Scope.Type.ROUTINE);
+    setCallParameters(procCtx.create_routine_params());
+    visit(procCtx.single_block_stmt());
+    exec.leaveScope();       
+    return true;
+  }
+  
+  /**
+   * Execute a stored procedure using CALL or EXEC statement passing parameters
+   */
+  public boolean execProc(HplsqlParser.Expr_func_paramsContext ctx, String name) {
+    if (trace) {
+      trace(ctx, "EXEC PROCEDURE " + name);
+    }
+    HplsqlParser.Create_procedure_stmtContext procCtx = procMap.get(name.toUpperCase());    
+    if (procCtx == null) {
+      trace(ctx, "Procedure not found");
+      return false;
+    }    
+    HashMap<String, Var> out = new HashMap<String, Var>();
+    exec.enterScope(Scope.Type.ROUTINE);
+    setCallParameters(ctx, procCtx.create_routine_params(), out);
+    visit(procCtx.single_block_stmt());
+    exec.leaveScope();       
+    for (Map.Entry<String, Var> i : out.entrySet()) {      // Set OUT parameters
+      exec.setVariable(i.getKey(), i.getValue());
+    }
+    return true;
+  }
+  
+  /**
+   * Set parameters for user-defined function call
+   */
+  void setCallParameters(HplsqlParser.Expr_func_paramsContext actual, 
+                         HplsqlParser.Create_routine_paramsContext formal,
+                         HashMap<String, Var> out) {
+    int actualCnt = actual.expr().size();
+    int formalCnt = formal.create_routine_param_item().size();
+    for (int i = 0; i < actualCnt; i++) {
+      if (i >= formalCnt) {
+        break;
+      }
+      HplsqlParser.ExprContext a = actual.expr(i); 
+      HplsqlParser.Create_routine_param_itemContext p = formal.create_routine_param_item(i);
+      String name = p.ident().getText();
+      String type = p.dtype().getText();
+      String len = null;
+      String scale = null;   
+      if (p.dtype_len() != null) {
+        len = p.dtype_len().L_INT(0).getText();
+        if (p.dtype_len().L_INT(1) != null) {
+          scale = p.dtype_len().L_INT(1).getText();
+        }
+      }
+      Var value = evalPop(a);
+      Var var = setCallParameter(name, type, len, scale, value);
+      if (trace) {
+        trace(actual, "SET PARAM " + name + " = " + var.toString());      
+      } 
+      if (out != null && a.expr_atom() != null && a.expr_atom().ident() != null &&
+          (p.T_OUT() != null || p.T_INOUT() != null)) {
+        String actualName = a.expr_atom().ident().getText();
+        if (actualName != null) {
+          out.put(actualName, var);  
+        }         
+      }
+    }
+  }
+  
+  /**
+   * Set parameters for entry-point call (Main procedure defined by -main option)
+   */
+  void setCallParameters(HplsqlParser.Create_routine_paramsContext ctx) {
+    int cnt = ctx.create_routine_param_item().size();
+    for (int i = 0; i < cnt; i++) {
+      HplsqlParser.Create_routine_param_itemContext p = ctx.create_routine_param_item(i);
+      String name = p.ident().getText();
+      String type = p.dtype().getText();
+      String len = null;
+      String scale = null;   
+      if (p.dtype_len() != null) {
+        len = p.dtype_len().L_INT(0).getText();
+        if (p.dtype_len().L_INT(1) != null) {
+          scale = p.dtype_len().L_INT(1).getText();
+        }
+      }
+      Var value = exec.findVariable(name);
+      Var var = setCallParameter(name, type, len, scale, value);
+      if (trace) {
+        trace(ctx, "SET PARAM " + name + " = " + var.toString());      
+      }      
+    }
+  }
+  
+  /**
+   * Create a function or procedure parameter and set its value
+   */
+  Var setCallParameter(String name, String type, String len, String scale, Var value) {
+    Var var = new Var(name, type, len, scale, null);
+    var.cast(value);
+    exec.addVariable(var);    
+    return var;
+  }
+  
+  /**
+   * Add a user-defined function
+   */
+  public void addUserFunction(HplsqlParser.Create_function_stmtContext ctx) {
+    String name = ctx.ident().getText();
+    if (trace) {
+      trace(ctx, "CREATE FUNCTION " + name);
+    }
+    userMap.put(name.toUpperCase(), ctx);
+  }
+  
+  /**
+   * Add a user-defined procedure
+   */
+  public void addUserProcedure(HplsqlParser.Create_procedure_stmtContext ctx) {
+    String name = ctx.ident(0).getText();
+    if (trace) {
+      trace(ctx, "CREATE PROCEDURE " + name);
+    }
+    procMap.put(name.toUpperCase(), ctx);
+  }
+    
+  /**
+   * Execute a special function
+   */
+  public void specExec(HplsqlParser.Expr_spec_funcContext ctx) {
+    String name = ctx.start.getText().toUpperCase();
+    if (trace && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
+      trace(ctx, "FUNC " + name);      
+    }
+    FuncSpecCommand func = specMap.get(name);    
+    if (func != null) {
+      func.run(ctx);
+    }
+    else if(ctx.T_MAX_PART_STRING() != null) {
+      execMaxPartString(ctx);
+    } else if(ctx.T_MIN_PART_STRING() != null) {
+      execMinPartString(ctx);
+    } else if(ctx.T_MAX_PART_INT() != null) {
+      execMaxPartInt(ctx);
+    } else if(ctx.T_MIN_PART_INT() != null) {
+      execMinPartInt(ctx);
+    } else if(ctx.T_MAX_PART_DATE() != null) {
+      execMaxPartDate(ctx);
+    } else if(ctx.T_MIN_PART_DATE() != null) {
+      execMinPartDate(ctx);
+    } else if(ctx.T_PART_LOC() != null) {
+      execPartLoc(ctx);
+    } else if(ctx.T_SYSDATE() != null) {
+      execCurrentTimestamp(ctx, 0);
+    } else {
+      evalNull();
+    }
+  }
+  
+  /**
+   * Execute a special function in executable SQL statement
+   */
+  public void specExecSql(HplsqlParser.Expr_spec_funcContext ctx) {
+    String name = ctx.start.getText().toUpperCase();
+    if (trace && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
+      trace(ctx, "FUNC " + name);      
+    }
+    FuncSpecCommand func = specSqlMap.get(name);    
+    if (func != null) {
+      func.run(ctx);
+    }
+    else {
+      exec.stackPush(exec.getFormattedText(ctx));
+    }
+  }
+  
+  /**
+   * Get the current date
+   */
+  public void execCurrentDate(HplsqlParser.Expr_spec_funcContext ctx) {
+    if(trace) {
+      trace(ctx, "CURRENT_DATE");
+    }
+    SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+    String s = f.format(Calendar.getInstance().getTime());
+    exec.stackPush(new Var(Var.Type.DATE, Utils.toDate(s))); 
+  }
+  
+  /**
+   * Get the current date and time
+   */
+  public void execCurrentTimestamp(HplsqlParser.Expr_spec_funcContext ctx, int defPrecision) {
+    trace(ctx, "CURRENT_TIMESTAMP");
+    int precision = evalPop(ctx.expr(0), defPrecision).intValue();
+    String format = "yyyy-MM-dd HH:mm:ss";
+    if(precision > 0 && precision <= 3) {
+      format += "." + StringUtils.repeat("S", precision);
+    }
+    SimpleDateFormat f = new SimpleDateFormat(format);
+    String s = f.format(Calendar.getInstance(TimeZone.getDefault()).getTime());
+    exec.stackPush(new Var(Utils.toTimestamp(s), precision)); 
+  }
+  
+  /**
+   * Execute MAX_PART_STRING function
+   */
+  public void execMaxPartString(HplsqlParser.Expr_spec_funcContext ctx) {
+    if(trace) {
+      trace(ctx, "MAX_PART_STRING");
+    }
+    execMinMaxPart(ctx, Var.Type.STRING, true /*max*/);
+  }
+  
+  /**
+   * Execute MIN_PART_STRING function
+   */
+  public void execMinPartString(HplsqlParser.Expr_spec_funcContext ctx) {
+    if(trace) {
+      trace(ctx, "MIN_PART_STRING");
+    }
+    execMinMaxPart(ctx, Var.Type.STRING, false /*max*/);
+  }
+
+  /**
+   * Execute MAX_PART_INT function
+   */
+  public void execMaxPartInt(HplsqlParser.Expr_spec_funcContext ctx) {
+    if(trace) {
+      trace(ctx, "MAX_PART_INT");
+    }
+    execMinMaxPart(ctx, Var.Type.BIGINT, true /*max*/);
+  }
+  
+  /**
+   * Execute MIN_PART_INT function
+   */
+  public void execMinPartInt(HplsqlParser.Expr_spec_funcContext ctx) {
+    if(trace) {
+      trace(ctx, "MIN_PART_INT");
+    }
+    execMinMaxPart(ctx, Var.Type.BIGINT, false /*max*/);
+  }
+
+  /**
+   * Execute MAX_PART_DATE function
+   */
+  public void execMaxPartDate(HplsqlParser.Expr_spec_funcContext ctx) {
+    if(trace) {
+      trace(ctx, "MAX_PART_DATE");
+    }
+    execMinMaxPart(ctx, Var.Type.DATE, true /*max*/);
+  }
+  
+  /**
+   * Execute MIN_PART_DATE function
+   */
+  public void execMinPartDate(HplsqlParser.Expr_spec_funcContext ctx) {
+    if(trace) {
+      trace(ctx, "MIN_PART_DATE");
+    }
+    execMinMaxPart(ctx, Var.Type.DATE, false /*max*/);
+  }
+  
+  /**
+   * Execute MIN or MAX partition function
+   */
+  public void execMinMaxPart(HplsqlParser.Expr_spec_funcContext ctx, Var.Type type, boolean max) {
+    String tabname = evalPop(ctx.expr(0)).toString();
+    String sql = "SHOW PARTITIONS " + tabname;    
+    String colname = null;    
+    int colnum = -1;
+    int exprnum = ctx.expr().size();    
+    // Column name 
+    if (ctx.expr(1) != null) {
+      colname = evalPop(ctx.expr(1)).toString();
+    } else {
+      colnum = 0;
+    }
+    // Partition filter
+    if (exprnum >= 4) {
+      sql += " PARTITION (";
+      int i = 2;
+      while (i + 1 < exprnum) {
+        String fcol = evalPop(ctx.expr(i)).toString();
+        String fval = evalPop(ctx.expr(i+1)).toSqlString();
+        if (i > 2) {
+          sql += ", ";
+        }
+        sql += fcol + "=" + fval;        
+        i += 2;
+      }
+      sql += ")";
+    }
+    if (trace) {
+      trace(ctx, "Query: " + sql);
+    }
+    if (exec.getOffline()) {
+      evalNull();
+      return;
+    }
+    Query query = exec.executeQuery(ctx, sql, exec.conf.defaultConnection);
+    if (query.error()) {
+      evalNullClose(query, exec.conf.defaultConnection);
+      return;
+    }
+    ResultSet rs = query.getResultSet();
+    try {
+      String resultString = null;
+      Long resultInt = null;
+      Date resultDate = null;      
+      while (rs.next()) {
+        String[] parts = rs.getString(1).split("/");
+        // Find partition column by name
+        if (colnum == -1) {
+          for (int i = 0; i < parts.length; i++) {
+            String[] name = parts[i].split("=");
+            if (name[0].equalsIgnoreCase(colname)) {
+              colnum = i;
+              break;
+            }
+          }
+          // No partition column with the specified name exists
+          if (colnum == -1) {
+            evalNullClose(query, exec.conf.defaultConnection);
+            return;
+          }
+        }
+        String[] pair = parts[colnum].split("=");
+        if (type == Var.Type.STRING) {
+          resultString = Utils.minMaxString(resultString, pair[1], max);          
+        } 
+        else if (type == Var.Type.BIGINT) {
+          resultInt = Utils.minMaxInt(resultInt, pair[1], max);          
+        } 
+        else if (type == Var.Type.DATE) {
+          resultDate = Utils.minMaxDate(resultDate, pair[1], max);
+        }
+      }
+      if (resultString != null) {
+        evalString(resultString);
+      } 
+      else if (resultInt != null) {
+        evalInt(resultInt);
+      } 
+      else if (resultDate != null) {
+        evalDate(resultDate);
+      } 
+      else {
+        evalNull();
+      }
+    } catch (SQLException e) {}  
+    exec.closeQuery(query, exec.conf.defaultConnection);
+  }
+  
+  /**
+   * Execute PART_LOC function
+   */
+  public void execPartLoc(HplsqlParser.Expr_spec_funcContext ctx) {
+    String tabname = evalPop(ctx.expr(0)).toString();
+    String sql = "DESCRIBE EXTENDED " + tabname;    
+    int exprnum = ctx.expr().size();   
+    boolean hostname = false;
+    // Partition filter
+    if (exprnum > 1) {
+      sql += " PARTITION (";
+      int i = 1;
+      while (i + 1 < exprnum) {
+        String col = evalPop(ctx.expr(i)).toString();
+        String val = evalPop(ctx.expr(i+1)).toSqlString();
+        if (i > 2) {
+          sql += ", ";
+        }
+        sql += col + "=" + val;        
+        i += 2;
+      }
+      sql += ")";
+    }
+    // With host name
+    if (exprnum % 2 == 0 && evalPop(ctx.expr(exprnum - 1)).intValue() == 1) {
+      hostname = true;
+    }
+    if (trace) {
+      trace(ctx, "Query: " + sql);
+    }
+    if (exec.getOffline()) {
+      evalNull();
+      return;
+    }
+    Query query = exec.executeQuery(ctx, sql, exec.conf.defaultConnection);
+    if (query.error()) {
+      evalNullClose(query, exec.conf.defaultConnection);
+      return;
+    }
+    String result = null;
+    ResultSet rs = query.getResultSet();
+    try {
+      while (rs.next()) {
+        if (rs.getString(1).startsWith("Detailed Partition Information")) {
+          Matcher m = Pattern.compile(".*, location:(.*?),.*").matcher(rs.getString(2));
+          if (m.find()) {
+            result = m.group(1);
+          }    
+        }
+      }
+    } catch (SQLException e) {}  
+    if (result != null) {
+      // Remove the host name
+      if (!hostname) {
+        Matcher m = Pattern.compile(".*://.*?(/.*)").matcher(result); 
+        if (m.find()) {
+          result = m.group(1);
+        }
+      }
+      evalString(result);
+    }    
+    else {
+      evalNull();
+    }
+    exec.closeQuery(query, exec.conf.defaultConnection);
+  }
+  
+  /**
+   * Evaluate the expression and push the value to the stack
+   */
+  void eval(ParserRuleContext ctx) {
+    exec.visit(ctx);
+  }
+
+  /**
+   * Evaluate the expression to the specified variable
+   */
+  void evalVar(Var var) {
+    exec.stackPush(var); 
+  }
+
+  /**
+   * Evaluate the expression to NULL
+   */
+  void evalNull() {
+    exec.stackPush(Var.Null); 
+  }
+  
+  /**
+   * 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 to specified Int value
+   */
+  void evalInt(Long i) {
+    exec.stackPush(new Var(i)); 
+  }
+  
+  /**
+   * Evaluate the expression to specified Date value
+   */
+  void evalDate(Date date) {
+    exec.stackPush(new Var(Var.Type.DATE, date)); 
+  }
+  
+  /**
+   * Evaluate the expression to NULL and close the query
+   */
+  void evalNullClose(Query query, String conn) {
+    exec.stackPush(Var.Null); 
+    exec.closeQuery(query, conn);
+    if(trace) {
+      query.printStackTrace();
+    }
+  }
+  
+  /**
+   * Evaluate the expression and pop value from the stack
+   */
+  Var evalPop(ParserRuleContext ctx) {
+    exec.visit(ctx);
+    return exec.stackPop();  
+  }
+  
+  Var evalPop(ParserRuleContext ctx, int value) {
+    if (ctx != null) {
+      return evalPop(ctx);
+    }
+    return new Var(new Long(value));
+  }
+  
+  /**
+   * Execute rules
+   */
+  Integer visit(ParserRuleContext ctx) {
+    return exec.visit(ctx);  
+  } 
+ 
+  /**
+   * Execute children rules
+   */
+  Integer visitChildren(ParserRuleContext ctx) {
+    return exec.visitChildren(ctx);  
+  }  
+  
+  /**
+   * Trace information
+   */
+  public void trace(ParserRuleContext ctx, String message) {
+    if (trace) {
+      exec.trace(ctx, message);
+    }
+  }
+  
+  public void trace(String message) {
+    trace(null, message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java
new file mode 100644
index 0000000..926eeeb
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java
@@ -0,0 +1,151 @@
+/**
+ * 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.functions;
+
+import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.TimeZone;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hive.hplsql.*;
+
+public class FunctionDatetime extends Function {
+  public FunctionDatetime(Exec e) {
+    super(e);
+  }
+
+  /** 
+   * Register functions
+   */
+  @Override
+  public void register(Function f) {
+    f.map.put("DATE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { date(ctx); }});
+    f.map.put("TIMESTAMP_ISO", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { timestampIso(ctx); }});
+    f.map.put("TO_TIMESTAMP", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { toTimestamp(ctx); }});
+  
+    f.specMap.put("CURRENT_DATE", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentDate(ctx); }});
+    f.specMap.put("CURRENT_TIMESTAMP", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentTimestamp(ctx); }});
+
+    f.specSqlMap.put("CURRENT_DATE", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentDateSql(ctx); }});
+    f.specSqlMap.put("CURRENT_TIMESTAMP", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentTimestampSql(ctx); }});
+ }
+  
+  /**
+   * CURRENT_DATE
+   */
+  public void currentDate(HplsqlParser.Expr_spec_funcContext ctx) {
+    evalVar(currentDate());
+  }
+  
+  public static Var currentDate() {
+    SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+    String s = f.format(Calendar.getInstance().getTime());
+    return new Var(Var.Type.DATE, Utils.toDate(s)); 
+  }
+  
+  /**
+   * CURRENT_DATE in executable SQL statement
+   */
+  public void currentDateSql(HplsqlParser.Expr_spec_funcContext ctx) {
+    if (exec.getConnectionType() == Conn.Type.HIVE) {
+      evalString("TO_DATE(FROM_UNIXTIME(UNIX_TIMESTAMP()))");
+    } 
+    else {
+      evalString(exec.getFormattedText(ctx));
+    }
+  }
+  
+  /**
+   * CURRENT_TIMESTAMP
+   */
+  public void currentTimestamp(HplsqlParser.Expr_spec_funcContext ctx) {
+    int precision = evalPop(ctx.expr(0), 3).intValue();
+    evalVar(currentTimestamp(precision));
+  }
+  
+  public static Var currentTimestamp(int precision) {
+    String format = "yyyy-MM-dd HH:mm:ss";
+    if (precision > 0 && precision <= 3) {
+      format += "." + StringUtils.repeat("S", precision);
+    }
+    SimpleDateFormat f = new SimpleDateFormat(format);
+    String s = f.format(Calendar.getInstance(TimeZone.getDefault()).getTime());
+    return new Var(Utils.toTimestamp(s), precision); 
+  }
+  
+  /**
+   * CURRENT_TIMESTAMP in executable SQL statement
+   */
+  public void currentTimestampSql(HplsqlParser.Expr_spec_funcContext ctx) {
+    if (exec.getConnectionType() == Conn.Type.HIVE) {
+      evalString("FROM_UNIXTIME(UNIX_TIMESTAMP())");
+    } 
+    else {
+      evalString(exec.getFormattedText(ctx));
+    }
+  }
+  
+  /**
+   * DATE function
+   */
+  void date(HplsqlParser.Expr_func_paramsContext ctx) {
+    if (ctx.expr().size() != 1) {
+      evalNull();
+      return;
+    }
+    Var var = new Var(Var.Type.DATE);
+    var.cast(evalPop(ctx.expr(0)));
+    evalVar(var);
+  }
+
+  /**
+   * TIMESTAMP_ISO function
+   */
+  void timestampIso(HplsqlParser.Expr_func_paramsContext ctx) {
+    if (ctx.expr().size() != 1) {
+      evalNull();
+      return;
+    }
+    Var var = new Var(Var.Type.TIMESTAMP);
+    var.cast(evalPop(ctx.expr(0)));
+    evalVar(var);
+  }
+  
+  /**
+   * TO_TIMESTAMP function
+   */
+  void toTimestamp(HplsqlParser.Expr_func_paramsContext ctx) {
+    if (ctx.expr().size() != 2) {
+      evalNull();
+      return;
+    }    
+    String value = evalPop(ctx.expr(0)).toString();
+    String sqlFormat = evalPop(ctx.expr(1)).toString();
+    String format = Utils.convertSqlDatetimeFormat(sqlFormat);
+    try {
+      long timeInMs = new SimpleDateFormat(format).parse(value).getTime();
+      evalVar(new Var(Var.Type.TIMESTAMP, new Timestamp(timeInMs)));
+    }
+    catch (Exception e) {
+      exec.signal(e);
+      evalNull();
+    }
+  }
+}  

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java
new file mode 100644
index 0000000..8d2456c
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java
@@ -0,0 +1,188 @@
+/**
+ * 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.functions;
+
+import org.apache.hive.hplsql.*;
+
+public class FunctionMisc extends Function {
+  public FunctionMisc(Exec e) {
+    super(e);
+  }
+
+  /** 
+   * Register functions
+   */
+  @Override
+  public void register(Function f) {
+    f.map.put("COALESCE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { nvl(ctx); }});
+    f.map.put("DECODE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { decode(ctx); }});
+    f.map.put("NVL", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { nvl(ctx); }});
+    f.map.put("NVL2", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { nvl2(ctx); }});
+    
+    f.specMap.put("ACTIVITY_COUNT", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { activityCount(ctx); }});
+    f.specMap.put("CAST", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { cast(ctx); }});
+    f.specMap.put("CURRENT", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { current(ctx); }});
+    f.specMap.put("CURRENT_USER", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentUser(ctx); }});
+    f.specMap.put("USER", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentUser(ctx); }});
+
+    f.specSqlMap.put("CURRENT", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentSql(ctx); }});
+  }
+  
+  /**
+   * ACTIVITY_COUNT function (built-in variable)
+   */
+  void activityCount(HplsqlParser.Expr_spec_funcContext ctx) {
+    evalInt(new Long(exec.getRowCount()));
+  }
+  
+  /**
+   * CAST function
+   */
+  void cast(HplsqlParser.Expr_spec_funcContext ctx) {
+    if (ctx.expr().size() != 1) {
+      evalNull();
+      return;
+    }
+    String type = ctx.dtype().getText();
+    String len = null;
+    String scale = null;
+    if (ctx.dtype_len() != null) {
+      len = ctx.dtype_len().L_INT(0).getText();
+      if (ctx.dtype_len().L_INT(1) != null) {
+        scale = ctx.dtype_len().L_INT(1).getText();
+      }
+    }    
+    Var var = new Var(null, type, len, scale, null);
+    var.cast(evalPop(ctx.expr(0)));
+    evalVar(var);
+  }
+  
+  /**
+   * CURRENT <VALUE> function
+   */
+  void current(HplsqlParser.Expr_spec_funcContext ctx) {
+    if (ctx.T_DATE() != null) {
+      evalVar(FunctionDatetime.currentDate()); 
+    }
+    else if (ctx.T_TIMESTAMP() != null) {
+      int precision = evalPop(ctx.expr(0), 3).intValue();
+      evalVar(FunctionDatetime.currentTimestamp(precision)); 
+    }
+    else if (ctx.T_USER() != null) {
+      evalVar(FunctionMisc.currentUser());
+    }
+    else {
+      evalNull();
+    }
+  }
+  
+  /**
+   * CURRENT <VALUE> function in executable SQL statement
+   */
+  void currentSql(HplsqlParser.Expr_spec_funcContext ctx) {
+    if (ctx.T_DATE() != null) {
+      if (exec.getConnectionType() == Conn.Type.HIVE) {
+        evalString("TO_DATE(FROM_UNIXTIME(UNIX_TIMESTAMP()))");
+      } 
+      else {
+        evalString("CURRENT_DATE");
+      }
+    }
+    else if (ctx.T_TIMESTAMP() != null) {
+      if (exec.getConnectionType() == Conn.Type.HIVE) {
+        evalString("FROM_UNIXTIME(UNIX_TIMESTAMP())");
+      } 
+      else {
+        evalString("CURRENT_TIMESTAMP");
+      }
+    }
+    else {
+      evalString(exec.getFormattedText(ctx));
+    }
+  }
+  
+  /**
+   * CURRENT_USER function
+   */
+  void currentUser(HplsqlParser.Expr_spec_funcContext ctx) {
+    evalVar(currentUser());
+  }
+  
+  public static Var currentUser() {
+    return new Var(System.getProperty("user.name"));
+  }
+  
+  /**
+   * DECODE function
+   */
+  void decode(HplsqlParser.Expr_func_paramsContext ctx) {
+    int cnt = ctx.expr().size();
+    if (cnt < 3) {
+      evalNull();
+      return;
+    }
+    Var value = evalPop(ctx.expr(0));
+    int i = 1;
+    while (i + 1 < cnt) {
+      Var when = evalPop(ctx.expr(i));
+      if ((value.isNull() && when.isNull()) || value.equals(when)) {
+        eval(ctx.expr(i + 1));
+        return;
+      }
+      i += 2;
+    }    
+    if (i < cnt) {           // ELSE expression
+      eval(ctx.expr(i));
+    }
+    else {
+      evalNull();
+    }
+  }
+  
+  /**
+   * NVL function - Return first non-NULL expression
+   */
+  void nvl(HplsqlParser.Expr_func_paramsContext ctx) {
+    for (int i=0; i < ctx.expr().size(); i++) {
+      Var v = evalPop(ctx.expr(i));
+      if (v.type != Var.Type.NULL) {
+        exec.stackPush(v);
+        return;
+      }
+    }
+    evalNull();
+  }
+  
+  /**
+   * NVL2 function - If expr1 is not NULL return expr2, otherwise expr3
+   */
+  void nvl2(HplsqlParser.Expr_func_paramsContext ctx) {
+    if (ctx.expr().size() == 3) {
+      if (!evalPop(ctx.expr(0)).isNull()) {
+        eval(ctx.expr(1));
+      }
+      else {
+        eval(ctx.expr(2));
+      }
+    }
+    else {
+      evalNull();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionOra.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionOra.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionOra.java
new file mode 100644
index 0000000..fec891a
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionOra.java
@@ -0,0 +1,231 @@
+/**
+ * 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.functions;
+
+import java.io.IOException;
+import java.io.EOFException;
+
+import org.apache.hive.hplsql.*;
+
+public class FunctionOra extends Function {
+  public FunctionOra(Exec e) {
+    super(e);
+  }
+
+  /** 
+   * Register functions
+   */
+  @Override
+  public void register(Function f) {  
+    f.map.put("DBMS_OUTPUT.PUT_LINE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { 
+      execDbmsOutputPutLine(ctx); }});
+    f.map.put("UTL_FILE.FOPEN", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { 
+      execUtlFileFopen(ctx); }});
+    f.map.put("UTL_FILE.GET_LINE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { 
+      execUtlFileGetLine(ctx); }});
+    f.map.put("UTL_FILE.PUT_LINE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { 
+      execUtlFilePutLine(ctx); }});
+    f.map.put("UTL_FILE.PUT", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { 
+      execUtlFilePut(ctx); }});
+    f.map.put("UTL_FILE.FCLOSE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { 
+      execUtlFileFclose(ctx); }});
+  }
+  
+  /**
+   * Print a text message
+   */
+  void execDbmsOutputPutLine(HplsqlParser.Expr_func_paramsContext ctx) {
+    if (ctx.expr().size() > 0) {
+      visit(ctx.expr(0));
+      System.out.println(exec.stackPop().toString());
+    }
+  }
+  
+  /**
+   * Execute UTL_FILE.FOPEN function
+   */
+  public void execUtlFileFopen(HplsqlParser.Expr_func_paramsContext ctx) {
+    String dir = "";
+    String name = "";
+    boolean write = true;
+    boolean overwrite = false;
+    int cnt = ctx.expr().size();    
+    // Directory
+    if (cnt > 0) {
+      dir = evalPop(ctx.expr(0)).toString();
+    }    
+    // File name
+    if (cnt > 1) {
+      name = evalPop(ctx.expr(1)).toString();
+    }    
+    // Mode
+    if (cnt >= 2) {
+      String mode = evalPop(ctx.expr(2)).toString();
+      if (mode.equalsIgnoreCase("r")) {
+        write = false;
+      }
+      else if (mode.equalsIgnoreCase("w")) {
+        write = true;
+        overwrite = true;
+      }
+    }    
+    File file = new File();    
+    if (write) {
+      file.create(dir, name, overwrite);
+    }
+    else {
+      file.open(dir, name);
+    }        
+    exec.stackPush(new Var(Var.Type.FILE, file));
+  }
+
+  /**
+   * Read a text line from an open file
+   */
+  void execUtlFileGetLine(HplsqlParser.Expr_func_paramsContext ctx) {
+    int cnt = ctx.expr().size();
+    Var file = null;
+    Var str = null;
+    StringBuilder out = new StringBuilder();
+    
+    // File handle
+    if(cnt > 0) {
+      visit(ctx.expr(0));
+      file = exec.stackPop();
+    }    
+    // String variable
+    if(cnt > 1) {
+      visit(ctx.expr(1));
+      str = exec.stackPop();
+    }
+    
+    if(file != null && file.type == Var.Type.FILE) {
+      File f = (File)file.value;
+      
+      if(trace) {
+        trace(ctx, "File: " + f.toString());      
+      }
+      
+      try {
+        while(true) {
+          char c = f.readChar();
+          if(c == '\n') {
+            break;
+          }
+          out.append(c);
+        }        
+      } catch (IOException e) {
+        if(!(e instanceof EOFException)) {
+          out.setLength(0);
+        }
+      }
+      
+      // Set the new value to the output string variable
+      if(str != null) {
+        str.setValue(out.toString());
+        
+        if(trace) {
+          trace(ctx, "OUT " + str.getName() + " = " + str.toString());      
+        }
+      }
+    }
+    else if(trace) {
+      trace(ctx, "Variable of FILE type not found");      
+    }
+  }
+  
+  /**
+   * Execute UTL_FILE.PUT_LINE function
+   */
+  public void execUtlFilePutLine(HplsqlParser.Expr_func_paramsContext ctx) {
+    execUtlFilePut(ctx, true /*newline*/);  
+  }
+
+  /**
+   * Execute UTL_FILE.PUT function
+   */
+  public void execUtlFilePut(HplsqlParser.Expr_func_paramsContext ctx) {
+    execUtlFilePut(ctx, false /*newline*/);  
+  }
+  
+  /**
+   * Write a string to file
+   */
+  void execUtlFilePut(HplsqlParser.Expr_func_paramsContext ctx, boolean newline) {
+    int cnt = ctx.expr().size();
+    Var file = null;
+    String str = "";
+    
+    // File handle
+    if(cnt > 0) {
+      visit(ctx.expr(0));
+      file = exec.stackPop();
+    }    
+    // Text string
+    if(cnt > 1) {
+      visit(ctx.expr(1));
+      str = exec.stackPop().toString();
+    }
+    
+    if(file != null && file.type == Var.Type.FILE) {
+      File f = (File)file.value;
+      
+      if(trace) {
+        trace(ctx, "File: " + f.toString());      
+      }
+      
+      f.writeString(str); 
+      
+      if(newline) {
+        f.writeString("\n");
+      }
+    }
+    else if(trace) {
+      trace(ctx, "Variable of FILE type not found");      
+    }
+  }
+  
+  /**
+   * Execute UTL_FILE.FCLOSE function
+   */
+  void execUtlFileFclose(HplsqlParser.Expr_func_paramsContext ctx) {
+    int cnt = ctx.expr().size();
+    Var file = null;
+    
+    // File handle
+    if(cnt > 0) {
+      visit(ctx.expr(0));
+      file = exec.stackPop();
+    }    
+        
+    if(file != null && file.type == Var.Type.FILE) {
+      File f = (File)file.value;
+      
+      if(trace) {
+        trace(ctx, "File: " + f.toString());      
+      }
+      
+      f.close();      
+      file.removeValue();
+    }
+    else if(trace) {
+      trace(ctx, "Variable of FILE type not found");      
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java
new file mode 100644
index 0000000..610ff60
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java
@@ -0,0 +1,276 @@
+/**
+ * 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.functions;
+
+import org.apache.hive.hplsql.*;
+
+public class FunctionString extends Function {
+  public FunctionString(Exec e) {
+    super(e);
+  }
+
+  /** 
+   * Register functions
+   */
+  @Override
+  public void register(Function f) {
+    f.map.put("CONCAT", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { concat(ctx); }});
+    f.map.put("CHAR", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { char_(ctx); }});
+    f.map.put("INSTR", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { instr(ctx); }});
+    f.map.put("LEN", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { len(ctx); }});
+    f.map.put("LENGTH", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { length(ctx); }});
+    f.map.put("LOWER", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { lower(ctx); }});
+    f.map.put("SUBSTR", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { substr(ctx); }});    
+    f.map.put("SUBSTRING", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { substr(ctx); }});
+    f.map.put("TO_CHAR", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { toChar(ctx); }});
+    f.map.put("UPPER", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { upper(ctx); }});
+    
+    f.specMap.put("SUBSTRING", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { substring(ctx); }});
+    f.specMap.put("TRIM", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { trim(ctx); }});
+  }
+  
+  /**
+   * CONCAT function
+   */
+  void concat(HplsqlParser.Expr_func_paramsContext ctx) {
+    StringBuilder val = new StringBuilder();
+    int cnt = ctx.expr().size();
+    boolean nulls = true;
+    for (int i = 0; i < cnt; i++) {
+      Var c = evalPop(ctx.expr(i));
+      if (!c.isNull()) {
+        val.append(c.toString());
+        nulls = false;
+      }
+    }
+    if (nulls) {
+      evalNull();
+    }
+    else {
+      evalString(val);
+    }
+  }
+  
+  /**
+   * CHAR function
+   */
+  void char_(HplsqlParser.Expr_func_paramsContext ctx) {
+    int cnt = ctx.expr().size();
+    if (cnt != 1) {
+      evalNull();
+      return;
+    }
+    String str = evalPop(ctx.expr(0)).toString(); 
+    evalString(str);
+  }
+  
+  /**
+   * INSTR function
+   */
+  void instr(HplsqlParser.Expr_func_paramsContext ctx) {
+    int cnt = ctx.expr().size();
+    if (cnt < 2) {
+      evalNull();
+      return;
+    }
+    String str = evalPop(ctx.expr(0)).toString();
+    if (str == null) {
+      evalNull();
+      return;
+    }
+    else if(str.isEmpty()) {
+      evalInt(new Long(0));
+      return;
+    }
+    String substr = evalPop(ctx.expr(1)).toString();
+    int pos = 1;
+    int occur = 1;
+    int idx = 0;
+    if (cnt >= 3) {
+      pos = evalPop(ctx.expr(2)).intValue();
+      if (pos == 0) {
+        pos = 1;
+      }
+    }
+    if (cnt >= 4) {
+      occur = evalPop(ctx.expr(3)).intValue();
+      if (occur < 0) {
+        occur = 1;
+      }
+    }
+    for (int i = occur; i > 0; i--) {
+      if (pos > 0) {
+        idx = str.indexOf(substr, pos - 1);
+      }
+      else {
+        str = str.substring(0, str.length() - pos*(-1));
+        idx = str.lastIndexOf(substr);
+      }
+      if (idx == -1) {
+        idx = 0;
+        break;
+      }
+      else {
+        idx++;
+      }
+      if (i > 1) {
+        if (pos > 0) {
+          pos = idx + 1;
+        }
+        else {
+          pos = (str.length() - idx + 1) * (-1);
+        }
+      }
+    }
+    evalInt(new Long(idx));
+  }
+  
+  /**
+   * LEN function (excluding trailing spaces)
+   */
+  void len(HplsqlParser.Expr_func_paramsContext ctx) {
+    if (ctx.expr().size() != 1) {
+      evalNull();
+      return;
+    }
+    int len = evalPop(ctx.expr(0)).toString().trim().length(); 
+    evalInt(new Long(len));
+  }
+  
+  /**
+   * LENGTH function
+   */
+  void length(HplsqlParser.Expr_func_paramsContext ctx) {
+    if (ctx.expr().size() != 1) {
+      evalNull();
+      return;
+    }
+    int len = evalPop(ctx.expr(0)).toString().length(); 
+    evalInt(new Long(len));
+  }
+  
+  /**
+   * LOWER function
+   */
+  void lower(HplsqlParser.Expr_func_paramsContext ctx) {
+    if (ctx.expr().size() != 1) {
+      evalNull();
+      return;
+    }
+    String str = evalPop(ctx.expr(0)).toString().toLowerCase(); 
+    evalString(str);
+  }
+  
+  /**
+   * SUBSTR and SUBSTRING function
+   */
+  void substr(HplsqlParser.Expr_func_paramsContext ctx) {
+    int cnt = ctx.expr().size();
+    if (cnt < 2) {
+      evalNull();
+      return;
+    }
+    String str = evalPop(ctx.expr(0)).toString(); 
+    int start = evalPop(ctx.expr(1)).intValue();
+    int len = -1;
+    if (start == 0) {
+      start = 1; 
+    }
+    if (cnt > 2) {
+      len = evalPop(ctx.expr(2)).intValue();
+    }
+    substr(str, start, len);
+  }
+  
+  void substr(String str, int start, int len) {
+    if (str == null) {
+      evalNull();
+      return;
+    }
+    else if (str.isEmpty()) {
+      evalString(str);
+      return;
+    }
+    if (start == 0) {
+      start = 1; 
+    }
+    if (len == -1) {
+      if (start > 0) {
+        evalString(str.substring(start - 1));
+      }
+    }
+    else {
+      evalString(str.substring(start - 1, start - 1 + len));      
+    }
+  }
+  
+  /**
+   * SUBSTRING FROM FOR function
+   */
+  void substring(HplsqlParser.Expr_spec_funcContext ctx) {
+    String str = evalPop(ctx.expr(0)).toString(); 
+    int start = evalPop(ctx.expr(1)).intValue();
+    int len = -1;
+    if (start == 0) {
+      start = 1; 
+    }
+    if (ctx.T_FOR() != null) {
+      len = evalPop(ctx.expr(2)).intValue();
+    }
+    substr(str, start, len);
+  }
+  
+  /**
+   * TRIM function
+   */
+  void trim(HplsqlParser.Expr_spec_funcContext ctx) {
+    int cnt = ctx.expr().size();
+    if (cnt != 1) {
+      evalNull();
+      return;
+    }
+    String str = evalPop(ctx.expr(0)).toString(); 
+    evalString(str.trim());
+  }
+  
+  /**
+   * TO_CHAR function
+   */
+  void toChar(HplsqlParser.Expr_func_paramsContext ctx) {
+    int cnt = ctx.expr().size();
+    if (cnt != 1) {
+      evalNull();
+      return;
+    }
+    String str = evalPop(ctx.expr(0)).toString(); 
+    evalString(str);
+  }
+  
+  /**
+   * UPPER function
+   */
+  void upper(HplsqlParser.Expr_func_paramsContext ctx) {
+    if (ctx.expr().size() != 1) {
+      evalNull();
+      return;
+    }
+    String str = evalPop(ctx.expr(0)).toString().toUpperCase(); 
+    evalString(str);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1c9b02f..f84f3e9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -39,6 +39,7 @@
     <module>contrib</module>
     <module>hbase-handler</module>
     <module>hcatalog</module>
+    <module>hplsql</module>
     <module>hwi</module>
     <module>jdbc</module>
     <module>metastore</module>