You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2021/01/29 11:27:54 UTC

[GitHub] [hive] kgyrtkirk commented on a change in pull request #1733: HIVE-24346. Store HPL/SQL packages into HMS (amagyar)

kgyrtkirk commented on a change in pull request #1733:
URL: https://github.com/apache/hive/pull/1733#discussion_r566720841



##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -1425,30 +1450,40 @@ public Integer visitCreate_function_stmt(HplsqlParser.Create_function_stmtContex
   @Override 
   public Integer visitCreate_package_stmt(HplsqlParser.Create_package_stmtContext ctx) { 
     String name = ctx.ident(0).getText().toUpperCase();
-    exec.currentPackageDecl = new Package(name, exec, builtinFunctions);
-    exec.packages.put(name, exec.currentPackageDecl);
-    trace(ctx, "CREATE PACKAGE");
-    exec.currentPackageDecl.createSpecification(ctx);
-    exec.currentPackageDecl = null;
+    if (exec.packageLoading) {
+      exec.currentPackageDecl = new Package(name, exec, builtinFunctions);
+      exec.packages.put(name, exec.currentPackageDecl);
+      exec.currentPackageDecl.createSpecification(ctx);
+      exec.currentPackageDecl = null;
+    } else {
+      trace(ctx, "CREATE PACKAGE");
+      exec.packages.remove(name);
+      exec.packageRegistry.createPackage(name, getFormattedText(ctx));

Review comment:
       I don't understand this - why do we need to put something into the registry? isn't it able to service stuff by itself?

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -1425,30 +1450,40 @@ public Integer visitCreate_function_stmt(HplsqlParser.Create_function_stmtContex
   @Override 
   public Integer visitCreate_package_stmt(HplsqlParser.Create_package_stmtContext ctx) { 
     String name = ctx.ident(0).getText().toUpperCase();
-    exec.currentPackageDecl = new Package(name, exec, builtinFunctions);
-    exec.packages.put(name, exec.currentPackageDecl);
-    trace(ctx, "CREATE PACKAGE");
-    exec.currentPackageDecl.createSpecification(ctx);
-    exec.currentPackageDecl = null;
+    if (exec.packageLoading) {
+      exec.currentPackageDecl = new Package(name, exec, builtinFunctions);
+      exec.packages.put(name, exec.currentPackageDecl);
+      exec.currentPackageDecl.createSpecification(ctx);
+      exec.currentPackageDecl = null;
+    } else {
+      trace(ctx, "CREATE PACKAGE");
+      exec.packages.remove(name);
+      exec.packageRegistry.createPackage(name, getFormattedText(ctx));
+    }
     return 0; 
   }
 
   /**
    * CREATE PACKAGE body statement
    */
   @Override 
-  public Integer visitCreate_package_body_stmt(HplsqlParser.Create_package_body_stmtContext ctx) { 
+  public Integer visitCreate_package_body_stmt(HplsqlParser.Create_package_body_stmtContext ctx) {
     String name = ctx.ident(0).getText().toUpperCase();
-    exec.currentPackageDecl = exec.packages.get(name);
-    if (exec.currentPackageDecl == null) {
-      exec.currentPackageDecl = new Package(name, exec, builtinFunctions);
-      exec.currentPackageDecl.setAllMembersPublic(true);
-      exec.packages.put(name, exec.currentPackageDecl);
+    if (exec.packageLoading) {

Review comment:
       this exec.packageLoading variable doesn't look right - isn't there another way around?

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
##########
@@ -4131,4 +4132,12 @@ ReplicationMetricList getReplicationMetrics(GetReplicationMetricsRequest
   void dropStoredProcedure(StoredProcedureRequest request) throws MetaException, NoSuchObjectException, TException;
 
   List<String> getAllStoredProcedures(ListStoredProcedureRequest request) throws MetaException, TException;
+
+  void addPackage(Package request) throws NoSuchObjectException, MetaException, TException;
+
+  Package findPackage(PackageRequest request) throws TException;
+
+  List<String> listPackages(ListPackageRequest request) throws TException;
+
+  void dropPackage(PackageRequest request) throws TException;

Review comment:
       use a separate object for the request `DropPackageRequest`
   and for addPackage as well

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
##########
@@ -10614,6 +10644,101 @@ public void dropStoredProcedure(String catName, String dbName, String funcName)
     return names;
   }
 
+  @Override
+  public void addPackage(Package request) throws NoSuchObjectException, MetaException {
+    boolean committed = false;
+    MPackage mPkg;
+    Query query = null;
+    String catName = normalizeIdentifier(request.getCatName());
+    String dbName = normalizeIdentifier(request.getDbName());
+    MDatabase db = getMDatabase(catName, dbName);
+    try {
+      openTransaction();
+      query = findPackageQuery();
+      mPkg = (MPackage) query.execute(request.getPackageName(), dbName, catName);
+      pm.retrieve(mPkg);
+      if (mPkg == null) { // create new
+        mPkg = new MPackage();
+        populate(request, db, mPkg);
+        pm.makePersistent(mPkg);
+      } else { // update existing
+        populate(request, db, mPkg);
+      }

Review comment:
       I think this is an add and update 2-in-1 because you have 2 add methods in the `Package` interface

##########
File path: hplsql/src/test/queries/local/drop_package.sql
##########
@@ -0,0 +1,21 @@
+CREATE PACKAGE Counter AS
+  count INT := 0;
+  FUNCTION current() RETURNS INT;
+  PROCEDURE inc(i INT);
+END;
+
+CREATE PACKAGE BODY Counter AS
+  FUNCTION current() RETURNS INT IS BEGIN RETURN count; END;
+  PROCEDURE inc(i INT) IS BEGIN count := count + i; END;
+END;

Review comment:
       I think the language should be changed - we have separate `create package` and `create pacakge body`
   
   and replace those 2 with a single `create package` which also contain the body.
   
   the current implementation contains quirks to handle this approach - which results in that package body can be overwritten at any time - I think if the package already exists a `create package` call must end with an exception
   

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/packages/HmsPackageRegistry.java
##########
@@ -0,0 +1,96 @@
+/*
+ *  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.packages;
+
+import java.util.Optional;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.PackageRequest;
+import org.apache.hadoop.hive.metastore.api.Package;
+import org.apache.hive.hplsql.HplSqlSessionState;
+import org.apache.thrift.TException;
+
+public class HmsPackageRegistry implements PackageRegistry {
+  private final IMetaStoreClient msc;
+  private final HplSqlSessionState hplSqlSession;
+
+  public HmsPackageRegistry(IMetaStoreClient msc, HplSqlSessionState hplSqlSession) {
+    this.msc = msc;
+    this.hplSqlSession = hplSqlSession;
+  }
+
+  @Override
+  public Optional<String> findPackage(String name) {
+    try {
+      Package pkg = msc.findPackage(request(name));
+      return pkg == null
+              ? Optional.empty()
+              : Optional.of(pkg.getHeader() + ";\n" + pkg.getBody());
+    } catch (TException e) {
+      throw new RuntimeException(e.getCause());
+    }
+  }
+
+  @Override
+  public void createPackage(String name, String header) {
+    try {
+      msc.addPackage(makePackage(name, header, ""));
+    } catch (TException e) {
+      throw new RuntimeException(e.getCause());
+    }
+  }
+
+  @Override
+  public void createPackageBody(String name, String body) {
+    try {
+      Package existing = msc.findPackage(request(name));
+      if (existing == null) {
+        msc.addPackage(makePackage(name, "", body));
+      } else {
+        existing.setBody(body);
+        msc.addPackage(existing);
+      }
+    } catch (TException e) {
+      throw new RuntimeException(e.getCause());
+    }

Review comment:
       this seem more like an update+add 2-in-1

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/packages/PackageRegistry.java
##########
@@ -0,0 +1,28 @@
+/*
+ *  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.packages;
+
+import java.util.Optional;
+
+public interface PackageRegistry {
+  Optional<String> findPackage(String name);

Review comment:
       I think `getPackage` would be a better name for this method

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
##########
@@ -4131,4 +4132,12 @@ ReplicationMetricList getReplicationMetrics(GetReplicationMetricsRequest
   void dropStoredProcedure(StoredProcedureRequest request) throws MetaException, NoSuchObjectException, TException;
 
   List<String> getAllStoredProcedures(ListStoredProcedureRequest request) throws MetaException, TException;
+
+  void addPackage(Package request) throws NoSuchObjectException, MetaException, TException;
+
+  Package findPackage(PackageRequest request) throws TException;

Review comment:
       seems like a `getPackage` rathere than "find"
   
   

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -1460,7 +1495,26 @@ public Integer visitCreate_procedure_stmt(HplsqlParser.Create_procedure_stmtCont
     addLocalUdf(ctx);                      // Add procedures as they can be invoked by functions
     return 0; 
   }
-  
+
+  public void dropProcedure(HplsqlParser.Drop_stmtContext ctx, String name, boolean checkIfExists) {
+    if (checkIfExists && !function.exists(name)) {
+      trace(ctx, name + " DOES NOT EXIST");
+      return;
+    }
+    function.remove(name);

Review comment:
       this variable name(`function`) and the name of the class(`Function`) is highly misleading - it seems more like a `functionRegistry`

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
##########
@@ -1633,11 +1687,11 @@ public Integer visitExpr_cursor_attribute(HplsqlParser.Expr_cursor_attributeCont
    */
   @Override 
   public Integer visitExpr_func(HplsqlParser.Expr_funcContext ctx) {
-    String name = ctx.ident().getText();  
+    String name = ctx.ident().getText();
     if (exec.buildSql) {
       exec.execSql(name, ctx.expr_func_params());
-    }
-    else {
+    } else {
+      name = name.toUpperCase();

Review comment:
       this `toUpperCase` stuff is flowing everywhere - wouldn't it be possible to push this stuff into the parser somehow? ...and do an uppercase conversion that way?
   
   I'm wondering if it would be possible to  introduce something like `uppercaseIdent` in the grammar; and do the uppercase transformation there - and use the `uppercaseIdent` in the `expr_func` / other  rules

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
##########
@@ -10614,6 +10644,101 @@ public void dropStoredProcedure(String catName, String dbName, String funcName)
     return names;
   }
 
+  @Override
+  public void addPackage(Package request) throws NoSuchObjectException, MetaException {
+    boolean committed = false;
+    MPackage mPkg;
+    Query query = null;
+    String catName = normalizeIdentifier(request.getCatName());
+    String dbName = normalizeIdentifier(request.getDbName());
+    MDatabase db = getMDatabase(catName, dbName);
+    try {
+      openTransaction();
+      query = findPackageQuery();
+      mPkg = (MPackage) query.execute(request.getPackageName(), dbName, catName);
+      pm.retrieve(mPkg);
+      if (mPkg == null) { // create new
+        mPkg = new MPackage();
+        populate(request, db, mPkg);
+        pm.makePersistent(mPkg);
+      } else { // update existing
+        populate(request, db, mPkg);
+      }
+      committed = commitTransaction();
+    } finally {
+      rollbackAndCleanup(committed, query);
+    }
+  }
+
+  @Override
+  public Package findPackage(PackageRequest request) {
+    MPackage mPkg = findMPackage(request.getCatName(), request.getDbName(), request.getPackageName());
+    return mPkg == null ? null : convertToPackage(request.getCatName(), mPkg);
+  }
+
+  public List<String> listPackages(ListPackageRequest request) {
+    boolean committed = false;
+    Query query = null;
+    final String catName = normalizeIdentifier(request.getCatName());
+    final String dbName = request.isSetDbName() ? normalizeIdentifier(request.getDbName()) : null;
+    List<String> names;
+    try {
+      openTransaction();
+      if (request.isSetDbName()) {
+        query = pm.newQuery("SELECT name FROM org.apache.hadoop.hive.metastore.model.MPackage " +
+                "WHERE database.catalogName == catName && database.name == db");
+        query.declareParameters("java.lang.String catName, java.lang.String db");
+        query.setResult("name");
+        names = new ArrayList<>((Collection<String>) query.execute(catName, dbName));
+      } else {
+        query = pm.newQuery("SELECT name FROM org.apache.hadoop.hive.metastore.model.MPackage " +
+                "WHERE database.catalogName == catName");
+        query.declareParameters("java.lang.String catName");
+        query.setResult("name");
+        names = new ArrayList<>((Collection<String>) query.execute(catName));
+      }
+      committed = commitTransaction();
+    } finally {
+      rollbackAndCleanup(committed, query);
+    }
+    return names;
+  }
+
+  public void dropPackage(PackageRequest request) {
+    boolean success = false;
+    try {
+      openTransaction();
+      MPackage proc = findMPackage(request.getCatName(), request.getDbName(), request.getPackageName());
+      pm.retrieve(proc);
+      if (proc != null) {

Review comment:
       note: this will not throw an exception in any case which will make it behave with and without `IF EXISTS` the same

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
##########
@@ -10526,6 +10530,14 @@ private static void populate(StoredProcedure proc, MDatabase mDatabase, MStoredP
     result.setDatabase(mDatabase);
   }
 
+  private static void populate(Package pkg, MDatabase mDatabase, MPackage result) throws MetaException {

Review comment:
       this seems like an `MPackage` constructor to me
   

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
##########
@@ -10526,6 +10530,14 @@ private static void populate(StoredProcedure proc, MDatabase mDatabase, MStoredP
     result.setDatabase(mDatabase);
   }
 
+  private static void populate(Package pkg, MDatabase mDatabase, MPackage result) throws MetaException {

Review comment:
       please try to avoid adding simply named(`populate`) methods to large classes like this one - which has 10K+ lines already 

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
##########
@@ -10567,6 +10587,16 @@ private StoredProcedure convertToStoredProc(String catName, MStoredProc proc) {
             proc.getSource());
   }
 
+  private Package convertToPackage(String catName, MPackage mPkg) {

Review comment:
       how about having this as `MPackage.toPackage()` ?

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/packages/InMemoryPackageRegistry.java
##########
@@ -0,0 +1,60 @@
+/*
+ *  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.packages;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+public class InMemoryPackageRegistry implements PackageRegistry {
+  private Map<String, Source> registry = new HashMap<>();
+
+  @Override
+  public Optional<String> findPackage(String name) {
+    Source src = registry.get(name.toUpperCase());

Review comment:
       `toUpperCase` stuff  flowed to here as well!

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/packages/PackageRegistry.java
##########
@@ -0,0 +1,28 @@
+/*
+ *  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.packages;
+
+import java.util.Optional;
+
+public interface PackageRegistry {
+  Optional<String> findPackage(String name);

Review comment:
       I would expect something more concreate than a plain string - from the other methods it seems to me that a package has a header and a body - so there should be a `Package` object which could contain those...and the registry could also accept and output that.

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
##########
@@ -10567,6 +10587,16 @@ private StoredProcedure convertToStoredProc(String catName, MStoredProc proc) {
             proc.getSource());
   }
 
+  private Package convertToPackage(String catName, MPackage mPkg) {
+    return new Package(
+            catName,

Review comment:
       doesn't we have something like `mPkg.getDatabase().getCatName()`? 

##########
File path: hplsql/src/main/java/org/apache/hive/hplsql/packages/PackageRegistry.java
##########
@@ -0,0 +1,28 @@
+/*
+ *  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.packages;
+
+import java.util.Optional;
+
+public interface PackageRegistry {
+  Optional<String> findPackage(String name);
+  void createPackage(String name, String header);
+  void createPackageBody(String name, String body);

Review comment:
       why do we need 2 methods to create a package - I think 1 method should be enough

##########
File path: standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
##########
@@ -1932,7 +1932,7 @@ 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" (
+  CREATE TABLE "STORED_PROCS" (

Review comment:
       nit: new spaces

##########
File path: standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
##########
@@ -1945,6 +1945,22 @@ CREATE TABLE "STORED_PROCS" (
 CREATE UNIQUE INDEX "UNIQUESTOREDPROC" ON "STORED_PROCS" ("NAME", "DB_ID");
 ALTER TABLE ONLY "STORED_PROCS" ADD CONSTRAINT "STOREDPROC_FK1" FOREIGN KEY ("DB_ID") REFERENCES "DBS" ("DB_ID") DEFERRABLE;
 
+-- Create stored procedure tables
+CREATE TABLE "PACKAGES" (
+  "PKG_ID" BIGINT NOT NULL,
+  "CREATE_TIME" INTEGER NOT NULL,
+  "DB_ID" BIGINT NOT NULL,
+  "NAME" VARCHAR(256) NOT NULL,
+  "OWNER_NAME" VARCHAR(128) NOT NULL,
+  "HEADER" TEXT NOT NULL,
+  "BODY" TEXT NOT NULL,
+  PRIMARY KEY ("PKG_ID")
+);
+
+CREATE UNIQUE INDEX "UNIQUEPKG" ON "PACKAGES" ("NAME", "DB_ID");
+ALTER TABLE ONLY "PACKAGES" ADD CONSTRAINT "PACKAGES_FK1" FOREIGN KEY ("DB_ID") REFERENCES "DBS" ("DB_ID")  DEFERRABLE;

Review comment:
       is there a reason to DEFER the FK instead of casecadeing the drop by using the FK?

##########
File path: hplsql/src/test/results/local/drop_package.out.txt
##########
@@ -0,0 +1,16 @@
+Ln:1 CREATE PACKAGE
+Ln:7 CREATE PACKAGE BODY
+Ln:2 DECLARE count INT = 0
+Ln:13 EXEC PACKAGE PROCEDURE COUNTER.INC
+Ln:13 SET PARAM i = 10
+Ln:8 SET count = 10
+Ln:15 PRINT
+EXEC PACKAGE FUNCTION COUNTER.CURRENT
+Ln:7 RETURN
+10
+Ln:17 DROP
+Ln:17 COUNTER DROPPED
+Ln:19 PRINT
+null

Review comment:
       why didn't this resulted in an exception? I think asking for an undefined variable inside an unknown package should be a hard error!
   
   this behaviour may help introduce bugs (and keep them hidden) into the hplsql codes users write.




----------------------------------------------------------------
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