You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2020/10/21 06:38:33 UTC

[impala] branch master updated (65a0325 -> ee4043e)

This is an automated email from the ASF dual-hosted git repository.

stigahuang pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git.


    from 65a0325  IMPALA-10178 Run-time profile shall report skews
     new 6dbf1ca  IMPALA-6628: Use unqualified table references in .test files run from test_queries.py
     new ee4043e  IMPALA-10168: Expose JSON catalog objects in catalogd's debug page

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/catalog/catalog-server.cc                   |  28 ++-
 be/src/catalog/catalog.cc                          |   5 +
 be/src/catalog/catalog.h                           |   6 +
 .../java/org/apache/impala/service/JniCatalog.java |  13 +
 .../queries/QueryTest/analytic-fns.test            | 105 ++++----
 .../functional-query/queries/QueryTest/empty.test  |  38 +--
 .../queries/QueryTest/inline-view-limit.test       |   2 +-
 .../queries/QueryTest/inline-view.test             |  20 +-
 .../functional-query/queries/QueryTest/limit.test  |   4 +-
 .../functional-query/queries/QueryTest/misc.test   |   4 +-
 .../functional-query/queries/QueryTest/sort.test   |   6 +-
 .../queries/QueryTest/subquery-single-node.test    |   8 +-
 .../queries/QueryTest/subquery.test                | 280 +++++++++++----------
 .../functional-query/queries/QueryTest/top-n.test  |  14 +-
 .../functional-query/queries/QueryTest/union.test  |   3 +
 .../queries/QueryTest/with-clause.test             |  30 +--
 tests/query_test/test_queries.py                   |   8 +
 tests/webserver/test_web_pages.py                  |  50 ++++
 18 files changed, 365 insertions(+), 259 deletions(-)


[impala] 02/02: IMPALA-10168: Expose JSON catalog objects in catalogd's debug page

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

stigahuang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit ee4043e1a0940ae5711c68336d1ad522631d0e35
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Tue Oct 13 09:24:20 2020 +0800

    IMPALA-10168: Expose JSON catalog objects in catalogd's debug page
    
    Catalogd has a debug page at '/catalog_object' showing catalog objects
    in thrift debug strings. It's inconvenient for tests to parse the thrift
    string and get interesting infos.
    
    This patch extends this page to support returning JSON results, which
    eases tests to extract complex infos from the catalog objects, e.g.
    partition ids of a hdfs table. Just like getting json results from other
    pages, the usage is adding a ‘json’ argument in the URL, e.g.
    http://localhost:25020/catalog_object?json&object_type=TABLE&object_name=db1.tbl1
    
    Implementation:
    Csaba helped to find that Thrift has a protocol, TSimpleJSONProtocol,
    which can convert thrift objects to human readable JSON strings. This
    simplifies the implementation a lot. However, TSimpleJSONProtocol is not
    implemented in cpp yet (THRIFT-2476). So we do the conversion in FE to
    use its java implementation.
    
    Tests:
     - Add tests to verify json fields existence.
    
    Change-Id: I15f256b4e3f5206c7140746694106e03b0a4ad92
    Reviewed-on: http://gerrit.cloudera.org:8080/16449
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/catalog/catalog-server.cc                   | 28 +++++++++---
 be/src/catalog/catalog.cc                          |  5 +++
 be/src/catalog/catalog.h                           |  6 +++
 .../java/org/apache/impala/service/JniCatalog.java | 13 ++++++
 tests/webserver/test_web_pages.py                  | 50 ++++++++++++++++++++++
 5 files changed, 96 insertions(+), 6 deletions(-)

diff --git a/be/src/catalog/catalog-server.cc b/be/src/catalog/catalog-server.cc
index 0148745..a66e256 100644
--- a/be/src/catalog/catalog-server.cc
+++ b/be/src/catalog/catalog-server.cc
@@ -19,6 +19,7 @@
 
 #include <gutil/strings/substitute.h>
 #include <thrift/protocol/TDebugProtocol.h>
+#include <thrift/protocol/TJSONProtocol.h>
 
 #include "catalog/catalog-util.h"
 #include "exec/read-write-util.h"
@@ -621,6 +622,7 @@ void CatalogServer::CatalogObjectsUrlCallback(const Webserver::WebRequest& req,
   const auto& args = req.parsed_args;
   Webserver::ArgumentMap::const_iterator object_type_arg = args.find("object_type");
   Webserver::ArgumentMap::const_iterator object_name_arg = args.find("object_name");
+  Webserver::ArgumentMap::const_iterator json_arg = args.find("json");
   if (object_type_arg != args.end() && object_name_arg != args.end()) {
     TCatalogObjectType::type object_type =
         TCatalogObjectTypeFromName(object_type_arg->second);
@@ -630,13 +632,27 @@ void CatalogServer::CatalogObjectsUrlCallback(const Webserver::WebRequest& req,
     Status status =
         TCatalogObjectFromObjectName(object_type, object_name_arg->second, &request);
 
-    // Get the object and dump its contents.
-    TCatalogObject result;
-    if (status.ok()) status = catalog_->GetCatalogObject(request, &result);
-    if (status.ok()) {
-      Value debug_string(ThriftDebugString(result).c_str(), document->GetAllocator());
-      document->AddMember("thrift_string", debug_string, document->GetAllocator());
+    if (json_arg != args.end()) {
+      // Get the JSON string from FE since Thrift doesn't have a cpp implementation for
+      // SimpleJsonProtocol (THRIFT-2476), so we use it's java implementation.
+      // TODO: switch to use cpp implementation of SimpleJsonProtocol after THRIFT-2476
+      //  is resolved.
+      string json_str;
+      if (status.ok()) status = catalog_->GetJsonCatalogObject(request, &json_str);
+      if (status.ok()) {
+        Value debug_string(json_str.c_str(), document->GetAllocator());
+        document->AddMember("json_string", debug_string, document->GetAllocator());
+      }
     } else {
+      // Get the object and dump its contents.
+      TCatalogObject result;
+      if (status.ok()) status = catalog_->GetCatalogObject(request, &result);
+      if (status.ok()) {
+        Value debug_string(ThriftDebugString(result).c_str(), document->GetAllocator());
+        document->AddMember("thrift_string", debug_string, document->GetAllocator());
+      }
+    }
+    if (!status.ok()) {
       Value error(status.GetDetail().c_str(), document->GetAllocator());
       document->AddMember("error", error, document->GetAllocator());
     }
diff --git a/be/src/catalog/catalog.cc b/be/src/catalog/catalog.cc
index e7401de..d75484f 100644
--- a/be/src/catalog/catalog.cc
+++ b/be/src/catalog/catalog.cc
@@ -57,6 +57,7 @@ Catalog::Catalog() {
     {"getDbs", "([B)[B", &get_dbs_id_},
     {"getFunctions", "([B)[B", &get_functions_id_},
     {"getCatalogObject", "([B)[B", &get_catalog_object_id_},
+    {"getJsonCatalogObject", "([B)Ljava/lang/String;", &get_json_catalog_object_id_},
     {"getPartialCatalogObject", "([B)[B", &get_partial_catalog_object_id_},
     {"getCatalogDelta", "([B)[B", &get_catalog_delta_id_},
     {"getCatalogUsage", "()[B", &get_catalog_usage_id_},
@@ -92,6 +93,10 @@ Status Catalog::GetCatalogObject(const TCatalogObject& req,
   return JniUtil::CallJniMethod(catalog_, get_catalog_object_id_, req, resp);
 }
 
+Status Catalog::GetJsonCatalogObject(const TCatalogObject& req, string* res) {
+  return JniUtil::CallJniMethod(catalog_, get_json_catalog_object_id_, req, res);
+}
+
 Status Catalog::GetPartialCatalogObject(const TGetPartialCatalogObjectRequest& req,
     TGetPartialCatalogObjectResponse* resp) {
   return JniUtil::CallJniMethod(catalog_, get_partial_catalog_object_id_, req, resp);
diff --git a/be/src/catalog/catalog.h b/be/src/catalog/catalog.h
index 1287532..10c2410 100644
--- a/be/src/catalog/catalog.h
+++ b/be/src/catalog/catalog.h
@@ -72,6 +72,11 @@ class Catalog {
   /// information on the error will be returned.
   Status GetCatalogObject(const TCatalogObject& request, TCatalogObject* response);
 
+  /// Like the above method but get the json string of the catalog object. The json
+  /// string can't be deserialized to Thrift objects so can only be used in showing debug
+  /// infos.
+  Status GetJsonCatalogObject(const TCatalogObject& req, std::string* res);
+
   /// Return partial information about a Catalog object.
   /// Returns OK if the operation was successful, otherwise a Status object with
   /// information on the error will be returned.
@@ -141,6 +146,7 @@ class Catalog {
   jmethodID exec_ddl_id_;  // JniCatalog.execDdl()
   jmethodID reset_metadata_id_;  // JniCatalog.resetMetdata()
   jmethodID get_catalog_object_id_;  // JniCatalog.getCatalogObject()
+  jmethodID get_json_catalog_object_id_;  // JniCatalog.getJsonCatalogObject()
   jmethodID get_partial_catalog_object_id_;  // JniCatalog.getPartialCatalogObject()
   jmethodID get_catalog_delta_id_;  // JniCatalog.getCatalogDelta()
   jmethodID get_catalog_version_id_;  // JniCatalog.getCatalogVersion()
diff --git a/fe/src/main/java/org/apache/impala/service/JniCatalog.java b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
index e76a2ab..3b0fbd4 100644
--- a/fe/src/main/java/org/apache/impala/service/JniCatalog.java
+++ b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
@@ -72,6 +72,7 @@ import org.apache.impala.util.PatternMatcher;
 import org.apache.thrift.TException;
 import org.apache.thrift.TSerializer;
 import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TSimpleJSONProtocol;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -245,6 +246,18 @@ public class JniCatalog {
     return serializer.serialize(catalog_.getTCatalogObject(objectDescription));
   }
 
+  /**
+   * Gets the json string of a catalog object. It can only be used in showing debug
+   * messages and can't be deserialized to a thrift object.
+   */
+  public String getJsonCatalogObject(byte[] thriftParams) throws ImpalaException,
+      TException {
+    TCatalogObject objectDescription = new TCatalogObject();
+    JniUtil.deserializeThrift(protocolFactory_, objectDescription, thriftParams);
+    TSerializer jsonSerializer = new TSerializer(new TSimpleJSONProtocol.Factory());
+    return jsonSerializer.toString(catalog_.getTCatalogObject(objectDescription));
+  }
+
   public byte[] getPartialCatalogObject(byte[] thriftParams) throws ImpalaException,
       TException {
     TGetPartialCatalogObjectRequest req =
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
index 4026e67..01661a4 100644
--- a/tests/webserver/test_web_pages.py
+++ b/tests/webserver/test_web_pages.py
@@ -299,6 +299,10 @@ class TestWebPage(ImpalaTestSuite):
     self.__test_catalog_object(unique_database, "foo_kudu", cluster_properties)
     self.__test_catalog_object(unique_database, "foo_part_parquet", cluster_properties)
     self.__test_catalog_object(unique_database, "foo", cluster_properties)
+    self.__test_json_db_object(unique_database)
+    self.__test_json_table_object(unique_database, "foo")
+    self.__test_json_table_object(unique_database, "foo_part")
+    self.__test_json_table_object(unique_database, "foo_part_parquet")
     self.__test_table_metrics(unique_database, "foo_part", "total-file-size-bytes")
     self.__test_table_metrics(unique_database, "foo_part", "num-files")
     self.__test_table_metrics(unique_database, "foo_part", "alter-duration")
@@ -335,6 +339,52 @@ class TestWebPage(ImpalaTestSuite):
       self.get_and_check_status(obj_url, impalad_expected_str,
           ports_to_test=self.IMPALAD_TEST_PORT)
 
+  def __test_json_db_object(self, db_name):
+    """Tests the /catalog_object?json endpoint of catalogd for the given db."""
+    obj_url = self.CATALOG_OBJECT_URL + \
+              "?json&object_type=DATABASE&object_name={0}".format(db_name)
+    responses = self.get_and_check_status(obj_url, ports_to_test=self.CATALOG_TEST_PORT)
+    obj = json.loads(json.loads(responses[0].text)["json_string"])
+    assert obj["type"] == 2, "type should be DATABASE"
+    assert "catalog_version" in obj, "TCatalogObject should have catalog_version"
+    db_obj = obj["db"]
+    assert db_obj["db_name"] == db_name
+    assert "metastore_db" in db_obj, "Loaded database should have metastore_db"
+
+  def __test_json_table_object(self, db_name, tbl_name):
+    """Tests the /catalog_object?json endpoint of catalogd for the given db/table. Runs
+    against an unloaded as well as a loaded table."""
+    obj_url = self.CATALOG_OBJECT_URL + \
+              "?json&object_type=TABLE&object_name={0}.{1}".format(db_name, tbl_name)
+    self.client.execute("invalidate metadata %s.%s" % (db_name, tbl_name))
+    responses = self.get_and_check_status(obj_url, ports_to_test=self.CATALOG_TEST_PORT)
+    obj = json.loads(json.loads(responses[0].text)["json_string"])
+    assert obj["type"] == 3, "type should be TABLE"
+    assert "catalog_version" in obj, "TCatalogObject should have catalog_version"
+    tbl_obj = obj["table"]
+    assert tbl_obj["db_name"] == db_name
+    assert tbl_obj["tbl_name"] == tbl_name
+    assert "hdfs_table" not in tbl_obj, "Unloaded table should not have hdfs_table"
+
+    self.client.execute("refresh %s.%s" % (db_name, tbl_name))
+    responses = self.get_and_check_status(obj_url, ports_to_test=self.CATALOG_TEST_PORT)
+    obj = json.loads(json.loads(responses[0].text)["json_string"])
+    assert obj["type"] == 3, "type should be TABLE"
+    assert "catalog_version" in obj, "TCatalogObject should have catalog_version"
+    tbl_obj = obj["table"]
+    assert tbl_obj["db_name"] == db_name
+    assert tbl_obj["tbl_name"] == tbl_name
+    assert "columns" in tbl_obj, "Loaded TTable should have columns"
+    assert tbl_obj["table_type"] == 0, "table_type should be HDFS_TABLE"
+    assert "metastore_table" in tbl_obj
+    hdfs_tbl_obj = tbl_obj["hdfs_table"]
+    assert "hdfsBaseDir" in hdfs_tbl_obj
+    assert "colNames" in hdfs_tbl_obj
+    assert "nullPartitionKeyValue" in hdfs_tbl_obj
+    assert "nullColumnValue" in hdfs_tbl_obj
+    assert "partitions" in hdfs_tbl_obj
+    assert "prototype_partition" in hdfs_tbl_obj
+
   def check_endpoint_is_disabled(self, url, string_to_search="", ports_to_test=None):
     """Helper method that verifies the given url does not exist."""
     if ports_to_test is None:


[impala] 01/02: IMPALA-6628: Use unqualified table references in .test files run from test_queries.py

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

stigahuang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 6dbf1ca09c9d8b53a0d37d9201508847e3247283
Author: Qifan Chen <qc...@cloudera.com>
AuthorDate: Thu Oct 15 10:18:05 2020 -0400

    IMPALA-6628: Use unqualified table references in .test files run from test_queries.py
    
    This fix modified the following tests launched from test_queries.py by
    removing references to database 'functional' whenever possible. The
    objective of the change is to allow more testing coverage with different
    databases than the single 'functional' database. In the fix, neither new
    tables were added nor expected results were altered.
    
      empty.test
      inline-view-limit.test
      inline-view.test
      limit.test
      misc.test
      sort.test
      subquery-single-node.test
      subquery.test
      top-n.test
      union.test
      with-clause.test
    
    It was determined that other tests in
    testdata/workloads/functional-query/queries/QueryTest do not refer to
    'functional' or the references are a must for some reason.
    
    Testing
       Ran query_tests on these changed tests with exhaustive exploration
       strategy.
    
    Change-Id: Idd50eaaaba25e3bedc2b30592a314d2b6b83f972
    Reviewed-on: http://gerrit.cloudera.org:8080/16603
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../queries/QueryTest/analytic-fns.test            | 105 ++++----
 .../functional-query/queries/QueryTest/empty.test  |  38 +--
 .../queries/QueryTest/inline-view-limit.test       |   2 +-
 .../queries/QueryTest/inline-view.test             |  20 +-
 .../functional-query/queries/QueryTest/limit.test  |   4 +-
 .../functional-query/queries/QueryTest/misc.test   |   4 +-
 .../functional-query/queries/QueryTest/sort.test   |   6 +-
 .../queries/QueryTest/subquery-single-node.test    |   8 +-
 .../queries/QueryTest/subquery.test                | 280 +++++++++++----------
 .../functional-query/queries/QueryTest/top-n.test  |  14 +-
 .../functional-query/queries/QueryTest/union.test  |   3 +
 .../queries/QueryTest/with-clause.test             |  30 +--
 tests/query_test/test_queries.py                   |   8 +
 13 files changed, 269 insertions(+), 253 deletions(-)

diff --git a/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test b/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
index e558f7f..45feae0 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
@@ -1,6 +1,9 @@
 ====
 ---- QUERY
 # The test used to contain avg(timestamp_col) but it had to be removed due to IMPALA-9584.
+#
+# Table date_tbl and decimal_tbl are qualified with 'functional' database since neither
+# is defined in many other databases.
 select year, month,
 count(int_col) over (partition by year, month),
 avg(int_col) over (partition by year, month),
@@ -320,7 +323,7 @@ max(int_col)
 sum(int_col)
   over(partition by tinyint_col order by int_col desc
   rows between unbounded preceding and 2 following)
-from functional.alltypesagg
+from alltypesagg
 where id < 20
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
 1,1,10,1,1,1,1,12
@@ -360,7 +363,7 @@ max(sum(bigint_col))
 min(sum(bigint_col))
   over(partition by min(tinyint_col) order by sum(int_col)
   rows between unbounded preceding and 4 following)
-from functional.alltypes
+from alltypes
 group by 1
 order by 1, 2, 3
 ---- RESULTS
@@ -493,7 +496,7 @@ select bool_col, a, b, a+b from
    count(int_col) over() a,
    sum(int_col + bigint_col) over(partition by bool_col) b
    from
-     (select * from functional.alltypes where id < 20) v1) v2
+     (select * from alltypes where id < 20) v1) v2
 order by 1, 2, 3
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
 false,20,550,570
@@ -522,13 +525,13 @@ BOOLEAN, BIGINT, BIGINT, BIGINT
 ---- QUERY
 # analytics and unions
 select 1 a, min(id) over (partition by int_col)
-  from functional.alltypes where id < 5
+  from alltypes where id < 5
 union distinct
 select 2, max(id) over (partition by bool_col)
-  from functional.alltypessmall where id >= 5 and id <= 10
+  from alltypessmall where id >= 5 and id <= 10
 union all
 (select 3, sum(bigint_col) over (partition by int_col order by id)
- from functional.alltypestiny where bool_col = false)
+ from alltypestiny where bool_col = false)
 order by 1, 2 desc nulls first
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
 1,4
@@ -547,10 +550,10 @@ TINYINT, BIGINT
 ====
 ---- QUERY
 # analytics in an uncorrelated subquery
-select id, int_col, bool_col from functional.alltypestiny t1
+select id, int_col, bool_col from alltypestiny t1
 where int_col in
   (select min(bigint_col) over(partition by bool_col)
-   from functional.alltypessmall t2 where t2.id < 10)
+   from alltypessmall t2 where t2.id < 10)
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
 0,0,true
 2,0,true
@@ -1086,10 +1089,10 @@ DOUBLE, BIGINT, BIGINT, BIGINT, BIGINT, BIGINT, BIGINT, BIGINT
 # analytic output are substituted with TupleIsNullPredicates referring to
 # the physical output.
 select t2.a, t2.id, t2.int_col
-from functional.alltypes t1
+from alltypes t1
 left outer join
   (select sum(int_col) over(order by id) a, id, int_col is null as int_col
-   from functional.alltypes
+   from alltypes
    where year = 2009 and month between 1 and 2) t2
 on (t1.id = t2.id)
 where year = 2009 and month = 1
@@ -1111,7 +1114,7 @@ BIGINT, INT, BOOLEAN
 ---- QUERY
 # IMPALA-1542: Analytic fn containing arithmetic expr on NULL crashed
 select lag(coalesce(505, 1 + NULL), 1) over (order by int_col desc)
-from functional.alltypestiny
+from alltypestiny
 ---- RESULTS
 NULL
 505
@@ -1188,7 +1191,7 @@ select
   avg(g) over (order by f),
   t2.a,
   t2.d
-from functional.alltypestiny t1
+from alltypestiny t1
 left outer join
   (select
      id as a,
@@ -1198,7 +1201,7 @@ left outer join
      bigint_col e,
      coalesce(bigint_col, 30) as f,
      coalesce(id + bigint_col, 40) as g
-   from functional.alltypestiny) t2
+   from alltypestiny) t2
 on (t1.id = t2.a + 100)
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
 12,4,NULL,NULL,NULL
@@ -1224,7 +1227,7 @@ from
     avg(g) over (order by f) af3,
     t2.a,
     t2.d
-  from functional.alltypestiny t1
+  from alltypestiny t1
   left outer join
     (select
        id as a,
@@ -1234,7 +1237,7 @@ from
        bigint_col e,
        coalesce(bigint_col, 30) as f,
        coalesce(id + bigint_col, 40) as g
-     from functional.alltypestiny) t2
+     from alltypestiny) t2
   on (t1.id = t2.a + 100)) t3
 group by d
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
@@ -1256,13 +1259,13 @@ select * from
    sum(x) over (partition by a.id) as x,
    ifnull(y, 10) as y,
    ifnull(z, "b") as z
-   from functional.alltypestiny a
+   from alltypestiny a
    left outer join
      (select id,
       ifnull(int_col, 1) x,
       count(bigint_col) over(partition by id) y,
       ifnull(string_col, "a") z
-      from functional.alltypestiny b) v1
+      from alltypestiny b) v1
    on (a.id = v1.id)) v2
 full outer join
   (select
@@ -1270,13 +1273,13 @@ full outer join
    sum(x) over (order by c.id) as x,
    ifnull(y, 10) as y,
    ifnull(z, "b") as z
-   from functional.alltypestiny c
+   from alltypestiny c
    left outer join
      (select id,
       ifnull(int_col, 1) x,
       count(bigint_col) over(order by id) y,
       ifnull(string_col, "a") z
-      from functional.alltypestiny d) v3
+      from alltypestiny d) v3
    on (c.id = v3.id)) v4
 on (v2.id = v4.id)
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
@@ -1300,7 +1303,7 @@ with w as
   (select coalesce(-668, -335, count(smallint_col)) as int_col,
           lead('670', 87) over
             (order by coalesce(-668, -335, count(smallint_col)) desc) as char_col
-   from functional.alltypes)
+   from alltypes)
 select
   lead(-410, 11) over
     (order by (t2.int_col) + (t2.int_col) desc,
@@ -1317,7 +1320,7 @@ from w t1
 right outer join
   (select (t1.month) + (t1.year) as int_col,
    least(coalesce(t1.float_col, 683.98), -213.01) as float_col
-   from functional.alltypestiny t1
+   from alltypestiny t1
    where (t1.int_col) not in (t1.smallint_col, t1.smallint_col)) t2
 on (t2.int_col = t1.int_col and t2.int_col = t1.int_col and t2.int_col = t1.int_col)
 ---- RESULTS
@@ -1328,11 +1331,11 @@ SMALLINT, INT, BIGINT, DOUBLE, BIGINT
 # IMPALA-1946: Check that On-clause predicates of an outer join assigned in a scan
 # are not wrapped in TupleIsNullPredicates.
 select /* +straight_join */ a.id, b.id
-from functional.alltypestiny a
+from alltypestiny a
 left outer join
   (select t1.id, ifnull(t1.int_col, 10) as int_col
-   from functional.alltypestiny t1
-   inner join functional.alltypestiny t2
+   from alltypestiny t1
+   inner join alltypestiny t2
    on (t1.id = t2.id)) b
 on (a.id = b.id and b.int_col < 10)
 ---- RESULTS
@@ -1354,7 +1357,7 @@ select t.year, t.month, t.id, t.pr, (t.r-1)/(t.c-1)
 from (select year, month, id, rank() over (partition by year, month order by id) as r,
 count() over(partition by year, month) as c,
 percent_rank() over(partition by year, month order by id) as pr
-from functional.alltypes where tinyint_col < 2 and month < 3 and (id % 100 < 10)) as t
+from alltypes where tinyint_col < 2 and month < 3 and (id % 100 < 10)) as t
 order by t.year, t.month, t.id;
 ---- RESULTS
 2009,1,0,0,0
@@ -1392,7 +1395,7 @@ from (select year, month, id,
 rank() over (partition by year, month order by id desc) as r,
 count() over(partition by year, month) as c,
 cume_dist() over(partition by year, month order by id) as cd
-from functional.alltypes where tinyint_col < 2 and month < 3 and (id % 100 < 10)) as t
+from alltypes where tinyint_col < 2 and month < 3 and (id % 100 < 10)) as t
 order by t.year, t.month, t.id;
 ---- RESULTS
 2009,1,0,0.125,0.125
@@ -1430,7 +1433,7 @@ from (select year, month, id,
 row_number() over (partition by year, month order by id) as rownum,
 count() over(partition by year, month) as c,
 ntile(3) over(partition by year, month order by id) as nt
-from functional.alltypes where tinyint_col < 2 and month < 3 and (id % 100 < 10)) as t
+from alltypes where tinyint_col < 2 and month < 3 and (id % 100 < 10)) as t
 order by t.year, t.month, t.id;
 ---- RESULTS
 2009,1,0,1,1
@@ -1500,7 +1503,7 @@ select bool_col, a, b, a + b from
    percent_rank() over(order by id) a,
    cume_dist() over(partition by bool_col order by id) b
    from
-     (select * from functional.alltypes where id < 20) v1) v2
+     (select * from alltypes where id < 20) v1) v2
 order by 1, 2, 3
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
 false,0.05263157894736842,0.1,0.1526315789473684
@@ -1547,7 +1550,7 @@ TINYINT, DOUBLE
 ====
 ---- QUERY
 # Test analytic function rewrite when the same expr appears twice
-select id, percent_rank() over(order by id) from functional.alltypestiny
+select id, percent_rank() over(order by id) from alltypestiny
 order by percent_rank() over (order by id)
 ---- RESULTS
 0,0
@@ -1563,7 +1566,7 @@ INT, DOUBLE
 ====
 ---- QUERY
 # Tests IMPALA-2158. Check that analytic functions with a limit succeed.
-select id, rank() over(order by id) from functional.alltypesagg limit 10
+select id, rank() over(order by id) from alltypesagg limit 10
 ---- RESULTS
 0,1
 0,1
@@ -1603,7 +1606,7 @@ TIMESTAMP, TIMESTAMP
 # IMPALA-2457 - PERCENT_RANK() returns NaN for row group with 1 row.
 select id, rank() over (partition by id order by tinyint_col),
 count() over (partition by id), percent_rank() over (partition by id order by tinyint_col)
-from functional.alltypestiny order by id
+from alltypestiny order by id
 ---- RESULTS
 0,1,1,0
 1,1,1,0
@@ -1646,7 +1649,7 @@ NULL
 # that wrapped constant expr.
 SELECT
 LEAD(996, 13) OVER (ORDER BY t1.int_col)
-FROM functional.alltypestiny t1
+FROM alltypestiny t1
 LEFT JOIN (
   SELECT
     v1.x,
@@ -1654,7 +1657,7 @@ LEFT JOIN (
   FROM (
     SELECT
       LAG(bigint_col, 34) OVER (ORDER BY t2.int_col) AS x
-    FROM functional.alltypestiny t2
+    FROM alltypestiny t2
     ) v1
   ) v2
 ON v2.x = t1.bigint_col AND v2.y = t1.bigint_col
@@ -1693,7 +1696,7 @@ select id, tinyint_col,
     (order by id range between unbounded preceding and current row),
   last_value(tinyint_col ignore nulls) over
     (order by id range between unbounded preceding and unbounded following)
-from functional.alltypesagg where id < 21
+from alltypesagg where id < 21
 ---- TYPES
 INT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, STRING, STRING, TINYINT, TINYINT
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
@@ -1735,7 +1738,7 @@ select id, tinyint_col,
     (order by id rows between 2 following and unbounded following),
   last_value(tinyint_col ignore nulls) over
     (order by id range between current row and unbounded following)
-from functional.alltypesagg where id < 21 order by id
+from alltypesagg where id < 21 order by id
 ---- TYPES
 INT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
@@ -1779,7 +1782,7 @@ select id, tinyint_col,
     (order by id range between unbounded preceding and current row),
   first_value(tinyint_col ignore nulls) over
     (order by id range between unbounded preceding and unbounded following)
-from functional.alltypesagg where id < 21
+from alltypesagg where id < 21
 ---- TYPES
 INT, TINYINT, TINYINT, TINYINT, STRING, STRING, TINYINT, TINYINT
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
@@ -1831,7 +1834,7 @@ select id, tinyint_col,
     (order by id rows between 2 following and unbounded following),
   first_value(tinyint_col ignore nulls) over
     (order by id range between current row and unbounded following)
-from functional.alltypesagg where id < 21
+from alltypesagg where id < 21
 ---- TYPES
 INT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
@@ -1872,7 +1875,7 @@ select id,
   last_value(double_col ignore nulls) over (order by id),
   last_value(string_col ignore nulls) over (order by id),
   last_value(timestamp_col ignore nulls) over (order by id)
-from functional.alltypesagg where id < 5
+from alltypesagg where id < 5
 ---- TYPES
 INT, BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, STRING, TIMESTAMP
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
@@ -1902,7 +1905,7 @@ select id,
   first_value(double_col ignore nulls) over (order by id),
   first_value(string_col ignore nulls) over (order by id),
   first_value(timestamp_col ignore nulls) over (order by id)
-from functional.alltypesagg where id < 5
+from alltypesagg where id < 5
 ---- TYPES
 INT, BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, STRING, TIMESTAMP
 ---- RESULTS: VERIFY_IS_EQUAL_SORTED
@@ -1965,7 +1968,7 @@ select count(*) from (
 select
   from_unixtime(lead(bigint_col, 1) over (order by id), 'yyyyMMddHH:mm:ss') as a,
   lead(from_unixtime(bigint_col, 'yyyyMMddHH:mm:ss'), 1) over (order by id) AS b
-from functional.alltypes) x
+from alltypes) x
 where x.a = x.b
 ---- TYPES
 BIGINT
@@ -1980,7 +1983,7 @@ select count(*) from (
 select
   from_unixtime(lag(bigint_col, 1) over (order by id), 'yyyyMMddHH:mm:ss') as a,
   lag(from_unixtime(bigint_col, 'yyyyMMddHH:mm:ss'), 1) over (order by id) AS b
-from functional.alltypes) x
+from alltypes) x
 where x.a = x.b
 ---- TYPES
 BIGINT
@@ -2005,7 +2008,7 @@ STRING, STRING
 create view if not exists imp5657_view as
 select
   last_value(tinyint_col ignore nulls) over (order by tinyint_col)
-from functional.alltypesagg
+from alltypesagg
 where id < 5;
 ====
 ---- QUERY
@@ -2024,7 +2027,7 @@ TINYINT
 # Test for IMPALA-1882: first_value works without order by clause. Therefore, output of
 # function depends on order of input rows, which in this case is DESC
 select id, bool_col, first_value(id) over (partition by bool_col) from
-(select * from functional.alltypessmall order by id desc limit 5) as t1
+(select * from alltypessmall order by id desc limit 5) as t1
 ---- TYPES
 INT, BOOLEAN, INT
 ---- RESULTS
@@ -2038,7 +2041,7 @@ INT, BOOLEAN, INT
 # Test for IMPALA-1882: first_value works without order by clause. Therefore, output of
 # function depends on order of input rows, which in this case is ASC
 select id, bool_col, first_value(id) over (partition by bool_col) from
-(select * from functional.alltypessmall order by id asc limit 5) as t1
+(select * from alltypessmall order by id asc limit 5) as t1
 ---- TYPES
 INT, BOOLEAN, INT
 ---- RESULTS
@@ -2051,7 +2054,7 @@ INT, BOOLEAN, INT
 # Test for IMPALA-1882: last_value works without order by clause. Therefore, output of
 # function depends on order of input rows, which in this case is DESC
 select id, bool_col, last_value(id) over (partition by bool_col) from
-(select * from functional.alltypessmall order by id desc limit 5) as t1
+(select * from alltypessmall order by id desc limit 5) as t1
 ---- TYPES
 INT, BOOLEAN, INT
 ---- RESULTS
@@ -2065,7 +2068,7 @@ INT, BOOLEAN, INT
 # Test for IMPALA-1882: last_value works without order by clause. Therefore, output of
 # function depends on order of input rows, which in this case is ASC
 select id, bool_col, last_value(id) over (partition by bool_col) from
-(select * from functional.alltypessmall order by id asc limit 5) as t1
+(select * from alltypessmall order by id asc limit 5) as t1
 ---- TYPES
 INT, BOOLEAN, INT
 ---- RESULTS
@@ -2079,7 +2082,7 @@ INT, BOOLEAN, INT
 # Test for IMPALA-1882: first_value works with ignore nulls and without order by clause.
 # Therefore, output of function depends on order of input rows, which in this case is ASC
 select bool_col, smallint_col, first_value(smallint_col ignore nulls) over
-(partition by bool_col) from (select * from functional.alltypesagg where
+(partition by bool_col) from (select * from alltypesagg where
 id > 99 order by id asc limit 10) as t1
 ---- TYPES
 BOOLEAN, SMALLINT, SMALLINT
@@ -2099,7 +2102,7 @@ true,8,2
 # Test for IMPALA-1882: first_value works with ignore nulls and without order by clause.
 # Therefore, output of function depends on order of input rows, which in this case is DESC
 select bool_col, smallint_col, first_value(smallint_col ignore nulls) over
-(partition by bool_col) from (select * from functional.alltypesagg where
+(partition by bool_col) from (select * from alltypesagg where
 id < 101 order by id desc limit 10) as t1;
 ---- TYPES
 BOOLEAN, SMALLINT, SMALLINT
@@ -2119,7 +2122,7 @@ true,92,98
 # Test for IMPALA-1882: last_value works with ignore nulls and without order by clause.
 # Therefore, output of function depends on order of input rows, which in this case is ASC
 select bool_col, smallint_col, last_value(smallint_col ignore nulls) over
-(partition by bool_col) from (select * from functional.alltypesagg where
+(partition by bool_col) from (select * from alltypesagg where
 id > 99 order by id asc limit 10) as t1;
 ---- TYPES
 BOOLEAN, SMALLINT, SMALLINT
@@ -2139,7 +2142,7 @@ true,8,8
 # Test for IMPALA-1882: last_value works with ignore nulls and without order by clause.
 # Therefore, output of function depends on order of input rows, which in this case is DESC
 select bool_col, smallint_col, last_value(smallint_col ignore nulls) over
-(partition by bool_col) from (select * from functional.alltypesagg where
+(partition by bool_col) from (select * from alltypesagg where
 id < 110 order by id desc limit 10) as t1;
 ---- TYPES
 BOOLEAN, SMALLINT, SMALLINT
@@ -2178,7 +2181,7 @@ TINYINT, BIGINT
 ---- QUERY
 # IMPALA-6323 Order by a constant is equivalent to no ordering.
 SELECT FIRST_VALUE(0) OVER (ORDER BY 0 ASC)
-FROM functional.alltypestiny;
+FROM alltypestiny;
 ---- TYPES
 TINYINT
 ---- RESULTS
diff --git a/testdata/workloads/functional-query/queries/QueryTest/empty.test b/testdata/workloads/functional-query/queries/QueryTest/empty.test
index 570dd97..184937c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/empty.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/empty.test
@@ -23,8 +23,8 @@ string
 ---- QUERY
 # Constant conjunct.
 select t1.id, t2.id
-from functional.alltypestiny t1
-left outer join functional.alltypes t2
+from alltypestiny t1
+left outer join alltypes t2
 on t1.id = t2.id
 where false
 ---- TYPES
@@ -34,7 +34,7 @@ int, int
 ---- QUERY
 # Constant conjunct in query block with an aggregation.
 select count(int_col), avg(double_col), count(*)
-from functional.alltypes
+from alltypes
 where null
 ---- TYPES
 bigint, double, bigint
@@ -44,11 +44,11 @@ bigint, double, bigint
 ---- QUERY
 # Constant conjunct in inline view.
 select e.id, f.id
-from functional.alltypessmall f
+from alltypessmall f
 inner join
   (select t1.id
-   from functional.alltypestiny t1
-   left outer join functional.alltypes t2
+   from alltypestiny t1
+   left outer join alltypes t2
    on t1.id = t2.id
    where 1 + 3 > 10) e
 on e.id = f.id
@@ -59,8 +59,8 @@ int, int
 ---- QUERY
 # Limit 0
 select t1.id, t2.id
-from functional.alltypestiny t1
-left outer join functional.alltypes t2
+from alltypestiny t1
+left outer join alltypes t2
 on t1.id = t2.id
 limit 0
 ---- TYPES
@@ -70,7 +70,7 @@ int, int
 ---- QUERY
 # Limit 0 in query block with an aggregation
 select count(int_col), avg(double_col), count(*)
-from functional.alltypes
+from alltypes
 limit 0
 ---- TYPES
 bigint, double, bigint
@@ -79,11 +79,11 @@ bigint, double, bigint
 ---- QUERY
 # Limit 0 in inline view
 select e.id, f.id
-from functional.alltypessmall f
+from alltypessmall f
 inner join
   (select t1.id
-   from functional.alltypestiny t1
-   left outer join functional.alltypes t2
+   from alltypestiny t1
+   left outer join alltypes t2
    on t1.id = t2.id
    limit 0) e
 on e.id = f.id
@@ -107,10 +107,10 @@ COALESCE(580, MAX(t2.year), 150) AS int_col FROM (
   COALESCE(LAG(-826, 69) OVER (
     ORDER BY t1.bool_col DESC, COALESCE(t2.int_col, t2.smallint_col, t2.id)), 279, -764) AS int_col,
   COALESCE(t2.int_col, t2.smallint_col, t2.id) AS int_col_2
-  FROM functional.alltypestiny t1
-  INNER JOIN functional.alltypestiny t2
+  FROM alltypestiny t1
+  INNER JOIN alltypestiny t2
   ON ((t2.bigint_col) = (t1.id)) AND ((t2.timestamp_col) = (t1.timestamp_col))) t1
-INNER JOIN functional.alltypes t2 ON (t2.year) = (t1.int_col_2)
+INNER JOIN alltypes t2 ON (t2.year) = (t1.int_col_2)
 WHERE (-339.22) IN (-922.68, -254.84)
 ---- RESULTS
 580
@@ -122,7 +122,7 @@ INT
 # the select stmt has an empty select-project-join portion.
 select sum(T.id), count(T.int_col)
 from
-  (select id, int_col, bigint_col from functional.alltypestiny) T
+  (select id, int_col, bigint_col from alltypestiny) T
 where false
 ---- RESULTS
 NULL,0
@@ -132,13 +132,13 @@ BIGINT, BIGINT
 ---- QUERY
 # IMPALA-2088: Test empty union operands with analytic functions.
 select lead(-496, 81) over (order by t1.double_col desc, t1.id asc)
-from functional.alltypestiny t1 where 5 = 6
+from alltypestiny t1 where 5 = 6
 union
 select 794.67
-from functional.alltypes t1 where 5 = 6
+from alltypes t1 where 5 = 6
 union all
 select coalesce(10.4, int_col)
-from functional.alltypes where false
+from alltypes where false
 ---- RESULTS
 ---- TYPES
 DECIMAL
diff --git a/testdata/workloads/functional-query/queries/QueryTest/inline-view-limit.test b/testdata/workloads/functional-query/queries/QueryTest/inline-view-limit.test
index 96e2fcc..a483f4f 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/inline-view-limit.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/inline-view-limit.test
@@ -96,7 +96,7 @@ INT, INT
 # but filters no rows.
 select count(*) from (
   select id from
-    (select id from functional.alltypes limit 7300) v1
+    (select id from alltypes limit 7300) v1
   where id >= 0) v2
 ---- RESULTS
 7300
diff --git a/testdata/workloads/functional-query/queries/QueryTest/inline-view.test b/testdata/workloads/functional-query/queries/QueryTest/inline-view.test
index 7a2bfae..284d3ea 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/inline-view.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/inline-view.test
@@ -343,7 +343,7 @@ tinyint, tinyint, tinyint, tinyint, tinyint, tinyint
 ====
 ---- QUERY
 # Semi and inner join on a table and on inline views made up of constant selects
-select x.date_string_col, z.* from functional.alltypessmall x
+select x.date_string_col, z.* from alltypessmall x
 left semi join
 (select 1 a, 3 b union all select 1 a, 3 b) y on y.a = x.id
 inner join
@@ -399,7 +399,7 @@ TINYINT, TINYINT, TINYINT, TINYINT, TINYINT, TINYINT
 ====
 ---- QUERY
 # Semi and inner join on a table and on inline views made up of values statements
-select x.date_string_col, z.* from functional.alltypessmall x
+select x.date_string_col, z.* from alltypessmall x
 left semi join
 (values(1 a, 3 b), (1, 3)) y on y.a = x.id
 inner join
@@ -412,7 +412,7 @@ STRING, TINYINT, TINYINT
 ====
 ---- QUERY
 # Select constant with unreferenced aggregate in subquery
-select 1 from (select count(*) from functional.alltypessmall) x
+select 1 from (select count(*) from alltypessmall) x
 ---- RESULTS
 1
 ---- TYPES
@@ -420,7 +420,7 @@ tinyint
 ====
 ---- QUERY
 # Select constant with unreferenced distinct aggregate in subquery
-select 1 from (select count(distinct tinyint_col) from functional.alltypessmall) x
+select 1 from (select count(distinct tinyint_col) from alltypessmall) x
 ---- RESULTS
 1
 ---- TYPES
@@ -428,7 +428,7 @@ tinyint
 ====
 ---- QUERY
 # Select aggregate from unreferenced aggregate in subquery
-select count(*) from (select count(*) from functional.alltypessmall) x
+select count(*) from (select count(*) from alltypessmall) x
 ---- RESULTS
 1
 ---- TYPES
@@ -436,7 +436,7 @@ bigint
 ====
 ---- QUERY
 # Select * from aggregate in subquery
-select * from (select count(*) from functional.alltypessmall) x
+select * from (select count(*) from alltypessmall) x
 ---- RESULTS
 100
 ---- TYPES
@@ -444,7 +444,7 @@ bigint
 ====
 ---- QUERY
 # Select from aliased aggregate in subquery
-select c from (select count(*) c from functional.alltypessmall) x
+select c from (select count(*) c from alltypessmall) x
 ---- RESULTS
 100
 ---- TYPES
@@ -452,7 +452,7 @@ bigint
 ====
 ---- QUERY
 # Select aggregate from aliased aggregate in subquery
-select count(c) from (select count(*) c from functional.alltypessmall) x
+select count(c) from (select count(*) c from alltypessmall) x
 ---- RESULTS
 1
 ---- TYPES
@@ -460,7 +460,7 @@ bigint
 ====
 ---- QUERY
 # Select aggregate from aggregate of basetable column in subquery
-select count(1) from (select count(tinyint_col) from functional.alltypessmall) x
+select count(1) from (select count(tinyint_col) from alltypessmall) x
 ---- RESULTS
 1
 ---- TYPES
@@ -469,7 +469,7 @@ bigint
 ---- QUERY
 # Select aggregate from aggregate in subquery with group by
 select count(1) from
-(select count(*) from functional.alltypessmall group by tinyint_col) x
+(select count(*) from alltypessmall group by tinyint_col) x
 ---- RESULTS
 10
 ---- TYPES
diff --git a/testdata/workloads/functional-query/queries/QueryTest/limit.test b/testdata/workloads/functional-query/queries/QueryTest/limit.test
index 49c6129..c11c328 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/limit.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/limit.test
@@ -324,7 +324,7 @@ int, boolean, tinyint, smallint, int, bigint, float, double, string, string, tim
 ====
 ---- QUERY
 # With an offset that reduces the result size (limit is much bigger than number of rows)
-select id from functional.alltypessmall order by 1 limit 10000 offset 99;
+select id from alltypessmall order by 1 limit 10000 offset 99;
 ---- RESULTS
 99
 ---- TYPES
@@ -332,7 +332,7 @@ int
 ====
 ---- QUERY
 # With an offset that is bigger than the number of rows
-select id from functional.alltypessmall order by 1 limit 1 offset 1000;
+select id from alltypessmall order by 1 limit 1 offset 1000;
 ---- RESULTS
 ---- TYPES
 int
diff --git a/testdata/workloads/functional-query/queries/QueryTest/misc.test b/testdata/workloads/functional-query/queries/QueryTest/misc.test
index bfe27d2..903101c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/misc.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/misc.test
@@ -122,7 +122,7 @@ int, string, bigint
 ====
 ---- QUERY
 # where clause is a SlotRef
-SELECT count(*) from functional.alltypes where bool_col
+SELECT count(*) from alltypes where bool_col
 ---- RESULTS
 3650
 ---- TYPES
@@ -130,7 +130,7 @@ bigint
 ====
 ---- QUERY
 # having clause is a SlotRef
-SELECT count(*) from functional.alltypes group by bool_col having bool_col
+SELECT count(*) from alltypes group by bool_col having bool_col
 ---- RESULTS
 3650
 ---- TYPES
diff --git a/testdata/workloads/functional-query/queries/QueryTest/sort.test b/testdata/workloads/functional-query/queries/QueryTest/sort.test
index 0fe7efa..3e6bc01 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/sort.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/sort.test
@@ -4094,7 +4094,7 @@ INT, BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, STRING, STRING, TIM
 ====
 ---- QUERY
 # Regression test for IMPALA-1148.
-select 1 from functional.alltypestiny order by 1
+select 1 from alltypestiny order by 1
 ---- RESULTS
 1
 1
@@ -4110,7 +4110,7 @@ TINYINT
 ---- QUERY
 # Test for corner case: fixed length data but only NULL var-len slots.
 select id, cast(NULL as string) str
-from functional.alltypestiny
+from alltypestiny
 order by 2, 1
 ---- RESULTS
 0,'NULL'
@@ -4127,7 +4127,7 @@ INT, STRING
 ---- QUERY
 # Test for corner case: fixed length data but only zero-length var len data.
 select id, ''
-from functional.alltypestiny
+from alltypestiny
 order by 2 desc, 1 desc
 ---- RESULTS
 7,''
diff --git a/testdata/workloads/functional-query/queries/QueryTest/subquery-single-node.test b/testdata/workloads/functional-query/queries/QueryTest/subquery-single-node.test
index 8cff141..433d1a1 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/subquery-single-node.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/subquery-single-node.test
@@ -2,10 +2,10 @@
 ---- QUERY
 # Uncorrelated subquery in binary predicate that returns scalar value at runtime.
 # num_nodes = 1 is set for this file by the python test.
-SELECT count(id) FROM functional.alltypes
+SELECT count(id) FROM alltypes
 WHERE int_col =
   (SELECT int_col
-   FROM functional.alltypessmall
+   FROM alltypessmall
    WHERE id = 1)
 ---- RESULTS
 730
@@ -14,10 +14,10 @@ BIGINT
 ====
 ---- QUERY
 # Uncorrelated subquery in arithmetic expr that returns scalar value at runtime.
-SELECT count(id) FROM functional.alltypes
+SELECT count(id) FROM alltypes
 WHERE int_col =
   3 * (SELECT int_col
-       FROM functional.alltypessmall
+       FROM alltypessmall
        WHERE id = 1)
 ---- RESULTS
 730
diff --git a/testdata/workloads/functional-query/queries/QueryTest/subquery.test b/testdata/workloads/functional-query/queries/QueryTest/subquery.test
index 0a38fb4..40f2ec3 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/subquery.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/subquery.test
@@ -2,8 +2,8 @@
 ---- QUERY
 # Uncorrelated IN subquery
 select a.id, a.int_col, a.string_col
-from functional.alltypessmall a
-where a.id in (select id from functional.alltypestiny where bool_col = false)
+from alltypessmall a
+where a.id in (select id from alltypestiny where bool_col = false)
 and a.id < 5
 ---- RESULTS
 1,1,'1'
@@ -14,8 +14,8 @@ INT, INT, STRING
 ---- QUERY
 # Uncorrelated IN subquery with IS NOT NULL predicate
 select a.id, a.int_col, a.string_col
-from functional.alltypes a
-where a.int_col in (select int_col from functional.alltypesagg where int_col is not null)
+from alltypes a
+where a.int_col in (select int_col from alltypesagg where int_col is not null)
 and a.id < 5
 ---- RESULTS
 1,1,'1'
@@ -28,8 +28,8 @@ INT, INT, STRING
 ---- QUERY
 # Uncorrelated IN subquery with IS NULL predicate (empty result)
 select a.id, a.int_col, a.string_col
-from functional.alltypes a
-where a.id in (select int_col from functional.alltypesagg where int_col is null)
+from alltypes a
+where a.id in (select int_col from alltypesagg where int_col is null)
 and a.id < 5
 ---- RESULTS
 ---- TYPES
@@ -38,8 +38,8 @@ INT, INT, STRING
 ---- QUERY
 # Uncorrelated NOT IN subquery
 select id, year, month
-from functional.alltypessmall
-where id not in (select id from functional.alltypestiny where bool_col = false)
+from alltypessmall
+where id not in (select id from alltypestiny where bool_col = false)
 and id < 5
 ---- RESULTS
 0,2009,1
@@ -51,8 +51,8 @@ INT, INT, INT
 ---- QUERY
 # Uncorrelated NOT IN subquery that returns only null values
 select id
-from functional.alltypestiny
-where int_col not in (select int_col from functional.alltypesagg where int_col is null)
+from alltypestiny
+where int_col not in (select int_col from alltypesagg where int_col is null)
 and id < 10
 ---- RESULTS
 ---- TYPES
@@ -61,8 +61,8 @@ INT
 ---- QUERY
 # Uncorrelated NOT IN subquery that returns null and non-null values
 select id
-from functional.alltypestiny
-where int_col not in (select int_col from functional.alltypesagg)
+from alltypestiny
+where int_col not in (select int_col from alltypesagg)
 ---- RESULTS
 ---- TYPES
 INT
@@ -71,8 +71,8 @@ INT
 # Uncorrelated NOT IN subquery that returns an empty set
 # (result should include all the tuples of the outer)
 select id, int_col
-from functional.alltypestiny
-where int_col not in (select int_col from functional.alltypesagg where id < 0)
+from alltypestiny
+where int_col not in (select int_col from alltypesagg where id < 0)
 order by id
 ---- RESULTS
 0,0
@@ -90,8 +90,8 @@ INT, INT
 # Outer with nulls and an uncorrelated NOT IN subquery that returns an empty set
 # (result should include NULLs in int_col)
 select id, int_col
-from functional.alltypesagg
-where int_col not in (select int_col from functional.alltypestiny where id < 0)
+from alltypesagg
+where int_col not in (select int_col from alltypestiny where id < 0)
 and id < 5
 order by id
 ---- RESULTS
@@ -107,9 +107,9 @@ INT, INT
 ---- QUERY
 # Outer with NULLS and an uncorrelated NOT IN
 select id
-from functional.alltypesagg
+from alltypesagg
 where int_col is null and
-int_col not in (select int_col from functional.alltypestiny)
+int_col not in (select int_col from alltypestiny)
 ---- RESULTS
 ---- TYPES
 INT
@@ -117,9 +117,9 @@ INT
 ---- QUERY
 # Outer with NULLS and an uncorrelated NOT IN subquery that returns an empty set
 select id, int_col
-from functional.alltypesagg
+from alltypesagg
 where int_col is null and
-int_col not in (select int_col from functional.alltypestiny where id < 0)
+int_col not in (select int_col from alltypestiny where id < 0)
 and id < 10
 order by id
 ---- RESULTS
@@ -131,8 +131,8 @@ INT, INT
 ---- QUERY
 # Correlated IN subquery
 select s.id, s.bool_col, s.int_col, s.date_string_col
-from functional.alltypessmall s
-where s.id in (select id from functional.alltypestiny t where t.int_col = s.int_col)
+from alltypessmall s
+where s.id in (select id from alltypestiny t where t.int_col = s.int_col)
 and s.bool_col = false
 ---- RESULTS
 1,false,1,'01/01/09'
@@ -142,10 +142,10 @@ INT, BOOLEAN, INT, STRING
 ---- QUERY
 # Correlated NOT IN subquery
 select s.id, s.bool_col, s.int_col
-from functional.alltypessmall s
+from alltypessmall s
 where s.id not in
   (select id
-   from functional.alltypestiny t
+   from alltypestiny t
    where s.int_col = t.int_col and t.bool_col = false)
 and s.id < 5
 order by s.id
@@ -162,9 +162,9 @@ INT, BOOLEAN, INT
 # (results should include all the tuples of the outer that pass
 # the remaining predicates)
 select id, int_col
-from functional.alltypestiny t
+from alltypestiny t
 where t.int_col not in
-  (select int_col from functional.alltypes a where t.id = a.id and a.bigint_col < 0)
+  (select int_col from alltypes a where t.id = a.id and a.bigint_col < 0)
 and id < 5
 order by id
 ---- RESULTS
@@ -179,9 +179,9 @@ INT, INT
 ---- QUERY
 # Correlated NOT IN subquery that returns nulls
 select id, int_col
-from functional.alltypestiny t
+from alltypestiny t
 where t.int_col not in
-  (select int_col from functional.alltypesagg a where int_col is null and a.id = t.id)
+  (select int_col from alltypesagg a where int_col is null and a.id = t.id)
 order by id
 ---- RESULTS
 1,1
@@ -198,9 +198,9 @@ INT, INT
 # Outer with nulls and a correlated NOT IN subquery that returns null and
 # non-null values
 select id, int_col
-from functional.alltypesagg a
+from alltypesagg a
 where a.int_col not in
-  (select int_col from functional.alltypesagg b where a.id = b.id)
+  (select int_col from alltypesagg b where a.id = b.id)
 and id < 10
 ---- RESULTS
 ---- TYPES
@@ -209,9 +209,9 @@ INT, INT
 ---- QUERY
 # Outer with nulls and a correlated NOT IN subquery that does not return nulls
 select id, int_col
-from functional.alltypesagg a
+from alltypesagg a
 where int_col not in
-  (select int_col from functional.alltypestiny t where a.id = t.id)
+  (select int_col from alltypestiny t where a.id = t.id)
 and bigint_col = 10
 order by id
 ---- RESULTS
@@ -230,7 +230,7 @@ INT, INT
 ---- QUERY
 # Correlated NOT IN subquery that returns an empty set
 select id, int_col, bigint_col
-from functional.alltypesagg a
+from alltypesagg a
 where int_col not in
   (select int_col from alltypestiny t where a.id = t.id and t.bigint_col < 0)
 and bigint_col = 10
@@ -275,8 +275,8 @@ BIGINT
 ---- QUERY
 # Correlated EXISTS subquery
 select count(*)
-from functional.alltypestiny t
-where exists (select * from functional.alltypessmall s where t.int_col = s.int_col)
+from alltypestiny t
+where exists (select * from alltypessmall s where t.int_col = s.int_col)
 and id < 4
 ---- RESULTS
 4
@@ -286,8 +286,8 @@ BIGINT
 ---- QUERY
 # Correlated NOT EXISTS subquery
 select id, int_col
-from functional.alltypessmall t
-where not exists (select 1 from functional.alltypestiny s where t.id = s.id)
+from alltypessmall t
+where not exists (select 1 from alltypestiny s where t.id = s.id)
 and month = 1 and int_col < 5
 order by id
 ---- RESULTS
@@ -307,8 +307,8 @@ INT, INT
 ---- QUERY
 # Uncorrelated EXISTS
 select id
-from functional.alltypestiny t
-where exists (select 1 from functional.alltypessmall where bool_col = false)
+from alltypestiny t
+where exists (select 1 from alltypessmall where bool_col = false)
 and bool_col = true
 order by id
 ---- RESULTS
@@ -322,8 +322,8 @@ INT
 ---- QUERY
 # Uncorrelated EXISTS that returns an empty set
 select 1
-from functional.alltypestiny t
-where exists (select null from functional.alltypessmall where id < 0)
+from alltypestiny t
+where exists (select null from alltypessmall where id < 0)
 and t.id > 0
 ---- RESULTS
 ---- TYPES
@@ -332,8 +332,8 @@ TINYINT
 ---- QUERY
 # Uncorrelated NOT EXISTS
 select id
-from functional.alltypestiny t
-where not exists (select 1 from functional.alltypessmall where bool_col = false)
+from alltypestiny t
+where not exists (select 1 from alltypessmall where bool_col = false)
 and bool_col = true
 ---- RESULTS
 ---- TYPES
@@ -342,8 +342,8 @@ INT
 ---- QUERY
 # Uncorrelated NOT EXISTS that returns an empty set
 select 1
-from functional.alltypestiny t
-where not exists (select null from functional.alltypessmall where id < 0)
+from alltypestiny t
+where not exists (select null from alltypessmall where id < 0)
 and t.id > 0
 ---- RESULTS
 1
@@ -359,8 +359,8 @@ TINYINT
 ---- QUERY
 # Uncorrelated aggregate subquery
 select count(*) from
-functional.alltypessmall t
-where t.id < (select max(id) from functional.alltypestiny)
+alltypessmall t
+where t.id < (select max(id) from alltypestiny)
 and t.bool_col = true
 ---- RESULTS
 4
@@ -370,8 +370,8 @@ BIGINT
 ---- QUERY
 # Uncorrelated aggregate subquery with count
 select id, int_col, year, month
-from functional.alltypessmall
-where int_col = (select count(*) from functional.alltypestiny)
+from alltypessmall
+where int_col = (select count(*) from alltypestiny)
 order by id
 ---- RESULTS
 8,8,2009,1
@@ -387,6 +387,8 @@ INT, INT, INT, INT
 ====
 ---- QUERY
 # Correlated aggregate subquery
+# Remove functional database qualification in the query below after IMPALA-10252 is
+# addressed.
 select id, int_col, year, month
 from functional.alltypessmall s
 where s.int_col = (select count(*) from functional.alltypestiny t where s.id = t.id)
@@ -410,13 +412,13 @@ INT, INT, INT, INT
 ---- QUERY
 # Multiple subquery predicates
 select id, bool_col, int_col, date_string_col
-from functional.alltypessmall s
-where s.id in (select id from functional.alltypestiny where bool_col = true)
+from alltypessmall s
+where s.id in (select id from alltypestiny where bool_col = true)
 and exists
   (select *
-   from functional.alltypesagg g
+   from alltypesagg g
    where s.int_col = g.int_col and g.bigint_col < 100)
-and s.int_col < (select count(*) from functional.alltypes where month = 1)
+and s.int_col < (select count(*) from alltypes where month = 1)
 ---- RESULTS
 2,true,2,'01/01/09'
 4,true,4,'01/01/09'
@@ -427,15 +429,15 @@ INT, BOOLEAN, INT, STRING
 ---- QUERY
 # Multiple nesting levels
 select month, count(*)
-from functional.alltypessmall s
+from alltypessmall s
 where id in
   (select id
-   from functional.alltypestiny t
+   from alltypestiny t
    where t.int_col <
      (select min(int_col)
-      from functional.alltypesagg a
+      from alltypesagg a
       where a.bool_col = false and exists
-      (select * from functional.alltypes b where b.id = a.id)))
+      (select * from alltypes b where b.id = a.id)))
 group by month
 ---- RESULTS
 1,4
@@ -445,11 +447,11 @@ INT, BIGINT
 ---- QUERY
 # Multiple tables in outer select block and in subqueries
 select t.id, t.month, t.year
-from functional.alltypestiny t left outer join functional.alltypessmall s
+from alltypestiny t left outer join alltypessmall s
 on s.id = t.id
 where t.int_col <
   (select avg(a.int_col) * 2
-   from functional.alltypesagg a left outer join functional.alltypes b
+   from alltypesagg a left outer join alltypes b
    on a.bigint_col = b.bigint_col
    where a.id = t.id and b.int_col < 10)
 order by t.id
@@ -468,13 +470,13 @@ INT, INT, INT
 # Subquery in the WITH clause
 with t as
   (select *
-   from functional.alltypessmall
+   from alltypessmall
    where id in
      (select id
-      from functional.alltypestiny
+      from alltypestiny
       where bool_col = false))
 select id, month, year from t where t.int_col =
-  (select count(*) from functional.alltypestiny where id < 5)
+  (select count(*) from alltypestiny where id < 5)
 ---- RESULTS
 5,1,2009
 ---- TYPES
@@ -483,12 +485,12 @@ INT, INT, INT
 ---- QUERY
 # Subquery in an inline view
 select s.id, s.year
-from functional.alltypessmall s left outer join
+from alltypessmall s left outer join
   (select *
-   from functional.alltypestiny t
-   where t.id in (select id from functional.alltypesagg)) b
+   from alltypestiny t
+   where t.id in (select id from alltypesagg)) b
 on (s.id = b.id)
-where s.int_col < (select max(int_col) from functional.alltypes) and s.id < 10
+where s.int_col < (select max(int_col) from alltypes) and s.id < 10
 order by s.id
 ---- RESULTS
 0,2009
@@ -506,7 +508,7 @@ INT, INT
 ---- QUERY
 # Subquery returning a decimal
 select id, double_col
-from functional.alltypestiny
+from alltypestiny
 where double_col < (select min(d3) from functional.decimal_tbl)
 order by id
 ---- RESULTS
@@ -521,7 +523,7 @@ INT, DOUBLE
 # Compare a decimal with the result of a subquery
 select d1, d2, d3
 from functional.decimal_tbl
-where d3 < (select max(double_col) from functional.alltypestiny)
+where d3 < (select max(double_col) from alltypestiny)
 ---- RESULTS
 1234,2222,1.2345678900
 ---- TYPES
@@ -530,7 +532,7 @@ DECIMAL, DECIMAL, DECIMAL
 ---- QUERY
 # Subquery returning a date
 select id, timestamp_col
-from functional.alltypestiny
+from alltypestiny
 where timestamp_col <= (select max(date_col)
   from functional.date_tbl)
 order by id;
@@ -551,7 +553,7 @@ INT, TIMESTAMP
 select date_col
 from functional.date_tbl
 where date_col >= DATE '1400-01-01' AND date_col >= (select max(timestamp_col)
-  from functional.alltypestiny);
+  from alltypestiny);
 ---- RESULTS
 2017-11-28
 2018-12-31
@@ -566,8 +568,8 @@ DATE
 ---- QUERY
 # Distinct in the outer select block
 select distinct bool_col
-from functional.alltypestiny t
-where 1 < (select count(*) from functional.alltypessmall)
+from alltypestiny t
+where 1 < (select count(*) from alltypessmall)
 ---- RESULTS
 false
 true
@@ -577,8 +579,8 @@ BOOLEAN
 ---- QUERY
 # Distinct with an unqualified star in the outer select block
 select distinct *
-from functional.alltypestiny t
-where 1 < (select avg(distinct id) from functional.alltypessmall)
+from alltypestiny t
+where 1 < (select avg(distinct id) from alltypessmall)
 and id < 2
 ---- RESULTS
 0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
@@ -589,8 +591,8 @@ INT, BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, FLOAT, DOUBLE, STRING, STRING, TIM
 ---- QUERY
 # Aggregate subquery in an IS NULL predicate
 select count(*)
-from functional.alltypestiny
-where (select max(int_col) from functional.alltypesagg where int_col is null) is null
+from alltypestiny
+where (select max(int_col) from alltypesagg where int_col is null) is null
 ---- RESULTS
 8
 ---- TYPES
@@ -599,8 +601,8 @@ BIGINT
 ---- QUERY
 # Aggregate subquery in an IS NOT NULL predicate
 select count(*)
-from functional.alltypestiny
-where (select max(int_col) from functional.alltypesagg where int_col is null) is not null
+from alltypestiny
+where (select max(int_col) from alltypesagg where int_col is null) is not null
 ---- RESULTS
 0
 ---- TYPES
@@ -609,8 +611,8 @@ BIGINT
 ---- QUERY
 # Correlated aggregate subquery in an IS NULL predicate
 select id, bool_col
-from functional.alltypestiny t
-where (select sum(int_col) from functional.alltypesagg g where t.id = g.id) is null
+from alltypestiny t
+where (select sum(int_col) from alltypesagg g where t.id = g.id) is null
 order by id
 ---- RESULTS
 0,true
@@ -620,8 +622,8 @@ INT, BOOLEAN
 ---- QUERY
 # Correlated aggregate subquery in an IS NOT NULL predicate
 select id, bool_col
-from functional.alltypestiny t
-where (select sum(int_col) from functional.alltypesagg g where t.id = g.id) is not null
+from alltypestiny t
+where (select sum(int_col) from alltypesagg g where t.id = g.id) is not null
 order by id
 ---- RESULTS
 1,false
@@ -637,9 +639,9 @@ INT, BOOLEAN
 ---- QUERY
 # Function with a scalar subquery
 select count(*)
-from functional.alltypestiny t
+from alltypestiny t
 where
- zeroifnull((select max(int_col) from functional.alltypesagg where int_col is null)) = 0
+ zeroifnull((select max(int_col) from alltypesagg where int_col is null)) = 0
 ---- RESULTS
 8
 ---- TYPES
@@ -648,9 +650,9 @@ BIGINT
 ---- QUERY
 # Function with a scalar subquery
 select id
-from functional.alltypestiny t
+from alltypestiny t
 where
-  nullifzero((select min(id) from functional.alltypesagg where int_col is null)) is null
+  nullifzero((select min(id) from alltypesagg where int_col is null)) is null
 and id < 5
 order by id
 ---- RESULTS
@@ -664,10 +666,10 @@ INT
 ====
 ---- QUERY
 # Between predicate with subqueries
-select id from functional.alltypessmall
+select id from alltypessmall
 where id between
-  (select min(bigint_col) from functional.alltypestiny) and
-  (select max(bigint_col) from functional.alltypestiny)
+  (select min(bigint_col) from alltypestiny) and
+  (select max(bigint_col) from alltypestiny)
 order by id
 ---- RESULTS
 0
@@ -687,10 +689,10 @@ INT
 ---- QUERY
 # Correlated EXISTS subquery with an analytic function and a group by clause
 select id, int_col
-from functional.alltypesagg a
+from alltypesagg a
 where exists
   (select id, sum(int_col) over (partition by bool_col)
-   from functional.alltypes b
+   from alltypes b
    where a.id = b.id and b.bigint_col < 100
    group by id, int_col, bool_col
    order by id)
@@ -768,9 +770,9 @@ BIGINT
 ---- QUERY
 # Regression test for IMPALA-1318.
 select count(t1.c) over () from
-  (select max(int_col) c from functional.alltypestiny) t1
+  (select max(int_col) c from alltypestiny) t1
 where t1.c not in
-  (select sum(t1.smallint_col) from functional.alltypes t1)
+  (select sum(t1.smallint_col) from alltypes t1)
 ---- RESULTS
 1
 ---- TYPES
@@ -875,10 +877,10 @@ TINYINT
 ---- QUERY
 # EXISTS subquery containing ORDER BY, LIMIT, and OFFSET (IMPALA-6934)
 SELECT count(*)
-FROM functional.alltypestiny t
+FROM alltypestiny t
 WHERE EXISTS
   (SELECT id
-   FROM functional.alltypestiny
+   FROM alltypestiny
    WHERE id < 5
    ORDER BY id LIMIT 10 OFFSET 6)
 ---- RESULTS
@@ -886,10 +888,10 @@ WHERE EXISTS
 ====
 ---- QUERY
 # Uncorrelated subquery in binary predicate that returns scalar value at runtime
-SELECT id FROM functional.alltypessmall
+SELECT id FROM alltypessmall
 WHERE int_col =
   (SELECT int_col
-   FROM functional.alltypessmall
+   FROM alltypessmall
    WHERE id = 1)
 ORDER BY id
 ---- RESULTS
@@ -910,10 +912,10 @@ INT
 ====
 ---- QUERY
 # Uncorrelated subquery in arithmetic expr that returns scalar value at runtime
-SELECT id FROM functional.alltypessmall
+SELECT id FROM alltypessmall
 WHERE int_col =
   3 * (SELECT int_col
-       FROM functional.alltypessmall
+       FROM alltypessmall
        WHERE id = 1)
 ORDER BY id
 ---- RESULTS
@@ -934,10 +936,10 @@ INT
 ====
 ---- QUERY
 # Uncorrelated subquery in binary predicate that returns no rows.
-SELECT id FROM functional.alltypessmall
+SELECT id FROM alltypessmall
 WHERE int_col =
   (SELECT int_col
-   FROM functional.alltypessmall
+   FROM alltypessmall
    WHERE id = -123)
 ORDER BY id
 ---- RESULTS
@@ -946,10 +948,10 @@ INT
 ====
 ---- QUERY
 # Uncorrelated subquery in arithmetic expr that returns no rows.
-SELECT id FROM functional.alltypessmall
+SELECT id FROM alltypessmall
 WHERE int_col =
   3 * (SELECT int_col
-       FROM functional.alltypessmall
+       FROM alltypessmall
        WHERE id = -123)
 ORDER BY id
 ---- RESULTS
@@ -958,10 +960,10 @@ INT
 ====
 ---- QUERY
 # Uncorrelated subquery in binary predicate that returns multiple rows
-SELECT id FROM functional.alltypessmall
+SELECT id FROM alltypessmall
 WHERE int_col =
   (SELECT int_col
-   FROM functional.alltypessmall)
+   FROM alltypessmall)
 ORDER BY id
 ---- RESULTS
 ---- CATCH
@@ -969,10 +971,10 @@ Subquery must not return more than one row:
 ====
 ---- QUERY
 # Uncorrelated subquery in arithmetic expr that returns multiple rows
-SELECT id FROM functional.alltypessmall
+SELECT id FROM alltypessmall
 WHERE int_col =
   3 * (SELECT int_col
-       FROM functional.alltypessmall)
+       FROM alltypessmall)
 ORDER BY id
 ---- RESULTS
 ---- CATCH
@@ -980,10 +982,10 @@ Subquery must not return more than one row:
 ====
 ---- QUERY
 # Uncorrelated subquery in binary predicate that returns scalar value at runtime
-SELECT count(id) FROM functional.alltypes
+SELECT count(id) FROM alltypes
 WHERE int_col =
   (SELECT int_col
-   FROM functional.alltypessmall
+   FROM alltypessmall
    WHERE id = 1)
 ---- RESULTS
 730
@@ -992,10 +994,10 @@ BIGINT
 ====
 ---- QUERY
 # Uncorrelated subquery in arithmetic expr that returns scalar value at runtime
-SELECT count(id) FROM functional.alltypes
+SELECT count(id) FROM alltypes
 WHERE int_col =
   3 * (SELECT int_col
-       FROM functional.alltypessmall
+       FROM alltypessmall
        WHERE id = 1)
 ---- RESULTS
 730
@@ -1013,7 +1015,7 @@ Subquery must not return more than one row:
 ---- QUERY
 # Subquery that returns more than one row
 # The error message must not reveal the definition of functional.alltypes_view
-SELECT id FROM functional.alltypes
+SELECT id FROM alltypes
 WHERE id = (SELECT bigint_col FROM functional.alltypes_view)
 ---- RESULTS
 ---- CATCH
@@ -1021,8 +1023,8 @@ Subquery must not return more than one row: SELECT bigint_col FROM functional.al
 ====
 ---- QUERY
 # Runtime scalar subquery with offset.
-select count(*) from functional.alltypes
-where 7 = (select id from functional.alltypestiny
+select count(*) from alltypes
+where 7 = (select id from alltypestiny
            order by id limit 8 offset 7)
 ---- RESULTS
 7300
@@ -1031,9 +1033,9 @@ BIGINT
 ====
 ---- QUERY
 # IMPALA-7108: Select from an inline view that returns a single row.
-select count(*) from functional.alltypes
+select count(*) from alltypes
 where int_col = (select f.id from (
-                 select * from functional.alltypes order by 1 limit 1) f)
+                 select * from alltypes order by 1 limit 1) f)
 ---- RESULTS
 730
 ---- TYPES
@@ -1041,8 +1043,8 @@ BIGINT
 ====
 ---- QUERY
 # Basic scalar in select list
-select id, 1+(select min(id) from functional.alltypessmall)
-  from functional.alltypestiny
+select id, 1+(select min(id) from alltypessmall)
+  from alltypestiny
 order by id
 ---- RESULTS
 0,1
@@ -1058,8 +1060,8 @@ INT, BIGINT
 ====
 ---- QUERY
 # No aggregate however limit 1
-select id, 1+(select id from functional.alltypessmall where id=0 group by id limit 1)
-  from functional.alltypestiny
+select id, 1+(select id from alltypessmall where id=0 group by id limit 1)
+  from alltypestiny
 order by id
 ---- RESULTS
 0,1
@@ -1088,8 +1090,8 @@ INT, BIGINT
 ====
 ---- QUERY
 # correlated scalar currently unsupported
-select id, (select count(id) from functional.alltypessmall where id=t.id)
-  from functional.alltypestiny t
+select id, (select count(id) from alltypessmall where id=t.id)
+  from alltypestiny t
 order by id
 ---- RESULTS
 ---- CATCH
@@ -1099,9 +1101,9 @@ A correlated scalar subquery is not supported in the expression:
 # Uncorrelated Scalar Aggregate in select list combined with aggregation in parent query
 select
 max(a.id),
-(select max(id) from functional.alltypestiny),
-(select min(id) from functional.alltypestiny)
-from functional.alltypessmall a
+(select max(id) from alltypestiny),
+(select min(id) from alltypestiny)
+from alltypessmall a
 ---- RESULTS
 99,7,0
 ---- TYPES
@@ -1111,9 +1113,9 @@ INT, INT, INT
 # Empty tables or false predicate should result in NULLs
 # Count aggregates on empty return 0
 select id,
-  (select sum(f2) from functional.emptytable),
-  (select count(id) from functional.alltypestiny where id > 100)
-from functional.alltypestiny
+  (select sum(f2) from emptytable),
+  (select count(id) from alltypestiny where id > 100)
+from alltypestiny
 order by id
 ---- RESULTS
 0,NULL,0
@@ -1131,8 +1133,8 @@ INT, BIGINT, BIGINT
 # nested subquery over a view with an outer predicate on the subquery column
 select count(id) from
   (select id,
-    (select count(*) as cnt from functional.alltypestiny where int_col
-    <= (select max(int_col) from functional.alltypes_view)) as c from functional.dimtbl) T
+    (select count(*) as cnt from alltypestiny where int_col
+    <= (select max(int_col) from functional.alltypes_view)) as c from dimtbl) T
   where t.c <10
 ---- RESULTS
 10
@@ -1141,8 +1143,8 @@ BIGINT
 ====
 ---- QUERY
 # Empty tables should result in NULLs
-select id, id in (select min(id) from functional.alltypessmall)
-  from functional.alltypestiny
+select id, id in (select min(id) from alltypessmall)
+  from alltypestiny
 order by id
 ---- RESULTS
 0,true
@@ -1159,8 +1161,8 @@ INT, BOOLEAN
 ---- QUERY
 # With a join in the subquery
 select id,
-  (select count(*) from functional.alltypessmall join functional.alltypestiny using (id))
-  from functional.alltypestiny
+  (select count(*) from alltypessmall join alltypestiny using (id))
+  from alltypestiny
 order by id
 ---- RESULTS
 0,8
@@ -1178,7 +1180,7 @@ INT, BIGINT
 # EXISTS predicates are handled at plantime so we need to special case their rewrites
 select id,
   exists (select max(id) from dimtbl)
-  from functional.alltypestiny
+  from alltypestiny
 order by id
 ---- RESULTS
 0,true
@@ -1410,9 +1412,9 @@ INT
 ---- QUERY
 # Subquery that only returns NULL values. The IN predicate should always
 # evaluate to false in this case, because the hand-picked values have a null 'int_col'.
-select id, int_col from functional.alltypes t
+select id, int_col from alltypes t
 where t.id = 42 or t.int_col IN (
-  select int_col from functional.alltypesagg where id in (1000,2000,3000,4000))
+  select int_col from alltypesagg where id in (1000,2000,3000,4000))
 ---- TYPES
 INT, INT
 ---- RESULTS
@@ -1420,8 +1422,8 @@ INT, INT
 ====
 ---- QUERY
 # IMPALA-9949: subquery in select list that returns 0 rows results in NULLs.
-select id, (select min(int_col) from functional.alltypes having min(int_col) < 0)
-from functional.alltypestiny
+select id, (select min(int_col) from alltypes having min(int_col) < 0)
+from alltypestiny
 ---- RESULTS
 0,NULL
 1,NULL
diff --git a/testdata/workloads/functional-query/queries/QueryTest/top-n.test b/testdata/workloads/functional-query/queries/QueryTest/top-n.test
index 37c986c..84afbbc 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/top-n.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/top-n.test
@@ -784,7 +784,7 @@ int,int,int,int
 ---- QUERY
 # Test limit 0 from sub query
 select sum(a.int_col) from
-  (select int_col from functional.alltypes order by int_col limit 0) a
+  (select int_col from alltypes order by int_col limit 0) a
 ---- RESULTS
 NULL
 ---- TYPES
@@ -883,7 +883,7 @@ int, boolean, tinyint, smallint, int, bigint, float, double, string, string, tim
 ---- QUERY
 # Taken from limit.test
 # With an offset that reduces the result size (limit is much bigger than number of rows)
-select id from functional.alltypessmall order by 1 limit 10000 offset 99;
+select id from alltypessmall order by 1 limit 10000 offset 99;
 ---- RESULTS
 99
 ---- TYPES
@@ -892,7 +892,7 @@ int
 ---- QUERY
 # Taken from limit.test
 # With an offset that is bigger than the number of rows
-select id from functional.alltypessmall order by 1 limit 1 offset 1000;
+select id from alltypessmall order by 1 limit 1 offset 1000;
 ---- RESULTS
 ---- TYPES
 int
@@ -931,7 +931,7 @@ int, int
 ---- QUERY
 # Taken from joins.test
 # cross join with nulls and constant table
-select id, tinyint_col, t1.c from functional.alltypesagg
+select id, tinyint_col, t1.c from alltypesagg
 cross join (values(NULL c, 1, 2)) as t1
 order by tinyint_col nulls first, id, t1.c
 limit 6
@@ -1316,7 +1316,7 @@ BIGINT, STRING
 ---- QUERY
 # Multiple views in with-clause. All views are used in a union.
 # Taken from with-clause.test
-with t1 as (select int_col x, bigint_col y from functional.alltypestiny),
+with t1 as (select int_col x, bigint_col y from alltypestiny),
 t2 as (select 1 x , 10 y), t3 as (values(2 x , 20 y), (3, 30))
 select * from t1 union all select * from t2 union all (select * from t3) order by x limit 20
 ---- RESULTS
@@ -1338,7 +1338,7 @@ INT, BIGINT
 # Self-join of with-clause table to make sure the join op is properly set
 # in the cloned inline-view instances.
 # Taken from with-clause.test
-with t as (select int_col x, bigint_col y from functional.alltypestiny order by id limit 2)
+with t as (select int_col x, bigint_col y from alltypestiny order by id limit 2)
 select * from t t1 left outer join t t2 on t1.y = t2.x full outer join t t3 on t2.y = t3.x
 order by t1.x limit 10
 ---- RESULTS
@@ -1391,4 +1391,4 @@ offset 9223372036854775807
 ---- RESULTS
 ---- TYPES
 STRING
-====
\ No newline at end of file
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/union.test b/testdata/workloads/functional-query/queries/QueryTest/union.test
index 6448fc7..18889f6 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/union.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/union.test
@@ -1133,6 +1133,9 @@ bigint
 # IMPALA-4883: The second union operand references a char column, which causes codegen
 # to fail and be disabled for that operand and all operands that follow it. For this query
 # codegen is enabled only for the first operand.
+#
+# Qualify table chars_tiny with functional database below as the same table is not
+# defined in other databases.
 select count(s) from (
   select cast(id as string) as s from alltypestiny
   union all
diff --git a/testdata/workloads/functional-query/queries/QueryTest/with-clause.test b/testdata/workloads/functional-query/queries/QueryTest/with-clause.test
index d86b682..151ac25 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/with-clause.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/with-clause.test
@@ -1,7 +1,7 @@
 ====
 ---- QUERY
 # Basic test with a single with-clause view.
-with t as (select int_col x, bigint_col y from functional.alltypestiny)
+with t as (select int_col x, bigint_col y from alltypestiny)
 select count(x), count(y) from t
 ---- RESULTS
 8,8
@@ -20,8 +20,8 @@ BIGINT, STRING
 ====
 ---- QUERY
 # Basic tests with a single with-clause view with column labels.
-with t(c1, c2) as (select int_col, bigint_col y from functional.alltypestiny)
-select * from t limit 1
+with t(c1, c2) as (select int_col, bigint_col y from alltypestiny)
+select * from t order by c1, c2 limit 1
 ---- RESULTS
 0,0
 ---- TYPES
@@ -30,8 +30,8 @@ INT, BIGINT
 C1, C2
 ====
 ---- QUERY
-with t(c1) as (select int_col, bigint_col from functional.alltypestiny)
-select * from t limit 1
+with t(c1) as (select int_col, bigint_col from alltypestiny)
+select * from t order by c1 limit 1
 ---- RESULTS
 0,0
 ---- TYPES
@@ -40,14 +40,14 @@ INT, BIGINT
 C1, BIGINT_COL
 ====
 ---- QUERY
-with t(c1, c2) as (select int_col from functional.alltypestiny)
-select * from t limit 1
+with t(c1, c2) as (select int_col from alltypestiny)
+select * from t order by c1,c2 limit 1
 ---- CATCH
 AnalysisException: WITH-clause view 't' returns 1 columns, but 2 labels were specified.
 ====
 ---- QUERY
 # Multiple views in with-clause. Only one view is used.
-with t1 as (select int_col x, bigint_col y from functional.alltypestiny),
+with t1 as (select int_col x, bigint_col y from alltypestiny),
 t2 as (select 1 x, 10 y), t3 as (values(2 x, 20 y), (3, 30))
 select x, y from t2
 ---- RESULTS
@@ -57,7 +57,7 @@ TINYINT, TINYINT
 ====
 ---- QUERY
 # Multiple views in with-clause. All views are used in a union.
-with t1 as (select int_col x, bigint_col y from functional.alltypestiny),
+with t1 as (select int_col x, bigint_col y from alltypestiny),
 t2 as (select 1 x, 10 y), t3 as (values(2 x, 20 y), (3, 30))
 select * from t1 union all select * from t2 union all (select * from t3) order by x limit 20
 ---- RESULTS
@@ -76,9 +76,9 @@ select * from t1 union all select * from t2 union all (select * from t3) order b
 INT, BIGINT
 ====
 ---- QUERY
-with t1(c1, c2) as (select int_col x, bigint_col y from functional.alltypestiny),
+with t1(c1, c2) as (select int_col x, bigint_col y from alltypestiny),
 t2(c3, c4) as (select 1 x, 10 y)
-select * from t1 limit 1 union all select * from t2 limit 1
+select * from t1 order by c1, c2 limit 1 union all select * from t2 order by c3, c4 limit 1
 ---- RESULTS
 0,0
 1,10
@@ -89,9 +89,9 @@ C1, C2
 ====
 ---- QUERY
 # Multiple views in with-clause. All views are used in a join.
-with t1 as (select int_col x, bigint_col y from functional.alltypes limit 2),
-t2 as (select int_col x, bigint_col y from functional.alltypestiny limit 2),
-t3 as (select int_col x, bigint_col y from functional.alltypessmall limit 2)
+with t1 as (select distinct int_col x, bigint_col y from alltypes order by 1,2 limit 2),
+t2 as (select distinct int_col x, bigint_col y from alltypestiny order by 1,2 limit 2),
+t3 as (select distinct int_col x, bigint_col y from alltypessmall order by 1,2 limit 2)
 select * from t1, t2, t3 where t1.x = t2.x and t2.x = t3.x
 ---- RESULTS
 0,0,0,0,0,0
@@ -102,7 +102,7 @@ INT, BIGINT, INT, BIGINT, INT, BIGINT
 ---- QUERY
 # Self-join of with-clause table to make sure the join op is properly set
 # in the cloned inline-view instances.
-with t as (select int_col x, bigint_col y from functional.alltypestiny order by id limit 2)
+with t as (select int_col x, bigint_col y from alltypestiny order by id limit 2)
 select * from t t1 left outer join t t2 on t1.y = t2.x full outer join t t3 on t2.y = t3.x
 order by t1.x limit 10
 ---- RESULTS
diff --git a/tests/query_test/test_queries.py b/tests/query_test/test_queries.py
index 61618f9..ed956b8 100644
--- a/tests/query_test/test_queries.py
+++ b/tests/query_test/test_queries.py
@@ -146,6 +146,14 @@ class TestQueries(ImpalaTestSuite):
     self.run_test_case('QueryTest/inline-view-limit', vector)
 
   def test_subquery(self, vector):
+    if vector.get_value('table_format').file_format == 'parquet':
+     pytest.xfail("IMPALA-10252: Query returns less number of rows.")
+    if vector.get_value('table_format').file_format == 'hbase':
+        pytest.xfail("Table alltypesagg is populated differently in database "
+                "functional and functional_hbase: there are nulls in column "
+                "int_col in the former and none in the latter. "
+                "Testing query: select int_col from alltypesagg where int_col "
+                "is null")
     self.run_test_case('QueryTest/subquery', vector)
 
   def test_subquery_single_node(self, vector):