You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by mj...@apache.org on 2017/08/08 22:17:38 UTC

[2/2] incubator-impala git commit: IMPALA-5760: Revert IMPALA-4795

IMPALA-5760: Revert IMPALA-4795

Revert commit 3059024bd8dc63bc1f4ecb91c99e11e2ae34ebb1 for
IMPALA-4795: Allow fetching function obj from catalog using
signature

This commit seems to cause TestUdfExecution.test_java_udfs
to fail periodically.

IMPALA-4795 wasn't a critical fix, so lets just revert it
until we know we can fix the flaky test at the same time.

Change-Id: Iae56a75e8ec44af6dae50f18869a486e5f8b608c
Reviewed-on: http://gerrit.cloudera.org:8080/7616
Reviewed-by: Henry Robinson <he...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/4456ead8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/4456ead8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/4456ead8

Branch: refs/heads/master
Commit: 4456ead8411ebb8b98282bdd646377a0e76ad6a3
Parents: 27dcc76
Author: Matthew Jacobs <mj...@cloudera.com>
Authored: Tue Aug 8 09:58:51 2017 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Tue Aug 8 22:13:11 2017 +0000

----------------------------------------------------------------------
 .../java/org/apache/impala/catalog/Catalog.java | 21 ++------------------
 .../org/apache/impala/catalog/Function.java     |  5 ++++-
 tests/common/impala_service.py                  | 15 +++++++-------
 tests/query_test/test_udfs.py                   | 18 +++--------------
 tests/webserver/test_web_pages.py               |  6 ------
 5 files changed, 17 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4456ead8/fe/src/main/java/org/apache/impala/catalog/Catalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Catalog.java b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
index 4e40e1d..994aa37 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Catalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Catalog.java
@@ -272,24 +272,6 @@ public abstract class Catalog {
     return db.getFunction(desc, mode);
   }
 
-  /**
-   * Returns the function that best matches 'tfn' and is registered with the
-   * catalog using 'INDISTINGUISHABLE' mode to check for matching. If scalar_fn or
-   * aggregate_fn field is not set in 'tfn' then function is looked up by signature.
-   * Returns null if function is not found or enough fields are not set in 'tfn'
-   */
-  private Function getFunction(TFunction tfn) {
-    Function fn = null;
-    if (tfn.isSetScalar_fn() || tfn.isSetAggregate_fn()) {
-      Function desc = Function.fromThrift(tfn);
-      fn = getFunction(desc, Function.CompareMode.IS_INDISTINGUISHABLE);
-    } else if (tfn.isSetSignature()) {
-      Db db = getDb(tfn.getName().getDb_name());
-      fn = (db == null) ? null : db.getFunction(tfn.getSignature());
-    }
-    return fn;
-  }
-
   public static Function getBuiltin(Function desc, Function.CompareMode mode) {
     return builtinsDb_.getFunction(desc, mode);
   }
@@ -479,7 +461,8 @@ public abstract class Catalog {
       }
       case FUNCTION: {
         TFunction tfn = objectDesc.getFn();
-        Function fn = getFunction(tfn);
+        Function desc = Function.fromThrift(tfn);
+        Function fn = getFunction(desc, Function.CompareMode.IS_INDISTINGUISHABLE);
         if (fn == null) {
           throw new CatalogException("Function not found: " + tfn);
         }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4456ead8/fe/src/main/java/org/apache/impala/catalog/Function.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Function.java b/fe/src/main/java/org/apache/impala/catalog/Function.java
index 19d835d..2f0859f 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Function.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Function.java
@@ -357,7 +357,10 @@ public class Function implements CatalogObject {
           aggFn.getMerge_fn_symbol(), aggFn.getGet_value_fn_symbol(),
           null, aggFn.getFinalize_fn_symbol());
     } else {
-      throw new IllegalStateException("Expected function type to be either UDA or UDF.");
+      // In the case where we are trying to look up the object, we only have the
+      // signature.
+      function = new Function(FunctionName.fromThrift(fn.getName()),
+          argTypes, Type.fromThrift(fn.getRet_type()), fn.isHas_var_args());
     }
     function.setBinaryType(fn.getBinary_type());
     function.setHasVarArgs(fn.isHas_var_args());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4456ead8/tests/common/impala_service.py
----------------------------------------------------------------------
diff --git a/tests/common/impala_service.py b/tests/common/impala_service.py
index 3c7ec91..3fb73bc 100644
--- a/tests/common/impala_service.py
+++ b/tests/common/impala_service.py
@@ -88,10 +88,6 @@ class BaseImpalaService(object):
     assert 0, 'Metric value %s did not reach value %s in %ss' %\
         (metric_name, expected_value, timeout)
 
-  def get_catalog_object_dump(self, object_type, object_name):
-    return self.read_debug_webpage('catalog_object?object_type=%s&object_name=%s' %\
-        (object_type, object_name))
-
 # Allows for interacting with an Impalad instance to perform operations such as creating
 # new connections or accessing the debug webpage.
 class ImpaladService(BaseImpalaService):
@@ -218,6 +214,10 @@ class ImpaladService(BaseImpalaService):
     hs2_client = TCLIService.Client(protocol)
     return hs2_client
 
+  def get_catalog_object_dump(self, object_type, object_name):
+    return self.read_debug_webpage('catalog_objects?object_type=%s&object_name=%s' %\
+        (object_type, object_name))
+
 
 # Allows for interacting with the StateStore service to perform operations such as
 # accessing the debug webpage.
@@ -229,9 +229,6 @@ class StateStoredService(BaseImpalaService):
     self.wait_for_metric_value('statestore.live-backends', num_subscribers,
                                timeout=timeout, interval=interval)
 
-  def get_catalog_object_dump(self, object_type, object_name):
-    raise Exception("Statestore does not contain catalog object")
-
 
 # Allows for interacting with the Catalog service to perform operations such as
 # accessing the debug webpage.
@@ -239,3 +236,7 @@ class CatalogdService(BaseImpalaService):
   def __init__(self, hostname, webserver_port, service_port):
     super(CatalogdService, self).__init__(hostname, webserver_port)
     self.service_port = service_port
+
+  def get_catalog_object_dump(self, object_type, object_name):
+    return self.read_debug_webpage('catalog_objects?object_type=%s&object_name=%s' %\
+        (object_type, object_name))

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4456ead8/tests/query_test/test_udfs.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_udfs.py b/tests/query_test/test_udfs.py
index c1d873d..eab5994 100644
--- a/tests/query_test/test_udfs.py
+++ b/tests/query_test/test_udfs.py
@@ -537,7 +537,7 @@ class TestUdfTargeted(TestUdfBase):
     self._run_query_all_impalads(
         exec_options, query_template.format(old_function_name), ["New UDF"])
 
-  def test_drop_then_add_function_while_running(self, vector, unique_database):
+  def test_drop_function_while_running(self, vector, unique_database):
     self.client.execute("drop function if exists `{0}`.drop_while_running(BIGINT)"
                         .format(unique_database))
     self.client.execute(
@@ -561,20 +561,8 @@ class TestUdfTargeted(TestUdfBase):
     self.client.execute(
         "drop function `{0}`.drop_while_running(BIGINT)".format(unique_database))
 
-    # Fetch some rows from the async query to make sure the UDF is being used
-    results = self.client.fetch(query, handle, 1)
-    assert results.success
-    assert len(results.data) == 1
-
-    # Re-create function associated with the same binary while the original query is
-    # running
-    self.client.execute(
-      "create function `{0}`.drop_while_running(BIGINT) returns "
-      "BIGINT LOCATION '{1}' SYMBOL='Identity'".format(
-        unique_database,
-        get_fs_path('/test-warehouse/libTestUdfs.so')))
-
     # Fetch the rest of the rows, this should still be able to run the UDF
     results = self.client.fetch(query, handle, -1)
     assert results.success
-    assert len(results.data) == 9998
+    assert len(results.data) == 9999
+

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/4456ead8/tests/webserver/test_web_pages.py
----------------------------------------------------------------------
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
index 0d62906..4a2d872 100644
--- a/tests/webserver/test_web_pages.py
+++ b/tests/webserver/test_web_pages.py
@@ -142,12 +142,6 @@ class TestWebPage(ImpalaTestSuite):
     self.__test_catalog_object("functional_parquet", "alltypes")
     self.__test_catalog_object("functional", "alltypesnopart")
     self.__test_catalog_object("functional_kudu", "alltypes")
-    # IMPALA-4795: Test getTCatalogObject() to fetch a function via the WebUI code path.
-    # Only tested on impalad since catalogd does not maintain metadata for builtin
-    # functions
-    result = self.impalad_test_service.get_catalog_object_dump("FUNCTION",
-        "_impala_builtins.abs(BIGINT)")
-    assert "abs(BIGINT)" in result
 
   def __test_catalog_object(self, db_name, tbl_name):
     """Tests the /catalog_object endpoint for the given db/table. Runs