Compare commits

...

13 Commits

Author SHA1 Message Date
Ilya Golshtein
cb5fc34f29
Merge 287aeba8eb into da2176d696 2024-11-21 08:33:35 +03:00
Alexey Milovidov
da2176d696
Merge pull request #72081 from ClickHouse/add-dashboard-selector
Add advanced dashboard selector
2024-11-21 05:06:51 +00:00
Alexey Milovidov
53e0036593
Merge pull request #72176 from ClickHouse/change-ldf-major-versions
Get rid of `major` tags in official docker images
2024-11-21 05:05:41 +00:00
Alexey Milovidov
25bd73ea5e
Merge pull request #72023 from ClickHouse/fix-bind
Fix comments
2024-11-21 05:03:24 +00:00
Ilya Golshtein
287aeba8eb mv_dependencies: get rid of random_device 2024-11-20 23:51:23 +03:00
Ilya Golshtein
85631dba54 mv_dependencies: style fixes 2024-11-20 18:51:58 +03:00
Ilya Golshtein
0a2b97c9c6 mv_dependencies: cascaded integration tests 2024-11-20 18:51:58 +03:00
Ilya Golshtein
1bbfe5340a mv_dependencies: remove redundant code 2024-11-20 18:51:58 +03:00
Ilya Golshtein
3fcd892a36 mv_dependencies: test passed 2024-11-20 18:51:58 +03:00
Ilya Golshtein
410951d64f mv_dependencies: initial 2024-11-20 18:51:11 +03:00
Mikhail f. Shiryaev
9a2a664b04
Get rid of major tags in official docker images 2024-11-20 16:36:50 +01:00
serxa
ad67608956 Add advanced dashboard selector 2024-11-19 13:18:21 +00:00
Alexey Milovidov
49589da56e Fix comments 2024-11-18 07:18:46 +01:00
21 changed files with 210 additions and 33 deletions

View File

@ -16,16 +16,18 @@ ClickHouse works 100-1000x faster than traditional database management systems,
For more information and documentation see https://clickhouse.com/.
<!-- This is not related to the docker official library, remove it before commit to https://github.com/docker-library/docs -->
## Versions
- The `latest` tag points to the latest release of the latest stable branch.
- Branch tags like `22.2` point to the latest release of the corresponding branch.
- Full version tags like `22.2.3.5` point to the corresponding release.
- Full version tags like `22.2.3` and `22.2.3.5` point to the corresponding release.
<!-- docker-official-library:off -->
<!-- This is not related to the docker official library, remove it before commit to https://github.com/docker-library/docs -->
- The tag `head` is built from the latest commit to the default branch.
- Each tag has optional `-alpine` suffix to reflect that it's built on top of `alpine`.
<!-- REMOVE UNTIL HERE -->
<!-- docker-official-library:on -->
### Compatibility
- The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3.

View File

@ -10,16 +10,18 @@ ClickHouse works 100-1000x faster than traditional database management systems,
For more information and documentation see https://clickhouse.com/.
<!-- This is not related to the docker official library, remove it before commit to https://github.com/docker-library/docs -->
## Versions
- The `latest` tag points to the latest release of the latest stable branch.
- Branch tags like `22.2` point to the latest release of the corresponding branch.
- Full version tags like `22.2.3.5` point to the corresponding release.
- Full version tags like `22.2.3` and `22.2.3.5` point to the corresponding release.
<!-- docker-official-library:off -->
<!-- This is not related to the docker official library, remove it before commit to https://github.com/docker-library/docs -->
- The tag `head` is built from the latest commit to the default branch.
- Each tag has optional `-alpine` suffix to reflect that it's built on top of `alpine`.
<!-- REMOVE UNTIL HERE -->
<!-- docker-official-library:on -->
### Compatibility
- The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3.

View File

@ -522,4 +522,3 @@ sidebar_label: 2024
* Backported in [#68518](https://github.com/ClickHouse/ClickHouse/issues/68518): Minor update in Dynamic/JSON serializations. [#68459](https://github.com/ClickHouse/ClickHouse/pull/68459) ([Kruglov Pavel](https://github.com/Avogar)).
* Backported in [#68558](https://github.com/ClickHouse/ClickHouse/issues/68558): CI: Minor release workflow fix. [#68536](https://github.com/ClickHouse/ClickHouse/pull/68536) ([Max K.](https://github.com/maxknv)).
* Backported in [#68576](https://github.com/ClickHouse/ClickHouse/issues/68576): CI: Tidy build timeout from 2h to 3h. [#68567](https://github.com/ClickHouse/ClickHouse/pull/68567) ([Max K.](https://github.com/maxknv)).

View File

@ -497,4 +497,3 @@ sidebar_label: 2024
* Backported in [#69899](https://github.com/ClickHouse/ClickHouse/issues/69899): Revert "Merge pull request [#69032](https://github.com/ClickHouse/ClickHouse/issues/69032) from alexon1234/include_real_time_execution_in_http_header". [#69885](https://github.com/ClickHouse/ClickHouse/pull/69885) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Backported in [#69931](https://github.com/ClickHouse/ClickHouse/issues/69931): RIPE is an acronym and thus should be capital. RIPE stands for **R**ACE **I**ntegrity **P**rimitives **E**valuation and RACE stands for **R**esearch and Development in **A**dvanced **C**ommunications **T**echnologies in **E**urope. [#69901](https://github.com/ClickHouse/ClickHouse/pull/69901) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Backported in [#70034](https://github.com/ClickHouse/ClickHouse/issues/70034): Revert "Add RIPEMD160 function". [#70005](https://github.com/ClickHouse/ClickHouse/pull/70005) ([Robert Schulze](https://github.com/rschu1ze)).

View File

@ -476,7 +476,7 @@
<input id="edit" type="button" value="✎" style="display: none;">
<input id="add" type="button" value="Add chart" style="display: none;">
<input id="reload" type="button" value="Reload">
<span id="search-span" class="nowrap" style="display: none;"><input id="search" type="button" value="🔎" title="Run query to obtain list of charts from ClickHouse"><input id="search-query" name="search" type="text" spellcheck="false"></span>
<span id="search-span" class="nowrap" style="display: none;"><input id="search" type="button" value="🔎" title="Run query to obtain list of charts from ClickHouse. Either select dashboard name or write your own query"><input id="search-query" name="search" list="search-options" type="text" spellcheck="false"><datalist id="search-options"></datalist></span>
<div id="chart-params"></div>
</div>
</form>
@ -532,9 +532,15 @@ const errorMessages = [
}
]
/// Dashboard selector
const dashboardSearchQuery = (dashboard_name) => `SELECT title, query FROM system.dashboards WHERE dashboard = '${dashboard_name}'`;
let dashboard_queries = {
"Overview": dashboardSearchQuery("Overview"),
};
const default_dashboard = 'Overview';
/// Query to fill `queries` list for the dashboard
let search_query = `SELECT title, query FROM system.dashboards WHERE dashboard = 'Overview'`;
let search_query = dashboardSearchQuery(default_dashboard);
let customized = false;
let queries = [];
@ -1439,7 +1445,7 @@ async function reloadAll(do_search) {
try {
updateParams();
if (do_search) {
search_query = document.getElementById('search-query').value;
search_query = toSearchQuery(document.getElementById('search-query').value);
queries = [];
refreshCustomized(false);
}
@ -1504,7 +1510,7 @@ function updateFromState() {
document.getElementById('url').value = host;
document.getElementById('user').value = user;
document.getElementById('password').value = password;
document.getElementById('search-query').value = search_query;
document.getElementById('search-query').value = fromSearchQuery(search_query);
refreshCustomized();
}
@ -1543,6 +1549,44 @@ if (window.location.hash) {
} catch {}
}
function fromSearchQuery(query) {
for (const dashboard_name in dashboard_queries) {
if (query == dashboard_queries[dashboard_name])
return dashboard_name;
}
return query;
}
function toSearchQuery(value) {
if (value in dashboard_queries)
return dashboard_queries[value];
else
return value;
}
async function populateSearchOptions() {
let {reply, error} = await doFetch("SELECT dashboard FROM system.dashboards GROUP BY dashboard ORDER BY ALL");
if (error) {
throw new Error(error);
}
let data = reply.data;
if (data.dashboard.length == 0) {
console.log("Unable to fetch dashboards list");
return;
}
dashboard_queries = {};
for (let i = 0; i < data.dashboard.length; i++) {
const dashboard = data.dashboard[i];
dashboard_queries[dashboard] = dashboardSearchQuery(dashboard);
}
const searchOptions = document.getElementById('search-options');
for (const dashboard in dashboard_queries) {
const opt = document.createElement('option');
opt.value = dashboard;
searchOptions.appendChild(opt);
}
}
async function start() {
try {
updateFromState();
@ -1558,6 +1602,7 @@ async function start() {
} else {
drawAll();
}
await populateSearchOptions();
} catch (e) {
showError(e.message);
}

View File

@ -528,7 +528,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromCompoundExpression(
*
* Resolve strategy:
* 1. Try to bind identifier to scope argument name to node map.
* 2. If identifier is binded but expression context and node type are incompatible return nullptr.
* 2. If identifier is bound but expression context and node type are incompatible return nullptr.
*
* It is important to support edge cases, where we lookup for table or function node, but argument has same name.
* Example: WITH (x -> x + 1) AS func, (func -> func(1) + func) AS lambda SELECT lambda(1);

View File

@ -362,7 +362,7 @@ ReplxxLineReader::ReplxxLineReader(
if (highlighter)
rx.set_highlighter_callback(highlighter);
/// By default C-p/C-n binded to COMPLETE_NEXT/COMPLETE_PREV,
/// By default C-p/C-n bound to COMPLETE_NEXT/COMPLETE_PREV,
/// bind C-p/C-n to history-previous/history-next like readline.
rx.bind_key(Replxx::KEY::control('N'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::HISTORY_NEXT, code); });
rx.bind_key(Replxx::KEY::control('P'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::HISTORY_PREVIOUS, code); });
@ -384,9 +384,9 @@ ReplxxLineReader::ReplxxLineReader(
rx.bind_key(Replxx::KEY::control('J'), commit_action);
rx.bind_key(Replxx::KEY::ENTER, commit_action);
/// By default COMPLETE_NEXT/COMPLETE_PREV was binded to C-p/C-n, re-bind
/// By default COMPLETE_NEXT/COMPLETE_PREV was bound to C-p/C-n, re-bind
/// to M-P/M-N (that was used for HISTORY_COMMON_PREFIX_SEARCH before, but
/// it also binded to M-p/M-n).
/// it also bound to M-p/M-n).
rx.bind_key(Replxx::KEY::meta('N'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::COMPLETE_NEXT, code); });
rx.bind_key(Replxx::KEY::meta('P'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::COMPLETE_PREVIOUS, code); });
/// By default M-BACKSPACE is KILL_TO_WHITESPACE_ON_LEFT, while in readline it is backward-kill-word

View File

@ -206,10 +206,12 @@ namespace DB
DECLARE(UInt64, threadpool_writer_pool_size, 100, "Size of background pool for write requests to object storages", 0) \
DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) \
DECLARE(UInt32, allowed_feature_tier, 0, "0 - All feature tiers allowed (experimental, beta, production). 1 - Only beta and production feature tiers allowed. 2 - Only production feature tier allowed", 0) \
DECLARE(UInt64, startup_mv_delay_ms, 0, "Debug parameter to simulate materizlied view creation delay", 0) \
// clang-format on
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, LIST_OF_SERVER_SETTINGS)

View File

@ -209,6 +209,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
{
size_t prev_tables_count = metadata.parsed_tables.size();
size_t prev_total_dictionaries = metadata.total_dictionaries;
size_t prev_total_materialized_views = metadata.total_materialized_views;
auto process_metadata = [&metadata, is_startup, local_context, this](const String & file_name)
{
@ -276,6 +277,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
std::lock_guard lock{metadata.mutex};
metadata.parsed_tables[qualified_name] = ParsedTableMetadata{full_path.string(), ast};
metadata.total_dictionaries += create_query->is_dictionary;
metadata.total_materialized_views += create_query->is_materialized_view;
}
}
catch (Exception & e)
@ -289,10 +291,17 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
size_t objects_in_database = metadata.parsed_tables.size() - prev_tables_count;
size_t dictionaries_in_database = metadata.total_dictionaries - prev_total_dictionaries;
size_t materialized_views_in_database = metadata.total_materialized_views - prev_total_materialized_views;
size_t tables_in_database = objects_in_database - dictionaries_in_database;
LOG_INFO(log, "Metadata processed, database {} has {} tables and {} dictionaries in total.",
TSA_SUPPRESS_WARNING_FOR_READ(database_name), tables_in_database, dictionaries_in_database);
LOG_INFO(log, "Metadata processed, database {} has {} tables, {} dictionaries and {} materialized views in total.",
TSA_SUPPRESS_WARNING_FOR_READ(database_name), tables_in_database, dictionaries_in_database, materialized_views_in_database);
// if (materialized_views_in_database)
// {
// }
}
void DatabaseOrdinary::loadTableFromMetadata(
@ -317,6 +326,8 @@ void DatabaseOrdinary::loadTableFromMetadata(
mode);
attachTable(local_context, table_name, table, getTableDataPath(query));
table->pushDependencies();
}
catch (Exception & e)
{

View File

@ -434,6 +434,8 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c
for (auto metric : getAttachedCountersForStorage(table))
CurrentMetrics::add(metric);
}
// if (DatabaseCatalog::iiMa
}
void DatabaseWithOwnTablesBase::shutdown()

View File

@ -44,6 +44,7 @@ struct ParsedTablesMetadata
/// For logging
size_t total_dictionaries = 0;
size_t total_materialized_views = 0;
};
/// Loads tables (and dictionaries) from specified databases

View File

@ -572,6 +572,11 @@ public:
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part moves between shards are not supported by storage {}", getName());
}
virtual void pushDependencies()
{
}
/** If the table have to do some complicated work on startup,
* that must be postponed after creation of table object
* (like launching some background threads),

View File

@ -187,6 +187,7 @@ StorageKafka::StorageKafka(
, thread_per_consumer((*kafka_settings)[KafkaSetting::kafka_thread_per_consumer].value)
, collection_name(collection_name_)
{
LOG_TRACE(log, "Top of StorageKafka ctor.");
kafka_settings->sanityCheck();
if ((*kafka_settings)[KafkaSetting::kafka_handle_error_mode] == StreamingHandleErrorMode::STREAM)
@ -524,6 +525,7 @@ size_t StorageKafka::getPollTimeoutMillisecond() const
void StorageKafka::threadFunc(size_t idx)
{
LOG_DEBUG(log, "Top of StorageKafka::threadFunc");
assert(idx < tasks.size());
auto task = tasks[idx];
std::string exception_str;
@ -533,15 +535,18 @@ void StorageKafka::threadFunc(size_t idx)
auto table_id = getStorageID();
// Check if at least one direct dependency is attached
size_t num_views = DatabaseCatalog::instance().getDependentViews(table_id).size();
LOG_DEBUG(log, "StorageKafka::threadFunc - before if");
if (num_views)
{
auto start_time = std::chrono::steady_clock::now();
mv_attached.store(true);
LOG_DEBUG(log, "StorageKafka::threadFunc - before while");
// Keep streaming as long as there are attached views and streaming is not cancelled
while (!task->stream_cancelled)
{
LOG_DEBUG(log, "StorageKafka::threadFunc - before StorageKafkaUtils::checkDependencies");
if (!StorageKafkaUtils::checkDependencies(table_id, getContext()))
break;
@ -564,6 +569,9 @@ void StorageKafka::threadFunc(size_t idx)
}
}
}
else
LOG_DEBUG(log, "No attached views");
}
catch (...)
{

View File

@ -425,15 +425,19 @@ bool checkDependencies(const StorageID & table_id, const ContextPtr& context)
// Check the dependencies are ready?
for (const auto & view_id : view_ids)
{
LOG_TRACE(&Poco::Logger::get("kafka checkDependencies"), "Top of for");
auto view = DatabaseCatalog::instance().tryGetTable(view_id, context);
if (!view)
return false;
LOG_TRACE(&Poco::Logger::get("kafka checkDependencies"), "Target table");
// If it materialized view, check it's target table
auto * materialized_view = dynamic_cast<StorageMaterializedView *>(view.get());
if (materialized_view && !materialized_view->tryGetTargetTable())
return false;
LOG_TRACE(&Poco::Logger::get("kafka checkDependencies"), "Transitive dependencies");
// Check all its dependencies
if (!checkDependencies(view_id, context))
return false;

View File

@ -29,6 +29,7 @@
#include <Common/typeid_cast.h>
#include <Common/checkStackSize.h>
#include <Common/randomSeed.h>
#include <Core/ServerSettings.h>
#include <Core/Settings.h>
#include <QueryPipeline/Pipe.h>
@ -40,6 +41,8 @@
#include <Backups/BackupEntriesCollector.h>
#include <Common/logger_useful.h>
namespace DB
{
namespace Setting
@ -51,6 +54,7 @@ namespace Setting
namespace ServerSetting
{
extern const ServerSettingsUInt64 max_materialized_views_count_for_table;
extern const ServerSettingsUInt64 startup_mv_delay_ms;
}
namespace RefreshSetting
@ -745,12 +749,34 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id)
refresher->rename(new_table_id, getTargetTableId());
}
void StorageMaterializedView::startup()
void StorageMaterializedView::pushDependencies()
{
// assert(!dependencies_are_tracked);
if (!dependencies_are_tracked)
{
auto metadata_snapshot = getInMemoryMetadataPtr();
const auto & select_query = metadata_snapshot->getSelectQuery();
if (!select_query.select_table_id.empty())
DatabaseCatalog::instance().addViewDependency(select_query.select_table_id, getStorageID());
dependencies_are_tracked = true;
}
}
void StorageMaterializedView::startup()
{
if (const auto configured_delay_ms = getContext()->getServerSettings()[ServerSetting::startup_mv_delay_ms]; configured_delay_ms)
{
pcg64_fast gen{randomSeed()};
const auto delay_ms = std::uniform_int_distribution<>(0, 1)(gen) ? configured_delay_ms : 0UL;
if (delay_ms)
{
LOG_DEBUG(&Poco::Logger::get("StorageMaterializedView"), "sleeping in startup of {}", getStorageID().table_name);
std::this_thread::sleep_for(std::chrono::milliseconds(delay_ms));
LOG_DEBUG(&Poco::Logger::get("StorageMaterializedView"), "woken up in startup of {}", getStorageID().table_name);
}
}
pushDependencies();
if (refresher)
refresher->startup();

View File

@ -67,6 +67,7 @@ public:
void renameInMemory(const StorageID & new_table_id) override;
void pushDependencies() override;
void startup() override;
void shutdown(bool is_drop) override;
@ -116,6 +117,8 @@ private:
/// have UUID, and we do inner table lookup by name instead.
bool fixed_uuid = true;
bool dependencies_are_tracked = false;
friend class RefreshTask;
void checkStatementCanBeForwarded() const;

View File

@ -299,8 +299,6 @@ class TagAttrs:
# Only one latest can exist
latest: ClickHouseVersion
# Only one can be a major one (the most fresh per a year)
majors: Dict[int, ClickHouseVersion]
# Only one lts version can exist
lts: Optional[ClickHouseVersion]
@ -345,14 +343,6 @@ def ldf_tags(version: ClickHouseVersion, distro: str, tag_attrs: TagAttrs) -> st
tags.append("lts")
tags.append(f"lts-{distro}")
# If the tag `22`, `23`, `24` etc. should be included in the tags
with_major = tag_attrs.majors.get(version.major) in (None, version)
if with_major:
tag_attrs.majors[version.major] = version
if without_distro:
tags.append(f"{version.major}")
tags.append(f"{version.major}-{distro}")
# Add all normal tags
for tag in (
f"{version.major}.{version.minor}",
@ -384,7 +374,7 @@ def generate_ldf(args: argparse.Namespace) -> None:
args.directory / git_runner(f"git -C {args.directory} rev-parse --show-cdup")
).absolute()
lines = ldf_header(git, directory)
tag_attrs = TagAttrs(versions[-1], {}, None)
tag_attrs = TagAttrs(versions[-1], None)
# We iterate from the most recent to the oldest version
for version in reversed(versions):

View File

@ -1,3 +1,4 @@
<clickhouse>
<async_load_databases>true</async_load_databases>
<startup_mv_delay_ms>10000</startup_mv_delay_ms>
</clickhouse>

View File

@ -242,3 +242,80 @@ def test_async_load_system_database(started_cluster):
for i in range(id - 1):
node2.query(f"drop table if exists system.text_log_{i + 1}_test")
node2.query(f"drop table if exists system.query_log_{i + 1}_test")
def test_materialzed_views(started_cluster):
query = node1.query
query("create database test_mv")
query("create table test_mv.t (Id UInt64) engine=MergeTree order by Id")
query("create table test_mv.a (Id UInt64) engine=MergeTree order by Id")
query("create table test_mv.z (Id UInt64) engine=MergeTree order by Id")
query("create materialized view t_to_a to test_mv.a as select Id from test_mv.t")
query("create materialized view t_to_z to test_mv.z as select Id from test_mv.t")
node1.restart_clickhouse()
query("insert into test_mv.t values(42)")
assert query("select * from test_mv.a Format CSV") == "42\n"
assert query("select * from test_mv.z Format CSV") == "42\n"
query("drop view t_to_a")
query("drop view t_to_z")
query("drop table test_mv.t")
query("drop table test_mv.a")
query("drop table test_mv.z")
query("drop database test_mv")
def test_materialzed_views_cascaded(started_cluster):
query = node1.query
query("create database test_mv")
query("create table test_mv.t (Id UInt64) engine=MergeTree order by Id")
query("create table test_mv.a (Id UInt64) engine=MergeTree order by Id")
query("create table test_mv.z (Id UInt64) engine=MergeTree order by Id")
query("create materialized view t_to_a to test_mv.a as select Id from test_mv.t")
query("create materialized view a_to_z to test_mv.z as select Id from test_mv.a")
node1.restart_clickhouse()
query("insert into test_mv.t values(42)")
assert query("select * from test_mv.a Format CSV") == "42\n"
assert query("select * from test_mv.z Format CSV") == "42\n"
query("drop view t_to_a")
query("drop view a_to_z")
query("drop table test_mv.t")
query("drop table test_mv.a")
query("drop table test_mv.z")
query("drop database test_mv")
def test_materialzed_views_cascaded_multiple(started_cluster):
query = node1.query
query("create database test_mv")
query("create table test_mv.t (Id UInt64) engine=MergeTree order by Id")
query("create table test_mv.a (Id UInt64) engine=MergeTree order by Id")
query("create table test_mv.x (IdText String) engine=MergeTree order by IdText")
query(
"create table test_mv.z (Id UInt64, IdTextLength UInt64) engine=MergeTree order by Id"
)
query("create materialized view t_to_a to test_mv.a as select Id from test_mv.t")
query(
"create materialized view t_to_x to test_mv.x as select toString(Id) as IdText from test_mv.t"
)
query(
"create materialized view ax_to_z to test_mv.z as select Id, (select max(length(IdText)) from test_mv.x) as IdTextLength from test_mv.a"
)
node1.restart_clickhouse()
query("insert into test_mv.t values(42)")
assert query("select * from test_mv.a Format CSV") == "42\n"
assert query("select * from test_mv.x Format CSV") == '"42"\n'
assert query("select * from test_mv.z Format CSV") == "42,2\n"
query("drop view t_to_a")
query("drop view t_to_x")
query("drop view ax_to_z")
query("drop table test_mv.t")
query("drop table test_mv.a")
query("drop table test_mv.x")
query("drop table test_mv.z")
query("drop database test_mv")

View File

@ -378,7 +378,7 @@ def test_reload_via_client(cluster, zk):
configure_from_zk(zk)
break
except QueryRuntimeException:
logging.exception("The new socket is not binded yet")
logging.exception("The new socket is not bound yet")
time.sleep(0.1)
if exception: