Files
impala/tests/custom_cluster/test_catalogd_ha.py
Riza Suminto 0b1a32fad8 IMPALA-13850 (part 4): Implement in-place reset for CatalogD
This patch improve the availability of CatalogD under huge INVALIDATE
METADATA operation. Previously, CatalogServiceCatalog.reset() hold
versionLock_.writeLock() for the whole reset duration. When the number
of database, tables, or functions are big, this write lock can be held
for a long time, preventing any other catalog operation from proceeding.

This patch improve the situation by:
1. Making CatalogServiceCatalog.reset() rebuild dbCache_ in place and
   occasionally release the write lock between rebuild stages.
2. Fetch databases, tables, and functions metadata from MetaStore in
   background using ExecutorService. Added catalog_reset_max_threads
   flag to control number of threads to do parallel fetch.

In order to do so, lexicographic order must be enforced during reset()
and ensure all Db invalidation within a single stage is complete before
releasing the write lock. Stages should run in approximately the same
amount of time. A catalog operation over a database must ensure that no
reset operation is currently running, or the database name is
lexicographically less than the current database-under-invalidation.

This patch adds CatalogResetManager to do background metadata fetching
and provide helper methods to help facilitate waiting for reset
progress. CatalogServiceCatalog must hold the versionLock_.writeLock()
before calling most of CatalogResetManager methods.

These are methods in CatalogServiceCatalog class that must wait for
CatalogResetManager.waitOngoingMetadataFetch():

addDb()
addFunction()
addIncompleteTable()
addTable()
invalidateTableIfExists()
removeDb()
removeFunction()
removeTable()
renameTable()
replaceTableIfUnchanged()
tryLock()
updateDb()
InvalidateAwareDbSnapshotIterator.hasNext()

Concurrent global IM must wait until currently running global IM
complete. The waiting happens by calling waitFullMetadataFetch().

CatalogServiceCatalog.getAllDbs() get a snapshot of dbCache_ values at a
time. With this patch, it is now possible that some Db in this snapshot
maybe removed from dbCache() by concurrent reset(). Caller that cares
about snapshot integrity like CatalogServiceCatalog.getCatalogDelta()
should be careful when iterating the snapshot. It must iterate in
lexicographic order, similar like reset(), and make sure that it does
not go beyond the current database-under-invalidation. It also must skip
the Db that it is currently being inspected if Db.isRemoved() is True.
Added helper class InvalidateAwareDbSnapshot for this kind of iteration

Override CatalogServiceCatalog.getDb() and
CatalogServiceCatalog.getDbs() to wait until first reset metadata
complete or looked up Db found in cache.

Expand test_restart_catalogd_twice to test_restart_legacy_catalogd_twice
and test_restart_local_catalogd_twice. Update
CustomClusterTestSuite.wait_for_wm_init_complete() to correctly pass
timeout values to helper methods that it calls. Reduce cluster_size from
10 to 3 in few tests of test_workload_mgmt_init.py to avoid flakiness.

Fixed HMS connection leak between tests in AuthorizationStmtTest (see
IMPALA-8073).

Testing:
- Pass exhaustive tests.

Change-Id: Ib4ae2154612746b34484391c5950e74b61f85c9d
Reviewed-on: http://gerrit.cloudera.org:8080/22640
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Quanlong Huang <huangquanlong@gmail.com>
2025-07-09 14:05:04 +00:00

586 lines
29 KiB
Python

# 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 __future__ import absolute_import, division, print_function
import json
import logging
import re
import requests
import time
from builtins import round
from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
from tests.common.environ import build_flavor_timeout
from tests.common.impala_connection import ERROR
from tests.common.parametrize import UniqueDatabase
from tests.common.test_vector import HS2
from tests.util.filesystem_utils import IS_S3, get_fs_path
from time import sleep
LOG = logging.getLogger('catalogd_ha_test')
DEFAULT_STATESTORE_SERVICE_PORT = 24000
DEFAULT_CATALOG_SERVICE_PORT = 26000
SLOW_BUILD_SYNC_DDL_DELAY_S = 20
SYNC_DDL_DELAY_S = build_flavor_timeout(
10, slow_build_timeout=SLOW_BUILD_SYNC_DDL_DELAY_S)
SS_AUTO_FAILOVER_FREQ_MS = 500
SS_AUTO_FAILOVER_ARGS = (
"--use_subscriber_id_as_catalogd_priority=true "
"--statestore_heartbeat_frequency_ms={0} "
"--active_catalogd_designation_monitoring_interval_ms={0} ").format(
SS_AUTO_FAILOVER_FREQ_MS)
# s3 can behave as a slow build.
if IS_S3:
SYNC_DDL_DELAY_S = SLOW_BUILD_SYNC_DDL_DELAY_S
class TestCatalogdHA(CustomClusterTestSuite):
"""A simple wrapper class to launch a cluster with catalogd HA enabled.
The cluster will be launched with two catalogd instances as Active-Passive HA pair.
statestored and catalogds are started with starting flag FLAGS_enable_catalogd_ha
as true. """
VARZ_URL = "http://localhost:{0}/varz"
CATALOG_HA_INFO_URL = "http://localhost:{0}/catalog_ha_info"
JSON_METRICS_URL = "http://localhost:{0}/jsonmetrics"
HEALTHZ_URL = "http://localhost:{0}/healthz"
SS_TEST_PORT = ["25010"]
@classmethod
def default_test_protocol(cls):
return HS2
# Verify port of the active catalogd of statestore is matching with the catalog
# service port of the given catalogd service.
def __verify_statestore_active_catalogd_port(self, catalogd_service):
statestore_service = self.cluster.statestored.service
active_catalogd_address = \
statestore_service.get_metric_value("statestore.active-catalogd-address")
_, catalog_service_port = active_catalogd_address.split(":")
assert int(catalog_service_port) == catalogd_service.get_catalog_service_port()
# Verify port of the active catalogd of impalad is matching with the catalog
# service port of the given catalogd service.
def __verify_impalad_active_catalogd_port(self, impalad_index, catalogd_service):
impalad_service = self.cluster.impalads[impalad_index].service
active_catalogd_address = \
impalad_service.get_metric_value("catalog.active-catalogd-address")
_, catalog_service_port = active_catalogd_address.split(":")
assert int(catalog_service_port) == catalogd_service.get_catalog_service_port()
def __run_simple_queries(self, unique_database, sync_ddl=False):
opts = {'sync_ddl': sync_ddl}
self.execute_query_expect_success(self.client, "USE " + unique_database, opts)
self.execute_query_expect_success(
self.client, "drop table if exists test_catalogd_ha", opts)
self.execute_query_expect_success(
self.client, "create table if not exists test_catalogd_ha (id int)", opts)
self.execute_query_expect_success(
self.client, "insert into table test_catalogd_ha values(1), (2), (3)", opts)
self.execute_query_expect_success(
self.client, "select count(*) from test_catalogd_ha", opts)
self.execute_query_expect_success(
self.client, "drop table if exists test_catalogd_ha", opts)
self.execute_query_expect_success(self.client, "USE default", opts)
def __get_catalogds(self):
"""Return tuple of (active_catalogd, standby_catalogd)."""
# Verify two catalogd instances are created with one as active.
catalogds = self.cluster.catalogds()
assert len(catalogds) == 2
# Assert that /healthz page is OK.
for catalogd in catalogds:
port = catalogd.get_webserver_port()
page = requests.get(self.HEALTHZ_URL.format(port))
assert page.status_code == requests.codes.ok
page = requests.head(self.HEALTHZ_URL.format(port))
assert page.status_code == requests.codes.ok
first_impalad = self.cluster.get_first_impalad()
page = requests.head(self.HEALTHZ_URL.format(first_impalad.get_webserver_port()))
assert page.status_code == requests.codes.ok
active_catalogd = catalogds[0]
standby_catalogd = catalogds[1]
if not active_catalogd.service.get_metric_value("catalog-server.active-status"):
active_catalogd, standby_catalogd = standby_catalogd, active_catalogd
assert active_catalogd.service.get_metric_value("catalog-server.active-status")
assert not standby_catalogd.service.get_metric_value("catalog-server.active-status")
return (active_catalogd, standby_catalogd)
@CustomClusterTestSuite.with_args(
statestored_args="--use_subscriber_id_as_catalogd_priority=true",
start_args="--enable_catalogd_ha")
def test_catalogd_ha_with_two_catalogd(self, unique_database):
self.__test_catalogd_ha_with_two_catalogd(unique_database)
def __test_catalogd_ha_with_two_catalogd(self, unique_database):
"""The test case for cluster started with catalogd HA enabled."""
(active_catalogd, standby_catalogd) = self.__get_catalogds()
catalogd_service_1 = active_catalogd.service
# Verify ports of the active catalogd of statestore and impalad are matching with
# the catalog service port of the current active catalogd.
self.__verify_statestore_active_catalogd_port(catalogd_service_1)
self.__verify_impalad_active_catalogd_port(0, catalogd_service_1)
self.__verify_impalad_active_catalogd_port(1, catalogd_service_1)
self.__verify_impalad_active_catalogd_port(2, catalogd_service_1)
# Verify simple queries are ran successfully.
self.__run_simple_queries(unique_database)
# Verify simple queries with sync_ddl as 1.
self.__run_simple_queries(unique_database, sync_ddl=True)
# Restart one coordinator. Verify it get active catalogd address from statestore.
self.cluster.impalads[1].restart()
self.cluster.impalads[1].service.wait_for_metric_value('impala-server.ready',
expected_value=1, timeout=30)
self.__verify_impalad_active_catalogd_port(1, catalogd_service_1)
@CustomClusterTestSuite.with_args(
statestored_args="--enable_catalogd_ha=true "
"--use_subscriber_id_as_catalogd_priority=true "
"--catalogd_ha_preemption_wait_period_ms=200",
catalogd_args="--enable_catalogd_ha=true")
def test_catalogd_ha_with_one_catalogd(self, unique_database):
"""The test case for cluster with only one catalogd when catalogd HA is enabled."""
# Verify the catalogd instances is created as active.
catalogds = self.cluster.catalogds()
assert len(catalogds) == 1
catalogd_service_1 = catalogds[0].service
assert catalogd_service_1.get_metric_value("catalog-server.active-status")
# Verify ports of the active catalogd of statestore and impalad are matching with
# the catalog service port of the current active catalogd.
self.__verify_statestore_active_catalogd_port(catalogd_service_1)
self.__verify_impalad_active_catalogd_port(0, catalogd_service_1)
self.__verify_impalad_active_catalogd_port(1, catalogd_service_1)
self.__verify_impalad_active_catalogd_port(2, catalogd_service_1)
# Verify simple queries are ran successfully.
self.__run_simple_queries(unique_database)
def __test_catalogd_auto_failover(self, unique_database):
"""Stop active catalogd and verify standby catalogd becomes active.
Restart original active catalogd. Verify that statestore does not resume its
active role. If test_query_fail_during_failover is True, run a query during failover
and comfirm that it is fail."""
(active_catalogd, standby_catalogd) = self.__get_catalogds()
catalogd_service_1 = active_catalogd.service
catalogd_service_2 = standby_catalogd.service
statestore_service = self.cluster.statestored.service
# Assert that cluster is set up with configs needed to run this test.
assert SS_AUTO_FAILOVER_FREQ_MS >= int(statestore_service.get_flag_current_value(
'active_catalogd_designation_monitoring_interval_ms'))
assert SS_AUTO_FAILOVER_FREQ_MS >= int(statestore_service.get_flag_current_value(
'statestore_heartbeat_frequency_ms'))
start_count_clear_topic_entries = statestore_service.get_metric_value(
"statestore.num-clear-topic-entries-requests")
# Kill active catalogd
active_catalogd.kill()
# Tes run a DDL query after active_catalogd killed.
# This query should fail if coordinator has not receive update from StatestoreD about
# the standby_catalogd becomes active.
self.execute_query_expect_failure(
self.client, "create table {}.table_creation_during_failover (id int)".format(
unique_database))
# Wait for long enough for the statestore to detect the failure of active catalogd
# and assign active role to standby catalogd.
catalogd_service_2.wait_for_metric_value(
"catalog-server.active-status", expected_value=True, timeout=30)
assert catalogd_service_2.get_metric_value(
"catalog-server.ha-number-active-status-change") > 0
assert catalogd_service_2.get_metric_value("catalog-server.active-status")
# Verify ports of the active catalogd of statestore and impalad are matching with
# the catalog service port of the current active catalogd.
self.__verify_statestore_active_catalogd_port(catalogd_service_2)
self.__verify_impalad_active_catalogd_port(0, catalogd_service_2)
self.__verify_impalad_active_catalogd_port(1, catalogd_service_2)
self.__verify_impalad_active_catalogd_port(2, catalogd_service_2)
# Verify simple queries are ran successfully.
self.__run_simple_queries(unique_database)
# Verify simple queries with sync_ddl as 1.
self.__run_simple_queries(unique_database, sync_ddl=True)
end_count_clear_topic_entries = statestore_service.get_metric_value(
"statestore.num-clear-topic-entries-requests")
assert end_count_clear_topic_entries > start_count_clear_topic_entries
# Restart original active catalogd. Verify that statestore does not resume it as
# active to avoid flip-flop.
active_catalogd.start(wait_until_ready=True)
sleep(1)
catalogd_service_1 = active_catalogd.service
assert not catalogd_service_1.get_metric_value("catalog-server.active-status")
assert catalogd_service_2.get_metric_value("catalog-server.active-status")
# Verify ports of the active catalogd of statestore and impalad are matching with
# the catalog service port of the current active catalogd.
self.__verify_statestore_active_catalogd_port(catalogd_service_2)
self.__verify_impalad_active_catalogd_port(0, catalogd_service_2)
self.__verify_impalad_active_catalogd_port(1, catalogd_service_2)
self.__verify_impalad_active_catalogd_port(2, catalogd_service_2)
@CustomClusterTestSuite.with_args(
statestored_args=SS_AUTO_FAILOVER_ARGS,
catalogd_args="--catalogd_ha_reset_metadata_on_failover=false",
start_args="--enable_catalogd_ha")
def test_catalogd_auto_failover(self, unique_database):
"""Tests for Catalog Service auto fail over without failed RPCs."""
self.__test_catalogd_auto_failover(unique_database)
statestore_service = self.cluster.statestored.service
successful_update_catalogd_rpc_num = statestore_service.get_metric_value(
"statestore.num-successful-update-catalogd-rpc")
failed_update_catalogd_rpc_num = statestore_service.get_metric_value(
"statestore.num-failed-update-catalogd-rpc")
assert successful_update_catalogd_rpc_num >= 6
assert failed_update_catalogd_rpc_num == 0
@CustomClusterTestSuite.with_args(
statestored_args=(
SS_AUTO_FAILOVER_ARGS
+ "--debug_actions=SEND_UPDATE_CATALOGD_RPC_FIRST_ATTEMPT:FAIL@1.0"),
catalogd_args="--catalogd_ha_reset_metadata_on_failover=false",
start_args="--enable_catalogd_ha")
def test_catalogd_auto_failover_with_failed_rpc(self, unique_database):
"""Tests for Catalog Service auto fail over with failed RPCs."""
self.__test_catalogd_auto_failover(unique_database)
statestore_service = self.cluster.statestored.service
successful_update_catalogd_rpc_num = statestore_service.get_metric_value(
"statestore.num-successful-update-catalogd-rpc")
failed_update_catalogd_rpc_num = statestore_service.get_metric_value(
"statestore.num-failed-update-catalogd-rpc")
assert successful_update_catalogd_rpc_num >= 6
assert failed_update_catalogd_rpc_num == successful_update_catalogd_rpc_num
@CustomClusterTestSuite.with_args(
statestored_args=(
SS_AUTO_FAILOVER_ARGS
+ "--debug_actions=SEND_UPDATE_CATALOGD_RPC_FIRST_ATTEMPT:SLEEP@3000"),
# minicluster has 68 Db when this test is written. So total sleep is ~3.4s.
catalogd_args="--reset_metadata_lock_duration_ms=100 "
"--debug_actions=reset_metadata_loop_locked:SLEEP@50",
start_args="--enable_catalogd_ha")
@UniqueDatabase.parametrize(name_prefix='aaa_test_catalogd_auto_failover_slow_first_db')
def test_catalogd_auto_failover_slow_first_db(self, unique_database):
"""Tests for Catalog Service auto fail over with both slow metadata reset and slow
statestore update. Set 'aaa_' as unique_database prefix to make the database among
the earliest in reset metadata order."""
self.__test_catalogd_auto_failover(unique_database)
@CustomClusterTestSuite.with_args(
statestored_args=(
SS_AUTO_FAILOVER_ARGS
+ "--debug_actions=SEND_UPDATE_CATALOGD_RPC_FIRST_ATTEMPT:SLEEP@3000"),
# minicluster has 68 Db when this test is written. So total sleep is ~3.4s.
catalogd_args="--reset_metadata_lock_duration_ms=100 "
"--debug_actions=reset_metadata_loop_locked:SLEEP@50",
start_args="--enable_catalogd_ha")
@UniqueDatabase.parametrize(name_prefix='zzz_test_catalogd_auto_failover_slow_last_db')
def test_catalogd_auto_failover_slow_last_db(self, unique_database):
"""Tests for Catalog Service auto fail over with both slow metadata reset and slow
statestore update. Set 'zzz_' as unique_database prefix to make the database among
the latest in reset metadata order."""
self.__test_catalogd_auto_failover(unique_database)
def __test_catalogd_manual_failover(self, unique_database):
"""Stop active catalogd and verify standby catalogd becomes active.
Restart original active catalogd with force_catalogd_active as true. Verify that
statestore resume it as active.
"""
(active_catalogd, standby_catalogd) = self.__get_catalogds()
catalogd_service_1 = active_catalogd.service
catalogd_service_2 = standby_catalogd.service
statestore_service = self.cluster.statestored.service
start_count_clear_topic_entries = statestore_service.get_metric_value(
"statestore.num-clear-topic-entries-requests")
# Kill active catalogd
active_catalogd.kill()
# Wait for long enough for the statestore to detect the failure of active catalogd
# and assign active role to standby catalogd.
catalogd_service_2.wait_for_metric_value(
"catalog-server.active-status", expected_value=True, timeout=30)
assert catalogd_service_2.get_metric_value(
"catalog-server.ha-number-active-status-change") > 0
assert catalogd_service_2.get_metric_value("catalog-server.active-status")
# Verify ports of the active catalogd of statestore and impalad are matching with
# the catalog service port of the current active catalogd.
self.__verify_statestore_active_catalogd_port(catalogd_service_2)
self.__verify_impalad_active_catalogd_port(0, catalogd_service_2)
self.__verify_impalad_active_catalogd_port(1, catalogd_service_2)
self.__verify_impalad_active_catalogd_port(2, catalogd_service_2)
# Verify simple queries are ran successfully.
self.__run_simple_queries(unique_database)
end_count_clear_topic_entries = statestore_service.get_metric_value(
"statestore.num-clear-topic-entries-requests")
assert end_count_clear_topic_entries > start_count_clear_topic_entries
start_count_clear_topic_entries = end_count_clear_topic_entries
# Restart original active catalogd with force_catalogd_active as true.
# Verify that statestore resume it as active.
active_catalogd.start(wait_until_ready=True,
additional_args="--force_catalogd_active=true")
catalogd_service_1 = active_catalogd.service
catalogd_service_1.wait_for_metric_value(
"catalog-server.active-status", expected_value=True, timeout=15)
assert catalogd_service_1.get_metric_value("catalog-server.active-status")
sleep_time_s = build_flavor_timeout(2, slow_build_timeout=5)
sleep(sleep_time_s)
assert not catalogd_service_2.get_metric_value("catalog-server.active-status")
# Verify ports of the active catalogd of statestore and impalad are matching with
# the catalog service port of the current active catalogd.
self.__verify_statestore_active_catalogd_port(catalogd_service_1)
self.__verify_impalad_active_catalogd_port(0, catalogd_service_1)
self.__verify_impalad_active_catalogd_port(1, catalogd_service_1)
self.__verify_impalad_active_catalogd_port(2, catalogd_service_1)
end_count_clear_topic_entries = statestore_service.get_metric_value(
"statestore.num-clear-topic-entries-requests")
assert end_count_clear_topic_entries > start_count_clear_topic_entries
@CustomClusterTestSuite.with_args(
statestored_args="--use_subscriber_id_as_catalogd_priority=true "
"--statestore_heartbeat_frequency_ms=1000",
catalogd_args="--catalogd_ha_reset_metadata_on_failover=false",
start_args="--enable_catalogd_ha")
def test_catalogd_manual_failover(self, unique_database):
"""Tests for Catalog Service manual fail over without failed RPCs."""
self.__test_catalogd_manual_failover(unique_database)
statestore_service = self.cluster.statestored.service
successful_update_catalogd_rpc_num = statestore_service.get_metric_value(
"statestore.num-successful-update-catalogd-rpc")
failed_update_catalogd_rpc_num = statestore_service.get_metric_value(
"statestore.num-failed-update-catalogd-rpc")
assert successful_update_catalogd_rpc_num >= 10
assert failed_update_catalogd_rpc_num == 0
@CustomClusterTestSuite.with_args(
statestored_args="--use_subscriber_id_as_catalogd_priority=true "
"--statestore_heartbeat_frequency_ms=1000 "
"--debug_actions=SEND_UPDATE_CATALOGD_RPC_FIRST_ATTEMPT:FAIL@1.0",
catalogd_args="--catalogd_ha_reset_metadata_on_failover=false",
start_args="--enable_catalogd_ha")
def test_catalogd_manual_failover_with_failed_rpc(self, unique_database):
"""Tests for Catalog Service manual fail over with failed RPCs."""
self.__test_catalogd_manual_failover(unique_database)
statestore_service = self.cluster.statestored.service
successful_update_catalogd_rpc_num = statestore_service.get_metric_value(
"statestore.num-successful-update-catalogd-rpc")
failed_update_catalogd_rpc_num = statestore_service.get_metric_value(
"statestore.num-failed-update-catalogd-rpc")
assert successful_update_catalogd_rpc_num >= 10
assert failed_update_catalogd_rpc_num == successful_update_catalogd_rpc_num
@CustomClusterTestSuite.with_args(
statestored_args="--use_subscriber_id_as_catalogd_priority=true "
"--statestore_heartbeat_frequency_ms=1000",
impalad_args="--debug_actions=IGNORE_NEW_ACTIVE_CATALOGD_ADDR:FAIL@1.0",
start_args="--enable_catalogd_ha")
def test_manual_failover_with_coord_ignore_notification(self):
"""Tests for Catalog Service manual failover with coordinators to ignore failover
notification."""
(active_catalogd, standby_catalogd) = self.__get_catalogds()
catalogd_service_1 = active_catalogd.service
# Restart standby catalogd with force_catalogd_active as true.
standby_catalogd.kill()
standby_catalogd.start(wait_until_ready=True,
additional_args="--force_catalogd_active=true")
# Wait until original active catalogd becomes in-active.
catalogd_service_1.wait_for_metric_value(
"catalog-server.active-status", expected_value=False, timeout=15)
assert not catalogd_service_1.get_metric_value("catalog-server.active-status")
# Run query to create a table. Coordinator still send request to catalogd_service_1
# so that the request will be rejected.
ddl_query = "CREATE TABLE coordinator_ignore_notification (c int)"
ex = self.execute_query_expect_failure(self.client, ddl_query)
assert "Request for Catalog service is rejected" in str(ex)
@CustomClusterTestSuite.with_args(
statestored_args="--use_subscriber_id_as_catalogd_priority=true",
start_args="--enable_catalogd_ha",
disable_log_buffering=True)
def test_restart_statestore(self, unique_database):
"""The test case for restarting statestore after the cluster is created with
catalogd HA enabled."""
(active_catalogd, standby_catalogd) = self.__get_catalogds()
catalogd_service_1 = active_catalogd.service
# Verify ports of the active catalogd of statestore and impalad are matching with
# the catalog service port of the current active catalogd.
self.__verify_statestore_active_catalogd_port(catalogd_service_1)
self.__verify_impalad_active_catalogd_port(0, catalogd_service_1)
self.__verify_impalad_active_catalogd_port(1, catalogd_service_1)
self.__verify_impalad_active_catalogd_port(2, catalogd_service_1)
# Restart statestore. Verify one catalogd is assigned as active, the other is
# assigned as standby.
self.cluster.statestored.restart()
wait_time_s = build_flavor_timeout(90, slow_build_timeout=180)
self.cluster.statestored.service.wait_for_metric_value('statestore.live-backends',
expected_value=5, timeout=wait_time_s)
sleep_time_s = build_flavor_timeout(2, slow_build_timeout=5)
sleep(sleep_time_s)
if not active_catalogd.service.get_metric_value("catalog-server.active-status"):
active_catalogd, standby_catalogd = standby_catalogd, active_catalogd
assert active_catalogd.service.get_metric_value("catalog-server.active-status")
assert not standby_catalogd.service.get_metric_value("catalog-server.active-status")
catalogd_service_1 = active_catalogd.service
# Verify ports of the active catalogd of statestore and impalad are matching with
# the catalog service port of the current active catalogd.
self.__verify_statestore_active_catalogd_port(catalogd_service_1)
self.__verify_impalad_active_catalogd_port(0, catalogd_service_1)
self.__verify_impalad_active_catalogd_port(1, catalogd_service_1)
self.__verify_impalad_active_catalogd_port(2, catalogd_service_1)
# Verify simple queries are ran successfully.
self.__run_simple_queries(unique_database)
unexpected_msg = re.compile("Ignore the update of active catalogd since more recent "
"update has been processed ([0-9]+ vs [0-9]+)")
self.assert_catalogd_log_contains("INFO", unexpected_msg, expected_count=0)
self.assert_impalad_log_contains("INFO", unexpected_msg, expected_count=0)
@CustomClusterTestSuite.with_args(
catalogd_args="--force_catalogd_active=true",
start_args="--enable_catalogd_ha")
def test_two_catalogd_with_force_active(self, unique_database):
"""The test case for cluster started with catalogd HA enabled and
both catalogds started with 'force_catalogd_active' as true.
Verify that one and only one catalogd is active."""
sleep_time_s = build_flavor_timeout(2, slow_build_timeout=5)
sleep(sleep_time_s)
self.__test_catalogd_ha_with_two_catalogd(unique_database)
@CustomClusterTestSuite.with_args(
statestored_args="--use_subscriber_id_as_catalogd_priority=true",
catalogd_args="--debug_actions='catalogd_wait_sync_ddl_version_delay:SLEEP@{0}'"
.format(SYNC_DDL_DELAY_S * 1000),
start_args="--enable_catalogd_ha")
def test_catalogd_failover_with_sync_ddl(self, unique_database):
"""Tests for Catalog Service force fail-over when running DDL with SYNC_DDL
enabled."""
(active_catalogd, standby_catalogd) = self.__get_catalogds()
catalogd_service_1 = active_catalogd.service
# Run DDL with SYNC_DDL enabled.
client = self.cluster.impalads[0].service.create_hs2_client()
assert client is not None
try:
client.set_configuration_option('sync_ddl', '1')
ddl_query = "CREATE TABLE {database}.failover_sync_ddl (c int)"
handle = client.execute_async(ddl_query.format(database=unique_database))
# Restart standby catalogd with force_catalogd_active as true.
start_s = time.time()
standby_catalogd.kill()
standby_catalogd.start(wait_until_ready=True,
additional_args="--force_catalogd_active=true")
# Wait until original active catalogd becomes in-active.
catalogd_service_1.wait_for_metric_value(
"catalog-server.active-status", expected_value=False, timeout=15)
assert not catalogd_service_1.get_metric_value("catalog-server.active-status")
elapsed_s = time.time() - start_s
assert elapsed_s < SYNC_DDL_DELAY_S, \
"Catalogd failover took %s seconds to complete" % (elapsed_s)
LOG.info("Catalogd failover took %s seconds to complete" % round(elapsed_s, 1))
# Verify that the query is failed due to the Catalogd HA fail-over.
client.wait_for_impala_state(handle, ERROR, SYNC_DDL_DELAY_S * 2 + 10)
finally:
client.close()
@CustomClusterTestSuite.with_args(
statestored_args="--use_subscriber_id_as_catalogd_priority=true",
catalogd_args="--catalogd_ha_reset_metadata_on_failover=true",
start_args="--enable_catalogd_ha")
def test_metadata_after_failover(self, unique_database):
"""Verify that the metadata is correct after failover."""
(active_catalogd, standby_catalogd) = self.__get_catalogds()
catalogd_service_2 = standby_catalogd.service
create_func_impala = ("create function {database}.identity_tmp(bigint) "
"returns bigint location '{location}' symbol='Identity'")
self.client.execute(create_func_impala.format(
database=unique_database,
location=get_fs_path('/test-warehouse/libTestUdfs.so')))
self.execute_query_expect_success(
self.client, "select %s.identity_tmp(10)" % unique_database)
# Kill active catalogd
active_catalogd.kill()
# Wait for long enough for the statestore to detect the failure of active catalogd
# and assign active role to standby catalogd.
catalogd_service_2.wait_for_metric_value(
"catalog-server.active-status", expected_value=True, timeout=30)
assert catalogd_service_2.get_metric_value(
"catalog-server.ha-number-active-status-change") > 0
assert catalogd_service_2.get_metric_value("catalog-server.active-status")
self.execute_query_expect_success(
self.client, "select %s.identity_tmp(10)" % unique_database)
def test_page_with_disable_ha(self):
self.__test_catalog_ha_info_page()
@CustomClusterTestSuite.with_args(start_args="--enable_catalogd_ha")
def test_page_with_enable_ha(self):
self.__test_catalog_ha_info_page()
def __test_catalog_ha_info_page(self):
for port in self.SS_TEST_PORT:
response = requests.get(self.VARZ_URL.format(port) + "?json")
assert response.status_code == requests.codes.ok
varz_json = json.loads(response.text)
ha_flags = [e for e in varz_json["flags"]
if e["name"] == "enable_catalogd_ha"]
assert len(ha_flags) == 1
assert ha_flags[0]["default"] == "false"
# High availability for the Catalog is enabled.
if ha_flags[0]["current"] == "true":
url = self.JSON_METRICS_URL.format(port) + "?json"
metrics = json.loads(requests.get(url).text)
if metrics["statestore.active-status"]:
url = self.CATALOG_HA_INFO_URL.format(port) + "?json"
catalog_ha_info = json.loads(requests.get(url).text)
assert catalog_ha_info["active_catalogd_address"]\
== metrics["statestore.active-catalogd-address"]
else:
reponse = requests.get(self.CATALOG_HA_INFO_URL.format(port)).text
assert reponse.__contains__("The current statestored is inactive.")
else:
page = requests.get(self.CATALOG_HA_INFO_URL.format(port))
assert page.status_code == requests.codes.not_found