mirror of
https://github.com/apache/impala.git
synced 2025-12-25 02:03:09 -05:00
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>
586 lines
29 KiB
Python
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
|