remove some debug code

This commit is contained in:
Alexander Tokmakov 2021-02-15 13:26:34 +03:00
parent 320fd6b264
commit 9c7cf9e92e
19 changed files with 109 additions and 78 deletions

View File

@ -60,4 +60,8 @@ fi
# more idiologically correct.
read -ra ADDITIONAL_OPTIONS <<< "${ADDITIONAL_OPTIONS:-}"
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--replicated-database')
fi
clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt

View File

@ -57,6 +57,10 @@ function run_tests()
ADDITIONAL_OPTIONS+=('4')
fi
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--replicated-database')
fi
clickhouse-test --testname --shard --zookeeper --hung-check --print-time \
--test-runs "$NUM_TRIES" \
"$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \

View File

@ -100,6 +100,7 @@ namespace CurrentMetrics
extern const Metric Revision;
extern const Metric VersionInteger;
extern const Metric MemoryTracking;
extern const Metric MaxDDLEntryID;
}
@ -997,7 +998,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
int pool_size = config().getInt("distributed_ddl.pool_size", 1);
if (pool_size < 1)
throw Exception("distributed_ddl.pool_size should be greater then 0", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
global_context->setDDLWorker(std::make_unique<DDLWorker>(pool_size, ddl_zookeeper_path, *global_context, &config(), "distributed_ddl"));
global_context->setDDLWorker(std::make_unique<DDLWorker>(pool_size, ddl_zookeeper_path, *global_context, &config(),
"distributed_ddl", "DDLWorker", &CurrentMetrics::MaxDDLEntryID));
}
std::unique_ptr<DNSCacheUpdater> dns_cache_updater;

View File

@ -420,6 +420,9 @@ class IColumn;
M(Bool, optimize_rewrite_sum_if_to_count_if, true, "Rewrite sumIf() and sum(if()) function countIf() function when logically equivalent", 0) \
M(UInt64, insert_shard_id, 0, "If non zero, when insert into a distributed table, the data will be inserted into the shard `insert_shard_id` synchronously. Possible values range from 1 to `shards_number` of corresponding distributed table", 0) \
M(Bool, allow_experimental_query_deduplication, false, "Allow sending parts' UUIDs for a query in order to deduplicate data parts if any", 0) \
M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \
M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \
M(Bool, database_replicated_ddl_output, true, "Return table with query execution status as a result of DDL query", 0) \
\
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
\

View File

@ -311,7 +311,8 @@ BlockIO DatabaseReplicated::propose(const ASTPtr & query, const Context & query_
Strings hosts_to_wait = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
auto stream = std::make_shared<DDLQueryStatusInputStream>(node_path, entry, query_context, hosts_to_wait);
io.in = std::move(stream);
if (query_context.getSettingsRef().database_replicated_ddl_output)
io.in = std::move(stream);
return io;
}

View File

@ -80,7 +80,6 @@ public:
ClusterPtr getCluster() const;
//FIXME
friend struct DatabaseReplicatedTask;
friend class DatabaseReplicatedDDLWorker;
private:

View File

@ -89,7 +89,7 @@ String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry)
return node_path;
}
String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entry, const Context & /*query_context*/)
String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entry, const Context & query_context)
{
/// NOTE Possibly it would be better to execute initial query on the most up-to-date node,
/// but it requires more complex logic around /try node.
@ -114,7 +114,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr
task->is_initial_query = true;
LOG_DEBUG(log, "Waiting for worker thread to process all entries before {}", entry_name);
UInt64 timeout = 600;
UInt64 timeout = query_context.getSettingsRef().database_replicated_initial_query_timeout_sec;
{
std::unique_lock lock{mutex};
bool processed = wait_current_task_change.wait_for(lock, std::chrono::seconds(timeout), [&]()

View File

@ -32,10 +32,6 @@
namespace fs = std::filesystem;
namespace CurrentMetrics
{
extern const Metric MaxDDLEntryID;
}
namespace DB
{
@ -152,12 +148,14 @@ std::unique_ptr<ZooKeeperLock> createSimpleZooKeeperLock(
DDLWorker::DDLWorker(int pool_size_, const std::string & zk_root_dir, const Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix,
const String & logger_name)
const String & logger_name, const CurrentMetrics::Metric * max_entry_metric_)
: context(context_)
, log(&Poco::Logger::get(logger_name))
, pool_size(pool_size_)
, max_entry_metric(max_entry_metric_)
{
CurrentMetrics::set(CurrentMetrics::MaxDDLEntryID, 0);
if (max_entry_metric)
CurrentMetrics::set(*max_entry_metric, 0);
if (1 < pool_size)
{
@ -456,7 +454,8 @@ void DDLWorker::updateMaxDDLEntryID(const String & entry_name)
{
if (max_id.compare_exchange_weak(prev_id, id))
{
CurrentMetrics::set(CurrentMetrics::MaxDDLEntryID, id);
if (max_entry_metric)
CurrentMetrics::set(*max_entry_metric, id);
break;
}
}
@ -596,7 +595,7 @@ void DDLWorker::processTask(DDLTaskBase & task)
}
bool DDLWorker::taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, const StoragePtr storage)
bool DDLWorker::taskShouldBeExecutedOnLeader(const ASTPtr & ast_ddl, const StoragePtr storage)
{
/// Pure DROP queries have to be executed on each node separately
if (auto * query = ast_ddl->as<ASTDropQuery>(); query && query->kind != ASTDropQuery::Kind::Truncate)

View File

@ -43,7 +43,7 @@ class DDLWorker
{
public:
DDLWorker(int pool_size_, const std::string & zk_root_dir, const Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix,
const String & logger_name = "DDLWorker");
const String & logger_name = "DDLWorker", const CurrentMetrics::Metric * max_entry_metric_ = nullptr);
virtual ~DDLWorker();
/// Pushes query into DDL queue, returns path to created node
@ -81,7 +81,7 @@ protected:
void updateMaxDDLEntryID(const String & entry_name);
/// Check that query should be executed on leader replica only
static bool taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, StoragePtr storage);
static bool taskShouldBeExecutedOnLeader(const ASTPtr & ast_ddl, StoragePtr storage);
/// Executes query only on leader replica in case of replicated table.
/// Queries like TRUNCATE/ALTER .../OPTIMIZE have to be executed only on one node of shard.
@ -144,6 +144,7 @@ protected:
size_t max_tasks_in_queue = 1000;
std::atomic<UInt64> max_id = 0;
const CurrentMetrics::Metric * max_entry_metric;
};

View File

@ -138,20 +138,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
bool old_style_database = context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary;
auto engine = std::make_shared<ASTFunction>();
auto storage = std::make_shared<ASTStorage>();
//FIXME revert it before merge
engine->name = "Atomic";
if (old_style_database)
{
if (database_name == "test")
engine->name = "Ordinary"; // for stateful tests
else
engine = makeASTFunction("Replicated",
std::make_shared<ASTLiteral>(fmt::format("/clickhouse/db/{}/", create.database)),
std::make_shared<ASTLiteral>("s1"),
std::make_shared<ASTLiteral>("r" + toString(getpid())));
}
engine->name = old_style_database ? "Ordinary" : "Atomic";
engine->no_empty_args = true;
storage->set(storage->engine, engine);
create.set(create.storage, storage);
@ -221,6 +208,12 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
"Enable allow_experimental_database_materialize_mysql to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
}
if (create.storage->engine->name == "Replicated" && !context.getSettingsRef().allow_experimental_database_replicated && !internal)
{
throw Exception("Replicated is an experimental database engine. "
"Enable allow_experimental_database_replicated to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
}
DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", context);
if (create.uuid != UUIDHelpers::Nil)

View File

@ -205,10 +205,6 @@ DDLQueryStatusInputStream::DDLQueryStatusInputStream(const String & zk_node_path
addTotalRowsApprox(waiting_hosts.size());
timeout_seconds = context.getSettingsRef().distributed_ddl_task_timeout;
//FIXME revert it before merge
if (context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary)
timeout_seconds = 10;
}
Block DDLQueryStatusInputStream::readImpl()
@ -252,7 +248,6 @@ Block DDLQueryStatusInputStream::readImpl()
sleepForMilliseconds(std::min<size_t>(1000, 50 * (try_number + 1)));
}
/// TODO: add shared lock
if (!zookeeper->exists(node_path))
{
throw Exception(ErrorCodes::UNFINISHED,
@ -301,12 +296,7 @@ Block DDLQueryStatusInputStream::readImpl()
res = sample.cloneWithColumns(std::move(columns));
}
//FIXME revert it before merge
bool is_functional_tests = !by_hostname && context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary;
if (is_functional_tests)
return {};
else
return res;
return res;
}
Strings DDLQueryStatusInputStream::getChildrenAllowNoNode(const std::shared_ptr<zkutil::ZooKeeper> & zookeeper, const String & node_path)

View File

@ -261,6 +261,18 @@
"with_coverage": false
}
},
"Functional stateful tests (release, DatabaseReplicated)": {
"required_build_properties": {
"compiler": "clang-11",
"package_type": "deb",
"build_type": "relwithdebuginfo",
"sanitizer": "none",
"bundled": "bundled",
"splitted": "unsplitted",
"clang-tidy": "disable",
"with_coverage": false
}
},
"Functional stateless tests (address)": {
"required_build_properties": {
"compiler": "clang-11",
@ -381,6 +393,18 @@
"with_coverage": false
}
},
"Functional stateless tests (release, DatabaseReplicated)": {
"required_build_properties": {
"compiler": "clang-11",
"package_type": "deb",
"build_type": "relwithdebuginfo",
"sanitizer": "none",
"bundled": "bundled",
"splitted": "unsplitted",
"clang-tidy": "disable",
"with_coverage": false
}
},
"Stress test (address)": {
"required_build_properties": {
"compiler": "clang-11",

View File

@ -105,7 +105,9 @@ def remove_control_characters(s):
s = re.sub(r"[\x00-\x08\x0b\x0e-\x1f\x7f]", "", s)
return s
def get_db_engine(args):
def get_db_engine(args, database_name):
if args.replicated_database:
return " ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')".format(database_name)
if args.db_engine:
return " ENGINE=" + args.db_engine
return "" # Will use default engine
@ -128,7 +130,7 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True)
try:
clickhouse_proc_create.communicate(("CREATE DATABASE " + database + get_db_engine(args)), timeout=args.timeout)
clickhouse_proc_create.communicate(("CREATE DATABASE " + database + get_db_engine(args, database)), timeout=args.timeout)
except TimeoutExpired:
total_time = (datetime.now() - start_time).total_seconds()
return clickhouse_proc_create, "", "Timeout creating database {} before test".format(database), total_time
@ -532,6 +534,8 @@ class BuildFlags():
RELEASE = 'release-build'
DATABASE_ORDINARY = 'database-ordinary'
POLYMORPHIC_PARTS = 'polymorphic-parts'
ANTLR = 'antlr'
DATABASE_REPLICATED = 'database-replicated'
def collect_build_flags(client):
@ -613,7 +617,9 @@ def main(args):
build_flags = collect_build_flags(args.client)
if args.antlr:
build_flags.append('antlr')
build_flags.append(BuildFlags.ANTLR)
if args.replicated_database:
build_flags.append(BuildFlags.DATABASE_REPLICATED)
if args.use_skip_list:
tests_to_skip_from_list = collect_tests_to_skip(args.skip_list_path, build_flags)
@ -666,10 +672,10 @@ def main(args):
if args.database and args.database != "test":
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True)
clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + args.database + get_db_engine(args)))
clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + args.database + get_db_engine(args, args.database)))
clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True)
clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS test" + get_db_engine(args)))
clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS test" + get_db_engine(args, 'test')))
def is_test_from_dir(suite_dir, case):
case_file = os.path.join(suite_dir, case)
@ -923,6 +929,7 @@ if __name__ == '__main__':
parser.add_argument('--skip-list-path', help="Path to skip-list file")
parser.add_argument('--use-skip-list', action='store_true', default=False, help="Use skip list to skip tests if found")
parser.add_argument('--db-engine', help='Database engine name')
parser.add_argument('--replicated-database', action='store_true', default=False, help='Run tests with Replicated database engine')
parser.add_argument('--antlr', action='store_true', default=False, dest='antlr', help='Use new ANTLR parser in tests')
parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests')

View File

@ -61,5 +61,8 @@ fi
if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; then
ln -sf $SRC_PATH/users.d/database_ordinary.xml $DEST_SERVER_PATH/users.d/
fi
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
ln -sf $SRC_PATH/users.d/database_replicated.xml $DEST_SERVER_PATH/users.d/
fi
ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml

View File

@ -0,0 +1,10 @@
<yandex>
<profiles>
<default>
<allow_experimental_database_replicated>1</allow_experimental_database_replicated>
<database_replicated_ddl_output>0</database_replicated_ddl_output>
<database_replicated_initial_query_timeout_sec>30</database_replicated_initial_query_timeout_sec>
<distributed_ddl_task_timeout>30</distributed_ddl_task_timeout>
</default>
</profiles>
</yandex>

View File

@ -14,7 +14,7 @@ DOCKER_COMPOSE_PATH = get_docker_compose_path()
cluster = ClickHouseCluster(__file__)
node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=False, stay_alive=True, with_zookeeper=True) #FIXME
node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=False, stay_alive=True)
node_db_atomic = cluster.add_instance('node2', user_configs=["configs/users_db_atomic.xml"], with_mysql=False, stay_alive=True)

View File

@ -0,0 +1,12 @@
<yandex>
<profiles>
<default>
<allow_experimental_database_replicated>1</allow_experimental_database_replicated>
</default>
</profiles>
<users>
<default>
<profile>default</profile>
</default>
</users>
</yandex>

View File

@ -8,11 +8,11 @@ from helpers.network import PartitionManager
cluster = ClickHouseCluster(__file__)
main_node = cluster.add_instance('main_node', main_configs=['configs/config.xml'], with_zookeeper=True, stay_alive=True, macros={"shard": 1, "replica": 1})
dummy_node = cluster.add_instance('dummy_node', main_configs=['configs/config.xml'], with_zookeeper=True, stay_alive=True, macros={"shard": 1, "replica": 2})
competing_node = cluster.add_instance('competing_node', main_configs=['configs/config.xml'], with_zookeeper=True, macros={"shard": 1, "replica": 3})
snapshotting_node = cluster.add_instance('snapshotting_node', main_configs=['configs/config.xml'], with_zookeeper=True, macros={"shard": 2, "replica": 1})
snapshot_recovering_node = cluster.add_instance('snapshot_recovering_node', main_configs=['configs/config.xml'], with_zookeeper=True, macros={"shard": 2, "replica": 2})
main_node = cluster.add_instance('main_node', main_configs=['configs/config.xml'], user_configs=['configs/settings.xml'], with_zookeeper=True, stay_alive=True, macros={"shard": 1, "replica": 1})
dummy_node = cluster.add_instance('dummy_node', main_configs=['configs/config.xml'], user_configs=['configs/settings.xml'], with_zookeeper=True, stay_alive=True, macros={"shard": 1, "replica": 2})
competing_node = cluster.add_instance('competing_node', main_configs=['configs/config.xml'], user_configs=['configs/settings.xml'], with_zookeeper=True, macros={"shard": 1, "replica": 3})
snapshotting_node = cluster.add_instance('snapshotting_node', main_configs=['configs/config.xml'], user_configs=['configs/settings.xml'], with_zookeeper=True, macros={"shard": 2, "replica": 1})
snapshot_recovering_node = cluster.add_instance('snapshot_recovering_node', main_configs=['configs/config.xml'], user_configs=['configs/settings.xml'], with_zookeeper=True, macros={"shard": 2, "replica": 2})
all_nodes = [main_node, dummy_node, competing_node, snapshotting_node, snapshot_recovering_node]

View File

@ -100,10 +100,15 @@
"00604_show_create_database",
"00609_mv_index_in_in",
"00510_materizlized_view_and_deduplication_zookeeper",
"memory_tracking", /// FIXME remove it before merge
"00738_lock_for_inner_table"
],
"database-replicated": [
"memory_tracking",
"memory_usage",
"live_view",
"01188_attach_table_from_pat",
"01415_sticking_mutations",
"01130_in_memory_parts",
"01110_dictionary_layout_without_arguments",
"01018_ddl_dictionaries_create",
"01018_ddl_dictionaries_select",
@ -167,7 +172,6 @@
"01493_alter_remove_properties_zookeeper",
"01475_read_subcolumns_storages",
"01475_read_subcolumns",
"01463_test_alter_live_view_refresh",
"01451_replicated_detach_drop_part",
"01451_detach_drop_part",
"01440_big_int_exotic_casts",
@ -180,9 +184,6 @@
"01355_alter_column_with_order",
"01291_geo_types",
"01270_optimize_skip_unused_shards_low_cardinality",
"01237_live_view_over_distributed_with_subquery_select_table_alias",
"01236_distributed_over_live_view_over_distributed",
"01235_live_view_over_distributed",
"01182_materialized_view_different_structure",
"01150_ddl_guard_rwr",
"01148_zookeeper_path_macros_unfolding",
@ -194,7 +195,6 @@
"01073_attach_if_not_exists",
"01072_optimize_skip_unused_shards_const_expr_eval",
"01071_prohibition_secondary_index_with_old_format_merge_tree",
"01071_live_view_detach_dependency",
"01062_alter_on_mutataion_zookeeper",
"01060_shutdown_table_after_detach",
"01056_create_table_as",
@ -207,27 +207,6 @@
"00989_parallel_parts_loading",
"00980_zookeeper_merge_tree_alter_settings",
"00980_merge_alter_settings",
"00980_create_temporary_live_view",
"00978_live_view_watch",
"00977_live_view_watch_events",
"00976_live_view_select_version",
"00975_live_view_create",
"00974_live_view_select_with_aggregation",
"00973_live_view_with_subquery_select_with_aggregation_in_subquery",
"00973_live_view_with_subquery_select_with_aggregation",
"00973_live_view_with_subquery_select_table_alias",
"00973_live_view_with_subquery_select_nested_with_aggregation_table_alias",
"00973_live_view_with_subquery_select_nested_with_aggregation",
"00973_live_view_with_subquery_select_nested",
"00973_live_view_with_subquery_select_join_no_alias",
"00973_live_view_with_subquery_select_join",
"00973_live_view_with_subquery_select",
"00973_live_view_select_prewhere",
"00973_live_view_select",
"00972_live_view_select_1",
"00969_live_view_watch_format_jsoneachrowwithprogress",
"00968_live_view_select_format_jsoneachrowwithprogress",
"00961_temporary_live_view_watch",
"00955_test_final_mark",
"00933_reserved_word",
"00926_zookeeper_adaptive_index_granularity_replicated_merge_tree",