You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ar...@apache.org on 2019/07/25 17:45:21 UTC

[impala] 03/04: IMPALA-8772: Import Testcase failed for SQL without table refs

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

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

commit 33b5c3dcbe4d0bd574122af5afd8af8a07ddc78a
Author: Jiawei Wang <ji...@cloudera.com>
AuthorDate: Mon Jul 22 14:28:49 2019 -0700

    IMPALA-8772: Import Testcase failed for SQL without table refs
    
    Description:
    Query like this: select 5 * 4; can generate valid testcase, but
    cannot be loaded because it does not involve any table and view
    references.
    
    Fix:
    Add null check for CatalogOpExecutor when doing testcase data load
    
    Tests:
    tests/metadata/test_testcase_builder
    
    Change-Id: I50d77d7de44bb140598a1e8db4da85a5ec87f31e
    TODO: NEED TO DELETE GENERATED TESTCASE AFTER THIS IS DONE
    Reviewed-on: http://gerrit.cloudera.org:8080/13893
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../apache/impala/service/CatalogOpExecutor.java   | 66 ++++++++++++----------
 tests/metadata/test_testcase_builder.py            | 57 +++++++++++++++++++
 2 files changed, 92 insertions(+), 31 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 0a15915..0951349 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -445,42 +445,46 @@ public class CatalogOpExecutor {
     // the state in-memory and do not flush it to HMS, the older state can be recovered
     // by loading everything back from HMS. For ex: INVALIDATE METADATA.
     int numDbsAdded = 0;
-    for (TDatabase thriftDb: testCaseData.getDbs()) {
-      Db db = Db.fromTDatabase(thriftDb);
-      // Set a new version to force an overwrite if a Db already exists with the same
-      // name.
-      db.setCatalogVersion(catalog_.incrementAndGetCatalogVersion());
-      Db ret = catalog_.addDb(db.getName(), db.getMetaStoreDb());
-      if (ret != null) {
-        ++numDbsAdded;
-        response.result.addToUpdated_catalog_objects(db.toTCatalogObject());
+    if (testCaseData.getDbs() != null) {
+      for (TDatabase thriftDb : testCaseData.getDbs()) {
+        Db db = Db.fromTDatabase(thriftDb);
+        // Set a new version to force an overwrite if a Db already exists with the same
+        // name.
+        db.setCatalogVersion(catalog_.incrementAndGetCatalogVersion());
+        Db ret = catalog_.addDb(db.getName(), db.getMetaStoreDb());
+        if (ret != null) {
+          ++numDbsAdded;
+          response.result.addToUpdated_catalog_objects(db.toTCatalogObject());
+        }
       }
     }
 
     int numTblsAdded = 0;
     int numViewsAdded = 0;
-    for(TTable tTable: testCaseData.tables_and_views) {
-      Db db = catalog_.getDb(tTable.db_name);
-      // Db should have been created by now.
-      Preconditions.checkNotNull(db, String.format("Missing db %s", tTable.db_name));
-      Table t = Table.fromThrift(db, tTable);
-      // Set a new version to force an overwrite if a table already exists with the same
-      // name.
-      t.setCatalogVersion(catalog_.incrementAndGetCatalogVersion());
-      catalog_.addTable(db, t);
-      if (t instanceof View) {
-        ++numViewsAdded;
-      } else {
-        ++numTblsAdded;
-      }
-      // The table lock is needed here since toTCatalogObject() calls Table#toThrift()
-      // which expects the current thread to hold this lock. For more details refer
-      // to IMPALA-4092.
-      t.getLock().lock();
-      try {
-        response.result.addToUpdated_catalog_objects(t.toTCatalogObject());
-      } finally {
-        t.getLock().unlock();
+    if (testCaseData.getTables_and_views() != null) {
+      for (TTable tTable : testCaseData.tables_and_views) {
+        Db db = catalog_.getDb(tTable.db_name);
+        // Db should have been created by now.
+        Preconditions.checkNotNull(db, String.format("Missing db %s", tTable.db_name));
+        Table t = Table.fromThrift(db, tTable);
+        // Set a new version to force an overwrite if a table already exists with the same
+        // name.
+        t.setCatalogVersion(catalog_.incrementAndGetCatalogVersion());
+        catalog_.addTable(db, t);
+        if (t instanceof View) {
+          ++numViewsAdded;
+        } else {
+          ++numTblsAdded;
+        }
+        // The table lock is needed here since toTCatalogObject() calls Table#toThrift()
+        // which expects the current thread to hold this lock. For more details refer
+        // to IMPALA-4092.
+        t.getLock().lock();
+        try {
+          response.result.addToUpdated_catalog_objects(t.toTCatalogObject());
+        } finally {
+          t.getLock().unlock();
+        }
       }
     }
     StringBuilder responseStr = new StringBuilder();
diff --git a/tests/metadata/test_testcase_builder.py b/tests/metadata/test_testcase_builder.py
new file mode 100644
index 0000000..242eae9
--- /dev/null
+++ b/tests/metadata/test_testcase_builder.py
@@ -0,0 +1,57 @@
+# 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.
+
+from tests.common.impala_test_suite import ImpalaTestSuite
+from tests.common.test_dimensions import (
+  create_single_exec_option_dimension,
+  create_uncompressed_text_dimension)
+
+
+class TestTestcaseBuilder(ImpalaTestSuite):
+
+  @classmethod
+  def get_workload(self):
+    return 'functional-query'
+
+  @classmethod
+  def add_test_dimensions(cls):
+    super(TestTestcaseBuilder, cls).add_test_dimensions()
+    # This test only needs to be run once.
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
+    cls.ImpalaTestMatrix.add_dimension(
+      create_uncompressed_text_dimension(cls.get_workload()))
+
+  def test_query_without_from(self):
+    # Generate Testcase Data for query without table reference
+    testcase_generate_query = """COPY TESTCASE TO 'hdfs:///tmp' SELECT 5 * 20"""
+    result = self.execute_query_expect_success(self.client, testcase_generate_query)
+    assert len(result.data) == 1, "Testcase builder wrong result: {0}".format(result.data)
+
+    # Check file exists
+    testcase_path = str(result.data)[1: -1]
+    index = testcase_path.index('/tmp')
+    hdfs_path = testcase_path[index:-1]
+    assert self.hdfs_client.exists(hdfs_path), "File not generated {0}".format(hdfs_path)
+
+    try:
+      # Test load testcase works
+      testcase_load_query = "COPY TESTCASE FROM {0}".format(testcase_path)
+      self.execute_query_expect_success(self.client, testcase_load_query)
+    finally:
+      # Delete testcase file from tmp
+      status = self.hdfs_client.delete_file_dir(hdfs_path)
+      assert status, "Delete generated testcase file failed with {0}".format(status)