You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2020/09/30 16:38:09 UTC

[GitHub] [hive] zeroflag opened a new pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

zeroflag opened a new pull request #1542:
URL: https://github.com/apache/hive/pull/1542


   work in progress


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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512060372



##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java
##########
@@ -36,20 +36,20 @@ public FunctionDatetime(Exec 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("FROM_UNIXTIME", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { fromUnixtime(ctx); }});
-    f.map.put("NOW", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { now(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.map.put("UNIX_TIMESTAMP", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { unixTimestamp(ctx); }});
+  public void register(BuiltinFunctions f) {
+    f.map.put("DATE", this::date);

Review comment:
       This is a general issue, many things are package private. We can refactor that little by little.
   
   
   .




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r498877894



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>
+      </field>
+      <field name="language">
+        <column name="LANG" jdbc-type="varchar" length="128" allows-null="false"/>
+      </field>
+      <field name="returnType">
+        <column name="RET_TYPE" jdbc-type="varchar" length="128" allows-null="true"/>
+      </field>
+      <field name="parameters" table="SP_POS_ARGS" >
+        <collection element-type="MPosParam"/>
+        <join>
+            <column name="SP_ID"/>
+        </join>
+        <element>
+          <embedded>
+            <field name="name">
+              <column name="NAME" jdbc-type="varchar" length="256" allows-null="false"/>
+            </field>
+            <field name="type">
+              <column name="TYPE" jdbc-type="varchar" length="128" allows-null="false"/>
+            </field>
+            <field name="length">
+              <column name="length" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="scale">
+              <column name="scale" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="isOut">
+              <column name="OUT" allows-null="false"/>
+            </field>
+          </embedded>
+        </element>
+      </field>

Review comment:
       The signature is already parsed by the time the procedure is being created. We would need to drop that information, get back the textual representation of the signature to store it in HMS, and reparse it on the client side when someone calls the procedure. That's maybe not a big deal but still unnecessary to parse it twice. Storing it in a structured way also ensures some degree of validity, you can't store a syntactically incorrect signature if we store it in a structured way.
   
   I'm not sure if they never participate in a query. If one wants to discover the stored procedures which are currently stored in a DB and find out on what data they operate they would need to do some clumsy string manipulations on the signature.
   
   Considering that other DB engines also store these information separately I would like to keep it as it is for now and see how it works in practice. Later on when we have multi language support we can revisit this issue. 




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r498868959



##########
File path: standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
##########
@@ -2830,6 +2848,11 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   void add_replication_metrics(1: ReplicationMetricList replicationMetricList) throws(1:MetaException o1)
   ReplicationMetricList get_replication_metrics(1: GetReplicationMetricsRequest rqst) throws(1:MetaException o1)
   GetOpenTxnsResponse get_open_txns_req(1: GetOpenTxnsRequest getOpenTxnsRequest)
+
+  void create_stored_procedure(1: string catName, 2: StoredProcedure proc) throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  StoredProcedure get_stored_procedure(1: string catName, 2: string db, 3: string name) throws (1:MetaException o1, 2:NoSuchObjectException o2)
+  void drop_stored_procedure(1: string catName, 2: string dbName, 3: string funcName) throws (1:MetaException o1, 2:NoSuchObjectException o2)
+  list<StoredProcedure> get_all_stored_procedures(1: string catName) throws (1:MetaException o1)

Review comment:
       You mean putting (1: string catName, 2: string dbName, 3: string funcName) into a request object? I can do that. But if we have only can parameter, like in the last case that would be an overkill in my opinion.




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r507534867



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>
+      </field>
+      <field name="language">
+        <column name="LANG" jdbc-type="varchar" length="128" allows-null="false"/>
+      </field>
+      <field name="returnType">
+        <column name="RET_TYPE" jdbc-type="varchar" length="128" allows-null="true"/>
+      </field>
+      <field name="parameters" table="SP_POS_ARGS" >
+        <collection element-type="MPosParam"/>
+        <join>
+            <column name="SP_ID"/>
+        </join>
+        <element>
+          <embedded>
+            <field name="name">
+              <column name="NAME" jdbc-type="varchar" length="256" allows-null="false"/>
+            </field>
+            <field name="type">
+              <column name="TYPE" jdbc-type="varchar" length="128" allows-null="false"/>
+            </field>
+            <field name="length">
+              <column name="length" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="scale">
+              <column name="scale" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="isOut">
+              <column name="OUT" allows-null="false"/>
+            </field>
+          </embedded>
+        </element>
+      </field>

Review comment:
       > That was a suggestion to provide a way to store a human readable..
   
   I understand it is only for showing the user, but it it feels like an afterthought, I don't really like it. It adds redundancy and it only has a single use. The structured information can be used to generate this output, (or in fact other outputs, like showing the parameters in a table) or to supply meta programming information to various tools, like code completion tools or method finders.
   
   For example, search for the return type:
   
   ```
   int i = fn.. // find every function that starts with "fn" and returns an int
   ```
   
   Although this is a search for the return type not on the parameter but it's the same problem since the return type is also part of the signature.
   
   I we had higher order functions (in fact JavaScript has and that might be the 2nd supported language) then:
   
   ```
   filter(fn.., listOfString); // find every function that starts with "fn" and takes a single string parameter
   ```
   Method finder:
   
   This might look like a bit scifi, there are only 2 programming environments I'm aware of, which know how to do this.
   
   ```
   MethodFinder.methodFor('somefile.txt', 'txt'); // find which method returns the extension part of filename, by an by example,
   ```
   
   It will return:
   ```
   "FilenameUtils.getExtension()"
   ```
   
   These things might sound unimportant but I think language design and tool development shouldn't be separated. Tool support should be considered from day 1 when creating a language.
   
   
   > If at some point in time the "parsing" will prove to be a bottle neck
   
   I'm was not talking about parsing as a bottleneck (though it could be) but interpreting the AST (method body) as a bottleneck. I can't think of any imperative language that can be taken seriously that works that way. Perhaps shell scripts, or early versions of Ruby which was notoriously slow so later they changed it.
   
   > I think we should clarify/separate 2 things
   
   If we only allow defining procedures in terms of the "host language", then this is true indeed. My assumption was that we might want to accept procedure definitions in terms of the "foreign language". For example function(x,y) {}. But ok, let's say this is not allowed. Then you're right, using the alternative format seems to be viable if we don't count the other issues.
   
   If we go to that route, what would you do with the other columns which are also part of the signature, like "LANG", "RET_TYPE", "NAME" ?
   
   ```
   CREATE FUNCTION func1(a int) RETURNS int LANGUAGE XX BEGIN ... END;
   ```
   
   > I don't know what feature you are refering 
   
   It's probably that one. I think that illustrates nicely why parsing is not always a prerequisite of calling a procedure. 
   




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512060986



##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -1659,13 +1665,70 @@ public Integer visitExpr_func(HplsqlParser.Expr_funcContext ctx) {
     }
     return 0;
   }
-  
+
+  /**
+   * 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("(");
+    if (ctx != null) {
+      int cnt = ctx.func_param().size();
+      for (int i = 0; i < cnt; i++) {
+        sql.append(evalPop(ctx.func_param(i).expr()));
+        if (i + 1 < cnt) {
+          sql.append(", ");
+        }
+      }
+    }
+    sql.append(")");
+    exec.stackPush(sql);
+  }
+
+  /**
+   * Execute a HPL/SQL user-defined function in a query
+   */
+  private boolean execUserSql(HplsqlParser.Expr_func_paramsContext ctx, String name) {
+    if (!function.exists(name.toUpperCase())) {
+      return false;
+    }
+    StringBuilder sql = new StringBuilder();
+    sql.append("hplsql('");

Review comment:
       I didn't check since this is not new code, and fixing every issue in the existing code was not the scope of this issue.




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512060541



##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/HmsFunction.java
##########
@@ -0,0 +1,232 @@
+/*
+ *
+ *  * 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 static org.apache.hive.hplsql.functions.InMemoryFunction.setCallParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.StoredProcedure;
+import org.apache.hadoop.hive.metastore.api.StoredProcedureRequest;
+import org.apache.hive.hplsql.Exec;
+import org.apache.hive.hplsql.HplsqlBaseVisitor;
+import org.apache.hive.hplsql.HplsqlLexer;
+import org.apache.hive.hplsql.HplsqlParser;
+import org.apache.hive.hplsql.Scope;
+import org.apache.hive.hplsql.Var;
+import org.apache.thrift.TException;
+
+public class HmsFunction implements Function {

Review comment:
       I agree, but this is how it used to work, and I didn't want to address these things as part of this patch. 




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r508638143



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>
+      </field>
+      <field name="language">
+        <column name="LANG" jdbc-type="varchar" length="128" allows-null="false"/>
+      </field>
+      <field name="returnType">
+        <column name="RET_TYPE" jdbc-type="varchar" length="128" allows-null="true"/>
+      </field>
+      <field name="parameters" table="SP_POS_ARGS" >
+        <collection element-type="MPosParam"/>
+        <join>
+            <column name="SP_ID"/>
+        </join>
+        <element>
+          <embedded>
+            <field name="name">
+              <column name="NAME" jdbc-type="varchar" length="256" allows-null="false"/>
+            </field>
+            <field name="type">
+              <column name="TYPE" jdbc-type="varchar" length="128" allows-null="false"/>
+            </field>
+            <field name="length">
+              <column name="length" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="scale">
+              <column name="scale" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="isOut">
+              <column name="OUT" allows-null="false"/>
+            </field>
+          </embedded>
+        </element>
+      </field>

Review comment:
       Discussed this further offline, let's try the string based approach for now and see how it goes. I'll modify the patch.




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r498878756



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>

Review comment:
       I'll double check it, I remember having some problem with another textual datatype, I can't remember which one was that, that's why MEDIUMTEXT was chosen.




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

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



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


[GitHub] [hive] kgyrtkirk commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
kgyrtkirk commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r507634498



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>
+      </field>
+      <field name="language">
+        <column name="LANG" jdbc-type="varchar" length="128" allows-null="false"/>
+      </field>
+      <field name="returnType">
+        <column name="RET_TYPE" jdbc-type="varchar" length="128" allows-null="true"/>
+      </field>
+      <field name="parameters" table="SP_POS_ARGS" >
+        <collection element-type="MPosParam"/>
+        <join>
+            <column name="SP_ID"/>
+        </join>
+        <element>
+          <embedded>
+            <field name="name">
+              <column name="NAME" jdbc-type="varchar" length="256" allows-null="false"/>
+            </field>
+            <field name="type">
+              <column name="TYPE" jdbc-type="varchar" length="128" allows-null="false"/>
+            </field>
+            <field name="length">
+              <column name="length" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="scale">
+              <column name="scale" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="isOut">
+              <column name="OUT" allows-null="false"/>
+            </field>
+          </embedded>
+        </element>
+      </field>

Review comment:
       > I understand it is only for showing the user, but it it feels like an afterthought, I don't really like it.
   
   Okay; then we don't need to add and store it - we could also use a simple UDF to get the signature if we want that.
   
   About  `search for the return type` I think we should come up with a "working solution first" and go after these things when we need it (maybe never?)
   I think that `MethodFinder.methodFor` will need a lot more than just a signature...so it's a little bit too much
   
   > These things might sound unimportant but I think language design and tool development shouldn't be separated. Tool support should be considered from day 1 when creating a language.
   
   We may leave this thing out completely right now - because we don't need it right now.
   I think instead of adding something which may not perfectly align with our future needs ; leaving something out will not lock us in at all.
   
   > but interpreting the AST (method body) as a bottleneck.
   
   that's the problem of the language implementation itself - I think in case the language function is defined in some kind of "text" then we should store it as "text" - to have a less convoluted contract with the embedded language.
   
   > If we only allow defining procedures in terms of the "host language", then this is true indeed. My assumption was that we might want to accept procedure definitions in terms of the "foreign language".
   
   yes; we should only store those procedures/functions which are usable from the HiveQL - don't we?
   
   > `CREATE FUNCTION func1(a int) RETURNS int LANGUAGE XX BEGIN ... END;`
   
   I think we should simply store the whole definition from `CREATE` to the closing `;`.
   Storing anything less could become unparsable.
   
   > > I don't know what feature you are refering
   
   > It's probably that one. I think that illustrates nicely why parsing is not always a prerequisite of calling a procedure.
   
   There might be some misunderstanding here by "parsing" - I mean to process it with the host sql language. 
   
   If the client language has performance issues doing parsing/etc is then that issue belong to the language itself.
   In case the language is an on-the fly interpreted language which also happens to have a compiled version ; then that other version could either be registered as a separate language (and refer to a binary somehow) or the language could add some caching/etc to avoid unneccessary parsing overhead.
   Couldn't we do something like this for hplsql?
   
   In any case: I think that extending the basic implementation with a "generic blob storage" option to provide additional services for the stored procedures (which could potentially be used to speed up sql function execution) should be a separate feature - and as such; should be discussed(and implemented) separetly.
   (Honestly I think there would be marginal benefits implementing this - and could be dodged by the client language implementation with a few caches/etc.)
   




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512058846



##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java
##########
@@ -27,7 +27,7 @@
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hive.hplsql.*;
 
-public class FunctionDatetime extends Function {
+public class FunctionDatetime extends BuiltinFunctions {

Review comment:
       It used to extend from Function which was a class before. I didn't really change how it used to work, it's still using implementation inheritance, which I personally don't like but didn't want to change it as part of this patch. We might want to move builtin functions into the DB later on, making this class unnecessary in the future.




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

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



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


[GitHub] [hive] kgyrtkirk commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
kgyrtkirk commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r511984518



##########
File path: standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-3.2.0-to-4.0.0.mysql.sql
##########
@@ -109,6 +109,20 @@ CREATE TABLE IF NOT EXISTS REPLICATION_METRICS (
 CREATE INDEX POLICY_IDX ON REPLICATION_METRICS (RM_POLICY);
 CREATE INDEX DUMP_IDX ON REPLICATION_METRICS (RM_DUMP_EXECUTION_ID);
 
+-- Create stored procedure tables
+CREATE TABLE STORED_PROCS (
+  `SP_ID` BIGINT(20) NOT NULL,
+  `CREATE_TIME` INT(11) NOT NULL,
+  `DB_ID` BIGINT(20) NOT NULL,
+  `NAME` VARCHAR(256) NOT NULL,
+  `OWNER_NAME` VARCHAR(128) NOT NULL,
+  `SOURCE` LONGTEXT NOT NULL,
+  PRIMARY KEY (`SP_ID`)
+);
+
+CREATE UNIQUE INDEX UNIQUESTOREDPROC ON STORED_PROCS (NAME, DB_ID);
+ALTER TABLE `STORED_PROCS` ADD CONSTRAINT `STOREDPROC_FK1` FOREIGN KEY (`DB_ID`) REFERENCES DBS (`DB_ID`);

Review comment:
       what will happen when the db is dropped? wouldn't this FK will restrict the DB from being dropped?

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
##########
@@ -1,780 +1,30 @@
 /*
- * 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
+ *  * Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       these lines start with `* *`

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/BuiltinFunctions.java
##########
@@ -0,0 +1,435 @@
+/*
+ *
+ *  * 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.Date;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.hive.hplsql.Exec;
+import org.apache.hive.hplsql.HplsqlParser;
+import org.apache.hive.hplsql.Query;
+import org.apache.hive.hplsql.Utils;
+import org.apache.hive.hplsql.Var;
+
+public class BuiltinFunctions {

Review comment:
       do we really need to define these function differently than others; I've taken a look at `MIN_PART_STRING` and it seenms like its an ordinary function...so it could probably use the registry way approach 

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
##########
@@ -1,780 +1,30 @@
 /*
- * 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
+ *  * 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.
  *
- * 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.ArrayList;
-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.hive.hplsql.HplsqlParser;
 
-import org.apache.commons.lang3.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(name, ctx)) {
-      return;
-    }
-    else if (isProc(name) && execProc(name, ctx, null)) {
-      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 != null && ctx.parent != null && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
-      trace(ctx, "FUNC " + name);      
-    }
-    FuncCommand func = map.get(name.toUpperCase());    
-    if (func != null) {
-      func.run(ctx);
-    }    
-    else {
-      info(ctx, "Function not found: " + name);
-      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("(");
-    if (ctx != null) {
-      int cnt = ctx.func_param().size();
-      for (int i = 0; i < cnt; i++) {
-        sql.append(evalPop(ctx.func_param(i).expr()));
-        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(String name, HplsqlParser.Expr_func_paramsContext ctx) {
-    HplsqlParser.Create_function_stmtContext userCtx = userMap.get(name.toUpperCase());
-    if (userCtx == null) {
-      return false;
-    }
-    if (trace) {
-      trace(ctx, "EXEC FUNCTION " + name);
-    }
-    ArrayList<Var> actualParams = getActualCallParameters(ctx);
-    exec.enterScope(Scope.Type.ROUTINE);
-    setCallParameters(ctx, actualParams, userCtx.create_routine_params(), null);
-    if (userCtx.declare_block_inplace() != null) {
-      visit(userCtx.declare_block_inplace());
-    }
-    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.func_param().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.func_param(i).expr()));
-      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);
-    exec.callStackPush(name);
-    if (procCtx.create_routine_params() != null) {
-      setCallParameters(procCtx.create_routine_params());
-    }
-    visit(procCtx.proc_block());
-    exec.callStackPop();
-    exec.leaveScope();       
-    return true;
-  }
-  
-  /**
-   * Check if the stored procedure with the specified name is defined
-   */
-  public boolean isProc(String name) {
-    if (procMap.get(name.toUpperCase()) != null) {
-      return true;
-    }
-    return false;
-  }
-  
-  /**
-   * Execute a stored procedure using CALL or EXEC statement passing parameters
-   */
-  public boolean execProc(String name, HplsqlParser.Expr_func_paramsContext ctx, ParserRuleContext callCtx) {
-    if (trace) {
-      trace(callCtx, "EXEC PROCEDURE " + name);
-    }
-    HplsqlParser.Create_procedure_stmtContext procCtx = procMap.get(name.toUpperCase());    
-    if (procCtx == null) {
-      trace(callCtx, "Procedure not found");
-      return false;
-    }    
-    ArrayList<Var> actualParams = getActualCallParameters(ctx);
-    HashMap<String, Var> out = new HashMap<String, Var>();
-    exec.enterScope(Scope.Type.ROUTINE);
-    exec.callStackPush(name);
-    if (procCtx.declare_block_inplace() != null) {
-      visit(procCtx.declare_block_inplace());
-    }
-    if (procCtx.create_routine_params() != null) {
-      setCallParameters(ctx, actualParams, procCtx.create_routine_params(), out);
-    }
-    visit(procCtx.proc_block());
-    exec.callStackPop();
-    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
-   */
-  public void setCallParameters(HplsqlParser.Expr_func_paramsContext actual, ArrayList<Var> actualValues, 
-                         HplsqlParser.Create_routine_paramsContext formal,
-                         HashMap<String, Var> out) {
-    if (actual == null || actual.func_param() == null || actualValues == null) {
-      return;
-    }
-    int actualCnt = actualValues.size();
-    int formalCnt = formal.create_routine_param_item().size();
-    for (int i = 0; i < actualCnt; i++) {
-      if (i >= formalCnt) {
-        break;
-      }
-      HplsqlParser.ExprContext a = actual.func_param(i).expr(); 
-      HplsqlParser.Create_routine_param_itemContext p = getCallParameter(actual, formal, 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 var = setCallParameter(name, type, len, scale, actualValues.get(i));
-      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;
-  }
-  
-  /**
-   * Get call parameter definition by name (if specified) or position
-   */
-  HplsqlParser.Create_routine_param_itemContext getCallParameter(HplsqlParser.Expr_func_paramsContext actual, 
-      HplsqlParser.Create_routine_paramsContext formal, int pos) {
-    String named = null;
-    int out_pos = pos;
-    if (actual.func_param(pos).ident() != null) {
-      named = actual.func_param(pos).ident().getText(); 
-      int cnt = formal.create_routine_param_item().size();
-      for (int i = 0; i < cnt; i++) {
-        if (named.equalsIgnoreCase(formal.create_routine_param_item(i).ident().getText())) {
-          out_pos = i;
-          break;
-        }
-      }
-    }
-    return formal.create_routine_param_item(out_pos);
-  }  
-  
-  /**
-   * Evaluate actual call parameters
-   */
-  public ArrayList<Var> getActualCallParameters(HplsqlParser.Expr_func_paramsContext actual) {
-    if (actual == null || actual.func_param() == null) {
-      return null;
-    }
-    int cnt = actual.func_param().size();
-    ArrayList<Var> values = new ArrayList<Var>(cnt);
-    for (int i = 0; i < cnt; i++) {
-      values.add(evalPop(actual.func_param(i).expr()));
-    }
-    return values;
-  }
-  
-  /**
-   * 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);
-  }
-  
-  /**
-   * Get the number of parameters in function call
-   */
-  public int getParamCount(HplsqlParser.Expr_func_paramsContext ctx) {
-    if (ctx == null) {
-      return 0;
-    }
-    return ctx.func_param().size();
-  }
-    
-  /**
-   * 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 {
-      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))); 
-  }
-  
-  /**
-   * 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)); 
-  }
-  
-  void evalInt(int i) {
-    evalInt(Long.valueOf(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(Long.valueOf(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);
-  }
-  
-  public void info(ParserRuleContext ctx, String message) {
-    exec.info(ctx, message);
-  }
+public interface Function {
+  boolean exec(String name, HplsqlParser.Expr_func_paramsContext ctx);
+  void addUserFunction(HplsqlParser.Create_function_stmtContext ctx);
+  void addUserProcedure(HplsqlParser.Create_procedure_stmtContext ctx);
+  boolean exists(String name);

Review comment:
       what does exists mean? an implementation of this interface is a `Function` - which should have a name ; so a `getName` would probably fit better

##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,31 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">

Review comment:
       right now I'm wondering how much this differs from the `MFunction` stuff; is it differ enough to have a separate table?

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -1659,13 +1665,70 @@ public Integer visitExpr_func(HplsqlParser.Expr_funcContext ctx) {
     }
     return 0;
   }
-  
+
+  /**
+   * 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("(");
+    if (ctx != null) {
+      int cnt = ctx.func_param().size();
+      for (int i = 0; i < cnt; i++) {
+        sql.append(evalPop(ctx.func_param(i).expr()));
+        if (i + 1 < cnt) {
+          sql.append(", ");
+        }
+      }
+    }
+    sql.append(")");
+    exec.stackPush(sql);
+  }
+
+  /**
+   * Execute a HPL/SQL user-defined function in a query
+   */
+  private boolean execUserSql(HplsqlParser.Expr_func_paramsContext ctx, String name) {
+    if (!function.exists(name.toUpperCase())) {
+      return false;
+    }
+    StringBuilder sql = new StringBuilder();
+    sql.append("hplsql('");

Review comment:
       will this escaping be enough in all cases?

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java
##########
@@ -36,20 +36,20 @@ public FunctionDatetime(Exec 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("FROM_UNIXTIME", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { fromUnixtime(ctx); }});
-    f.map.put("NOW", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { now(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.map.put("UNIX_TIMESTAMP", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { unixTimestamp(ctx); }});
+  public void register(BuiltinFunctions f) {
+    f.map.put("DATE", this::date);

Review comment:
       note: registrator exposes it's internals

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MPosParam.java
##########
@@ -0,0 +1,69 @@
+/*
+ *
+ *  * 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.hadoop.hive.metastore.model;
+
+public class MPosParam {

Review comment:
       is this an unused class?

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/HmsFunction.java
##########
@@ -0,0 +1,232 @@
+/*
+ *
+ *  * 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 static org.apache.hive.hplsql.functions.InMemoryFunction.setCallParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.StoredProcedure;
+import org.apache.hadoop.hive.metastore.api.StoredProcedureRequest;
+import org.apache.hive.hplsql.Exec;
+import org.apache.hive.hplsql.HplsqlBaseVisitor;
+import org.apache.hive.hplsql.HplsqlLexer;
+import org.apache.hive.hplsql.HplsqlParser;
+import org.apache.hive.hplsql.Scope;
+import org.apache.hive.hplsql.Var;
+import org.apache.thrift.TException;
+
+public class HmsFunction implements Function {

Review comment:
       I think `Function`-s should be independent from where they are defined (builtin or hms) - and there should be some kind of registry which knows about all the functions.
   
   I feel that the `Function` interface is not really intuituve - its more like a cointainer of functions or something like that.
   
   Right now this class seem to contain some parts of a registry; which could also do chaining - plus  the registration and execution logic for functions defined in the metastore.
   It seems to me that there is no real registry right now - if there would be one then I think this class could be some kind of "factory of functions" ?
   

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java
##########
@@ -27,7 +27,7 @@
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hive.hplsql.*;
 
-public class FunctionDatetime extends Function {
+public class FunctionDatetime extends BuiltinFunctions {

Review comment:
       I think this class should implement the `Function` interface and not extends a class which has a name which suggest that it's a "container of functions"

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
##########
@@ -1,780 +1,30 @@
 /*
- * 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
+ *  * 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.
  *
- * 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.ArrayList;
-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.hive.hplsql.HplsqlParser;
 
-import org.apache.commons.lang3.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(name, ctx)) {
-      return;
-    }
-    else if (isProc(name) && execProc(name, ctx, null)) {
-      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 != null && ctx.parent != null && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
-      trace(ctx, "FUNC " + name);      
-    }
-    FuncCommand func = map.get(name.toUpperCase());    
-    if (func != null) {
-      func.run(ctx);
-    }    
-    else {
-      info(ctx, "Function not found: " + name);
-      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("(");
-    if (ctx != null) {
-      int cnt = ctx.func_param().size();
-      for (int i = 0; i < cnt; i++) {
-        sql.append(evalPop(ctx.func_param(i).expr()));
-        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(String name, HplsqlParser.Expr_func_paramsContext ctx) {
-    HplsqlParser.Create_function_stmtContext userCtx = userMap.get(name.toUpperCase());
-    if (userCtx == null) {
-      return false;
-    }
-    if (trace) {
-      trace(ctx, "EXEC FUNCTION " + name);
-    }
-    ArrayList<Var> actualParams = getActualCallParameters(ctx);
-    exec.enterScope(Scope.Type.ROUTINE);
-    setCallParameters(ctx, actualParams, userCtx.create_routine_params(), null);
-    if (userCtx.declare_block_inplace() != null) {
-      visit(userCtx.declare_block_inplace());
-    }
-    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.func_param().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.func_param(i).expr()));
-      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);
-    exec.callStackPush(name);
-    if (procCtx.create_routine_params() != null) {
-      setCallParameters(procCtx.create_routine_params());
-    }
-    visit(procCtx.proc_block());
-    exec.callStackPop();
-    exec.leaveScope();       
-    return true;
-  }
-  
-  /**
-   * Check if the stored procedure with the specified name is defined
-   */
-  public boolean isProc(String name) {
-    if (procMap.get(name.toUpperCase()) != null) {
-      return true;
-    }
-    return false;
-  }
-  
-  /**
-   * Execute a stored procedure using CALL or EXEC statement passing parameters
-   */
-  public boolean execProc(String name, HplsqlParser.Expr_func_paramsContext ctx, ParserRuleContext callCtx) {
-    if (trace) {
-      trace(callCtx, "EXEC PROCEDURE " + name);
-    }
-    HplsqlParser.Create_procedure_stmtContext procCtx = procMap.get(name.toUpperCase());    
-    if (procCtx == null) {
-      trace(callCtx, "Procedure not found");
-      return false;
-    }    
-    ArrayList<Var> actualParams = getActualCallParameters(ctx);
-    HashMap<String, Var> out = new HashMap<String, Var>();
-    exec.enterScope(Scope.Type.ROUTINE);
-    exec.callStackPush(name);
-    if (procCtx.declare_block_inplace() != null) {
-      visit(procCtx.declare_block_inplace());
-    }
-    if (procCtx.create_routine_params() != null) {
-      setCallParameters(ctx, actualParams, procCtx.create_routine_params(), out);
-    }
-    visit(procCtx.proc_block());
-    exec.callStackPop();
-    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
-   */
-  public void setCallParameters(HplsqlParser.Expr_func_paramsContext actual, ArrayList<Var> actualValues, 
-                         HplsqlParser.Create_routine_paramsContext formal,
-                         HashMap<String, Var> out) {
-    if (actual == null || actual.func_param() == null || actualValues == null) {
-      return;
-    }
-    int actualCnt = actualValues.size();
-    int formalCnt = formal.create_routine_param_item().size();
-    for (int i = 0; i < actualCnt; i++) {
-      if (i >= formalCnt) {
-        break;
-      }
-      HplsqlParser.ExprContext a = actual.func_param(i).expr(); 
-      HplsqlParser.Create_routine_param_itemContext p = getCallParameter(actual, formal, 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 var = setCallParameter(name, type, len, scale, actualValues.get(i));
-      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;
-  }
-  
-  /**
-   * Get call parameter definition by name (if specified) or position
-   */
-  HplsqlParser.Create_routine_param_itemContext getCallParameter(HplsqlParser.Expr_func_paramsContext actual, 
-      HplsqlParser.Create_routine_paramsContext formal, int pos) {
-    String named = null;
-    int out_pos = pos;
-    if (actual.func_param(pos).ident() != null) {
-      named = actual.func_param(pos).ident().getText(); 
-      int cnt = formal.create_routine_param_item().size();
-      for (int i = 0; i < cnt; i++) {
-        if (named.equalsIgnoreCase(formal.create_routine_param_item(i).ident().getText())) {
-          out_pos = i;
-          break;
-        }
-      }
-    }
-    return formal.create_routine_param_item(out_pos);
-  }  
-  
-  /**
-   * Evaluate actual call parameters
-   */
-  public ArrayList<Var> getActualCallParameters(HplsqlParser.Expr_func_paramsContext actual) {
-    if (actual == null || actual.func_param() == null) {
-      return null;
-    }
-    int cnt = actual.func_param().size();
-    ArrayList<Var> values = new ArrayList<Var>(cnt);
-    for (int i = 0; i < cnt; i++) {
-      values.add(evalPop(actual.func_param(i).expr()));
-    }
-    return values;
-  }
-  
-  /**
-   * 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);
-  }
-  
-  /**
-   * Get the number of parameters in function call
-   */
-  public int getParamCount(HplsqlParser.Expr_func_paramsContext ctx) {
-    if (ctx == null) {
-      return 0;
-    }
-    return ctx.func_param().size();
-  }
-    
-  /**
-   * 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 {
-      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))); 
-  }
-  
-  /**
-   * 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)); 
-  }
-  
-  void evalInt(int i) {
-    evalInt(Long.valueOf(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(Long.valueOf(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);
-  }
-  
-  public void info(ParserRuleContext ctx, String message) {
-    exec.info(ctx, message);
-  }
+public interface Function {
+  boolean exec(String name, HplsqlParser.Expr_func_paramsContext ctx);

Review comment:
       what does this returned boolean mean?

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -799,30 +801,35 @@ Integer init(String[] args) throws Exception {
     select = new Select(this);
     stmt = new Stmt(this);
     converter = new Converter(this);
-        
-    function = new Function(this);
-    new FunctionDatetime(this).register(function);
-    new FunctionMisc(this).register(function);
-    new FunctionString(this).register(function);
-    new FunctionOra(this).register(function);
+
+    builtinFunctions = new BuiltinFunctions(this);
+    new FunctionDatetime(this).register(builtinFunctions);
+    new FunctionMisc(this).register(builtinFunctions);
+    new FunctionString(this).register(builtinFunctions);
+    new FunctionOra(this).register(builtinFunctions);
+    if ("hms".equalsIgnoreCase(System.getProperty("hplsql.storage"))) {
+      function = new HmsFunction(this, getMsc(System.getProperty("hplsq.metastore.uris", "thrift://localhost:9083")), builtinFunctions);

Review comment:
       typo: `hplsq`
   note: this `getMsc()` method could be placed closer to the `Hms` related stuff 




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

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



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


[GitHub] [hive] kgyrtkirk merged pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
kgyrtkirk merged pull request #1542:
URL: https://github.com/apache/hive/pull/1542


   


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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512590386



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MPosParam.java
##########
@@ -0,0 +1,69 @@
+/*
+ *
+ *  * 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.hadoop.hive.metastore.model;
+
+public class MPosParam {

Review comment:
       removed

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
##########
@@ -1,780 +1,30 @@
 /*
- * 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
+ *  * Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       fixed




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

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



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


[GitHub] [hive] kgyrtkirk commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
kgyrtkirk commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r498830563



##########
File path: standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
##########
@@ -2830,6 +2848,11 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   void add_replication_metrics(1: ReplicationMetricList replicationMetricList) throws(1:MetaException o1)
   ReplicationMetricList get_replication_metrics(1: GetReplicationMetricsRequest rqst) throws(1:MetaException o1)
   GetOpenTxnsResponse get_open_txns_req(1: GetOpenTxnsRequest getOpenTxnsRequest)
+
+  void create_stored_procedure(1: string catName, 2: StoredProcedure proc) throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  StoredProcedure get_stored_procedure(1: string catName, 2: string db, 3: string name) throws (1:MetaException o1, 2:NoSuchObjectException o2)
+  void drop_stored_procedure(1: string catName, 2: string dbName, 3: string funcName) throws (1:MetaException o1, 2:NoSuchObjectException o2)
+  list<StoredProcedure> get_all_stored_procedures(1: string catName) throws (1:MetaException o1)

Review comment:
       could you please follow the convention of other methods and define a struct for the requests arguments

##########
File path: standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql
##########
@@ -786,6 +786,35 @@ CREATE TABLE "APP"."REPLICATION_METRICS" (
 CREATE INDEX "POLICY_IDX" ON "APP"."REPLICATION_METRICS" ("RM_POLICY");
 CREATE INDEX "DUMP_IDX" ON "APP"."REPLICATION_METRICS" ("RM_DUMP_EXECUTION_ID");
 
+-- Create stored procedure tables
+CREATE TABLE "APP"."STORED_PROCS" (
+  "SP_ID" BIGINT NOT NULL,
+  "CREATE_TIME" INTEGER NOT NULL,
+  "LAST_ACCESS_TIME" INTEGER NOT NULL,

Review comment:
       I think we should only add fields which are actually usefull and in use - because right now the accesstime would not be updated at all I don't think we should add it.

##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>
+      </field>
+      <field name="language">
+        <column name="LANG" jdbc-type="varchar" length="128" allows-null="false"/>
+      </field>
+      <field name="returnType">
+        <column name="RET_TYPE" jdbc-type="varchar" length="128" allows-null="true"/>
+      </field>
+      <field name="parameters" table="SP_POS_ARGS" >
+        <collection element-type="MPosParam"/>
+        <join>
+            <column name="SP_ID"/>
+        </join>
+        <element>
+          <embedded>
+            <field name="name">
+              <column name="NAME" jdbc-type="varchar" length="256" allows-null="false"/>
+            </field>
+            <field name="type">
+              <column name="TYPE" jdbc-type="varchar" length="128" allows-null="false"/>
+            </field>
+            <field name="length">
+              <column name="length" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="scale">
+              <column name="scale" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="isOut">
+              <column name="OUT" allows-null="false"/>
+            </field>
+          </embedded>
+        </element>
+      </field>

Review comment:
       I think instead of storing the return_type/argument types and such in the metastore - as they would never participate in a query or anything "usefull"; they will just travel as payload in the messages.
   Given the fact that they are effectively implicit data which can be figured out from the function defintion - I think we may leave it to the execution engine; it should be able to figure it out (since it should be able to use it) .
   
   optionally; to give ourselfs(and users) some type of clarity we could add a "signature" string to the table - which could provide a human readable signature

##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>

Review comment:
       this is the first occurence of MEDIUMTEXT in package.jdo - I don't know how well that will work
   
   we had quite a few problems with "long" tableproperty values - and PARAM_VALUE was updated to use CLOB in oracle/etc
   
   the most important would be to make sure that we can store the procedure in all supported metastore databases - if possible this should also be tested in some way (at least by hand)




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r498867496



##########
File path: standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql
##########
@@ -786,6 +786,35 @@ CREATE TABLE "APP"."REPLICATION_METRICS" (
 CREATE INDEX "POLICY_IDX" ON "APP"."REPLICATION_METRICS" ("RM_POLICY");
 CREATE INDEX "DUMP_IDX" ON "APP"."REPLICATION_METRICS" ("RM_DUMP_EXECUTION_ID");
 
+-- Create stored procedure tables
+CREATE TABLE "APP"."STORED_PROCS" (
+  "SP_ID" BIGINT NOT NULL,
+  "CREATE_TIME" INTEGER NOT NULL,
+  "LAST_ACCESS_TIME" INTEGER NOT NULL,

Review comment:
       the intention was to have something the represents the last modification date (maybe the name was chosen poorly), but ok I'll remove it, it is not used




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512058024



##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
##########
@@ -1,780 +1,30 @@
 /*
- * 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
+ *  * 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.
  *
- * 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.ArrayList;
-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.hive.hplsql.HplsqlParser;
 
-import org.apache.commons.lang3.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(name, ctx)) {
-      return;
-    }
-    else if (isProc(name) && execProc(name, ctx, null)) {
-      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 != null && ctx.parent != null && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
-      trace(ctx, "FUNC " + name);      
-    }
-    FuncCommand func = map.get(name.toUpperCase());    
-    if (func != null) {
-      func.run(ctx);
-    }    
-    else {
-      info(ctx, "Function not found: " + name);
-      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("(");
-    if (ctx != null) {
-      int cnt = ctx.func_param().size();
-      for (int i = 0; i < cnt; i++) {
-        sql.append(evalPop(ctx.func_param(i).expr()));
-        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(String name, HplsqlParser.Expr_func_paramsContext ctx) {
-    HplsqlParser.Create_function_stmtContext userCtx = userMap.get(name.toUpperCase());
-    if (userCtx == null) {
-      return false;
-    }
-    if (trace) {
-      trace(ctx, "EXEC FUNCTION " + name);
-    }
-    ArrayList<Var> actualParams = getActualCallParameters(ctx);
-    exec.enterScope(Scope.Type.ROUTINE);
-    setCallParameters(ctx, actualParams, userCtx.create_routine_params(), null);
-    if (userCtx.declare_block_inplace() != null) {
-      visit(userCtx.declare_block_inplace());
-    }
-    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.func_param().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.func_param(i).expr()));
-      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);
-    exec.callStackPush(name);
-    if (procCtx.create_routine_params() != null) {
-      setCallParameters(procCtx.create_routine_params());
-    }
-    visit(procCtx.proc_block());
-    exec.callStackPop();
-    exec.leaveScope();       
-    return true;
-  }
-  
-  /**
-   * Check if the stored procedure with the specified name is defined
-   */
-  public boolean isProc(String name) {
-    if (procMap.get(name.toUpperCase()) != null) {
-      return true;
-    }
-    return false;
-  }
-  
-  /**
-   * Execute a stored procedure using CALL or EXEC statement passing parameters
-   */
-  public boolean execProc(String name, HplsqlParser.Expr_func_paramsContext ctx, ParserRuleContext callCtx) {
-    if (trace) {
-      trace(callCtx, "EXEC PROCEDURE " + name);
-    }
-    HplsqlParser.Create_procedure_stmtContext procCtx = procMap.get(name.toUpperCase());    
-    if (procCtx == null) {
-      trace(callCtx, "Procedure not found");
-      return false;
-    }    
-    ArrayList<Var> actualParams = getActualCallParameters(ctx);
-    HashMap<String, Var> out = new HashMap<String, Var>();
-    exec.enterScope(Scope.Type.ROUTINE);
-    exec.callStackPush(name);
-    if (procCtx.declare_block_inplace() != null) {
-      visit(procCtx.declare_block_inplace());
-    }
-    if (procCtx.create_routine_params() != null) {
-      setCallParameters(ctx, actualParams, procCtx.create_routine_params(), out);
-    }
-    visit(procCtx.proc_block());
-    exec.callStackPop();
-    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
-   */
-  public void setCallParameters(HplsqlParser.Expr_func_paramsContext actual, ArrayList<Var> actualValues, 
-                         HplsqlParser.Create_routine_paramsContext formal,
-                         HashMap<String, Var> out) {
-    if (actual == null || actual.func_param() == null || actualValues == null) {
-      return;
-    }
-    int actualCnt = actualValues.size();
-    int formalCnt = formal.create_routine_param_item().size();
-    for (int i = 0; i < actualCnt; i++) {
-      if (i >= formalCnt) {
-        break;
-      }
-      HplsqlParser.ExprContext a = actual.func_param(i).expr(); 
-      HplsqlParser.Create_routine_param_itemContext p = getCallParameter(actual, formal, 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 var = setCallParameter(name, type, len, scale, actualValues.get(i));
-      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;
-  }
-  
-  /**
-   * Get call parameter definition by name (if specified) or position
-   */
-  HplsqlParser.Create_routine_param_itemContext getCallParameter(HplsqlParser.Expr_func_paramsContext actual, 
-      HplsqlParser.Create_routine_paramsContext formal, int pos) {
-    String named = null;
-    int out_pos = pos;
-    if (actual.func_param(pos).ident() != null) {
-      named = actual.func_param(pos).ident().getText(); 
-      int cnt = formal.create_routine_param_item().size();
-      for (int i = 0; i < cnt; i++) {
-        if (named.equalsIgnoreCase(formal.create_routine_param_item(i).ident().getText())) {
-          out_pos = i;
-          break;
-        }
-      }
-    }
-    return formal.create_routine_param_item(out_pos);
-  }  
-  
-  /**
-   * Evaluate actual call parameters
-   */
-  public ArrayList<Var> getActualCallParameters(HplsqlParser.Expr_func_paramsContext actual) {
-    if (actual == null || actual.func_param() == null) {
-      return null;
-    }
-    int cnt = actual.func_param().size();
-    ArrayList<Var> values = new ArrayList<Var>(cnt);
-    for (int i = 0; i < cnt; i++) {
-      values.add(evalPop(actual.func_param(i).expr()));
-    }
-    return values;
-  }
-  
-  /**
-   * 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);
-  }
-  
-  /**
-   * Get the number of parameters in function call
-   */
-  public int getParamCount(HplsqlParser.Expr_func_paramsContext ctx) {
-    if (ctx == null) {
-      return 0;
-    }
-    return ctx.func_param().size();
-  }
-    
-  /**
-   * 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 {
-      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))); 
-  }
-  
-  /**
-   * 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)); 
-  }
-  
-  void evalInt(int i) {
-    evalInt(Long.valueOf(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(Long.valueOf(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);
-  }
-  
-  public void info(ParserRuleContext ctx, String message) {
-    exec.info(ctx, message);
-  }
+public interface Function {
+  boolean exec(String name, HplsqlParser.Expr_func_paramsContext ctx);

Review comment:
       true: function was found and executed, false if it was not found. The return value was already used and expected by the existing code so I kept it. There are some issues with how the existing implementation works, for example calling an undefined function evaluates to null, instead of throwing an error. We might want to revisit these things later, but this wasn't the scope of this patch.




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

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



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


[GitHub] [hive] kgyrtkirk commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
kgyrtkirk commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r502512642



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>
+      </field>
+      <field name="language">
+        <column name="LANG" jdbc-type="varchar" length="128" allows-null="false"/>
+      </field>
+      <field name="returnType">
+        <column name="RET_TYPE" jdbc-type="varchar" length="128" allows-null="true"/>
+      </field>
+      <field name="parameters" table="SP_POS_ARGS" >
+        <collection element-type="MPosParam"/>
+        <join>
+            <column name="SP_ID"/>
+        </join>
+        <element>
+          <embedded>
+            <field name="name">
+              <column name="NAME" jdbc-type="varchar" length="256" allows-null="false"/>
+            </field>
+            <field name="type">
+              <column name="TYPE" jdbc-type="varchar" length="128" allows-null="false"/>
+            </field>
+            <field name="length">
+              <column name="length" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="scale">
+              <column name="scale" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="isOut">
+              <column name="OUT" allows-null="false"/>
+            </field>
+          </embedded>
+        </element>
+      </field>

Review comment:
       
   > I'm not sure if they never participate in a query. If one wants to discover the stored procedures which are currently stored in a DB and find out on what data they operate they would need to do some clumsy string manipulations on the signature.
   
   I believe you are thinking about `information_schema` stuff - its not set in stone that we have to get all that data from the metastore db - for this case we might add a few UDFs parameter info into an array or something ; so we will still store simple things in the metastore - but we could transform it into more readable in. 
   
   > Considering that other DB engines also store these information separately I would like to keep it as it is for now and see how it works in practice. Later on when we have multi language support we can revisit this issue.
   
   yes it might be..but it would be better to revisit stuff like this if its really needed; and not after we have introduced "something" which later we should care for even if we don't want to
   
   I still think there will be no real benefit of "storing it" in a decomposed manner - it will be harder to go forward in case stuff changes - and right now will not use it for anything ; so let's remove it..and add it only if there is a real need for it.
   




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

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



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


[GitHub] [hive] zeroflag commented on pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on pull request #1542:
URL: https://github.com/apache/hive/pull/1542#issuecomment-702171541


   cc: @ashutoshc, @nrg4878, @maheshk114 


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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r498868959



##########
File path: standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
##########
@@ -2830,6 +2848,11 @@ PartitionsResponse get_partitions_req(1:PartitionsRequest req)
   void add_replication_metrics(1: ReplicationMetricList replicationMetricList) throws(1:MetaException o1)
   ReplicationMetricList get_replication_metrics(1: GetReplicationMetricsRequest rqst) throws(1:MetaException o1)
   GetOpenTxnsResponse get_open_txns_req(1: GetOpenTxnsRequest getOpenTxnsRequest)
+
+  void create_stored_procedure(1: string catName, 2: StoredProcedure proc) throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  StoredProcedure get_stored_procedure(1: string catName, 2: string db, 3: string name) throws (1:MetaException o1, 2:NoSuchObjectException o2)
+  void drop_stored_procedure(1: string catName, 2: string dbName, 3: string funcName) throws (1:MetaException o1, 2:NoSuchObjectException o2)
+  list<StoredProcedure> get_all_stored_procedures(1: string catName) throws (1:MetaException o1)

Review comment:
       You mean putting (1: string catName, 2: string dbName, 3: string funcName) into a request object? I can do that. But if we have only one parameter, like in the last case that would be an overkill in my opinion.




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512060858



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,31 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">

Review comment:
       I feel like the similarity is accidental and in the future we can expect more diversion between the two. We would likely have columns which are only applicable to one case and not the other. There are already things like className, resourceUri, resourceType in MFunction. 




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

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



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


[GitHub] [hive] kgyrtkirk commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
kgyrtkirk commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r506176698



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>
+      </field>
+      <field name="language">
+        <column name="LANG" jdbc-type="varchar" length="128" allows-null="false"/>
+      </field>
+      <field name="returnType">
+        <column name="RET_TYPE" jdbc-type="varchar" length="128" allows-null="true"/>
+      </field>
+      <field name="parameters" table="SP_POS_ARGS" >
+        <collection element-type="MPosParam"/>
+        <join>
+            <column name="SP_ID"/>
+        </join>
+        <element>
+          <embedded>
+            <field name="name">
+              <column name="NAME" jdbc-type="varchar" length="256" allows-null="false"/>
+            </field>
+            <field name="type">
+              <column name="TYPE" jdbc-type="varchar" length="128" allows-null="false"/>
+            </field>
+            <field name="length">
+              <column name="length" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="scale">
+              <column name="scale" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="isOut">
+              <column name="OUT" allows-null="false"/>
+            </field>
+          </embedded>
+        </element>
+      </field>

Review comment:
       > What was proposed (but @kgyrtkirk correct me if I'm wrong) as an alternative solution is store the signature together with the body and optionally add a signature string to the table as well (this would be redundant). I think regardless of the representation, backward compatibility will always be a concern.
   
   That was a suggestion to provide a way to store a human readable 1 liner for the function - I never thinked that we should parse it; the system should rely on the definition - the human readable field would be just for "show" to be used sysdb related tables/etc.
   
   > This is only true for now because HPL/SQL is an AST interpreter but if we ever want to make this performant probably we'll need to introduce a byte code VM at some point in the near future.
   
   If at some point in time the "parsing" will prove to be a bottle neck - then we should address it at that point ; rather than try to address a problem which we may not even face - correct me if I'm wrong but I think we will only parse the function 1 time during the HS2 life cycle - which doesn't seem to be a big deal to me.
   
   > One thing I don't like about storing the signature in text is that the assumption that invoking the procedure will always require parsing. This is only true for now because HPL/SQL is an AST interpreter but if we ever want to make this performant probably we'll need to introduce a byte code VM at some point in the near future. 
   
   I think we should clarify/separate 2 things - for stored procedures usually there will be 2 languages at play:
   * the "host" language - in this case HiveQL - which will accept the function definition
   * the "foreign" language - which may or may not need parsing; in case of "HPL/SQL" it will but that will not be true for all languages
   
   because this patch doesn't contain any HiveQL parser changes - I don't know how that part will work; could you give some example how these functions could be used by the user?
   ```
   create procedure x(a integer) as $$
   insert into t values(a);
   $$ langugage hplsql;
   
   call x(12);
   ```
   
   
   > For example postgres allows you to invoke procedures implemented in C where runtime information about the signature is not available.
   
   I don't know what feature you are refering to ; but afaik to register a function in postgres you should [run a `create function` stmt](https://www.postgresql.org/docs/9.1/sql-createfunction.html) - which will contain all the arguments/etc; and because of that I don't see any issue with other languages - as a create procedure call for any language; must contain the full defintion - which includes argument types/etc - so we will store that for any supported language..
   
   




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

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



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


[GitHub] [hive] jcamachor commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
jcamachor commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r504964132



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>
+      </field>
+      <field name="language">
+        <column name="LANG" jdbc-type="varchar" length="128" allows-null="false"/>
+      </field>
+      <field name="returnType">
+        <column name="RET_TYPE" jdbc-type="varchar" length="128" allows-null="true"/>
+      </field>
+      <field name="parameters" table="SP_POS_ARGS" >
+        <collection element-type="MPosParam"/>
+        <join>
+            <column name="SP_ID"/>
+        </join>
+        <element>
+          <embedded>
+            <field name="name">
+              <column name="NAME" jdbc-type="varchar" length="256" allows-null="false"/>
+            </field>
+            <field name="type">
+              <column name="TYPE" jdbc-type="varchar" length="128" allows-null="false"/>
+            </field>
+            <field name="length">
+              <column name="length" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="scale">
+              <column name="scale" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="isOut">
+              <column name="OUT" allows-null="false"/>
+            </field>
+          </embedded>
+        </element>
+      </field>

Review comment:
       Thanks @zeroflag for working on this , this is very cool. 
   
   There are a few challenges with keeping redundant information outside of the procedure text that I can think of. One of them is that while the semantics of the procedure definition may be well defined, the representation for the other fields may not be defined clearly. Another usual challenge is that if there are any changes in the future, you will have to ensure backwards compatibility for those fields too.
   
   Going through the specific implementation of type handling, it seems you are keeping length and scale in different fields. This is not done when we store types in HMS for Hive. Any reason for that? Also, checking the documentation, it seems HPL/SQL can apply some transformations to the field type. Are those transformations applied before storing the definition or later on?
   
   Based on that, I think keeping a lean representation in HMS has its advantages like @kgyrtkirk mentioned, specifically if those fields are not actually being used for the time being. It's preferable to add those fields later on if needed, rather than backtracking and removing fields.




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512058576



##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/BuiltinFunctions.java
##########
@@ -0,0 +1,435 @@
+/*
+ *
+ *  * 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.Date;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.hive.hplsql.Exec;
+import org.apache.hive.hplsql.HplsqlParser;
+import org.apache.hive.hplsql.Query;
+import org.apache.hive.hplsql.Utils;
+import org.apache.hive.hplsql.Var;
+
+public class BuiltinFunctions {

Review comment:
       This was just extracted out from the existing code, it's a not a new thing, I think this is out of scope now.




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512590211



##########
File path: standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-3.2.0-to-4.0.0.mysql.sql
##########
@@ -109,6 +109,20 @@ CREATE TABLE IF NOT EXISTS REPLICATION_METRICS (
 CREATE INDEX POLICY_IDX ON REPLICATION_METRICS (RM_POLICY);
 CREATE INDEX DUMP_IDX ON REPLICATION_METRICS (RM_DUMP_EXECUTION_ID);
 
+-- Create stored procedure tables
+CREATE TABLE STORED_PROCS (
+  `SP_ID` BIGINT(20) NOT NULL,
+  `CREATE_TIME` INT(11) NOT NULL,
+  `DB_ID` BIGINT(20) NOT NULL,
+  `NAME` VARCHAR(256) NOT NULL,
+  `OWNER_NAME` VARCHAR(128) NOT NULL,
+  `SOURCE` LONGTEXT NOT NULL,
+  PRIMARY KEY (`SP_ID`)
+);
+
+CREATE UNIQUE INDEX UNIQUESTOREDPROC ON STORED_PROCS (NAME, DB_ID);
+ALTER TABLE `STORED_PROCS` ADD CONSTRAINT `STOREDPROC_FK1` FOREIGN KEY (`DB_ID`) REFERENCES DBS (`DB_ID`);

Review comment:
       good catch, fixed it.




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r499644273



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>

Review comment:
       I changed it to CLOB, that is already used at multiple places.




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512061184



##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -799,30 +801,35 @@ Integer init(String[] args) throws Exception {
     select = new Select(this);
     stmt = new Stmt(this);
     converter = new Converter(this);
-        
-    function = new Function(this);
-    new FunctionDatetime(this).register(function);
-    new FunctionMisc(this).register(function);
-    new FunctionString(this).register(function);
-    new FunctionOra(this).register(function);
+
+    builtinFunctions = new BuiltinFunctions(this);
+    new FunctionDatetime(this).register(builtinFunctions);
+    new FunctionMisc(this).register(builtinFunctions);
+    new FunctionString(this).register(builtinFunctions);
+    new FunctionOra(this).register(builtinFunctions);
+    if ("hms".equalsIgnoreCase(System.getProperty("hplsql.storage"))) {
+      function = new HmsFunction(this, getMsc(System.getProperty("hplsq.metastore.uris", "thrift://localhost:9083")), builtinFunctions);

Review comment:
       This part is removed from subsequent patch.




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r505499995



##########
File path: standalone-metastore/metastore-server/src/main/resources/package.jdo
##########
@@ -1549,6 +1549,83 @@
         <column name="RM_DUMP_EXECUTION_ID"/>
       </index>
     </class>
+
+    <class name="MStoredProc" table="STORED_PROCS" identity-type="datastore" detachable="true">
+      <datastore-identity>
+        <column name="SP_ID"/>
+      </datastore-identity>
+      <field name="createTime">
+        <column name="CREATE_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="lastAccessTime">
+        <column name="LAST_ACCESS_TIME" jdbc-type="integer" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="DB_ID" allows-null="false"/>
+      </field>
+      <field name="name">
+        <column name="NAME" length="256" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="owner">
+        <column name="OWNER_NAME" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="source">
+        <column name="SOURCE" jdbc-type="MEDIUMTEXT" allows-null="false"/>
+      </field>
+      <field name="language">
+        <column name="LANG" jdbc-type="varchar" length="128" allows-null="false"/>
+      </field>
+      <field name="returnType">
+        <column name="RET_TYPE" jdbc-type="varchar" length="128" allows-null="true"/>
+      </field>
+      <field name="parameters" table="SP_POS_ARGS" >
+        <collection element-type="MPosParam"/>
+        <join>
+            <column name="SP_ID"/>
+        </join>
+        <element>
+          <embedded>
+            <field name="name">
+              <column name="NAME" jdbc-type="varchar" length="256" allows-null="false"/>
+            </field>
+            <field name="type">
+              <column name="TYPE" jdbc-type="varchar" length="128" allows-null="false"/>
+            </field>
+            <field name="length">
+              <column name="length" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="scale">
+              <column name="scale" jdbc-type="integer" allows-null="true"/>
+            </field>
+            <field name="isOut">
+              <column name="OUT" allows-null="false"/>
+            </field>
+          </embedded>
+        </element>
+      </field>

Review comment:
       Hey @jcamachor thanks for the feedback, I'm glad you chimed in. Maybe there is a misunderstanding, this information is not redundant in any way, we're making use of it when invoking a procedure. This information must be stored somewhere in some form the only question is what representation to use. Just to clarify, currently I store the signature in a structured way + procedure body (without the signature) in text. 
   
   What was proposed (but @kgyrtkirk correct me if I'm wrong) as an alternative solution is store the signature together with the body and optionally add a signature string to the table as well (this would be redundant). I think regardless of the representation, backward compatibility will always be a concern.
   
   One thing I don't like about storing the signature in text is that the assumption that invoking the procedure will always require parsing. This is only true for now because HPL/SQL is an AST interpreter but if we ever want to make this performant probably we'll need to introduce a byte code VM at some point in the near future. While creating a procedure needs parsing but invoking it wouldn't, if we stored the byte code. How would this work in this case? I suppose we can add runtime information into the byte code, but that's not always an option. For example postgres allows you to invoke procedures implemented in C where runtime information about the signature is not available. This might be one reason why they also choose to store the signature separately in a structured way.
   
   Multi language support was already raised by customers and adding it would be the easiest if we had common bytecode for different languages. One might want to call a procedure implemented in language A from a different language B. Then A would need to use the parser of language B to get the signature information, if the signature was stored in text.
   
   We can keep speculating on this, but at this point this is still an experimental and undocumented feature, I'm open to change it later if we have proof that one way is better than the other. 
   
   But if we decide to go with the alternative solution from now on, I suggest we choose a language agnostic representation (JSON or whatever) of the signature, instead of the unparsed text.
   
   > Also, checking the documentation, it seems HPL/SQL can apply some transformations to the field type. Are those transformations applied before storing the definition or later on?
   
   Yes, and that only affects create table statements.
   




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

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



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


[GitHub] [hive] zeroflag commented on a change in pull request #1542: HIVE-24217. HMS storage backend for HPL/SQL stored procedures (amagyar)

Posted by GitBox <gi...@apache.org>.
zeroflag commented on a change in pull request #1542:
URL: https://github.com/apache/hive/pull/1542#discussion_r512058291



##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
##########
@@ -1,780 +1,30 @@
 /*
- * 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
+ *  * 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.
  *
- * 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.ArrayList;
-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.hive.hplsql.HplsqlParser;
 
-import org.apache.commons.lang3.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(name, ctx)) {
-      return;
-    }
-    else if (isProc(name) && execProc(name, ctx, null)) {
-      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 != null && ctx.parent != null && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
-      trace(ctx, "FUNC " + name);      
-    }
-    FuncCommand func = map.get(name.toUpperCase());    
-    if (func != null) {
-      func.run(ctx);
-    }    
-    else {
-      info(ctx, "Function not found: " + name);
-      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("(");
-    if (ctx != null) {
-      int cnt = ctx.func_param().size();
-      for (int i = 0; i < cnt; i++) {
-        sql.append(evalPop(ctx.func_param(i).expr()));
-        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(String name, HplsqlParser.Expr_func_paramsContext ctx) {
-    HplsqlParser.Create_function_stmtContext userCtx = userMap.get(name.toUpperCase());
-    if (userCtx == null) {
-      return false;
-    }
-    if (trace) {
-      trace(ctx, "EXEC FUNCTION " + name);
-    }
-    ArrayList<Var> actualParams = getActualCallParameters(ctx);
-    exec.enterScope(Scope.Type.ROUTINE);
-    setCallParameters(ctx, actualParams, userCtx.create_routine_params(), null);
-    if (userCtx.declare_block_inplace() != null) {
-      visit(userCtx.declare_block_inplace());
-    }
-    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.func_param().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.func_param(i).expr()));
-      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);
-    exec.callStackPush(name);
-    if (procCtx.create_routine_params() != null) {
-      setCallParameters(procCtx.create_routine_params());
-    }
-    visit(procCtx.proc_block());
-    exec.callStackPop();
-    exec.leaveScope();       
-    return true;
-  }
-  
-  /**
-   * Check if the stored procedure with the specified name is defined
-   */
-  public boolean isProc(String name) {
-    if (procMap.get(name.toUpperCase()) != null) {
-      return true;
-    }
-    return false;
-  }
-  
-  /**
-   * Execute a stored procedure using CALL or EXEC statement passing parameters
-   */
-  public boolean execProc(String name, HplsqlParser.Expr_func_paramsContext ctx, ParserRuleContext callCtx) {
-    if (trace) {
-      trace(callCtx, "EXEC PROCEDURE " + name);
-    }
-    HplsqlParser.Create_procedure_stmtContext procCtx = procMap.get(name.toUpperCase());    
-    if (procCtx == null) {
-      trace(callCtx, "Procedure not found");
-      return false;
-    }    
-    ArrayList<Var> actualParams = getActualCallParameters(ctx);
-    HashMap<String, Var> out = new HashMap<String, Var>();
-    exec.enterScope(Scope.Type.ROUTINE);
-    exec.callStackPush(name);
-    if (procCtx.declare_block_inplace() != null) {
-      visit(procCtx.declare_block_inplace());
-    }
-    if (procCtx.create_routine_params() != null) {
-      setCallParameters(ctx, actualParams, procCtx.create_routine_params(), out);
-    }
-    visit(procCtx.proc_block());
-    exec.callStackPop();
-    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
-   */
-  public void setCallParameters(HplsqlParser.Expr_func_paramsContext actual, ArrayList<Var> actualValues, 
-                         HplsqlParser.Create_routine_paramsContext formal,
-                         HashMap<String, Var> out) {
-    if (actual == null || actual.func_param() == null || actualValues == null) {
-      return;
-    }
-    int actualCnt = actualValues.size();
-    int formalCnt = formal.create_routine_param_item().size();
-    for (int i = 0; i < actualCnt; i++) {
-      if (i >= formalCnt) {
-        break;
-      }
-      HplsqlParser.ExprContext a = actual.func_param(i).expr(); 
-      HplsqlParser.Create_routine_param_itemContext p = getCallParameter(actual, formal, 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 var = setCallParameter(name, type, len, scale, actualValues.get(i));
-      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;
-  }
-  
-  /**
-   * Get call parameter definition by name (if specified) or position
-   */
-  HplsqlParser.Create_routine_param_itemContext getCallParameter(HplsqlParser.Expr_func_paramsContext actual, 
-      HplsqlParser.Create_routine_paramsContext formal, int pos) {
-    String named = null;
-    int out_pos = pos;
-    if (actual.func_param(pos).ident() != null) {
-      named = actual.func_param(pos).ident().getText(); 
-      int cnt = formal.create_routine_param_item().size();
-      for (int i = 0; i < cnt; i++) {
-        if (named.equalsIgnoreCase(formal.create_routine_param_item(i).ident().getText())) {
-          out_pos = i;
-          break;
-        }
-      }
-    }
-    return formal.create_routine_param_item(out_pos);
-  }  
-  
-  /**
-   * Evaluate actual call parameters
-   */
-  public ArrayList<Var> getActualCallParameters(HplsqlParser.Expr_func_paramsContext actual) {
-    if (actual == null || actual.func_param() == null) {
-      return null;
-    }
-    int cnt = actual.func_param().size();
-    ArrayList<Var> values = new ArrayList<Var>(cnt);
-    for (int i = 0; i < cnt; i++) {
-      values.add(evalPop(actual.func_param(i).expr()));
-    }
-    return values;
-  }
-  
-  /**
-   * 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);
-  }
-  
-  /**
-   * Get the number of parameters in function call
-   */
-  public int getParamCount(HplsqlParser.Expr_func_paramsContext ctx) {
-    if (ctx == null) {
-      return 0;
-    }
-    return ctx.func_param().size();
-  }
-    
-  /**
-   * 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 {
-      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))); 
-  }
-  
-  /**
-   * 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)); 
-  }
-  
-  void evalInt(int i) {
-    evalInt(Long.valueOf(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(Long.valueOf(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);
-  }
-  
-  public void info(ParserRuleContext ctx, String message) {
-    exec.info(ctx, message);
-  }
+public interface Function {
+  boolean exec(String name, HplsqlParser.Expr_func_paramsContext ctx);
+  void addUserFunction(HplsqlParser.Create_function_stmtContext ctx);
+  void addUserProcedure(HplsqlParser.Create_procedure_stmtContext ctx);
+  boolean exists(String name);

Review comment:
       The terminology came from the existing code, it doesn't represent a particular function but it was already called this way (it was a class not an interface but worked the same way). It's more like a registry for functions but I didn't want to change it as part of this patch. Fixing this right now is out of scope.




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

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



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