Merge branch 'master' into read-cgroup-memory-usage-async-metrics

This commit is contained in:
Antonio Andelic 2024-07-18 09:01:54 +02:00
commit c413e7a494
81 changed files with 1367 additions and 388 deletions

2
contrib/grpc vendored

@ -1 +1 @@
Subproject commit f5b7fdc2dff09ada06dbf6c75df298fb40f898df
Subproject commit 1716359d2e28d304a250f9df0e6c0ccad03de8db

View File

@ -26,7 +26,10 @@ RUN apt-get update \
zstd \
--yes --no-install-recommends \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \
&& groupadd --system --gid 1000 clickhouse \
&& useradd --system --gid 1000 --uid 1000 -m clickhouse
# ^ For some reason, groupadd and useradd are needed for tests with 'expect', but I don't know, why.
COPY requirements.txt /
RUN pip3 install --no-cache-dir -r /requirements.txt

View File

@ -9,7 +9,7 @@ trap 'kill $(jobs -pr) ||:' EXIT
stage=${stage:-}
# Compiler version, normally set by Dockerfile
export LLVM_VERSION=${LLVM_VERSION:-17}
export LLVM_VERSION=${LLVM_VERSION:-18}
# A variable to pass additional flags to CMake.
# Here we explicitly default it to nothing so that bash doesn't complain about

View File

@ -17,6 +17,7 @@ ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=18
RUN apt-get update \
&& apt-get install \
sudo \
apt-transport-https \
apt-utils \
ca-certificates \

View File

@ -185,6 +185,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va
- `--format, -f` Use the specified default format to output the result.
- `--vertical, -E` If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables.
- `--time, -t` If specified, print the query execution time to stderr in non-interactive mode.
- `--memory-usage` If specified, print memory usage to stderr in non-interactive mode]. Possible values: 'none' - do not print memory usage, 'default' - print number of bytes, 'readable' - print memory usage in human-readable format.
- `--stacktrace` If specified, also print the stack trace if an exception occurs.
- `--config-file` The name of the configuration file.
- `--secure` If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl).

View File

@ -0,0 +1,35 @@
---
slug: /en/operations/system-tables/detached_tables
---
# detached_tables
Contains information of each detached table.
Columns:
- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the table is in.
- `table` ([String](../../sql-reference/data-types/string.md)) — Table name.
- `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Table uuid (Atomic database).
- `metadata_path` ([String](../../sql-reference/data-types/string.md)) - Path to the table metadata in the file system.
- `is_permanently` ([UInt8](../../sql-reference/data-types/int-uint.md)) - Flag indicates that the table was detached PERMANENTLY.
**Example**
```sql
SELECT * FROM system.detached_tables FORMAT Vertical;
```
```text
Row 1:
──────
database: base
table: t1
uuid: 81b1c20a-b7c6-4116-a2ce-7583fb6b6736
metadata_path: /var/lib/clickhouse/store/461/461cf698-fd0b-406d-8c01-5d8fd5748a91/t1.sql
is_permanently: 1
```

View File

@ -56,7 +56,6 @@ Functions:
## Related content
- [Reducing ClickHouse Storage Cost with the Low Cardinality Type Lessons from an Instana Engineer](https://altinity.com/blog/2020-5-20-reducing-clickhouse-storage-cost-with-the-low-cardinality-type-lessons-from-an-instana-engineer)
- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf)
- Blog: [Optimizing ClickHouse with Schemas and Codecs](https://clickhouse.com/blog/optimize-clickhouse-codecs-compression-schema)
- Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse)
- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf)

View File

@ -76,7 +76,7 @@ WHERE macro = 'test';
└───────┴──────────────┘
```
## FQDN
## fqdn
Returns the fully qualified domain name of the ClickHouse server.
@ -86,7 +86,7 @@ Returns the fully qualified domain name of the ClickHouse server.
fqdn();
```
Aliases: `fullHostName`, 'FQDN'.
Aliases: `fullHostName`, `FQDN`.
**Returned value**

View File

@ -356,7 +356,7 @@ sidebar_label: "\u53D8\u66F4\u65E5\u5FD7"
#### 新功能 {#new-feature-1}
- 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。 这个新功能是由Altinity的特殊要求添加到错误修正版本中的。
- 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。
[#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy)
### ClickHouse版本v20.1.2.4,2020-01-22 {#clickhouse-release-v20-1-2-4-2020-01-22}

View File

@ -423,6 +423,7 @@ void LocalServer::connect()
{
connection_parameters = ConnectionParameters(getClientConfiguration(), "localhost");
/// This is needed for table function input(...).
ReadBuffer * in;
auto table_file = getClientConfiguration().getString("table-file", "-");
if (table_file == "-" || table_file == "stdin")

View File

@ -2070,9 +2070,18 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
progress_indication.writeFinalProgress();
output_stream << std::endl << std::endl;
}
else if (getClientConfiguration().getBool("print-time-to-stderr", false))
else
{
const auto & config = getClientConfiguration();
if (config.getBool("print-time-to-stderr", false))
error_stream << progress_indication.elapsedSeconds() << "\n";
const auto & print_memory_mode = config.getString("print-memory-to-stderr", "");
auto peak_memeory_usage = std::max<Int64>(progress_indication.getMemoryUsage().peak, 0);
if (print_memory_mode == "default")
error_stream << peak_memeory_usage << "\n";
else if (print_memory_mode == "readable")
error_stream << formatReadableSizeWithBinarySuffix(peak_memeory_usage) << "\n";
}
if (!is_interactive && getClientConfiguration().getBool("print-num-processed-rows", false))
@ -3036,6 +3045,7 @@ void ClientBase::init(int argc, char ** argv)
("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.")
("wait_for_suggestions_to_load", "Load suggestion data synchonously.")
("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)")
("memory-usage", po::value<std::string>()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'")
("echo", "in batch mode, print query before execution")
@ -3121,6 +3131,14 @@ void ClientBase::init(int argc, char ** argv)
/// Output execution time to stderr in batch mode.
if (options.count("time"))
getClientConfiguration().setBool("print-time-to-stderr", true);
if (options.count("memory-usage"))
{
const auto & memory_usage_mode = options["memory-usage"].as<std::string>();
if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode);
getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode);
}
if (options.count("query"))
queries = options["query"].as<std::vector<std::string>>();
if (options.count("query_id"))

View File

@ -54,8 +54,6 @@ public:
struct ReplicaInfo
{
bool collaborate_with_initiator{false};
size_t all_replicas_count{0};
size_t number_of_current_replica{0};
};

View File

@ -175,4 +175,5 @@ private:
ReadBuffer * in;
};
}

View File

@ -142,14 +142,13 @@ void MultiplexedConnections::sendQuery(
modified_settings.group_by_two_level_threshold = 0;
modified_settings.group_by_two_level_threshold_bytes = 0;
}
}
if (replica_info)
{
client_info.collaborate_with_initiator = true;
client_info.count_participating_replicas = replica_info->all_replicas_count;
client_info.number_of_current_replica = replica_info->number_of_current_replica;
}
}
/// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting.
/// Make the analyzer being set, so it will be effectively applied on the remote server.

View File

@ -187,12 +187,9 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
#if USE_GWP_ASAN
if (unlikely(GWPAsan::GuardedAlloc.shouldSample()))
{
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
if (void * ptr = GWPAsan::GuardedAlloc.allocate(new_size, alignment))
{
auto trace_free = CurrentMemoryTracker::free(old_size);
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
trace_free.onFree(buf, old_size);
memcpy(ptr, buf, std::min(old_size, new_size));
free(buf, old_size);
trace_alloc.onAlloc(buf, new_size);
@ -209,6 +206,7 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
}
else
{
[[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size);
ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed);
}
}
@ -231,13 +229,17 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
if (alignment <= MALLOC_MIN_ALIGNMENT)
{
/// Resize malloc'd memory region with no special alignment requirement.
auto trace_free = CurrentMemoryTracker::free(old_size);
/// Realloc can do 2 possible things:
/// - expand existing memory region
/// - allocate new memory block and free the old one
/// Because we don't know which option will be picked we need to make sure there is enough
/// memory for all options
auto trace_alloc = CurrentMemoryTracker::alloc(new_size);
trace_free.onFree(buf, old_size);
void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf)
{
[[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size);
throw DB::ErrnoException(
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY,
"Allocator: Cannot realloc from {} to {}",
@ -246,6 +248,8 @@ void * Allocator<clear_memory_, populate>::realloc(void * buf, size_t old_size,
}
buf = new_buf;
auto trace_free = CurrentMemoryTracker::free(old_size);
trace_free.onFree(buf, old_size);
trace_alloc.onAlloc(buf, new_size);
if constexpr (clear_memory)

View File

@ -72,11 +72,6 @@ public:
/// How much seconds passed since query execution start.
double elapsedSeconds() const { return getElapsedNanoseconds() / 1e9; }
void updateThreadEventData(HostToTimesMap & new_hosts_data);
private:
double getCPUUsage();
struct MemoryUsage
{
UInt64 total = 0;
@ -86,6 +81,11 @@ private:
MemoryUsage getMemoryUsage() const;
void updateThreadEventData(HostToTimesMap & new_hosts_data);
private:
double getCPUUsage();
UInt64 getElapsedNanoseconds() const;
/// This flag controls whether to show the progress bar. We start showing it after

View File

@ -1306,6 +1306,10 @@ void BaseDaemon::setupWatchdog()
int status = 0;
do
{
// Close log files to prevent keeping descriptors of unlinked rotated files.
// On next log write files will be reopened.
closeLogs(logger());
if (-1 != waitpid(pid, &status, WUNTRACED | WCONTINUED) || errno == ECHILD)
{
if (WIFSTOPPED(status))

View File

@ -39,8 +39,10 @@ namespace ErrorCodes
class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator
{
public:
explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base)
: DatabaseTablesSnapshotIterator(std::move(base)) {}
explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base) noexcept
: DatabaseTablesSnapshotIterator(std::move(base))
{
}
UUID uuid() const override { return table()->getStorageID().uuid; }
};
@ -111,12 +113,12 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String &
// it is important to call the destructors of not_in_use without
// locked mutex to avoid potential deadlock.
DetachedTables not_in_use;
StoragePtr table;
StoragePtr detached_table;
{
std::lock_guard lock(mutex);
table = DatabaseOrdinary::detachTableUnlocked(name);
detached_table = DatabaseOrdinary::detachTableUnlocked(name);
table_name_to_path.erase(name);
detached_tables.emplace(table->getStorageID().uuid, table);
detached_tables.emplace(detached_table->getStorageID().uuid, detached_table);
not_in_use = cleanupDetachedTables();
}
@ -126,7 +128,7 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String &
LOG_DEBUG(log, "Finished removing not used detached tables");
}
return table;
return detached_table;
}
void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool sync)

View File

@ -1,7 +1,8 @@
#pragma once
#include <Databases/DatabasesCommon.h>
#include <Databases/DatabaseOrdinary.h>
#include <Databases/DatabasesCommon.h>
#include <Storages/IStorage_fwd.h>
namespace DB

View File

@ -188,6 +188,13 @@ void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_n
it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name);
LOG_DEBUG(log, "Add info for detached table {} to snapshot.", backQuote(table_name));
if (snapshot_detached_tables.contains(table_name))
{
LOG_DEBUG(log, "Clean info about detached table {} from snapshot.", backQuote(table_name));
snapshot_detached_tables.erase(table_name);
}
CurrentMetrics::add(CurrentMetrics::AttachedTable, 1);
}
@ -204,6 +211,15 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta
if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator);
tables_cache.erase(it);
LOG_DEBUG(log, "Add info for detached table {} to snapshot.", backQuote(table_name));
snapshot_detached_tables.emplace(
table_name,
SnapshotDetachedTable{
.database = res->getStorageID().database_name,
.table = res->getStorageID().table_name,
.uuid = res->getStorageID().uuid,
.metadata_path = getObjectMetadataPath(table_name),
.is_permanently = false});
CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1);
}

View File

@ -23,6 +23,7 @@
#include <Storages/StorageFactory.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/CurrentMetrics.h>
#include <Common/Exception.h>
#include <Common/assert_cast.h>
#include <Common/escapeForFileName.h>
#include <Common/filesystemHelpers.h>
@ -308,6 +309,16 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri
try
{
FS::createFile(detached_permanently_flag);
std::lock_guard lock(mutex);
if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end())
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name);
}
else
{
it->second.is_permanently = true;
}
}
catch (Exception & e)
{

View File

@ -189,7 +189,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;
auto process_metadata = [&metadata, is_startup, this](const String & file_name)
auto process_metadata = [&metadata, is_startup, local_context, this](const String & file_name)
{
fs::path path(getMetadataPath());
fs::path file_path(file_name);
@ -197,7 +197,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
try
{
auto ast = parseQueryFromMetadata(log, getContext(), full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false);
auto ast = parseQueryFromMetadata(log, local_context, full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false);
if (ast)
{
FunctionNameNormalizer::visit(ast.get());
@ -226,8 +226,23 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
if (fs::exists(full_path.string() + detached_suffix))
{
const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4));
permanently_detached_tables.push_back(table_name);
LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name));
std::lock_guard lock(mutex);
permanently_detached_tables.push_back(table_name);
const auto detached_table_name = create_query->getTable();
snapshot_detached_tables.emplace(
detached_table_name,
SnapshotDetachedTable{
.database = create_query->getDatabase(),
.table = detached_table_name,
.uuid = create_query->uuid,
.metadata_path = getObjectMetadataPath(detached_table_name),
.is_permanently = true});
LOG_TRACE(log, "Add permanently detached table {} to system.detached_tables", detached_table_name);
return;
}
@ -489,6 +504,12 @@ DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_c
return DatabaseWithOwnTablesBase::getTablesIterator(local_context, filter_by_table_name, skip_not_loaded);
}
DatabaseDetachedTablesSnapshotIteratorPtr DatabaseOrdinary::getDetachedTablesIterator(
ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const
{
return DatabaseWithOwnTablesBase::getDetachedTablesIterator(local_context, filter_by_table_name, skip_not_loaded);
}
Strings DatabaseOrdinary::getAllTableNames(ContextPtr) const
{
std::set<String> unique_names;

View File

@ -57,6 +57,9 @@ public:
LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override;
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override;
DatabaseDetachedTablesSnapshotIteratorPtr getDetachedTablesIterator(
ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override;
Strings getAllTableNames(ContextPtr context) const override;
void alterTable(
@ -64,7 +67,11 @@ public:
const StorageID & table_id,
const StorageInMemoryMetadata & metadata) override;
Strings getNamesOfPermanentlyDetachedTables() const override { return permanently_detached_tables; }
Strings getNamesOfPermanentlyDetachedTables() const override
{
std::lock_guard lock(mutex);
return permanently_detached_tables;
}
protected:
virtual void commitAlterTable(
@ -74,7 +81,7 @@ protected:
const String & statement,
ContextPtr query_context);
Strings permanently_detached_tables;
Strings permanently_detached_tables TSA_GUARDED_BY(mutex);
std::unordered_map<String, LoadTaskPtr> load_table TSA_GUARDED_BY(mutex);
std::unordered_map<String, LoadTaskPtr> startup_table TSA_GUARDED_BY(mutex);

View File

@ -2,12 +2,9 @@
#include <Backups/BackupEntriesCollector.h>
#include <Backups/RestorerFromBackup.h>
#include <Common/typeid_cast.h>
#include <Common/CurrentMetrics.h>
#include <Common/escapeForFileName.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ParserCreateQuery.h>
@ -16,6 +13,10 @@
#include <Storages/StorageFactory.h>
#include <Storages/Utils.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/CurrentMetrics.h>
#include <Common/escapeForFileName.h>
#include <Common/logger_useful.h>
#include <Common/typeid_cast.h>
namespace DB
@ -237,6 +238,24 @@ DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPt
return std::make_unique<DatabaseTablesSnapshotIterator>(std::move(filtered_tables), database_name);
}
DatabaseDetachedTablesSnapshotIteratorPtr DatabaseWithOwnTablesBase::getDetachedTablesIterator(
ContextPtr, const FilterByNameFunction & filter_by_table_name, bool /* skip_not_loaded */) const
{
std::lock_guard lock(mutex);
if (!filter_by_table_name)
return std::make_unique<DatabaseDetachedTablesSnapshotIterator>(snapshot_detached_tables);
SnapshotDetachedTables filtered_detached_tables;
for (const auto & [detached_table_name, snapshot] : snapshot_detached_tables)
if (filter_by_table_name(detached_table_name))
{
filtered_detached_tables.emplace(detached_table_name, snapshot);
}
return std::make_unique<DatabaseDetachedTablesSnapshotIterator>(std::move(filtered_detached_tables));
}
bool DatabaseWithOwnTablesBase::empty() const
{
std::lock_guard lock(mutex);
@ -251,27 +270,36 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(ContextPtr /* context_ */, con
StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_name)
{
StoragePtr res;
auto it = tables.find(table_name);
if (it == tables.end())
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
backQuote(database_name), backQuote(table_name));
res = it->second;
auto table_storage = it->second;
snapshot_detached_tables.emplace(
table_name,
SnapshotDetachedTable{
.database = it->second->getStorageID().getDatabaseName(),
.table = table_name,
.uuid = it->second->getStorageID().uuid,
.metadata_path = getObjectMetadataPath(table_name),
.is_permanently = false});
tables.erase(it);
res->is_detached = true;
table_storage->is_detached = true;
if (res->isSystemStorage() == false)
CurrentMetrics::sub(getAttachedCounterForStorage(res), 1);
if (table_storage->isSystemStorage() == false)
CurrentMetrics::sub(getAttachedCounterForStorage(table_storage), 1);
auto table_id = res->getStorageID();
auto table_id = table_storage->getStorageID();
if (table_id.hasUUID())
{
assert(database_name == DatabaseCatalog::TEMPORARY_DATABASE || getUUID() != UUIDHelpers::Nil);
DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid);
}
return res;
return table_storage;
}
void DatabaseWithOwnTablesBase::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &)
@ -300,6 +328,8 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", table_id.getFullTableName());
}
snapshot_detached_tables.erase(table_name);
/// It is important to reset is_detached here since in case of RENAME in
/// non-Atomic database the is_detached is set to true before RENAME.
table->is_detached = false;
@ -337,6 +367,7 @@ void DatabaseWithOwnTablesBase::shutdown()
std::lock_guard lock(mutex);
tables.clear();
snapshot_detached_tables.clear();
}
DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase()

View File

@ -37,6 +37,9 @@ public:
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override;
DatabaseDetachedTablesSnapshotIteratorPtr
getDetachedTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override;
std::vector<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override;
void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr<IRestoreCoordination> restore_coordination, UInt64 timeout_ms) override;
@ -46,6 +49,7 @@ public:
protected:
Tables tables TSA_GUARDED_BY(mutex);
SnapshotDetachedTables snapshot_detached_tables TSA_GUARDED_BY(mutex);
LoggerPtr log;
DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context);

View File

@ -5,20 +5,22 @@
#include <Interpreters/Context_fwd.h>
#include <Interpreters/executeQuery.h>
#include <Parsers/IAST_fwd.h>
#include <QueryPipeline/BlockIO.h>
#include <Storages/IStorage.h>
#include <Storages/IStorage_fwd.h>
#include <base/types.h>
#include <Common/Exception.h>
#include <Common/AsyncLoader.h>
#include <Common/Exception.h>
#include <Common/PoolId.h>
#include <Common/ThreadPool_fwd.h>
#include <QueryPipeline/BlockIO.h>
#include <ctime>
#include <functional>
#include <map>
#include <memory>
#include <mutex>
#include <stdexcept>
#include <vector>
#include <map>
namespace DB
@ -110,6 +112,57 @@ public:
using DatabaseTablesIteratorPtr = std::unique_ptr<IDatabaseTablesIterator>;
struct SnapshotDetachedTable final
{
String database;
String table;
UUID uuid = UUIDHelpers::Nil;
String metadata_path;
bool is_permanently{};
};
class DatabaseDetachedTablesSnapshotIterator
{
private:
SnapshotDetachedTables snapshot;
SnapshotDetachedTables::iterator it;
protected:
DatabaseDetachedTablesSnapshotIterator(DatabaseDetachedTablesSnapshotIterator && other) noexcept
{
size_t idx = std::distance(other.snapshot.begin(), other.it);
std::swap(snapshot, other.snapshot);
other.it = other.snapshot.end();
it = snapshot.begin();
std::advance(it, idx);
}
public:
explicit DatabaseDetachedTablesSnapshotIterator(const SnapshotDetachedTables & tables_) : snapshot(tables_), it(snapshot.begin())
{
}
explicit DatabaseDetachedTablesSnapshotIterator(SnapshotDetachedTables && tables_) : snapshot(std::move(tables_)), it(snapshot.begin())
{
}
void next() { ++it; }
bool isValid() const { return it != snapshot.end(); }
String database() const { return it->second.database; }
String table() const { return it->second.table; }
UUID uuid() const { return it->second.uuid; }
String metadataPath() const { return it->second.metadata_path; }
bool isPermanently() const { return it->second.is_permanently; }
};
using DatabaseDetachedTablesSnapshotIteratorPtr = std::unique_ptr<DatabaseDetachedTablesSnapshotIterator>;
/** Database engine.
* It is responsible for:
@ -232,6 +285,12 @@ public:
/// Wait for all tables to be loaded and started up. If `skip_not_loaded` is true, then not yet loaded or not yet started up (at the moment of iterator creation) tables are excluded.
virtual DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name = {}, bool skip_not_loaded = false) const = 0; /// NOLINT
virtual DatabaseDetachedTablesSnapshotIteratorPtr getDetachedTablesIterator(
ContextPtr /*context*/, const FilterByNameFunction & /*filter_by_table_name = {}*/, bool /*skip_not_loaded = false*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get detached tables for Database{}", getEngineName());
}
/// Returns list of table names.
virtual Strings getAllTableNames(ContextPtr context) const
{

View File

@ -1223,6 +1223,9 @@ private:
else executeGeneric<first>(key_cols, icolumn, vec_to);
}
/// Return a fixed random-looking magic number when input is empty.
static constexpr auto filler = 0xe28dbde7fe22e41c;
void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector<ToType>::Container & vec_to, bool & is_first) const
{
/// Flattening of tuples.
@ -1231,6 +1234,11 @@ private:
const auto & tuple_columns = tuple->getColumns();
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(*type).getElements();
size_t tuple_size = tuple_columns.size();
if (0 == tuple_size && is_first)
for (auto & hash : vec_to)
hash = static_cast<ToType>(filler);
for (size_t i = 0; i < tuple_size; ++i)
executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first);
}
@ -1239,6 +1247,11 @@ private:
const auto & tuple_columns = tuple_const->getColumns();
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(*type).getElements();
size_t tuple_size = tuple_columns.size();
if (0 == tuple_size && is_first)
for (auto & hash : vec_to)
hash = static_cast<ToType>(filler);
for (size_t i = 0; i < tuple_size; ++i)
{
auto tmp = ColumnConst::create(tuple_columns[i], column->size());
@ -1300,10 +1313,7 @@ public:
constexpr size_t first_data_argument = Keyed;
if (arguments.size() <= first_data_argument)
{
/// Return a fixed random-looking magic number when input is empty
vec_to.assign(input_rows_count, static_cast<ToType>(0xe28dbde7fe22e41c));
}
vec_to.assign(input_rows_count, static_cast<ToType>(filler));
KeyColumnsType key_cols{};
if constexpr (Keyed)

View File

@ -114,6 +114,34 @@ namespace
else if (query.grantees)
user.grantees = *query.grantees;
}
time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context)
{
if (context)
valid_until = evaluateConstantExpressionAsLiteral(valid_until, context);
const String valid_until_str = checkAndGetLiteralArgument<String>(valid_until, "valid_until");
if (valid_until_str == "infinity")
return 0;
time_t time = 0;
ReadBufferFromString in(valid_until_str);
if (context)
{
const auto & time_zone = DateLUT::instance("");
const auto & utc_time_zone = DateLUT::instance("UTC");
parseDateTimeBestEffort(time, in, time_zone, utc_time_zone);
}
else
{
readDateTimeText(time, in);
}
return time;
}
}
BlockIO InterpreterCreateUserQuery::execute()
@ -134,23 +162,7 @@ BlockIO InterpreterCreateUserQuery::execute()
std::optional<time_t> valid_until;
if (query.valid_until)
{
const ASTPtr valid_until_literal = evaluateConstantExpressionAsLiteral(query.valid_until, getContext());
const String valid_until_str = checkAndGetLiteralArgument<String>(valid_until_literal, "valid_until");
time_t time = 0;
if (valid_until_str != "infinity")
{
const auto & time_zone = DateLUT::instance("");
const auto & utc_time_zone = DateLUT::instance("UTC");
ReadBufferFromString in(valid_until_str);
parseDateTimeBestEffort(time, in, time_zone, utc_time_zone);
}
valid_until = time;
}
valid_until = getValidUntilFromAST(query.valid_until, getContext());
std::optional<RolesOrUsersSet> default_roles_from_query;
if (query.default_roles)
@ -259,7 +271,11 @@ void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreat
if (query.auth_data)
auth_data = AuthenticationData::fromAST(*query.auth_data, {}, !query.attach);
updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, {}, allow_no_password, allow_plaintext_password, true);
std::optional<time_t> valid_until;
if (query.valid_until)
valid_until = getValidUntilFromAST(query.valid_until, {});
updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, valid_until, allow_no_password, allow_plaintext_password, true);
}
void registerInterpreterCreateUserQuery(InterpreterFactory & factory)

View File

@ -95,7 +95,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const
if (server_protocol_revision >= DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS)
{
writeVarUInt(static_cast<UInt64>(collaborate_with_initiator), out);
writeVarUInt(count_participating_replicas, out);
writeVarUInt(obsolete_count_participating_replicas, out);
writeVarUInt(number_of_current_replica, out);
}
}
@ -185,7 +185,7 @@ void ClientInfo::read(ReadBuffer & in, UInt64 client_protocol_revision)
UInt64 value;
readVarUInt(value, in);
collaborate_with_initiator = static_cast<bool>(value);
readVarUInt(count_participating_replicas, in);
readVarUInt(obsolete_count_participating_replicas, in);
readVarUInt(number_of_current_replica, in);
}
}

View File

@ -127,7 +127,7 @@ public:
/// For parallel processing on replicas
bool collaborate_with_initiator{false};
UInt64 count_participating_replicas{0};
UInt64 obsolete_count_participating_replicas{0};
UInt64 number_of_current_replica{0};
enum class BackgroundOperationType : uint8_t

View File

@ -5004,13 +5004,6 @@ void Context::setConnectionClientVersion(UInt64 client_version_major, UInt64 cli
client_info.connection_tcp_protocol_version = client_tcp_protocol_version;
}
void Context::setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica)
{
client_info.collaborate_with_initiator = collaborate_with_initiator;
client_info.count_participating_replicas = all_replicas_count;
client_info.number_of_current_replica = number_of_current_replica;
}
void Context::increaseDistributedDepth()
{
++client_info.distributed_depth;

View File

@ -700,7 +700,6 @@ public:
void setInitialQueryStartTime(std::chrono::time_point<std::chrono::system_clock> initial_query_start_time);
void setQuotaClientKey(const String & quota_key);
void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version);
void setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica);
void increaseDistributedDepth();
const OpenTelemetry::TracingContext & getClientTraceContext() const { return client_info.client_trace_context; }
OpenTelemetry::TracingContext & getClientTraceContext() { return client_info.client_trace_context; }

View File

@ -8,6 +8,7 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;

View File

@ -385,7 +385,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons
if (database->getEngineName() != "Ordinary")
return;
Strings permanently_detached_tables = database->getNamesOfPermanentlyDetachedTables();
const Strings permanently_detached_tables = database->getNamesOfPermanentlyDetachedTables();
if (!permanently_detached_tables.empty())
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot automatically convert database {} from Ordinary to Atomic, "

View File

@ -265,6 +265,16 @@ ASTPtr ASTCreateQuery::clone() const
return res;
}
String ASTCreateQuery::getID(char delim) const
{
String res = attach ? "AttachQuery" : "CreateQuery";
String database = getDatabase();
if (!database.empty())
res += (delim + getDatabase());
res += (delim + getTable());
return res;
}
void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;

View File

@ -136,7 +136,7 @@ public:
bool create_or_replace{false};
/** Get the text that identifies this element. */
String getID(char delim) const override { return (attach ? "AttachQuery" : "CreateQuery") + (delim + getDatabase()) + delim + getTable(); }
String getID(char delim) const override;
ASTPtr clone() const override;

View File

@ -412,8 +412,8 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
const auto & shard = cluster->getShardsInfo().at(0);
size_t all_replicas_count = current_settings.max_parallel_replicas;
if (all_replicas_count > shard.getAllNodeCount())
size_t max_replicas_to_use = current_settings.max_parallel_replicas;
if (max_replicas_to_use > shard.getAllNodeCount())
{
LOG_INFO(
getLogger("ReadFromParallelRemoteReplicasStep"),
@ -421,14 +421,14 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder
"Will use the latter number to execute the query.",
current_settings.max_parallel_replicas,
shard.getAllNodeCount());
all_replicas_count = shard.getAllNodeCount();
max_replicas_to_use = shard.getAllNodeCount();
}
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool> shuffled_pool;
if (all_replicas_count < shard.getAllNodeCount())
if (max_replicas_to_use < shard.getAllNodeCount())
{
shuffled_pool = shard.pool->getShuffledPools(current_settings);
shuffled_pool.resize(all_replicas_count);
shuffled_pool.resize(max_replicas_to_use);
}
else
{
@ -438,11 +438,10 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder
shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func);
}
for (size_t i=0; i < all_replicas_count; ++i)
for (size_t i=0; i < max_replicas_to_use; ++i)
{
IConnections::ReplicaInfo replica_info
{
.all_replicas_count = all_replicas_count,
/// we should use this number specifically because efficiency of data distribution by consistent hash depends on it.
.number_of_current_replica = i,
};

View File

@ -36,18 +36,33 @@ inline void iotaWithStepOptimized(T * begin, size_t count, T first_value, T step
iotaWithStep(begin, count, first_value, step);
}
/// The range is defined as [start, end)
UInt64 itemCountInRange(UInt64 start, UInt64 end, UInt64 step)
{
const auto range_count = end - start;
if (step == 1)
return range_count;
return (range_count - 1) / step + 1;
}
class NumbersSource : public ISource
{
public:
NumbersSource(UInt64 block_size_, UInt64 offset_, std::optional<UInt64> limit_, UInt64 chunk_step_, const std::string & column_name, UInt64 step_)
NumbersSource(
UInt64 block_size_,
UInt64 offset_,
std::optional<UInt64> end_,
const std::string & column_name,
UInt64 step_in_chunk_,
UInt64 step_between_chunks_)
: ISource(createHeader(column_name))
, block_size(block_size_)
, next(offset_)
, chunk_step(chunk_step_)
, step(step_)
, end(end_)
, step_in_chunk(step_in_chunk_)
, step_between_chunks(step_between_chunks_)
{
if (limit_.has_value())
end = limit_.value() + offset_;
}
String getName() const override { return "Numbers"; }
@ -64,7 +79,10 @@ protected:
{
if (end.value() <= next)
return {};
real_block_size = std::min(block_size, end.value() - next);
auto max_items_to_generate = itemCountInRange(next, *end, step_in_chunk);
real_block_size = std::min(block_size, max_items_to_generate);
}
auto column = ColumnUInt64::create(real_block_size);
ColumnUInt64::Container & vec = column->getData();
@ -74,21 +92,20 @@ protected:
UInt64 * current_end = &vec[real_block_size];
iotaWithStepOptimized(pos, static_cast<size_t>(current_end - pos), curr, step);
iotaWithStepOptimized(pos, static_cast<size_t>(current_end - pos), curr, step_in_chunk);
next += chunk_step;
next += step_between_chunks;
progress(column->size(), column->byteSize());
return {Columns{std::move(column)}, real_block_size};
}
private:
UInt64 block_size;
UInt64 next;
UInt64 chunk_step;
std::optional<UInt64> end; /// not included
UInt64 step;
UInt64 step_in_chunk;
UInt64 step_between_chunks;
};
struct RangeWithStep
@ -549,20 +566,39 @@ Pipe ReadFromSystemNumbersStep::makePipe()
return pipe;
}
const auto end = std::invoke(
[&]() -> std::optional<UInt64>
{
if (numbers_storage.limit.has_value())
return *(numbers_storage.limit) + numbers_storage.offset;
return {};
});
/// Fall back to NumbersSource
/// Range in a single block
const auto block_range = max_block_size * numbers_storage.step;
/// Step between chunks in a single source.
/// It is bigger than block_range in case of multiple threads, because we have to account for other sources as well.
const auto step_between_chunks = num_streams * block_range;
for (size_t i = 0; i < num_streams; ++i)
{
const auto source_offset = i * block_range;
if (numbers_storage.limit.has_value() && *numbers_storage.limit < source_offset)
break;
const auto source_start = numbers_storage.offset + source_offset;
auto source = std::make_shared<NumbersSource>(
max_block_size,
numbers_storage.offset + i * max_block_size * numbers_storage.step,
numbers_storage.limit,
num_streams * max_block_size * numbers_storage.step,
source_start,
end,
numbers_storage.column_name,
numbers_storage.step);
numbers_storage.step,
step_between_chunks);
if (numbers_storage.limit && i == 0)
{
auto rows_appr = (*numbers_storage.limit - 1) / numbers_storage.step + 1;
auto rows_appr = itemCountInRange(numbers_storage.offset, *numbers_storage.limit, numbers_storage.step);
if (limit > 0 && limit < rows_appr)
rows_appr = query_info_limit;
source->addTotalRowsApprox(rows_appr);
@ -571,19 +607,6 @@ Pipe ReadFromSystemNumbersStep::makePipe()
pipe.addSource(std::move(source));
}
if (numbers_storage.limit)
{
size_t i = 0;
auto storage_limit = (*numbers_storage.limit - 1) / numbers_storage.step + 1;
/// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly.
pipe.addSimpleTransform(
[&](const Block & header)
{
++i;
return std::make_shared<LimitTransform>(header, storage_limit * i / num_streams - storage_limit * (i - 1) / num_streams, 0);
});
}
return pipe;
}

View File

@ -622,8 +622,9 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response
{
LOG_INFO(
log,
"Total time to process a request took too long ({}ms).\nRequest info: {}",
elapsed,
"Total time to process a request in session {} took too long ({}ms).\nRequest info: {}",
session_id,
elapsed_ms,
request->toString(/*short_format=*/true));
}

View File

@ -30,17 +30,10 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe
response.setContentType("text/plain; version=0.0.4; charset=UTF-8");
WriteBufferFromHTTPServerResponse wb(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout, write_event);
try
{
metrics_writer->write(wb);
wb.finalize();
}
catch (...)
{
wb.finalize();
}
}
catch (...)
{
tryLogCurrentException("PrometheusRequestHandler");
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <Core/Types_fwd.h>
#include <base/types.h>
#include <map>
@ -9,9 +10,10 @@ namespace DB
{
class IStorage;
struct SnapshotDetachedTable;
using ConstStoragePtr = std::shared_ptr<const IStorage>;
using StoragePtr = std::shared_ptr<IStorage>;
using Tables = std::map<String, StoragePtr>;
using SnapshotDetachedTables = std::map<String, SnapshotDetachedTable>;
}

View File

@ -21,7 +21,6 @@
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTTablesInSelectQuery.h>

View File

@ -0,0 +1,241 @@
#include "StorageSystemDetachedTables.h"
#include <Access/ContextAccess.h>
#include <Core/NamesAndTypes.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/SourceStepWithFilter.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/ProjectionsDescription.h>
#include <Storages/StorageInMemoryMetadata.h>
#include <Storages/System/StorageSystemTables.h>
#include <Storages/System/getQueriedColumnsMaskAndHeader.h>
#include <Storages/VirtualColumnUtils.h>
#include <boost/range/adaptor/map.hpp>
namespace DB
{
namespace
{
class DetachedTablesBlockSource : public ISource
{
public:
DetachedTablesBlockSource(
std::vector<UInt8> columns_mask_,
Block header_,
UInt64 max_block_size_,
ColumnPtr databases_,
ColumnPtr detached_tables_,
ContextPtr context_)
: ISource(std::move(header_))
, columns_mask(std::move(columns_mask_))
, max_block_size(max_block_size_)
, databases(std::move(databases_))
, context(Context::createCopy(context_))
{
size_t size = detached_tables_->size();
detached_tables.reserve(size);
for (size_t idx = 0; idx < size; ++idx)
{
detached_tables.insert(detached_tables_->getDataAt(idx).toString());
}
}
String getName() const override { return "DetachedTables"; }
protected:
Chunk generate() override
{
if (done)
return {};
MutableColumns result_columns = getPort().getHeader().cloneEmptyColumns();
const auto access = context->getAccess();
const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
size_t database_idx = 0;
size_t rows_count = 0;
for (; database_idx < databases->size() && rows_count < max_block_size; ++database_idx)
{
database_name = databases->getDataAt(database_idx).toString();
database = DatabaseCatalog::instance().tryGetDatabase(database_name);
if (!database)
continue;
const bool need_to_check_access_for_tables
= need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name);
if (!detached_tables_it || !detached_tables_it->isValid())
detached_tables_it = database->getDetachedTablesIterator(context, {}, false);
for (; rows_count < max_block_size && detached_tables_it->isValid(); detached_tables_it->next())
{
const auto detached_table_name = detached_tables_it->table();
if (!detached_tables.contains(detached_table_name))
continue;
if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, detached_table_name))
continue;
fillResultColumnsByDetachedTableIterator(result_columns);
++rows_count;
}
}
if (databases->size() == database_idx && (!detached_tables_it || !detached_tables_it->isValid()))
{
done = true;
}
const UInt64 num_rows = result_columns.at(0)->size();
return Chunk(std::move(result_columns), num_rows);
}
private:
const std::vector<UInt8> columns_mask;
const UInt64 max_block_size;
const ColumnPtr databases;
NameSet detached_tables;
DatabaseDetachedTablesSnapshotIteratorPtr detached_tables_it;
ContextPtr context;
bool done = false;
DatabasePtr database;
std::string database_name;
void fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const
{
size_t src_index = 0;
size_t res_index = 0;
if (columns_mask[src_index++])
result_columns[res_index++]->insert(detached_tables_it->database());
if (columns_mask[src_index++])
result_columns[res_index++]->insert(detached_tables_it->table());
if (columns_mask[src_index++])
result_columns[res_index++]->insert(detached_tables_it->uuid());
if (columns_mask[src_index++])
result_columns[res_index++]->insert(detached_tables_it->metadataPath());
if (columns_mask[src_index++])
result_columns[res_index++]->insert(detached_tables_it->isPermanently());
}
};
}
class ReadFromSystemDetachedTables : public SourceStepWithFilter
{
public:
std::string getName() const override { return "ReadFromSystemDetachedTables"; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
ReadFromSystemDetachedTables(
const Names & column_names_,
const SelectQueryInfo & query_info_,
const StorageSnapshotPtr & storage_snapshot_,
const ContextPtr & context_,
Block sample_block,
std::vector<UInt8> columns_mask_,
size_t max_block_size_);
void applyFilters(ActionDAGNodes added_filter_nodes) override;
private:
std::vector<UInt8> columns_mask;
size_t max_block_size;
ColumnPtr filtered_databases_column;
ColumnPtr filtered_tables_column;
};
StorageSystemDetachedTables::StorageSystemDetachedTables(const StorageID & table_id_) : IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
auto description = ColumnsDescription{
ColumnDescription{"database", std::make_shared<DataTypeString>(), "The name of the database the table is in."},
ColumnDescription{"table", std::make_shared<DataTypeString>(), "Table name."},
ColumnDescription{"uuid", std::make_shared<DataTypeUUID>(), "Table uuid (Atomic database)."},
ColumnDescription{"metadata_path", std::make_shared<DataTypeString>(), "Path to the table metadata in the file system."},
ColumnDescription{"is_permanently", std::make_shared<DataTypeUInt8>(), "Table was detached permanently."},
};
storage_metadata.setColumns(std::move(description));
setInMemoryMetadata(storage_metadata);
}
void StorageSystemDetachedTables::read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum /*processed_stage*/,
const size_t max_block_size,
size_t /*num_streams*/)
{
storage_snapshot->check(column_names);
auto sample_block = storage_snapshot->metadata->getSampleBlock();
auto [columns_mask, res_block] = getQueriedColumnsMaskAndHeader(sample_block, column_names);
auto reading = std::make_unique<ReadFromSystemDetachedTables>(
column_names, query_info, storage_snapshot, context, std::move(res_block), std::move(columns_mask), max_block_size);
query_plan.addStep(std::move(reading));
}
ReadFromSystemDetachedTables::ReadFromSystemDetachedTables(
const Names & column_names_,
const SelectQueryInfo & query_info_,
const StorageSnapshotPtr & storage_snapshot_,
const ContextPtr & context_,
Block sample_block,
std::vector<UInt8> columns_mask_,
size_t max_block_size_)
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_)
, columns_mask(std::move(columns_mask_))
, max_block_size(max_block_size_)
{
}
void ReadFromSystemDetachedTables::applyFilters(ActionDAGNodes added_filter_nodes)
{
SourceStepWithFilter::applyFilters(std::move(added_filter_nodes));
const ActionsDAG::Node * predicate = nullptr;
if (filter_actions_dag)
predicate = filter_actions_dag->getOutputs().at(0);
filtered_databases_column = detail::getFilteredDatabases(predicate, context);
filtered_tables_column = detail::getFilteredTables(predicate, filtered_databases_column, context, true);
}
void ReadFromSystemDetachedTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
auto pipe = Pipe(std::make_shared<DetachedTablesBlockSource>(
std::move(columns_mask),
getOutputStream().header,
max_block_size,
std::move(filtered_databases_column),
std::move(filtered_tables_column),
context));
pipeline.init(std::move(pipe));
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <Storages/IStorage.h>
namespace DB
{
class Context;
/** Implements the system table `detached_tables`, which allows you to get information about detached tables.
*/
class StorageSystemDetachedTables final : public IStorage
{
public:
explicit StorageSystemDetachedTables(const StorageID & table_id_);
std::string getName() const override { return "SystemDetachedTables"; }
void read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & /*query_info*/,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams) override;
bool isSystemStorage() const override { return true; }
};
}

View File

@ -1,29 +1,30 @@
#include <Storages/System/StorageSystemTables.h>
#include <Access/ContextAccess.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeNullable.h>
#include <Storages/System/StorageSystemTables.h>
#include <Storages/System/getQueriedColumnsMaskAndHeader.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/VirtualColumnUtils.h>
#include <Access/ContextAccess.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypesNumber.h>
#include <Disks/IStoragePolicy.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/formatWithPossiblyHidingSecrets.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Common/typeid_cast.h>
#include <Common/StringUtils.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <Disks/IStoragePolicy.h>
#include <Processors/ISource.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/SourceStepWithFilter.h>
#include <QueryPipeline/Pipe.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <DataTypes/DataTypeUUID.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/System/getQueriedColumnsMaskAndHeader.h>
#include <Storages/VirtualColumnUtils.h>
#include <Common/StringUtils.h>
#include <Common/typeid_cast.h>
#include <boost/range/adaptor/map.hpp>
@ -31,6 +32,105 @@
namespace DB
{
namespace
{
/// Avoid heavy operation on tables if we only queried columns that we can get without table object.
/// Otherwise it will require table initialization for Lazy database.
bool needTable(const DatabasePtr & database, const Block & header)
{
if (database->getEngineName() != "Lazy")
return true;
static const std::set<std::string> columns_without_table = {"database", "name", "uuid", "metadata_modification_time"};
for (const auto & column : header.getColumnsWithTypeAndName())
{
if (columns_without_table.find(column.name) == columns_without_table.end())
return true;
}
return false;
}
}
namespace detail
{
ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context)
{
MutableColumnPtr column = ColumnString::create();
const auto databases = DatabaseCatalog::instance().getDatabases();
for (const auto & database_name : databases | boost::adaptors::map_keys)
{
if (database_name == DatabaseCatalog::TEMPORARY_DATABASE)
continue; /// We don't want to show the internal database for temporary tables in system.tables
column->insert(database_name);
}
Block block{ColumnWithTypeAndName(std::move(column), std::make_shared<DataTypeString>(), "database")};
VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context);
return block.getByPosition(0).column;
}
ColumnPtr getFilteredTables(
const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context, const bool is_detached)
{
Block sample{
ColumnWithTypeAndName(nullptr, std::make_shared<DataTypeString>(), "name"),
ColumnWithTypeAndName(nullptr, std::make_shared<DataTypeString>(), "engine")};
MutableColumnPtr database_column = ColumnString::create();
MutableColumnPtr engine_column;
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample);
if (dag)
{
bool filter_by_engine = false;
for (const auto * input : dag->getInputs())
if (input->result_name == "engine")
filter_by_engine = true;
if (filter_by_engine)
engine_column = ColumnString::create();
}
for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx)
{
const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString();
DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name);
if (!database)
continue;
if (is_detached)
{
auto table_it = database->getDetachedTablesIterator(context, {}, false);
for (; table_it->isValid(); table_it->next())
{
database_column->insert(table_it->table());
}
}
else
{
for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next())
{
database_column->insert(table_it->name());
if (engine_column)
engine_column->insert(table_it->table()->getName());
}
}
}
Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared<DataTypeString>(), "name")};
if (engine_column)
block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared<DataTypeString>(), "engine"));
if (dag)
VirtualColumnUtils::filterBlockWithDAG(dag, block, context);
return block.getByPosition(0).column;
}
}
StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
: IStorage(table_id_)
@ -105,92 +205,6 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
setInMemoryMetadata(storage_metadata);
}
namespace
{
ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context)
{
MutableColumnPtr column = ColumnString::create();
const auto databases = DatabaseCatalog::instance().getDatabases();
for (const auto & database_name : databases | boost::adaptors::map_keys)
{
if (database_name == DatabaseCatalog::TEMPORARY_DATABASE)
continue; /// We don't want to show the internal database for temporary tables in system.tables
column->insert(database_name);
}
Block block { ColumnWithTypeAndName(std::move(column), std::make_shared<DataTypeString>(), "database") };
VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context);
return block.getByPosition(0).column;
}
ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context)
{
Block sample {
ColumnWithTypeAndName(nullptr, std::make_shared<DataTypeString>(), "name"),
ColumnWithTypeAndName(nullptr, std::make_shared<DataTypeString>(), "engine")
};
MutableColumnPtr database_column = ColumnString::create();
MutableColumnPtr engine_column;
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample);
if (dag)
{
bool filter_by_engine = false;
for (const auto * input : dag->getInputs())
if (input->result_name == "engine")
filter_by_engine = true;
if (filter_by_engine)
engine_column = ColumnString::create();
}
for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx)
{
const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString();
DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name);
if (!database)
continue;
for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next())
{
database_column->insert(table_it->name());
if (engine_column)
engine_column->insert(table_it->table()->getName());
}
}
Block block {ColumnWithTypeAndName(std::move(database_column), std::make_shared<DataTypeString>(), "name")};
if (engine_column)
block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared<DataTypeString>(), "engine"));
if (dag)
VirtualColumnUtils::filterBlockWithDAG(dag, block, context);
return block.getByPosition(0).column;
}
/// Avoid heavy operation on tables if we only queried columns that we can get without table object.
/// Otherwise it will require table initialization for Lazy database.
bool needTable(const DatabasePtr & database, const Block & header)
{
if (database->getEngineName() != "Lazy")
return true;
static const std::set<std::string> columns_without_table = { "database", "name", "uuid", "metadata_modification_time" };
for (const auto & column : header.getColumnsWithTypeAndName())
{
if (columns_without_table.find(column.name) == columns_without_table.end())
return true;
}
return false;
}
class TablesBlockSource : public ISource
{
public:
@ -690,8 +704,6 @@ private:
std::string database_name;
};
}
class ReadFromSystemTables : public SourceStepWithFilter
{
public:
@ -756,8 +768,8 @@ void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes)
if (filter_actions_dag)
predicate = filter_actions_dag->getOutputs().at(0);
filtered_databases_column = getFilteredDatabases(predicate, context);
filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context);
filtered_databases_column = detail::getFilteredDatabases(predicate, context);
filtered_tables_column = detail::getFilteredTables(predicate, filtered_databases_column, context, false);
}
void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)

View File

@ -8,6 +8,15 @@ namespace DB
class Context;
namespace detail
{
ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context);
ColumnPtr
getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context, bool is_detached);
}
/** Implements the system table `tables`, which allows you to get information about all tables.
*/

View File

@ -18,6 +18,7 @@
#include <Storages/System/StorageSystemDataSkippingIndices.h>
#include <Storages/System/StorageSystemDataTypeFamilies.h>
#include <Storages/System/StorageSystemDetachedParts.h>
#include <Storages/System/StorageSystemDetachedTables.h>
#include <Storages/System/StorageSystemDictionaries.h>
#include <Storages/System/StorageSystemEvents.h>
#include <Storages/System/StorageSystemFormats.h>
@ -129,6 +130,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b
attachNoDescription<StorageSystemZeros>(context, system_database, "zeros_mt", "Multithreaded version of system.zeros.", true);
attach<StorageSystemDatabases>(context, system_database, "databases", "Lists all databases of the current server.");
attachNoDescription<StorageSystemTables>(context, system_database, "tables", "Lists all tables of the current server.");
attachNoDescription<StorageSystemDetachedTables>(context, system_database, "detached_tables", "Lists all detached tables of the current server.");
attachNoDescription<StorageSystemColumns>(context, system_database, "columns", "Lists all columns from all tables of the current server.");
attach<StorageSystemFunctions>(context, system_database, "functions", "Contains a list of all available ordinary and aggregate functions with their descriptions.");
attach<StorageSystemEvents>(context, system_database, "events", "Contains profiling events and their current value.");

View File

@ -99,7 +99,8 @@ def set_capacity(
continue
raise ValueError("Queue status is not in ['in_progress', 'queued']")
scale_down, scale_up = get_scales(runner_type)
# scale_down, scale_up = get_scales(runner_type)
_, scale_up = get_scales(runner_type)
# With lyfecycle hooks some instances are actually free because some of
# them are in 'Terminating:Wait' state
effective_capacity = max(
@ -110,7 +111,7 @@ def set_capacity(
# How much nodes are free (positive) or need to be added (negative)
capacity_reserve = effective_capacity - running - queued
stop = False
if capacity_reserve < 0:
if capacity_reserve <= 0:
# This part is about scaling up
capacity_deficit = -capacity_reserve
# It looks that we are still OK, since no queued jobs exist
@ -158,41 +159,43 @@ def set_capacity(
)
return
# Now we will calculate if we need to scale down
stop = stop or asg["DesiredCapacity"] == asg["MinSize"]
new_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down)
new_capacity = max(new_capacity, asg["MinSize"])
new_capacity = min(new_capacity, asg["MaxSize"])
stop = stop or asg["DesiredCapacity"] == new_capacity
if stop:
logging.info(
"Do not decrease ASG %s capacity, current capacity=%s, effective "
"capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s",
asg["AutoScalingGroupName"],
asg["DesiredCapacity"],
effective_capacity,
asg["MinSize"],
running,
queued,
)
return
logging.info(
"The ASG %s capacity will be decreased to %s, current capacity=%s, effective "
"capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s",
asg["AutoScalingGroupName"],
new_capacity,
asg["DesiredCapacity"],
effective_capacity,
asg["MinSize"],
running,
queued,
)
if not dry_run:
client.set_desired_capacity(
AutoScalingGroupName=asg["AutoScalingGroupName"],
DesiredCapacity=new_capacity,
)
# FIXME: try decreasing capacity from runners that finished their jobs and have no job assigned
# IMPORTANT: Runner init script must be of version that supports ASG decrease
# # Now we will calculate if we need to scale down
# stop = stop or asg["DesiredCapacity"] == asg["MinSize"]
# new_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down)
# new_capacity = max(new_capacity, asg["MinSize"])
# new_capacity = min(new_capacity, asg["MaxSize"])
# stop = stop or asg["DesiredCapacity"] == new_capacity
# if stop:
# logging.info(
# "Do not decrease ASG %s capacity, current capacity=%s, effective "
# "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s",
# asg["AutoScalingGroupName"],
# asg["DesiredCapacity"],
# effective_capacity,
# asg["MinSize"],
# running,
# queued,
# )
# return
#
# logging.info(
# "The ASG %s capacity will be decreased to %s, current capacity=%s, effective "
# "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s",
# asg["AutoScalingGroupName"],
# new_capacity,
# asg["DesiredCapacity"],
# effective_capacity,
# asg["MinSize"],
# running,
# queued,
# )
# if not dry_run:
# client.set_desired_capacity(
# AutoScalingGroupName=asg["AutoScalingGroupName"],
# DesiredCapacity=new_capacity,
# )
def main(dry_run: bool = True) -> None:

View File

@ -97,19 +97,34 @@ class TestSetCapacity(unittest.TestCase):
),
TestCase("lower-min", 10, 5, 20, [Queue("queued", 5, "lower-min")], 10),
# Decrease capacity
TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 5),
# FIXME: Tests changed for lambda that can only scale up
# TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 5),
TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], -1),
# TestCase(
# "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], 5
# ),
TestCase(
"style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], 5
"style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], -1
),
TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 17),
TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 3),
# TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 17),
TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], -1),
# TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 3),
TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], -1),
# TestCase(
# "style-checker",
# 1,
# 13,
# 20,
# [Queue("in_progress", 5, "style-checker")],
# 5,
# ),
TestCase(
"style-checker",
1,
13,
20,
[Queue("in_progress", 5, "style-checker")],
5,
-1,
),
)
for t in test_cases:

View File

@ -415,7 +415,8 @@ class CI:
JobNames.INTEGRATION_TEST_FLAKY: CommonJobConfigs.INTEGRATION_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_ASAN],
pr_only=True,
reference_job_name=JobNames.INTEGRATION_TEST_TSAN,
# TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf
# reference_job_name=JobNames.INTEGRATION_TEST_TSAN,
),
JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties(
required_builds=[BuildNames.PACKAGE_RELEASE],
@ -460,7 +461,8 @@ class CI:
required_builds=[BuildNames.PACKAGE_ASAN],
pr_only=True,
timeout=3600,
reference_job_name=JobNames.STATELESS_TEST_RELEASE,
# TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf
# reference_job_name=JobNames.STATELESS_TEST_RELEASE,
),
JobNames.JEPSEN_KEEPER: JobConfig(
required_builds=[BuildNames.BINARY_RELEASE],

View File

@ -17,9 +17,19 @@ from download_release_packages import download_last_release
from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH
from get_robot_token import get_parameter_from_ssm
from pr_info import PRInfo
from report import ERROR, SUCCESS, JobReport, StatusType, TestResults, read_test_results
from report import (
ERROR,
SUCCESS,
JobReport,
StatusType,
TestResults,
read_test_results,
FAILURE,
)
from stopwatch import Stopwatch
from tee_popen import TeePopen
from ci_config import CI
from ci_utils import Utils
NO_CHANGES_MSG = "Nothing to run"
@ -351,7 +361,23 @@ def main():
additional_files=additional_logs,
).dump(to_file=args.report_to_file if args.report_to_file else None)
should_block_ci = False
if state != SUCCESS:
should_block_ci = True
if state == FAILURE and CI.is_required(check_name):
failed_cnt = Utils.get_failed_tests_number(description)
print(
f"Job status is [{state}] with [{failed_cnt}] failed test cases. status description [{description}]"
)
if (
failed_cnt
and failed_cnt <= CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI
):
print(f"Won't block the CI workflow")
should_block_ci = False
if should_block_ci:
sys.exit(1)

View File

@ -23,10 +23,13 @@ from report import (
TestResult,
TestResults,
read_test_results,
FAILURE,
)
from stopwatch import Stopwatch
import integration_tests_runner as runner
from ci_config import CI
from ci_utils import Utils
def get_json_params_dict(
@ -233,7 +236,23 @@ def main():
additional_files=additional_logs,
).dump(to_file=args.report_to_file if args.report_to_file else None)
should_block_ci = False
if state != SUCCESS:
should_block_ci = True
if state == FAILURE and CI.is_required(check_name):
failed_cnt = Utils.get_failed_tests_number(description)
print(
f"Job status is [{state}] with [{failed_cnt}] failed test cases. status description [{description}]"
)
if (
failed_cnt
and failed_cnt <= CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI
):
print(f"Won't block the CI workflow")
should_block_ci = False
if should_block_ci:
sys.exit(1)

View File

@ -3,48 +3,27 @@
import re
from typing import Tuple
# Individual trusted contirbutors who are not in any trusted organization.
# Individual trusted contributors who are not in any trusted organization.
# Can be changed in runtime: we will append users that we learned to be in
# a trusted org, to save GitHub API calls.
TRUSTED_CONTRIBUTORS = {
e.lower()
for e in [
"achimbab", # Kakao corp
"Algunenano", # Raúl Marín, ClickHouse, Inc
"amosbird",
"azat", # SEMRush
"bharatnc", # Many contributions.
"bobrik", # Seasoned contributor, CloudFlare
"cwurm", # ClickHouse, Inc
"den-crane", # Documentation contributor
"hagen1778", # Roman Khavronenko, seasoned contributor
"hczhcz",
"hexiaoting", # Seasoned contributor
"ildus", # adjust, ex-pgpro
"javisantana", # a Spanish ClickHouse enthusiast, ex-Carto
"kreuzerkrieg",
"nikvas0",
"nvartolomei", # Seasoned contributor, CloudFlare
"spongedu", # Seasoned contributor
"taiyang-li",
"ucasFL", # Amos Bird's friend
"vdimir", # ClickHouse, Inc
"YiuRULE",
"zlobober", # Developer of YT
"ilejn", # Arenadata, responsible for Kerberized Kafka
"thomoco", # ClickHouse, Inc
"BoloniniD", # Seasoned contributor, HSE
"tonickkozlov", # Cloudflare
"tylerhannan", # ClickHouse, Inc
"myrrc", # Mike Kot, DoubleCloud
"thevar1able", # ClickHouse, Inc
"aalexfvk",
"MikhailBurdukov",
"tsolodov", # ClickHouse, Inc
"kitaisreal",
"k-morozov", # Konstantin Morozov, Yandex Cloud
"justindeguzman", # ClickHouse, Inc
"jrdi", # ClickHouse contributor, TinyBird
"XuJia0210", # ClickHouse, Inc
]
}

View File

@ -27,7 +27,6 @@ from report import SUCCESS, FAILURE
from env_helper import GITHUB_UPSTREAM_REPOSITORY, GITHUB_REPOSITORY
from synchronizer_utils import SYNC_BRANCH_PREFIX
from ci_config import CI
from ci_utils import Utils
# The team name for accepted approvals
TEAM_NAME = getenv("GITHUB_TEAM_NAME", "core")
@ -249,74 +248,22 @@ def main():
repo = gh.get_repo(args.repo)
if args.set_ci_status:
# set Mergeable check status and exit
assert args.wf_status in (FAILURE, SUCCESS)
# set mergeable check status and exit
commit = get_commit(gh, args.pr_info.sha)
statuses = get_commit_filtered_statuses(commit)
max_failed_tests_per_job = 0
job_name_with_max_failures = None
total_failed_tests = 0
failed_to_get_info = False
has_failed_statuses = False
for status in statuses:
if not CI.is_required(status.context) or status.context in (
CI.StatusNames.SYNC,
CI.StatusNames.PR_CHECK,
):
# CI.StatusNames.SYNC or CI.StatusNames.PR_CHECK should not be checked
continue
print(f"Check status [{status.context}], [{status.state}]")
if status.state == FAILURE:
if CI.is_required(status.context) and status.state != SUCCESS:
print(f"WARNING: Failed status [{status.context}], [{status.state}]")
has_failed_statuses = True
failed_cnt = Utils.get_failed_tests_number(status.description)
if failed_cnt is None:
failed_to_get_info = True
print(
f"WARNING: failed to get number of failed tests from [{status.description}]"
)
else:
if failed_cnt > max_failed_tests_per_job:
job_name_with_max_failures = status.context
max_failed_tests_per_job = failed_cnt
total_failed_tests += failed_cnt
print(
f"Failed test cases in [{status.context}] is [{failed_cnt}], total failures [{total_failed_tests}]"
)
elif status.state != SUCCESS and status.context not in (
CI.StatusNames.SYNC,
CI.StatusNames.PR_CHECK,
):
# do not block CI on failures in (CI.StatusNames.SYNC, CI.StatusNames.PR_CHECK)
has_failed_statuses = True
print(
f"Unexpected status for [{status.context}]: [{status.state}] - block further testing"
)
failed_to_get_info = True
can_continue = True
if total_failed_tests > CI.MAX_TOTAL_FAILURES_BEFORE_BLOCKING_CI:
print(
f"Required check has [{total_failed_tests}] failed - block further testing"
)
can_continue = False
if max_failed_tests_per_job > CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI:
print(
f"Job [{job_name_with_max_failures}] has [{max_failed_tests_per_job}] failures - block further testing"
)
can_continue = False
if failed_to_get_info:
print("Unexpected commit status state - block further testing")
can_continue = False
if args.wf_status != SUCCESS and not has_failed_statuses:
# workflow failed but reason is unknown as no failed statuses present
can_continue = False
print(
"WARNING: Either the runner is faulty or the operating status is unknown. The first is self-healing, the second requires investigation."
)
if args.wf_status == SUCCESS or has_failed_statuses:
# do not set mergeable check status if args.wf_status == failure, apparently it has died runners and is to be restarted
# set Mergeable check if workflow is successful (green)
# or if we have GH statuses with failures (red)
# to avoid false-green on a died runner
state = trigger_mergeable_check(
commit,
statuses,
@ -333,10 +280,10 @@ def main():
print(
"Workflow failed but no failed statuses found (died runner?) - cannot set Mergeable Check status"
)
if not can_continue:
sys.exit(1)
if args.wf_status == SUCCESS and not has_failed_statuses:
sys.exit(0)
else:
sys.exit(1)
# An ugly and not nice fix to patch the wrong organization URL,
# see https://github.com/PyGithub/PyGithub/issues/2395#issuecomment-1378629710

View File

@ -1,5 +1,7 @@
#!/usr/bin/env bash
set -e
usage() {
echo "Usage: $0 ENVIRONMENT" >&2
echo "Valid values for ENVIRONMENT: staging, production" >&2
@ -55,7 +57,7 @@ EOF
body() {
local first_line
first_line=$(sed -n '/^# THE SCRIPT START$/{=;q}' "$SOURCE_SCRIPT")
first_line=$(sed -n '/^# THE SCRIPT START$/{=;q;}' "$SOURCE_SCRIPT")
if [ -z "$first_line" ]; then
echo "The pattern '# THE SCRIPT START' is not found in $SOURCE_SCRIPT" >&2
exit 1

View File

@ -50,7 +50,7 @@ set -uo pipefail
# set accordingly to a runner role #
####################################
echo "Running init script"
echo "Running init v1"
export DEBIAN_FRONTEND=noninteractive
export RUNNER_HOME=/home/ubuntu/actions-runner
@ -90,7 +90,6 @@ terminate_delayed() {
# IF `sleep` IS CHANGED, CHANGE ANOTHER VALUE IN `pgrep`
sleep=13.14159265358979323846
echo "Going to terminate the runner's instance in $sleep seconds"
INSTANCE_ID=$(ec2metadata --instance-id)
# We execute it with `at` to not have it as an orphan process, but launched independently
# GH Runners kill all remain processes
echo "sleep '$sleep'; aws ec2 terminate-instances --instance-ids $INSTANCE_ID" | at now || \
@ -111,11 +110,17 @@ declare -f terminate_delayed >> /tmp/actions-hooks/common.sh
terminate_and_exit() {
# Terminate instance and exit from the script instantly
echo "Going to terminate the runner's instance"
INSTANCE_ID=$(ec2metadata --instance-id)
aws ec2 terminate-instances --instance-ids "$INSTANCE_ID"
exit 0
}
terminate_decrease_and_exit() {
# Terminate instance and exit from the script instantly
echo "Going to terminate the runner's instance and decrease asg capacity"
aws autoscaling terminate-instance-in-auto-scaling-group --instance-id "$INSTANCE_ID" --should-decrement-desired-capacity
exit 0
}
declare -f terminate_and_exit >> /tmp/actions-hooks/common.sh
check_spot_instance_is_old() {
@ -324,7 +329,7 @@ while true; do
sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" \
|| continue
echo "Runner didn't launch or have assigned jobs after ${RUNNER_AGE} seconds, shutting down"
terminate_and_exit
terminate_decrease_and_exit
fi
fi
else

View File

@ -0,0 +1,21 @@
<clickhouse>
<!-- this update period also syncs MemoryTracking with RSS, disable this, by using period = 1 day -->
<asynchronous_metrics_update_period_s>86400</asynchronous_metrics_update_period_s>
<query_masking_rules remove="remove"/>
<query_thread_log remove="remove"/>
<query_log remove="remove" />
<query_views_log remove="remove" />
<metric_log remove="remove"/>
<error_log remove="remove"/>
<text_log remove="remove"/>
<trace_log remove="remove"/>
<asynchronous_metric_log remove="remove" />
<session_log remove="remove" />
<part_log remove="remove" />
<crash_log remove="remove" />
<opentelemetry_span_log remove="remove" />
<!-- just in case it will be enabled by default -->
<zookeeper_log remove="remove" />
<transactions_info_log remove="remove" />
</clickhouse>

View File

@ -0,0 +1,54 @@
#!/usr/bin/env python3
import logging
import time
import pytest
from helpers.cluster import ClickHouseCluster
from multiprocessing.dummy import Pool
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=[
"configs/async_metrics_no.xml",
],
mem_limit="4g",
env_variables={"MALLOC_CONF": "dirty_decay_ms:0"},
)
@pytest.fixture(scope="module", autouse=True)
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_multiple_queries():
if node.is_built_with_sanitizer():
return
p = Pool(15)
def run_query(node):
try:
node.query("SELECT * FROM system.numbers GROUP BY number")
except Exception as ex:
print("Exception", ex)
raise ex
tasks = []
for i in range(30):
tasks.append(p.apply_async(run_query, (node,)))
time.sleep(i * 0.1)
for task in tasks:
try:
task.get()
except Exception as ex:
print("Exception", ex)
# test that we didn't kill the server
node.query("SELECT 1")

View File

@ -0,0 +1,13 @@
<clickhouse>
<remote_servers>
<test_cluster>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>replica1</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</clickhouse>

View File

@ -0,0 +1,112 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"replica1",
with_zookeeper=True,
main_configs=["configs/remote_servers.xml"],
macros={"replica": "replica1"},
stay_alive=True,
)
@pytest.fixture(scope="module", autouse=True)
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
@pytest.mark.parametrize(
"db_name,db_engine,table_engine,table_engine_params",
[
pytest.param(
"test_db_atomic",
"Atomic",
"MergeTree",
"ORDER BY n",
id="Atomic db with MergeTree table",
),
pytest.param(
"test_db_lazy",
"Lazy(60)",
"Log",
"",
id="Lazy db with Log table",
),
pytest.param(
"test_db_repl",
"Replicated('/clickhouse/tables/test_table','shard1', 'replica1')",
"ReplicatedMergeTree",
"ORDER BY n",
id="Replicated db with ReplicatedMergeTree table",
),
],
)
def test_system_detached_tables(
start_cluster, db_name, db_engine, table_engine, table_engine_params
):
node.query(f"CREATE DATABASE IF NOT EXISTS {db_name} ENGINE={db_engine};")
node.query(
f"CREATE TABLE {db_name}.test_table (n Int64) ENGINE={table_engine} {table_engine_params};"
)
node.query(
f"CREATE TABLE {db_name}.test_table_perm (n Int64) ENGINE={table_engine} {table_engine_params};"
)
test_table_uuid = node.query(
"SELECT uuid FROM system.tables WHERE table='test_table'"
).rstrip("\n")
test_table_metadata_path = node.query(
"SELECT metadata_path FROM system.tables WHERE table='test_table'"
).rstrip("\n")
test_table_perm_uuid = node.query(
"SELECT uuid FROM system.tables WHERE table='test_table_perm'"
).rstrip("\n")
test_table_perm_metadata_path = node.query(
"SELECT metadata_path FROM system.tables WHERE table='test_table_perm'"
).rstrip("\n")
assert "" == node.query(
f"SELECT * FROM system.detached_tables WHERE database='{db_name}'"
)
node.query(
f"SET database_replicated_always_detach_permanently=1; DETACH TABLE {db_name}.test_table"
)
node.query(f"DETACH TABLE {db_name}.test_table_perm PERMANENTLY")
querry = f"SELECT database, table, is_permanently, uuid, metadata_path FROM system.detached_tables WHERE database='{db_name}' FORMAT Values"
result = node.query(querry)
if db_engine.startswith("Repl"):
expected_before_restart = f"('{db_name}','test_table',1,'{test_table_uuid}','{test_table_metadata_path}'),('{db_name}','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')"
else:
expected_before_restart = f"('{db_name}','test_table',0,'{test_table_uuid}','{test_table_metadata_path}'),('{db_name}','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')"
assert result == expected_before_restart
if db_engine.startswith("Lazy"):
return
node.restart_clickhouse()
if db_engine.startswith("Repl"):
expected_after_restart = expected_before_restart
else:
expected_after_restart = f"('{db_name}','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')"
result = node.query(querry)
assert result == expected_after_restart
node.restart_clickhouse()
result = node.query(querry)
assert result == expected_after_restart
node.query(f"DROP DATABASE {db_name}")

View File

@ -5,7 +5,7 @@ from time import sleep
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance("node")
node = cluster.add_instance("node", stay_alive=True)
@pytest.fixture(scope="module")
@ -84,3 +84,22 @@ def test_details(started_cluster):
node.query("SHOW CREATE USER user_details_time_only")
== f"CREATE USER user_details_time_only VALID UNTIL \\'{until_year}-01-01 22:03:40\\'\n"
)
def test_restart(started_cluster):
node.query("CREATE USER user_restart VALID UNTIL '06/11/2010 08:03:20 Z+3'")
assert (
node.query("SHOW CREATE USER user_restart")
== "CREATE USER user_restart VALID UNTIL \\'2010-11-06 05:03:20\\'\n"
)
node.restart_clickhouse()
assert (
node.query("SHOW CREATE USER user_restart")
== "CREATE USER user_restart VALID UNTIL \\'2010-11-06 05:03:20\\'\n"
)
error = "Authentication failed"
assert error in node.query_and_get_error("SELECT 1", user="user_restart")

View File

@ -8,7 +8,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
i=0 retries=5
# Connecting to wrong address and checking for race condition
# http_max_tries is limited to 2 because with the default 10 retries the execution time might go as high as around 3 minutes (because of exponential back-off).
# because of that we might see wrong 'tests hung' reports depending on how close to the end of tests run this particular test was executed.
# proper fix should be implemented in https://github.com/ClickHouse/ClickHouse/issues/66656
while [[ $i -lt $retries ]]; do
timeout 5s ${CLICKHOUSE_CLIENT} --max_threads 10 --query "SELECT * FROM url('http://128.0.0.{1..10}:${CLICKHOUSE_PORT_HTTP}/?query=SELECT+sleep(1)', TSV, 'x UInt8')" --format Null 2>/dev/null
timeout 5s ${CLICKHOUSE_CLIENT} --max_threads 10 --http_max_tries 2 --query "SELECT * FROM url('http://128.0.0.{1..10}:${CLICKHOUSE_PORT_HTTP}/?query=SELECT+sleep(1)', TSV, 'x UInt8')" --format Null 2>/dev/null
((++i))
done

View File

@ -29,7 +29,7 @@ from in_02231
group by key;
set optimize_trivial_insert_select = 1;
insert into in_02231 select * from numbers(10e6) settings max_memory_usage='310Mi', max_threads=1;
insert into in_02231 select * from numbers(10e6) settings max_memory_usage='400Mi', max_threads=1;
drop table buffer_02231;
drop table out_02231;

View File

@ -1,4 +1,4 @@
-- Tags: no-fasttest
-- Tags: no-fasttest, no-tsan, no-asan, no-msan, no-ubsan
-- This tests depends on internet access, but it does not matter, because it only has to check that there is no abort due to a bug in Apache Arrow library.
SET optimize_trivial_insert_select=1;
INSERT INTO TABLE FUNCTION url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet') SELECT * FROM url('https://clickhouse-public-datasets.s3.amazonaws.com/hits_compatible/athena_partitioned/hits_9.parquet'); -- { serverError CANNOT_WRITE_TO_OSTREAM, RECEIVED_ERROR_FROM_REMOTE_IO_SERVER, POCO_EXCEPTION }

View File

@ -5,6 +5,7 @@
501
50
17928
17928
0
10
13

View File

@ -5,6 +5,7 @@ SELECT count() FROM generate_series(7, 77, 10);
SELECT count() FROM generate_series(0, 1000, 2);
SELECT count() FROM generate_series(0, 999, 20);
SELECT sum(generate_series) FROM generate_series(4, 1008, 4) WHERE generate_series % 7 = 1;
SELECT sum(generate_series) FROM generate_series(4, 1008, 4) WHERE generate_series % 7 = 1 SETTINGS max_block_size = 71;
SELECT * FROM generate_series(5, 4);
SELECT * FROM generate_series(0, 0);

View File

@ -0,0 +1,11 @@
database atomic tests
test03172_system_detached_tables test_table 0
test03172_system_detached_tables test_table_perm 1
test03172_system_detached_tables test_table 0
test03172_system_detached_tables test_table_perm 1
test03172_system_detached_tables test_table 0
-----------------------
database lazy tests
before attach test03172_system_detached_tables_lazy test_table 0
before attach test03172_system_detached_tables_lazy test_table_perm 1
DROP TABLE

View File

@ -0,0 +1,53 @@
-- Tags: no-parallel
SELECT 'database atomic tests';
DROP DATABASE IF EXISTS test03172_system_detached_tables;
CREATE DATABASE IF NOT EXISTS test03172_system_detached_tables ENGINE=Atomic;
CREATE TABLE test03172_system_detached_tables.test_table (n Int64) ENGINE=MergeTree ORDER BY n;
SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables';
DETACH TABLE test03172_system_detached_tables.test_table;
SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables';
ATTACH TABLE test03172_system_detached_tables.test_table;
CREATE TABLE test03172_system_detached_tables.test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n;
SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables';
DETACH TABLE test03172_system_detached_tables.test_table_perm PERMANENTLY;
SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables';
DETACH TABLE test03172_system_detached_tables.test_table SYNC;
SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables';
SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables' AND table='test_table';
DROP DATABASE test03172_system_detached_tables SYNC;
SELECT '-----------------------';
SELECT 'database lazy tests';
DROP DATABASE IF EXISTS test03172_system_detached_tables_lazy;
CREATE DATABASE test03172_system_detached_tables_lazy Engine=Lazy(10);
CREATE TABLE test03172_system_detached_tables_lazy.test_table (number UInt64) engine=Log;
INSERT INTO test03172_system_detached_tables_lazy.test_table SELECT * FROM numbers(100);
DETACH TABLE test03172_system_detached_tables_lazy.test_table;
CREATE TABLE test03172_system_detached_tables_lazy.test_table_perm (number UInt64) engine=Log;
INSERT INTO test03172_system_detached_tables_lazy.test_table_perm SELECT * FROM numbers(100);
DETACH table test03172_system_detached_tables_lazy.test_table_perm PERMANENTLY;
SELECT 'before attach', database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy';
ATTACH TABLE test03172_system_detached_tables_lazy.test_table;
ATTACH TABLE test03172_system_detached_tables_lazy.test_table_perm;
SELECT 'after attach', database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy';
SELECT 'DROP TABLE';
DROP TABLE test03172_system_detached_tables_lazy.test_table SYNC;
DROP TABLE test03172_system_detached_tables_lazy.test_table_perm SYNC;
DROP DATABASE test03172_system_detached_tables_lazy SYNC;

View File

@ -1,4 +1,5 @@
-- Tags: no-random-settings, no-s3-storage
-- Tags: no-random-settings, no-object-storage
-- Tag no-object-storage: this test relies on the number of opened files in MergeTree that can differ in object storages
SET allow_experimental_dynamic_type = 1;
DROP TABLE IF EXISTS test_dynamic;

View File

@ -0,0 +1,6 @@
Ok
Ok
Ok
Ok
Ok
Ok

View File

@ -0,0 +1,13 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -t -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1 | grep -q "^2\." && echo "Ok" || echo "Fail"
${CLICKHOUSE_CLIENT} --time -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1 | grep -q "^2\." && echo "Ok" || echo "Fail"
${CLICKHOUSE_CLIENT} --memory-usage -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9]\+$" && echo "Ok" || echo "Fail"
${CLICKHOUSE_CLIENT} --memory-usage=none -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" # expected no output
${CLICKHOUSE_CLIENT} --memory-usage=default -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9]\+$" && echo "Ok" || echo "Fail"
${CLICKHOUSE_CLIENT} --memory-usage=readable -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9].*B$" && echo "Ok" || echo "Fail"
${CLICKHOUSE_CLIENT} --memory-usage=unknown -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "BAD_ARGUMENTS" && echo "Ok" || echo "Fail"

View File

@ -0,0 +1,14 @@
18679 31
0
10
20
30
40
50
60
70
80
90
100
110
4250

View File

@ -0,0 +1,38 @@
--- The following query was buggy before, so let's use it as a test case
WITH
(num > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < num) AND ((num % y) = 0)), range(toUInt64(sqrt(num)) + 1))) = 0) AS is_prime_slow
SELECT
num,
ds,
FROM
(
WITH
arraySum(arrayMap(y -> toUInt8(y), splitByString('', toString(num)))) AS digits_sum
SELECT
1 + (number * 2) AS num,
digits_sum AS ds
FROM numbers_mt(10000)
WHERE ds IN (
WITH
(number > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < number) AND ((number % y) = 0)), range(toUInt64(sqrt(number)) + 1))) = 0) AS is_prime_slow
SELECT number
FROM numbers(180 + 1)
WHERE is_prime_slow
)
)
WHERE is_prime_slow
ORDER BY num ASC
LIMIT 998, 1
SETTINGS max_block_size = 64, max_threads=16;
SELECT number
FROM numbers_mt(120)
WHERE (number % 10) = 0
ORDER BY number ASC
SETTINGS max_block_size = 31, max_threads = 11;
SELECT number
FROM numbers_mt(4242, 9)
WHERE (number % 10) = 0
ORDER BY number ASC
SETTINGS max_block_size = 31, max_threads = 11;

View File

@ -0,0 +1,6 @@
case 1
9900
9910
9920
case 2
9990

View File

@ -0,0 +1,11 @@
SELECT 'case 1';
SELECT number FROM numbers_mt(10000)
WHERE (number % 10) = 0
ORDER BY number ASC
LIMIT 990, 3;
SELECT 'case 2';
SELECT number FROM numbers_mt(10000)
WHERE (number % 10) = 0
ORDER BY number ASC
LIMIT 999, 20 SETTINGS max_block_size = 31;

View File

@ -0,0 +1,22 @@
16324913028386710556
16324913028386710556
5049034479224883533
7385293435322750976
12248912094175844631
5049034479224883533
5887129541803688833
5887129541803688833
13747979201178469747
5887129541803688833
15520217392480966957
16324913028386710556
16324913028386710556
5049034479224883533
7385293435322750976
12248912094175844631
5049034479224883533
5887129541803688833
5887129541803688833
13747979201178469747
5887129541803688833
15520217392480966957

View File

@ -0,0 +1,23 @@
SELECT sipHash64(());
SELECT sipHash64((), ());
SELECT sipHash64((), 1);
SELECT sipHash64(1, ());
SELECT sipHash64(1, (), 1);
SELECT sipHash64((), 1, ());
SELECT sipHash64((), (1, 2));
SELECT sipHash64((), (1, 2));
SELECT sipHash64((1, 2), ());
SELECT sipHash64((), (1, 2), ());
SELECT sipHash64((1, 2), (), (3, 4));
SELECT sipHash64(materialize(()));
SELECT sipHash64(materialize(()), materialize(()));
SELECT sipHash64(materialize(()), 1);
SELECT sipHash64(1, materialize(()));
SELECT sipHash64(1, materialize(()), 1);
SELECT sipHash64((), 1, materialize(()));
SELECT sipHash64(materialize(()), (1, 2));
SELECT sipHash64(materialize(()), (1, 2));
SELECT sipHash64((1, 2), materialize(()));
SELECT sipHash64(materialize(()), (1, 2), ());
SELECT sipHash64((1, 2), materialize(()), (3, 4));

View File

@ -1653,6 +1653,7 @@ formated
formatschema
formatter
formatters
fqdn
frac
freezed
fromDaysSinceYearZero