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 2023/07/27 06:07:37 UTC

[impala] branch master updated: IMPALA-11941: (Addendum) ease testing other JDKs

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


The following commit(s) were added to refs/heads/master by this push:
     new e27e4eb54 IMPALA-11941: (Addendum) ease testing other JDKs
e27e4eb54 is described below

commit e27e4eb54a97b299e079079905626c0264e1adc7
Author: Michael Smith <mi...@cloudera.com>
AuthorDate: Thu Jun 29 09:58:45 2023 -0700

    IMPALA-11941: (Addendum) ease testing other JDKs
    
    Makes it simpler to build with one JDK and run tests with another.
    TEST_JDK_VERSION sets IMPALA_JDK_VERSION before running tests, so the
    Impala cluster is started with that JDK. TEST_JAVA_HOME_OVERRIDE sets
    IMPALA_JAVA_HOME_OVERRIDE if a non-OS version of Java is required.
    
    Restart Kudu with original JAVA_HOME in frontend tests.
    
    Also skips restarting Hive, Kudu, and Ranger in tests as they'll restart
    with a different JDK than originally started with.
    
    Testing:
    1. built normally
    2. ran "TEST_JDK_VERSION=17 run-all-tests.sh"
    3. verified various logs contain "java.specification.version:17"
    
    Change-Id: I46b5515efd9537d63b843dbc42aa93b376efce00
    Reviewed-on: http://gerrit.cloudera.org:8080/20143
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 bin/run-all-tests.sh                                           |  9 +++++++++
 .../apache/impala/customservice/KuduHMSIntegrationTest.java    | 10 ++++++++++
 tests/authorization/test_ranger.py                             |  4 +++-
 tests/common/environ.py                                        |  4 ++++
 tests/common/skip.py                                           |  3 ++-
 tests/custom_cluster/test_catalog_hms_failures.py              |  3 +++
 tests/custom_cluster/test_custom_hive_configs.py               |  3 ++-
 tests/custom_cluster/test_kudu.py                              |  4 +++-
 tests/custom_cluster/test_kudu_table_create_without_hms.py     |  3 ++-
 tests/custom_cluster/test_metastore_events_cleanup.py          |  2 ++
 10 files changed, 40 insertions(+), 5 deletions(-)

diff --git a/bin/run-all-tests.sh b/bin/run-all-tests.sh
index 8f419c777..a114aacaa 100755
--- a/bin/run-all-tests.sh
+++ b/bin/run-all-tests.sh
@@ -32,6 +32,15 @@ setup_report_build_error
 : ${TIMEOUT_FOR_RUN_ALL_TESTS_MINS:=1200}
 KERB_ARGS=""
 
+# Use a different JDK for testing. Picked up by impala-config and start-impala-cluster.
+if [ ! -z "${TEST_JAVA_HOME_OVERRIDE:-}" ]; then
+  export MINICLUSTER_JAVA_HOME="${JAVA_HOME}"
+  export IMPALA_JAVA_HOME_OVERRIDE="${TEST_JAVA_HOME_OVERRIDE}"
+elif [ ! -z "${TEST_JDK_VERSION:-}" ]; then
+  export MINICLUSTER_JAVA_HOME="${JAVA_HOME}"
+  export IMPALA_JDK_VERSION="${TEST_JDK_VERSION}"
+fi
+
 . "${IMPALA_HOME}/bin/impala-config.sh" > /dev/null 2>&1
 . "${IMPALA_HOME}/testdata/bin/run-step.sh"
 if "${CLUSTER_DIR}/admin" is_kerberized; then
diff --git a/fe/src/test/java/org/apache/impala/customservice/KuduHMSIntegrationTest.java b/fe/src/test/java/org/apache/impala/customservice/KuduHMSIntegrationTest.java
index 2019c23d3..a822079df 100644
--- a/fe/src/test/java/org/apache/impala/customservice/KuduHMSIntegrationTest.java
+++ b/fe/src/test/java/org/apache/impala/customservice/KuduHMSIntegrationTest.java
@@ -18,6 +18,7 @@
 package org.apache.impala.customservice;
 
 import static org.junit.Assert.assertEquals;
+import org.apache.commons.lang.StringUtils;
 
 import org.apache.impala.analysis.AnalyzeKuduDDLTest;
 import org.apache.impala.analysis.AuditingKuduTest;
@@ -47,6 +48,15 @@ public class KuduHMSIntegrationTest {
   private static void restartKudu(boolean enableHMSIntegration)
       throws Exception {
     List<String> envp = getSystemEnv(enableHMSIntegration);
+    String altJavaHome = System.getenv("MINICLUSTER_JAVA_HOME");
+    if (!StringUtils.isEmpty(altJavaHome)) {
+      // Restart Kudu with the same Java version. Skip loading libjsig.
+      envp.removeIf(s -> s.startsWith("LD_PRELOAD="));
+      envp.removeIf(s -> s.startsWith("JAVA="));
+      envp.removeIf(s -> s.startsWith("JAVA_HOME="));
+      envp.add("JAVA=" + altJavaHome + "/bin/java");
+      envp.add("JAVA_HOME=" + altJavaHome);
+    }
     int exitVal = CustomServiceRunner.RestartMiniclusterComponent(
         "kudu", envp.toArray(new String[envp.size()]));
     assertEquals(0, exitVal);
diff --git a/tests/authorization/test_ranger.py b/tests/authorization/test_ranger.py
index ae5a39065..5b089bf88 100644
--- a/tests/authorization/test_ranger.py
+++ b/tests/authorization/test_ranger.py
@@ -29,7 +29,7 @@ from subprocess import check_call
 
 from getpass import getuser
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
-from tests.common.skip import SkipIfFS, SkipIfHive2
+from tests.common.skip import SkipIfFS, SkipIfHive2, SkipIf
 from tests.common.test_dimensions import (create_client_protocol_dimension,
     create_exec_option_dimension, create_orc_dimension)
 from tests.util.hdfs_util import NAMENODE
@@ -724,6 +724,7 @@ class TestRanger(CustomClusterTestSuite):
                            user=ADMIN)
 
   @pytest.mark.execute_serially
+  @SkipIf.is_test_jdk
   @CustomClusterTestSuite.with_args(
     impalad_args=IMPALAD_ARGS, catalogd_args=CATALOGD_ARGS, reset_ranger=True)
   def test_show_grant_hive_privilege(self, unique_name):
@@ -2027,6 +2028,7 @@ class TestRanger(CustomClusterTestSuite):
                            user=ADMIN)
 
   @pytest.mark.execute_serially
+  @SkipIf.is_test_jdk
   @SkipIfFS.hive
   @SkipIfHive2.ranger_auth
   @CustomClusterTestSuite.with_args()
diff --git a/tests/common/environ.py b/tests/common/environ.py
index d5c7f8f9f..b3e354c9d 100644
--- a/tests/common/environ.py
+++ b/tests/common/environ.py
@@ -99,6 +99,10 @@ if kernel_version_match is not None and len(kernel_version_match.groups()) == 4:
   kernel_version = [int(x) for x in list(kernel_version_match.groups())]
 IS_BUGGY_EL6_KERNEL = 'el6' in kernel_release and kernel_version < [2, 6, 32, 674]
 
+# Detect if we're testing a different JDK than we used to build and start minicluster.
+IS_TEST_JDK = os.environ.get("TEST_JAVA_HOME_OVERRIDE",
+                             os.environ.get("TEST_JDK_VERSION", "")) != ""
+
 class ImpalaBuildFlavors:
   """
   Represents the possible CMAKE_BUILD_TYPE values. These build flavors are needed
diff --git a/tests/common/skip.py b/tests/common/skip.py
index bb7326be2..dc419a0ef 100644
--- a/tests/common/skip.py
+++ b/tests/common/skip.py
@@ -27,7 +27,7 @@ from functools import partial
 from tests.common.environ import (ImpalaTestClusterProperties,
                                   IS_DOCKERIZED_TEST_CLUSTER, IS_BUGGY_EL6_KERNEL,
                                   HIVE_MAJOR_VERSION, IS_REDHAT_6_DERIVATIVE,
-                                  IS_APACHE_HIVE)
+                                  IS_APACHE_HIVE, IS_TEST_JDK)
 from tests.common.kudu_test_suite import get_kudu_master_flag
 from tests.util.filesystem_utils import (
     IS_ABFS,
@@ -115,6 +115,7 @@ class SkipIf:
       reason="Secondary filesystem needed")
   is_buggy_el6_kernel = pytest.mark.skipif(
       IS_BUGGY_EL6_KERNEL, reason="Kernel is affected by KUDU-1508")
+  is_test_jdk = pytest.mark.skipif(IS_TEST_JDK, reason="Testing with different JDK")
 
 class SkipIfLocal:
   # These are skipped due to product limitations.
diff --git a/tests/custom_cluster/test_catalog_hms_failures.py b/tests/custom_cluster/test_catalog_hms_failures.py
index 09e26a433..9c1623bb8 100644
--- a/tests/custom_cluster/test_catalog_hms_failures.py
+++ b/tests/custom_cluster/test_catalog_hms_failures.py
@@ -25,12 +25,14 @@ from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
 from tests.common.custom_cluster_test_suite import (
     CustomClusterTestSuite,
     DEFAULT_CLUSTER_SIZE)
+from tests.common.skip import SkipIf
 from tests.util.filesystem_utils import IS_ISILON, IS_LOCAL
 
 
 NUM_SUBSCRIBERS = DEFAULT_CLUSTER_SIZE + 1
 
 
+@SkipIf.is_test_jdk
 class TestHiveMetaStoreFailure(CustomClusterTestSuite):
   """Tests to validate the Catalog Service continues to function even if the HMS
   fails."""
@@ -113,6 +115,7 @@ class TestHiveMetaStoreFailure(CustomClusterTestSuite):
         "MetaStoreClient lost connection. Attempting to reconnect", expected_count=-1)
 
 
+@SkipIf.is_test_jdk
 class TestCatalogHMSFailures(CustomClusterTestSuite):
   """Test Catalog behavior when HMS is not present."""
 
diff --git a/tests/custom_cluster/test_custom_hive_configs.py b/tests/custom_cluster/test_custom_hive_configs.py
index 4fcb88e72..b574aa3ce 100644
--- a/tests/custom_cluster/test_custom_hive_configs.py
+++ b/tests/custom_cluster/test_custom_hive_configs.py
@@ -20,7 +20,7 @@ import pytest
 from os import getenv
 
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
-from tests.common.skip import SkipIfHive2
+from tests.common.skip import SkipIfHive2, SkipIf
 
 HIVE_SITE_EXT_DIR = getenv('IMPALA_HOME') + '/fe/src/test/resources/hive-site-ext'
 
@@ -35,6 +35,7 @@ class TestCustomHiveConfigs(CustomClusterTestSuite):
     super(TestCustomHiveConfigs, cls).setup_class()
 
   @SkipIfHive2.acid
+  @SkipIf.is_test_jdk
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(hive_conf_dir=HIVE_SITE_EXT_DIR)
   def test_ctas_read_write_consistence(self, unique_database):
diff --git a/tests/custom_cluster/test_kudu.py b/tests/custom_cluster/test_kudu.py
index 98772d69c..49c4c4ce1 100644
--- a/tests/custom_cluster/test_kudu.py
+++ b/tests/custom_cluster/test_kudu.py
@@ -26,7 +26,7 @@ from time import sleep
 from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 from tests.common.kudu_test_suite import KuduTestSuite
-from tests.common.skip import SkipIfKudu, SkipIfBuildType
+from tests.common.skip import SkipIfKudu, SkipIfBuildType, SkipIf
 from tests.common.test_dimensions import add_exec_option_dimension
 from tests.util.event_processor_utils import EventProcessorUtils
 
@@ -139,6 +139,7 @@ class TestKuduClientTimeout(CustomKuduTest):
     self.run_test_case('QueryTest/kudu-timeouts-impalad', vector)
 
 
+@SkipIf.is_test_jdk
 class TestKuduHMSIntegration(CustomKuduTest):
   START_END_TIME_LINEAGE_LOG_DIR = tempfile.mkdtemp(prefix="start_end_time")
 
@@ -740,6 +741,7 @@ class TestKuduTransactionIgnoreConflict(TestKuduTransactionBase):
     self._test_kudu_txn_ctas(cursor, unique_database, False, "no error")
 
 
+@SkipIf.is_test_jdk
 class TestKuduTxnKeepalive(CustomKuduTest):
   """
   Tests the Kudu transaction to ensure the transaction handle kept by the front-end in
diff --git a/tests/custom_cluster/test_kudu_table_create_without_hms.py b/tests/custom_cluster/test_kudu_table_create_without_hms.py
index fecbd1e4c..1b3084ee8 100644
--- a/tests/custom_cluster/test_kudu_table_create_without_hms.py
+++ b/tests/custom_cluster/test_kudu_table_create_without_hms.py
@@ -20,7 +20,7 @@ import pytest
 import os
 
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
-from tests.common.skip import SkipIfHive3
+from tests.common.skip import SkipIfHive3, SkipIf
 
 IMPALA_HOME = os.getenv('IMPALA_HOME')
 HIVE_SITE_WITHOUT_HMS_DIR = IMPALA_HOME + '/fe/src/test/resources/hive-site-without-hms'
@@ -31,6 +31,7 @@ class TestCreatingKuduTableWithoutHMS(CustomClusterTestSuite):
   """Test creating kudu managed table without hms"""
 
   @SkipIfHive3.without_hms_not_supported
+  @SkipIf.is_test_jdk
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(hive_conf_dir=HIVE_SITE_WITHOUT_HMS_DIR)
   def test_kudu_table_create_without_hms(self, unique_database):
diff --git a/tests/custom_cluster/test_metastore_events_cleanup.py b/tests/custom_cluster/test_metastore_events_cleanup.py
index a2f578422..02638a1af 100644
--- a/tests/custom_cluster/test_metastore_events_cleanup.py
+++ b/tests/custom_cluster/test_metastore_events_cleanup.py
@@ -20,6 +20,7 @@ import pytest
 import os
 
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
+from tests.common.skip import SkipIf
 
 IMPALA_HOME = os.getenv('IMPALA_HOME')
 HIVE_SITE_EVENTS_CLEANUP = IMPALA_HOME + '/fe/src/test/resources/hive-site-events-cleanup'
@@ -28,6 +29,7 @@ HIVE_SITE_EVENTS_CLEANUP = IMPALA_HOME + '/fe/src/test/resources/hive-site-event
 class TestTableLoadingWithEventsCleanUp(CustomClusterTestSuite):
 
   @pytest.mark.execute_serially
+  @SkipIf.is_test_jdk
   @CustomClusterTestSuite.with_args(hive_conf_dir=HIVE_SITE_EVENTS_CLEANUP)
   def test_table_load_with_events_cleanup(self, unique_database):
     """Regression test for IMPALA-11028"""